diff options
author | Reynold Xin <rxin@databricks.com> | 2015-01-13 17:16:41 -0800 |
---|---|---|
committer | Reynold Xin <rxin@databricks.com> | 2015-01-13 17:16:41 -0800 |
commit | f9969098c8cb15e36c718b80c6cf5b534a6cf7c3 (patch) | |
tree | f7335abaafcd8e044d09565f6f1f21f24d5bc544 /sql/core | |
parent | 14e3f114efb906937b2d7b7ac04484b2814a3b48 (diff) | |
download | spark-f9969098c8cb15e36c718b80c6cf5b534a6cf7c3.tar.gz spark-f9969098c8cb15e36c718b80c6cf5b534a6cf7c3.tar.bz2 spark-f9969098c8cb15e36c718b80c6cf5b534a6cf7c3.zip |
[SPARK-5123][SQL] Reconcile Java/Scala API for data types.
Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box.
As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code.
This subsumes https://github.com/apache/spark/pull/3925
Author: Reynold Xin <rxin@databricks.com>
Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits:
66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).
Diffstat (limited to 'sql/core')
83 files changed, 226 insertions, 2206 deletions
diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java deleted file mode 100644 index b73a371e93..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing Lists. - * An ArrayType object comprises two fields, {@code DataType elementType} and - * {@code boolean containsNull}. The field of {@code elementType} is used to specify the type of - * array elements. The field of {@code containsNull} is used to specify if the array has - * {@code null} values. - * - * To create an {@link ArrayType}, - * {@link DataType#createArrayType(DataType)} or - * {@link DataType#createArrayType(DataType, boolean)} - * should be used. - */ -public class ArrayType extends DataType { - private DataType elementType; - private boolean containsNull; - - protected ArrayType(DataType elementType, boolean containsNull) { - this.elementType = elementType; - this.containsNull = containsNull; - } - - public DataType getElementType() { - return elementType; - } - - public boolean isContainsNull() { - return containsNull; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ArrayType arrayType = (ArrayType) o; - - if (containsNull != arrayType.containsNull) return false; - if (!elementType.equals(arrayType.elementType)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = elementType.hashCode(); - result = 31 * result + (containsNull ? 1 : 0); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java deleted file mode 100644 index 7daad60f62..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing byte[] values. - * - * {@code BinaryType} is represented by the singleton object {@link DataType#BinaryType}. - */ -public class BinaryType extends DataType { - protected BinaryType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java deleted file mode 100644 index 5a1f527256..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing boolean and Boolean values. - * - * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}. - */ -public class BooleanType extends DataType { - protected BooleanType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java deleted file mode 100644 index e5cdf06b21..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing byte and Byte values. - * - * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}. - */ -public class ByteType extends DataType { - protected ByteType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java deleted file mode 100644 index c69bbd5736..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import java.util.*; - -/** - * The base type of all Spark SQL data types. - * - * To get/create specific data type, users should use singleton objects and factory methods - * provided by this class. - */ -public abstract class DataType { - - /** - * Gets the StringType object. - */ - public static final StringType StringType = new StringType(); - - /** - * Gets the BinaryType object. - */ - public static final BinaryType BinaryType = new BinaryType(); - - /** - * Gets the BooleanType object. - */ - public static final BooleanType BooleanType = new BooleanType(); - - /** - * Gets the DateType object. - */ - public static final DateType DateType = new DateType(); - - /** - * Gets the TimestampType object. - */ - public static final TimestampType TimestampType = new TimestampType(); - - /** - * Gets the DoubleType object. - */ - public static final DoubleType DoubleType = new DoubleType(); - - /** - * Gets the FloatType object. - */ - public static final FloatType FloatType = new FloatType(); - - /** - * Gets the ByteType object. - */ - public static final ByteType ByteType = new ByteType(); - - /** - * Gets the IntegerType object. - */ - public static final IntegerType IntegerType = new IntegerType(); - - /** - * Gets the LongType object. - */ - public static final LongType LongType = new LongType(); - - /** - * Gets the ShortType object. - */ - public static final ShortType ShortType = new ShortType(); - - /** - * Gets the NullType object. - */ - public static final NullType NullType = new NullType(); - - /** - * Creates an ArrayType by specifying the data type of elements ({@code elementType}). - * The field of {@code containsNull} is set to {@code true}. - */ - public static ArrayType createArrayType(DataType elementType) { - if (elementType == null) { - throw new IllegalArgumentException("elementType should not be null."); - } - - return new ArrayType(elementType, true); - } - - /** - * Creates an ArrayType by specifying the data type of elements ({@code elementType}) and - * whether the array contains null values ({@code containsNull}). - */ - public static ArrayType createArrayType(DataType elementType, boolean containsNull) { - if (elementType == null) { - throw new IllegalArgumentException("elementType should not be null."); - } - - return new ArrayType(elementType, containsNull); - } - - /** - * Creates a MapType by specifying the data type of keys ({@code keyType}) and values - * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}. - */ - public static MapType createMapType(DataType keyType, DataType valueType) { - if (keyType == null) { - throw new IllegalArgumentException("keyType should not be null."); - } - if (valueType == null) { - throw new IllegalArgumentException("valueType should not be null."); - } - - return new MapType(keyType, valueType, true); - } - - /** - * Creates a MapType by specifying the data type of keys ({@code keyType}), the data type of - * values ({@code keyType}), and whether values contain any null value - * ({@code valueContainsNull}). - */ - public static MapType createMapType( - DataType keyType, - DataType valueType, - boolean valueContainsNull) { - if (keyType == null) { - throw new IllegalArgumentException("keyType should not be null."); - } - if (valueType == null) { - throw new IllegalArgumentException("valueType should not be null."); - } - - return new MapType(keyType, valueType, valueContainsNull); - } - - /** - * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and - * whether values of this field can be null values ({@code nullable}). - */ - public static StructField createStructField( - String name, - DataType dataType, - boolean nullable, - Metadata metadata) { - if (name == null) { - throw new IllegalArgumentException("name should not be null."); - } - if (dataType == null) { - throw new IllegalArgumentException("dataType should not be null."); - } - if (metadata == null) { - throw new IllegalArgumentException("metadata should not be null."); - } - - return new StructField(name, dataType, nullable, metadata); - } - - /** - * Creates a StructField with empty metadata. - * - * @see #createStructField(String, DataType, boolean, Metadata) - */ - public static StructField createStructField(String name, DataType dataType, boolean nullable) { - return createStructField(name, dataType, nullable, (new MetadataBuilder()).build()); - } - - /** - * Creates a StructType with the given list of StructFields ({@code fields}). - */ - public static StructType createStructType(List<StructField> fields) { - return createStructType(fields.toArray(new StructField[0])); - } - - /** - * Creates a StructType with the given StructField array ({@code fields}). - */ - public static StructType createStructType(StructField[] fields) { - if (fields == null) { - throw new IllegalArgumentException("fields should not be null."); - } - Set<String> distinctNames = new HashSet<String>(); - for (StructField field: fields) { - if (field == null) { - throw new IllegalArgumentException( - "fields should not contain any null."); - } - - distinctNames.add(field.getName()); - } - if (distinctNames.size() != fields.length) { - throw new IllegalArgumentException("fields should have distinct names."); - } - - return new StructType(fields); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java deleted file mode 100644 index 6677793baa..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing java.sql.Date values. - * - * {@code DateType} is represented by the singleton object {@link DataType#DateType}. - */ -public class DateType extends DataType { - protected DateType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java deleted file mode 100644 index 60752451ec..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing java.math.BigDecimal values. - */ -public class DecimalType extends DataType { - private boolean hasPrecisionInfo; - private int precision; - private int scale; - - public DecimalType(int precision, int scale) { - this.hasPrecisionInfo = true; - this.precision = precision; - this.scale = scale; - } - - public DecimalType() { - this.hasPrecisionInfo = false; - this.precision = -1; - this.scale = -1; - } - - public boolean isUnlimited() { - return !hasPrecisionInfo; - } - - public boolean isFixed() { - return hasPrecisionInfo; - } - - /** Return the precision, or -1 if no precision is set */ - public int getPrecision() { - return precision; - } - - /** Return the scale, or -1 if no precision is set */ - public int getScale() { - return scale; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - DecimalType that = (DecimalType) o; - - if (hasPrecisionInfo != that.hasPrecisionInfo) return false; - if (precision != that.precision) return false; - if (scale != that.scale) return false; - - return true; - } - - @Override - public int hashCode() { - int result = (hasPrecisionInfo ? 1 : 0); - result = 31 * result + precision; - result = 31 * result + scale; - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java deleted file mode 100644 index f0060d0bcf..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing double and Double values. - * - * {@code DoubleType} is represented by the singleton object {@link DataType#DoubleType}. - */ -public class DoubleType extends DataType { - protected DoubleType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java deleted file mode 100644 index 4a6a37f691..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing float and Float values. - * - * {@code FloatType} is represented by the singleton object {@link DataType#FloatType}. - */ -public class FloatType extends DataType { - protected FloatType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java deleted file mode 100644 index bfd70490bb..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing int and Integer values. - * - * {@code IntegerType} is represented by the singleton object {@link DataType#IntegerType}. - */ -public class IntegerType extends DataType { - protected IntegerType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java deleted file mode 100644 index af13a46eb1..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing long and Long values. - * - * {@code LongType} is represented by the singleton object {@link DataType#LongType}. - */ -public class LongType extends DataType { - protected LongType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java deleted file mode 100644 index 063e6b34ab..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing Maps. A MapType object comprises two fields, - * {@code DataType keyType}, {@code DataType valueType}, and {@code boolean valueContainsNull}. - * The field of {@code keyType} is used to specify the type of keys in the map. - * The field of {@code valueType} is used to specify the type of values in the map. - * The field of {@code valueContainsNull} is used to specify if map values have - * {@code null} values. - * For values of a MapType column, keys are not allowed to have {@code null} values. - * - * To create a {@link MapType}, - * {@link DataType#createMapType(DataType, DataType)} or - * {@link DataType#createMapType(DataType, DataType, boolean)} - * should be used. - */ -public class MapType extends DataType { - private DataType keyType; - private DataType valueType; - private boolean valueContainsNull; - - protected MapType(DataType keyType, DataType valueType, boolean valueContainsNull) { - this.keyType = keyType; - this.valueType = valueType; - this.valueContainsNull = valueContainsNull; - } - - public DataType getKeyType() { - return keyType; - } - - public DataType getValueType() { - return valueType; - } - - public boolean isValueContainsNull() { - return valueContainsNull; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - MapType mapType = (MapType) o; - - if (valueContainsNull != mapType.valueContainsNull) return false; - if (!keyType.equals(mapType.keyType)) return false; - if (!valueType.equals(mapType.valueType)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = keyType.hashCode(); - result = 31 * result + valueType.hashCode(); - result = 31 * result + (valueContainsNull ? 1 : 0); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java deleted file mode 100644 index 0f819fb01a..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, - * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and - * Array[Metadata]. JSON is used for serialization. - * - * The default constructor is private. User should use [[MetadataBuilder]]. - */ -class Metadata extends org.apache.spark.sql.catalyst.util.Metadata { - Metadata(scala.collection.immutable.Map<String, Object> map) { - super(map); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java deleted file mode 100644 index 6e6b12f072..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. - */ -public class MetadataBuilder extends org.apache.spark.sql.catalyst.util.MetadataBuilder { - @Override - public Metadata build() { - return new Metadata(getMap()); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java deleted file mode 100644 index 6d5ecdf46e..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing null and NULL values. - * - * {@code NullType} is represented by the singleton object {@link DataType#NullType}. - */ -public class NullType extends DataType { - protected NullType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java deleted file mode 100644 index 7d7604b4e3..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing short and Short values. - * - * {@code ShortType} is represented by the singleton object {@link DataType#ShortType}. - */ -public class ShortType extends DataType { - protected ShortType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java deleted file mode 100644 index f4ba0c07c9..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing String values. - * - * {@code StringType} is represented by the singleton object {@link DataType#StringType}. - */ -public class StringType extends DataType { - protected StringType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java deleted file mode 100644 index 7c60d492bc..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import java.util.Map; - -/** - * A StructField object represents a field in a StructType object. - * A StructField object comprises three fields, {@code String name}, {@code DataType dataType}, - * and {@code boolean nullable}. The field of {@code name} is the name of a StructField. - * The field of {@code dataType} specifies the data type of a StructField. - * The field of {@code nullable} specifies if values of a StructField can contain {@code null} - * values. - * The field of {@code metadata} provides extra information of the StructField. - * - * To create a {@link StructField}, - * {@link DataType#createStructField(String, DataType, boolean, Metadata)} - * should be used. - */ -public class StructField { - private String name; - private DataType dataType; - private boolean nullable; - private Metadata metadata; - - protected StructField( - String name, - DataType dataType, - boolean nullable, - Metadata metadata) { - this.name = name; - this.dataType = dataType; - this.nullable = nullable; - this.metadata = metadata; - } - - public String getName() { - return name; - } - - public DataType getDataType() { - return dataType; - } - - public boolean isNullable() { - return nullable; - } - - public Metadata getMetadata() { - return metadata; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StructField that = (StructField) o; - - if (nullable != that.nullable) return false; - if (!dataType.equals(that.dataType)) return false; - if (!name.equals(that.name)) return false; - if (!metadata.equals(that.metadata)) return false; - - return true; - } - - @Override - public int hashCode() { - int result = name.hashCode(); - result = 31 * result + dataType.hashCode(); - result = 31 * result + (nullable ? 1 : 0); - result = 31 * result + metadata.hashCode(); - return result; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java deleted file mode 100644 index a4b501efd9..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import java.util.Arrays; - -/** - * The data type representing Rows. - * A StructType object comprises an array of StructFields. - * - * To create an {@link StructType}, - * {@link DataType#createStructType(java.util.List)} or - * {@link DataType#createStructType(StructField[])} - * should be used. - */ -public class StructType extends DataType { - private StructField[] fields; - - protected StructType(StructField[] fields) { - this.fields = fields; - } - - public StructField[] getFields() { - return fields; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - StructType that = (StructType) o; - - if (!Arrays.equals(fields, that.fields)) return false; - - return true; - } - - @Override - public int hashCode() { - return Arrays.hashCode(fields); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java deleted file mode 100644 index 06d44c731c..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -/** - * The data type representing java.sql.Timestamp values. - * - * {@code TimestampType} is represented by the singleton object {@link DataType#TimestampType}. - */ -public class TimestampType extends DataType { - protected TimestampType() {} -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java deleted file mode 100644 index f0d079d25b..0000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import java.io.Serializable; - -import org.apache.spark.annotation.DeveloperApi; - -/** - * ::DeveloperApi:: - * The data type representing User-Defined Types (UDTs). - * UDTs may use any other DataType for an underlying representation. - */ -@DeveloperApi -public abstract class UserDefinedType<UserType> extends DataType implements Serializable { - - protected UserDefinedType() { } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - @SuppressWarnings("unchecked") - UserDefinedType<UserType> that = (UserDefinedType<UserType>) o; - return this.sqlType().equals(that.sqlType()); - } - - /** Underlying storage type for this UDT */ - public abstract DataType sqlType(); - - /** Convert the user type to a SQL datum */ - public abstract Object serialize(Object obj); - - /** Convert a SQL datum to the user type */ - public abstract UserType deserialize(Object datum); - - /** Class object for the UserType */ - public abstract Class<UserType> userClass(); -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d8efce0cb4..d9f3b3a53f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,7 +24,6 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration - import org.apache.spark.SparkContext import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD @@ -32,14 +31,14 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} +import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types.UserDefinedType -import org.apache.spark.sql.execution.{SparkStrategies, _} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{DataSourceStrategy, BaseRelation, DDLParser, LogicalRelation} +import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation} +import org.apache.spark.sql.types._ /** * :: AlphaComponent :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 80787b61ce..686bcdfbb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql -import java.util.{Map => JMap, List => JList} - +import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -37,8 +36,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.{BooleanType, StructType} import org.apache.spark.storage.StorageLevel /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala index 65358b7d4e..f10ee7b66f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSQLParser.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql +import scala.util.parsing.combinator.RegexParsers + import org.apache.spark.sql.catalyst.{SqlLexical, AbstractSparkSQLParser} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{UncacheTableCommand, CacheTableCommand, SetCommand} +import org.apache.spark.sql.types.StringType -import scala.util.parsing.combinator.RegexParsers /** * The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 7f868cd4af..a75f559928 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -23,15 +23,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.sql.{SQLContext, StructType => SStructType} -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation} -import org.apache.spark.sql.types.util.DataTypeConversions -import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils /** @@ -126,9 +124,8 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { @DeveloperApi def applySchema(rowRDD: JavaRDD[Row], schema: StructType): JavaSchemaRDD = { val scalaRowRDD = rowRDD.rdd.map(r => r.row) - val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] val logicalPlan = - LogicalRDD(scalaSchema.toAttributes, scalaRowRDD)(sqlContext) + LogicalRDD(schema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -184,10 +181,10 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord val appliedScalaSchema = - Option(asScalaDataType(schema)).getOrElse( + Option(schema).getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( - json.rdd, 1.0, columnNameOfCorruptJsonRecord))).asInstanceOf[SStructType] + json.rdd, 1.0, columnNameOfCorruptJsonRecord))) val scalaRowRDD = JsonRDD.jsonStringToRow( json.rdd, appliedScalaSchema, columnNameOfCorruptJsonRecord) val logicalPlan = @@ -218,43 +215,25 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val (dataType, nullable) = property.getPropertyType match { case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) - case c: Class[_] if c == classOf[java.lang.String] => - (org.apache.spark.sql.StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => - (org.apache.spark.sql.ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => - (org.apache.spark.sql.IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => - (org.apache.spark.sql.LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => - (org.apache.spark.sql.DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => - (org.apache.spark.sql.ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => - (org.apache.spark.sql.FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => - (org.apache.spark.sql.BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => - (org.apache.spark.sql.ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => - (org.apache.spark.sql.IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => - (org.apache.spark.sql.LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => - (org.apache.spark.sql.DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => - (org.apache.spark.sql.ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => - (org.apache.spark.sql.FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => - (org.apache.spark.sql.BooleanType, true) - case c: Class[_] if c == classOf[java.math.BigDecimal] => - (org.apache.spark.sql.DecimalType(), true) - case c: Class[_] if c == classOf[java.sql.Date] => - (org.apache.spark.sql.DateType, true) - case c: Class[_] if c == classOf[java.sql.Timestamp] => - (org.apache.spark.sql.TimestampType, true) + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType(), true) + case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) } AttributeReference(property.getName, dataType, nullable)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index 5b9c612487..9e10e532fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.api.java import java.util.{List => JList} import org.apache.spark.Partitioner -import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.{JavaRDD, JavaRDDLike} import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.sql.types.util.DataTypeConversions +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import DataTypeConversions._ -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel /** @@ -59,8 +58,7 @@ class JavaSchemaRDD( override def toString: String = baseSchemaRDD.toString /** Returns the schema of this JavaSchemaRDD (represented by a StructType). */ - def schema: StructType = - asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType] + def schema: StructType = baseSchemaRDD.schema.asInstanceOf[StructType] // ======================================================================= // Base RDD functions that do NOT change schema diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 401798e317..207e2805ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.api.java -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.annotation.varargs import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} import scala.collection.JavaConversions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala index 158f26e3d4..4186c27451 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.api.java import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} -import org.apache.spark.sql.types.util.DataTypeConversions._ +import org.apache.spark.sql.types.DataType /** * A collection of functions that allow Java users to register UDFs. In order to handle functions @@ -38,10 +38,9 @@ private[java] trait UDFRegistration { println(s""" |def registerFunction( | name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { - | val scalaType = asScalaDataType(dataType) | sqlContext.functionRegistry.registerFunction( | name, - | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e)) + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), dataType, e)) |} """.stripMargin) } @@ -94,159 +93,159 @@ private[java] trait UDFRegistration { */ // scalastyle:off - def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF1[_, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), dataType, e)) } - def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF2[_, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF3[_, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF4[_, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF5[_, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF6[_, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF7[_, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF8[_, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + + def registerFunction( + name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { - val scalaType = asScalaDataType(dataType) + def registerFunction( + name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], @transient dataType: DataType) = { sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) } - // scalastyle:on } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala deleted file mode 100644 index a7d0f4f127..0000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java - -import org.apache.spark.sql.catalyst.types.{UserDefinedType => ScalaUserDefinedType} -import org.apache.spark.sql.{DataType => ScalaDataType} -import org.apache.spark.sql.types.util.DataTypeConversions - -/** - * Scala wrapper for a Java UserDefinedType - */ -private[sql] class JavaToScalaUDTWrapper[UserType](val javaUDT: UserDefinedType[UserType]) - extends ScalaUserDefinedType[UserType] with Serializable { - - /** Underlying storage type for this UDT */ - val sqlType: ScalaDataType = DataTypeConversions.asScalaDataType(javaUDT.sqlType()) - - /** Convert the user type to a SQL datum */ - def serialize(obj: Any): Any = javaUDT.serialize(obj) - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType = javaUDT.deserialize(datum) - - val userClass: java.lang.Class[UserType] = javaUDT.userClass() -} - -/** - * Java wrapper for a Scala UserDefinedType - */ -private[sql] class ScalaToJavaUDTWrapper[UserType](val scalaUDT: ScalaUserDefinedType[UserType]) - extends UserDefinedType[UserType] with Serializable { - - /** Underlying storage type for this UDT */ - val sqlType: DataType = DataTypeConversions.asJavaDataType(scalaUDT.sqlType) - - /** Convert the user type to a SQL datum */ - def serialize(obj: Any): java.lang.Object = scalaUDT.serialize(obj).asInstanceOf[java.lang.Object] - - /** Convert a SQL datum to the user type */ - def deserialize(datum: Any): UserType = scalaUDT.deserialize(datum) - - val userClass: java.lang.Class[UserType] = scalaUDT.userClass -} - -private[sql] object UDTWrappers { - - def wrapAsScala(udtType: UserDefinedType[_]): ScalaUserDefinedType[_] = { - udtType match { - case t: ScalaToJavaUDTWrapper[_] => t.scalaUDT - case _ => new JavaToScalaUDTWrapper(udtType) - } - } - - def wrapAsJava(udtType: ScalaUserDefinedType[_]): UserDefinedType[_] = { - udtType match { - case t: JavaToScalaUDTWrapper[_] => t.javaUDT - case _ => new ScalaToJavaUDTWrapper(udtType) - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 538dd5b734..91c4c105b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor +import org.apache.spark.sql.types.{BinaryType, DataType, NativeType} /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c68dceef3b..3a4977b836 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnBuilder._ import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder} +import org.apache.spark.sql.types._ private[sql] trait ColumnBuilder { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 668efe4a3b..391b3dae5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -21,7 +21,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index ab66c85c4f..fcf2faa091 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -24,8 +24,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ /** * An abstract class that represents type of a column. Used to append/extract Java objects into/from diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index 27ac5f4dbd..7dff9deac8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.columnar.compression import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} +import org.apache.spark.sql.types.NativeType private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { this: NativeColumnAccessor[T] => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 628d9cec41..aead768ecd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.Logging import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} +import org.apache.spark.sql.types.NativeType /** * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index acb06cb537..879d29bcfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -21,8 +21,8 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} +import org.apache.spark.sql.types.NativeType private[sql] trait Encoder[T <: NativeType] { def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 29edcf1724..6467324839 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -25,10 +25,11 @@ import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils + private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 069e950195..20b14834bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{StructType, Row, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} +import org.apache.spark.sql.types.StructType /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 7c3bf947e7..4abe26fe4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ case class AggregateEvaluation( schema: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 84d96e612f..131146012e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -29,7 +29,7 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0652d2ff7c..0cc9d049c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,16 +17,16 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.parquet._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 61be5ed2db..46245cd5a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -24,7 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext._ import org.apache.spark.sql.{SchemaRDD, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 5a41399971..741ccb8fb8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -33,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ import org.apache.spark.{Accumulator, Logging => SparkLogging} /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f5c02224c8..1af96c28d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.json import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType + private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 00449c2007..c92ec543e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.types.util.DataTypeConversions - import java.io.StringWriter import scala.collection.Map @@ -34,8 +31,9 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { @@ -246,7 +244,7 @@ private[sql] object JsonRDD extends Logging { // The value associated with the key is an array. // Handle inner structs of an array. def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { - case ArrayType(StructType(Nil), containsNull) => { + case ArrayType(e: StructType, containsNull) => { // The elements of this arrays are structs. v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap { element => allKeysWithValueTypes(element) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 1fd8e6220f..b75266d5aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -117,357 +117,8 @@ package object sql { val Row = catalyst.expressions.Row /** - * :: DeveloperApi :: - * - * The base type of all Spark SQL data types. - * - * @group dataType - */ - @DeveloperApi - type DataType = catalyst.types.DataType - - @DeveloperApi - val DataType = catalyst.types.DataType - - /** - * :: DeveloperApi :: - * - * The data type representing `String` values - * - * @group dataType - */ - @DeveloperApi - val StringType = catalyst.types.StringType - - /** - * :: DeveloperApi :: - * - * The data type representing `Array[Byte]` values. - * - * @group dataType - */ - @DeveloperApi - val BinaryType = catalyst.types.BinaryType - - /** - * :: DeveloperApi :: - * - * The data type representing `Boolean` values. - * - *@group dataType - */ - @DeveloperApi - val BooleanType = catalyst.types.BooleanType - - /** - * :: DeveloperApi :: - * - * The data type representing `java.sql.Timestamp` values. - * - * @group dataType - */ - @DeveloperApi - val TimestampType = catalyst.types.TimestampType - - /** - * :: DeveloperApi :: - * - * The data type representing `java.sql.Date` values. - * - * @group dataType - */ - @DeveloperApi - val DateType = catalyst.types.DateType - - /** - * :: DeveloperApi :: - * - * The data type representing `scala.math.BigDecimal` values. - * - * TODO(matei): explain precision and scale - * - * @group dataType - */ - @DeveloperApi - type DecimalType = catalyst.types.DecimalType - - /** - * :: DeveloperApi :: - * - * The data type representing `scala.math.BigDecimal` values. - * - * TODO(matei): explain precision and scale - * - * @group dataType - */ - @DeveloperApi - val DecimalType = catalyst.types.DecimalType - - /** - * :: DeveloperApi :: - * - * The data type representing `Double` values. - * - * @group dataType - */ - @DeveloperApi - val DoubleType = catalyst.types.DoubleType - - /** - * :: DeveloperApi :: - * - * The data type representing `Float` values. - * - * @group dataType - */ - @DeveloperApi - val FloatType = catalyst.types.FloatType - - /** - * :: DeveloperApi :: - * - * The data type representing `Byte` values. - * - * @group dataType - */ - @DeveloperApi - val ByteType = catalyst.types.ByteType - - /** - * :: DeveloperApi :: - * - * The data type representing `Int` values. - * - * @group dataType - */ - @DeveloperApi - val IntegerType = catalyst.types.IntegerType - - /** - * :: DeveloperApi :: - * - * The data type representing `Long` values. - * - * @group dataType - */ - @DeveloperApi - val LongType = catalyst.types.LongType - - /** - * :: DeveloperApi :: - * - * The data type representing `Short` values. - * - * @group dataType - */ - @DeveloperApi - val ShortType = catalyst.types.ShortType - - /** - * :: DeveloperApi :: - * - * The data type representing `NULL` values. - * - * @group dataType - */ - @DeveloperApi - val NullType = catalyst.types.NullType - - /** - * :: DeveloperApi :: - * - * The data type for collections of multiple values. - * Internally these are represented as columns that contain a ``scala.collection.Seq``. - * - * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and - * `containsNull: Boolean`. The field of `elementType` is used to specify the type of - * array elements. The field of `containsNull` is used to specify if the array has `null` values. - * - * @group dataType - */ - @DeveloperApi - type ArrayType = catalyst.types.ArrayType - - /** - * :: DeveloperApi :: - * - * An [[ArrayType]] object can be constructed with two ways, - * {{{ - * ArrayType(elementType: DataType, containsNull: Boolean) - * }}} and - * {{{ - * ArrayType(elementType: DataType) - * }}} - * For `ArrayType(elementType)`, the field of `containsNull` is set to `false`. - * - * @group dataType - */ - @DeveloperApi - val ArrayType = catalyst.types.ArrayType - - /** - * :: DeveloperApi :: - * - * The data type representing `Map`s. A [[MapType]] object comprises three fields, - * `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`. - * The field of `keyType` is used to specify the type of keys in the map. - * The field of `valueType` is used to specify the type of values in the map. - * The field of `valueContainsNull` is used to specify if values of this map has `null` values. - * For values of a MapType column, keys are not allowed to have `null` values. - * - * @group dataType - */ - @DeveloperApi - type MapType = catalyst.types.MapType - - /** - * :: DeveloperApi :: - * - * A [[MapType]] object can be constructed with two ways, - * {{{ - * MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) - * }}} and - * {{{ - * MapType(keyType: DataType, valueType: DataType) - * }}} - * For `MapType(keyType: DataType, valueType: DataType)`, - * the field of `valueContainsNull` is set to `true`. - * - * @group dataType - */ - @DeveloperApi - val MapType = catalyst.types.MapType - - /** - * :: DeveloperApi :: - * - * The data type representing [[Row]]s. - * A [[StructType]] object comprises a [[Seq]] of [[StructField]]s. - * - * @group dataType - */ - @DeveloperApi - type StructType = catalyst.types.StructType - - /** - * :: DeveloperApi :: - * - * A [[StructType]] object can be constructed by - * {{{ - * StructType(fields: Seq[StructField]) - * }}} - * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. - * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. - * If a provided name does not have a matching field, it will be ignored. For the case - * of extracting a single StructField, a `null` will be returned. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val struct = - * StructType( - * StructField("a", IntegerType, true) :: - * StructField("b", LongType, false) :: - * StructField("c", BooleanType, false) :: Nil) - * - * // Extract a single StructField. - * val singleField = struct("b") - * // singleField: StructField = StructField(b,LongType,false) - * - * // This struct does not have a field called "d". null will be returned. - * val nonExisting = struct("d") - * // nonExisting: StructField = null - * - * // Extract multiple StructFields. Field names are provided in a set. - * // A StructType object will be returned. - * val twoFields = struct(Set("b", "c")) - * // twoFields: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * - * // Those names do not have matching fields will be ignored. - * // For the case shown below, "d" will be ignored and - * // it is treated as struct(Set("b", "c")). - * val ignoreNonExisting = struct(Set("b", "c", "d")) - * // ignoreNonExisting: StructType = - * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) - * }}} - * - * A [[Row]] object is used as a value of the StructType. - * Example: - * {{{ - * import org.apache.spark.sql._ - * - * val innerStruct = - * StructType( - * StructField("f1", IntegerType, true) :: - * StructField("f2", LongType, false) :: - * StructField("f3", BooleanType, false) :: Nil) - * - * val struct = StructType( - * StructField("a", innerStruct, true) :: Nil) - * - * // Create a Row with the schema defined by struct - * val row = Row(Row(1, 2, true)) - * // row: Row = [[1,2,true]] - * }}} - * - * @group dataType - */ - @DeveloperApi - val StructType = catalyst.types.StructType - - /** - * :: DeveloperApi :: - * - * A [[StructField]] object represents a field in a [[StructType]] object. - * A [[StructField]] object comprises three fields, `name: [[String]]`, `dataType: [[DataType]]`, - * and `nullable: Boolean`. The field of `name` is the name of a `StructField`. The field of - * `dataType` specifies the data type of a `StructField`. - * The field of `nullable` specifies if values of a `StructField` can contain `null` values. - * - * @group field - */ - @DeveloperApi - type StructField = catalyst.types.StructField - - /** - * :: DeveloperApi :: - * - * A [[StructField]] object can be constructed by - * {{{ - * StructField(name: String, dataType: DataType, nullable: Boolean) - * }}} - * - * @group dataType - */ - @DeveloperApi - val StructField = catalyst.types.StructField - - /** * Converts a logical plan into zero or more SparkPlans. */ @DeveloperApi type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] - - /** - * :: DeveloperApi :: - * - * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, - * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and - * Array[Metadata]. JSON is used for serialization. - * - * The default constructor is private. User should use either [[MetadataBuilder]] or - * [[Metadata$#fromJson]] to create Metadata instances. - * - * @param map an immutable map that stores the data - */ - @DeveloperApi - type Metadata = catalyst.util.Metadata - - /** - * :: DeveloperApi :: - * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. - */ - @DeveloperApi - type MetadataBuilder = catalyst.util.MetadataBuilder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 1bbb66aaa1..7f437c4077 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql.parquet -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** * Collection of converters of Parquet types (group and primitive types) that @@ -91,8 +90,8 @@ private[sql] object CatalystConverter { case ArrayType(elementType: DataType, true) => { new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) } - case StructType(fields: Seq[StructField]) => { - new CatalystStructConverter(fields.toArray, fieldIndex, parent) + case StructType(fields: Array[StructField]) => { + new CatalystStructConverter(fields, fieldIndex, parent) } case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { new CatalystMapConverter( @@ -436,7 +435,7 @@ private[parquet] object CatalystArrayConverter { * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (complex or primitive) * @param index The position of this (array) field inside its parent converter @@ -500,7 +499,7 @@ private[parquet] class CatalystArrayConverter( * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (native) * @param index The position of this (array) field inside its parent converter @@ -621,7 +620,7 @@ private[parquet] class CatalystNativeArrayConverter( * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array contains null (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * [[org.apache.spark.sql.types.ArrayType]]. * * @param elementType The type of the array elements (complex or primitive) * @param index The position of this (array) field inside its parent converter @@ -727,7 +726,7 @@ private[parquet] class CatalystStructConverter( * A `parquet.io.api.GroupConverter` that converts two-element groups that * match the characteristics of a map (see * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.catalyst.types.MapType]]. + * [[org.apache.spark.sql.types.MapType]]. * * @param schema * @param index diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 56e7d11b2f..f08350878f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -29,7 +29,7 @@ import parquet.io.api.Binary import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 9049eb5932..af7248fdf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -29,8 +29,8 @@ import parquet.schema.MessageType import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal /** * A `parquet.io.api.RecordMaterializer` for Rows. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 97447871a1..6d8c682ccc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -36,7 +36,7 @@ import parquet.schema.Type.Repetition import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ // Implicits import scala.collection.JavaConversions._ @@ -80,7 +80,7 @@ private[parquet] object ParquetTypesConverter extends Logging { /** * Converts a given Parquet `Type` into the corresponding - * [[org.apache.spark.sql.catalyst.types.DataType]]. + * [[org.apache.spark.sql.types.DataType]]. * * We apply the following conversion rules: * <ul> @@ -191,7 +191,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } /** - * For a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] return + * For a given Catalyst [[org.apache.spark.sql.types.DataType]] return * the name of the corresponding Parquet primitive type or None if the given type * is not primitive. * @@ -231,21 +231,21 @@ private[parquet] object ParquetTypesConverter extends Logging { } /** - * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into + * Converts a given Catalyst [[org.apache.spark.sql.types.DataType]] into * the corresponding Parquet `Type`. * * The conversion follows the rules below: * <ul> * <li> Primitive types are converted into Parquet's primitive types.</li> - * <li> [[org.apache.spark.sql.catalyst.types.StructType]]s are converted + * <li> [[org.apache.spark.sql.types.StructType]]s are converted * into Parquet's `GroupType` with the corresponding field types.</li> - * <li> [[org.apache.spark.sql.catalyst.types.ArrayType]]s are converted + * <li> [[org.apache.spark.sql.types.ArrayType]]s are converted * into a 2-level nested group, where the outer group has the inner * group as sole field. The inner group has name `values` and * repetition level `REPEATED` and has the element type of * the array as schema. We use Parquet's `ConversionPatterns` for this * purpose.</li> - * <li> [[org.apache.spark.sql.catalyst.types.MapType]]s are converted + * <li> [[org.apache.spark.sql.types.MapType]]s are converted * into a nested (2-level) Parquet `GroupType` with two fields: a key * type and a value type. The nested group has repetition level * `REPEATED` and name `map`. We use Parquet's `ConversionPatterns` @@ -319,7 +319,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } - new ParquetGroupType(repetition, name, fields) + new ParquetGroupType(repetition, name, fields.toSeq) } case MapType(keyType, valueType, valueContainsNull) => { val parquetKeyType = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 55a2728a85..1b50afbbab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.parquet import java.util.{List => JList} +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job} -import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import parquet.hadoop.ParquetInputFormat import parquet.hadoop.util.ContextUtil @@ -30,13 +31,11 @@ import parquet.hadoop.util.ContextUtil import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{Partition => SparkPartition, Logging} import org.apache.spark.rdd.{NewHadoopPartition, RDD} - import org.apache.spark.sql.{SQLConf, Row, SQLContext} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, StructField, StructType} import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -import scala.collection.JavaConversions._ /** * Allows creation of parquet based tables using the syntax diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala index 4d87f6817d..12b59ba20b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.AttributeMap +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LeafNode, LogicalPlan} /** @@ -27,7 +27,7 @@ private[sql] case class LogicalRelation(relation: BaseRelation) extends LeafNode with MultiInstanceRelation { - override val output = relation.schema.toAttributes + override val output: Seq[AttributeReference] = relation.schema.toAttributes // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any) = other match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index f8741e0082..4cc9641c4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -23,11 +23,11 @@ import scala.util.parsing.combinator.PackratParsers import org.apache.spark.Logging import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.RunnableCommand -import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.SqlLexical +import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils /** * A parser for foreign DDL commands. @@ -162,10 +162,10 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi protected lazy val structType: Parser[DataType] = (STRUCT ~> "<" ~> repsep(structField, ",") <~ ">" ^^ { - case fields => new StructType(fields) + case fields => StructType(fields) }) | (STRUCT ~> "<>" ^^ { - case fields => new StructType(Nil) + case fields => StructType(Nil) }) private[sql] lazy val dataType: Parser[DataType] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 7f5564baa0..cd82cc6ecb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.sources import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext, StructType} +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} +import org.apache.spark.sql.types.StructType /** * ::DeveloperApi:: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index b9569e96c0..006b16fbe0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -20,9 +20,7 @@ package org.apache.spark.sql.test import java.util import scala.collection.JavaConverters._ - -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ /** * An example class to demonstrate UDT in Scala, Java, and Python. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala deleted file mode 100644 index d4ef517981..0000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.types.util - -import java.text.SimpleDateFormat - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql._ -import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, - MetadataBuilder => JMetaDataBuilder, UDTWrappers} -import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.types.UserDefinedType - -protected[sql] object DataTypeConversions { - - /** - * Returns the equivalent StructField in Scala for the given StructField in Java. - */ - def asJavaStructField(scalaStructField: StructField): JStructField = { - JDataType.createStructField( - scalaStructField.name, - asJavaDataType(scalaStructField.dataType), - scalaStructField.nullable, - (new JMetaDataBuilder).withMetadata(scalaStructField.metadata).build()) - } - - /** - * Returns the equivalent DataType in Java for the given DataType in Scala. - */ - def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match { - case udtType: UserDefinedType[_] => - UDTWrappers.wrapAsJava(udtType) - - case StringType => JDataType.StringType - case BinaryType => JDataType.BinaryType - case BooleanType => JDataType.BooleanType - case DateType => JDataType.DateType - case TimestampType => JDataType.TimestampType - case DecimalType.Fixed(precision, scale) => new JDecimalType(precision, scale) - case DecimalType.Unlimited => new JDecimalType() - case DoubleType => JDataType.DoubleType - case FloatType => JDataType.FloatType - case ByteType => JDataType.ByteType - case IntegerType => JDataType.IntegerType - case LongType => JDataType.LongType - case ShortType => JDataType.ShortType - case NullType => JDataType.NullType - - case arrayType: ArrayType => JDataType.createArrayType( - asJavaDataType(arrayType.elementType), arrayType.containsNull) - case mapType: MapType => JDataType.createMapType( - asJavaDataType(mapType.keyType), - asJavaDataType(mapType.valueType), - mapType.valueContainsNull) - case structType: StructType => JDataType.createStructType( - structType.fields.map(asJavaStructField).asJava) - } - - /** - * Returns the equivalent StructField in Scala for the given StructField in Java. - */ - def asScalaStructField(javaStructField: JStructField): StructField = { - StructField( - javaStructField.getName, - asScalaDataType(javaStructField.getDataType), - javaStructField.isNullable, - javaStructField.getMetadata) - } - - /** - * Returns the equivalent DataType in Scala for the given DataType in Java. - */ - def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { - case udtType: org.apache.spark.sql.api.java.UserDefinedType[_] => - UDTWrappers.wrapAsScala(udtType) - - case stringType: org.apache.spark.sql.api.java.StringType => - StringType - case binaryType: org.apache.spark.sql.api.java.BinaryType => - BinaryType - case booleanType: org.apache.spark.sql.api.java.BooleanType => - BooleanType - case dateType: org.apache.spark.sql.api.java.DateType => - DateType - case timestampType: org.apache.spark.sql.api.java.TimestampType => - TimestampType - case decimalType: org.apache.spark.sql.api.java.DecimalType => - if (decimalType.isFixed) { - DecimalType(decimalType.getPrecision, decimalType.getScale) - } else { - DecimalType.Unlimited - } - case doubleType: org.apache.spark.sql.api.java.DoubleType => - DoubleType - case floatType: org.apache.spark.sql.api.java.FloatType => - FloatType - case byteType: org.apache.spark.sql.api.java.ByteType => - ByteType - case integerType: org.apache.spark.sql.api.java.IntegerType => - IntegerType - case longType: org.apache.spark.sql.api.java.LongType => - LongType - case shortType: org.apache.spark.sql.api.java.ShortType => - ShortType - - case arrayType: org.apache.spark.sql.api.java.ArrayType => - ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull) - case mapType: org.apache.spark.sql.api.java.MapType => - MapType( - asScalaDataType(mapType.getKeyType), - asScalaDataType(mapType.getValueType), - mapType.isValueContainsNull) - case structType: org.apache.spark.sql.api.java.StructType => - StructType(structType.getFields.map(asScalaStructField)) - } - - def stringToTime(s: String): java.util.Date = { - if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - java.sql.Timestamp.valueOf(s) - } else { - java.sql.Date.valueOf(s) - } - } else if (s.endsWith("Z")) { - // this is zero timezone of ISO8601 - stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") - } else if (s.indexOf("GMT") == -1) { - // timezone with ISO8601 - val inset = "+00.00".length - val s0 = s.substring(0, s.length - inset) - val s1 = s.substring(s.length - inset, s.length) - if (s0.substring(s0.lastIndexOf(':')).contains('.')) { - stringToTime(s0 + "GMT" + s1) - } else { - stringToTime(s0 + ".0GMT" + s1) - } - } else { - // ISO8601 with GMT insert - val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) - ISO8601GMT.parse(s) - } - } - - /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type - case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d)) - case (other, _) => other - } - - /** Converts Java objects to catalyst rows / types */ - def convertCatalystToJava(a: Any): Any = a match { - case d: scala.math.BigDecimal => d.underlying() - case other => other - } -} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index a9a11285de..88017eb47d 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -19,15 +19,12 @@ package org.apache.spark.sql.api.java; import java.io.Serializable; -import org.apache.spark.sql.api.java.UDF1; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runners.Suite; -import org.junit.runner.RunWith; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.types.DataTypes; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -60,7 +57,7 @@ public class JavaAPISuite implements Serializable { public Integer call(String str) throws Exception { return str.length(); } - }, DataType.IntegerType); + }, DataTypes.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); @@ -81,7 +78,7 @@ public class JavaAPISuite implements Serializable { public Integer call(String str1, String str2) throws Exception { return str1.length() + str2.length(); } - }, DataType.IntegerType); + }, DataTypes.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index a04b8060cd..de586ba635 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -31,6 +31,7 @@ import org.junit.Test; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; +import org.apache.spark.sql.types.*; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -93,9 +94,9 @@ public class JavaApplySchemaSuite implements Serializable { }); List<StructField> fields = new ArrayList<StructField>(2); - fields.add(DataType.createStructField("name", DataType.StringType, false)); - fields.add(DataType.createStructField("age", DataType.IntegerType, false)); - StructType schema = DataType.createStructType(fields); + fields.add(DataTypes.createStructField("name", DataTypes.StringType, false)); + fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); + StructType schema = DataTypes.createStructType(fields); JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema); schemaRDD.registerTempTable("people"); @@ -118,14 +119,14 @@ public class JavaApplySchemaSuite implements Serializable { "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + "\"boolean\":false, \"null\":null}")); List<StructField> fields = new ArrayList<StructField>(7); - fields.add(DataType.createStructField("bigInteger", new DecimalType(), true)); - fields.add(DataType.createStructField("boolean", DataType.BooleanType, true)); - fields.add(DataType.createStructField("double", DataType.DoubleType, true)); - fields.add(DataType.createStructField("integer", DataType.IntegerType, true)); - fields.add(DataType.createStructField("long", DataType.LongType, true)); - fields.add(DataType.createStructField("null", DataType.StringType, true)); - fields.add(DataType.createStructField("string", DataType.StringType, true)); - StructType expectedSchema = DataType.createStructType(fields); + fields.add(DataTypes.createStructField("bigInteger", DataTypes.createDecimalType(), true)); + fields.add(DataTypes.createStructField("boolean", DataTypes.BooleanType, true)); + fields.add(DataTypes.createStructField("double", DataTypes.DoubleType, true)); + fields.add(DataTypes.createStructField("integer", DataTypes.IntegerType, true)); + fields.add(DataTypes.createStructField("long", DataTypes.LongType, true)); + fields.add(DataTypes.createStructField("null", DataTypes.StringType, true)); + fields.add(DataTypes.createStructField("string", DataTypes.StringType, true)); + StructType expectedSchema = DataTypes.createStructType(fields); List<Row> expectedResult = new ArrayList<Row>(2); expectedResult.add( Row.create( diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java deleted file mode 100644 index 8396a29c61..0000000000 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java; - -import java.util.List; -import java.util.ArrayList; - -import org.junit.Assert; -import org.junit.Test; - -import org.apache.spark.sql.types.util.DataTypeConversions; - -public class JavaSideDataTypeConversionSuite { - public void checkDataType(DataType javaDataType) { - org.apache.spark.sql.catalyst.types.DataType scalaDataType = - DataTypeConversions.asScalaDataType(javaDataType); - DataType actual = DataTypeConversions.asJavaDataType(scalaDataType); - Assert.assertEquals(javaDataType, actual); - } - - @Test - public void createDataTypes() { - // Simple DataTypes. - checkDataType(DataType.StringType); - checkDataType(DataType.BinaryType); - checkDataType(DataType.BooleanType); - checkDataType(DataType.DateType); - checkDataType(DataType.TimestampType); - checkDataType(new DecimalType()); - checkDataType(new DecimalType(10, 4)); - checkDataType(DataType.DoubleType); - checkDataType(DataType.FloatType); - checkDataType(DataType.ByteType); - checkDataType(DataType.IntegerType); - checkDataType(DataType.LongType); - checkDataType(DataType.ShortType); - - // Simple ArrayType. - DataType simpleJavaArrayType = DataType.createArrayType(DataType.StringType, true); - checkDataType(simpleJavaArrayType); - - // Simple MapType. - DataType simpleJavaMapType = DataType.createMapType(DataType.StringType, DataType.LongType); - checkDataType(simpleJavaMapType); - - // Simple StructType. - List<StructField> simpleFields = new ArrayList<StructField>(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false)); - DataType simpleJavaStructType = DataType.createStructType(simpleFields); - checkDataType(simpleJavaStructType); - - // Complex StructType. - List<StructField> complexFields = new ArrayList<StructField>(); - complexFields.add(DataType.createStructField("simpleArray", simpleJavaArrayType, true)); - complexFields.add(DataType.createStructField("simpleMap", simpleJavaMapType, true)); - complexFields.add(DataType.createStructField("simpleStruct", simpleJavaStructType, true)); - complexFields.add(DataType.createStructField("boolean", DataType.BooleanType, false)); - DataType complexJavaStructType = DataType.createStructType(complexFields); - checkDataType(complexJavaStructType); - - // Complex ArrayType. - DataType complexJavaArrayType = DataType.createArrayType(complexJavaStructType, true); - checkDataType(complexJavaArrayType); - - // Complex MapType. - DataType complexJavaMapType = - DataType.createMapType(complexJavaStructType, complexJavaArrayType, false); - checkDataType(complexJavaMapType); - } - - @Test - public void illegalArgument() { - // ArrayType - try { - DataType.createArrayType(null, true); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - - // MapType - try { - DataType.createMapType(null, DataType.StringType); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createMapType(DataType.StringType, null); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createMapType(null, null); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - - // StructField - try { - DataType.createStructField(null, DataType.StringType, true); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createStructField("name", null, true); - } catch (IllegalArgumentException expectedException) { - } - try { - DataType.createStructField(null, null, true); - } catch (IllegalArgumentException expectedException) { - } - - // StructType - try { - List<StructField> simpleFields = new ArrayList<StructField>(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - simpleFields.add(null); - DataType.createStructType(simpleFields); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - try { - List<StructField> simpleFields = new ArrayList<StructField>(); - simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); - simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true)); - simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); - DataType.createStructType(simpleFields); - Assert.fail(); - } catch (IllegalArgumentException expectedException) { - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala deleted file mode 100644 index e9740d913c..0000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* -* Licensed to the Apache Software Foundation (ASF) under one or more -* contributor license agreements. See the NOTICE file distributed with -* this work for additional information regarding copyright ownership. -* The ASF licenses this file to You under the Apache License, Version 2.0 -* (the "License"); you may not use this file except in compliance with -* the License. You may obtain a copy of the License at -* -* http://www.apache.org/licenses/LICENSE-2.0 -* -* Unless required by applicable law or agreed to in writing, software -* distributed under the License is distributed on an "AS IS" BASIS, -* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -* See the License for the specific language governing permissions and -* limitations under the License. -*/ - -package org.apache.spark.sql - -import org.scalatest.FunSuite - -class DataTypeSuite extends FunSuite { - - test("construct an ArrayType") { - val array = ArrayType(StringType) - - assert(ArrayType(StringType, true) === array) - } - - test("construct an MapType") { - val map = MapType(StringType, IntegerType) - - assert(MapType(StringType, IntegerType, true) === map) - } - - test("extract fields from a StructType") { - val struct = StructType( - StructField("a", IntegerType, true) :: - StructField("b", LongType, false) :: - StructField("c", StringType, true) :: - StructField("d", FloatType, true) :: Nil) - - assert(StructField("b", LongType, false) === struct("b")) - - intercept[IllegalArgumentException] { - struct("e") - } - - val expectedStruct = StructType( - StructField("b", LongType, false) :: - StructField("d", FloatType, true) :: Nil) - - assert(expectedStruct === struct(Set("b", "d"))) - intercept[IllegalArgumentException] { - struct(Set("b", "d", "e", "f")) - } - } - - def checkDataTypeJsonRepr(dataType: DataType): Unit = { - test(s"JSON - $dataType") { - assert(DataType.fromJson(dataType.json) === dataType) - } - } - - checkDataTypeJsonRepr(BooleanType) - checkDataTypeJsonRepr(ByteType) - checkDataTypeJsonRepr(ShortType) - checkDataTypeJsonRepr(IntegerType) - checkDataTypeJsonRepr(LongType) - checkDataTypeJsonRepr(FloatType) - checkDataTypeJsonRepr(DoubleType) - checkDataTypeJsonRepr(DecimalType.Unlimited) - checkDataTypeJsonRepr(TimestampType) - checkDataTypeJsonRepr(StringType) - checkDataTypeJsonRepr(BinaryType) - checkDataTypeJsonRepr(ArrayType(DoubleType, true)) - checkDataTypeJsonRepr(ArrayType(StringType, false)) - checkDataTypeJsonRepr(MapType(IntegerType, StringType, true)) - checkDataTypeJsonRepr(MapType(IntegerType, ArrayType(DoubleType), false)) - val metadata = new MetadataBuilder() - .putString("name", "age") - .build() - checkDataTypeJsonRepr( - StructType(Seq( - StructField("a", IntegerType, nullable = true), - StructField("b", ArrayType(DoubleType), nullable = false), - StructField("c", DoubleType, nullable = false, metadata)))) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index ab88f3ad10..efe622f8bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ /* Implicits */ import org.apache.spark.sql.catalyst.dsl._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 811319e0a6..f5b945f468 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} +import org.apache.spark.sql.types._ class RowSuite extends FunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bc72daf088..cbdb3e64bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ /* Implicits */ import org.apache.spark.sql.TestData._ @@ -748,7 +749,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { val metadata = new MetadataBuilder() .putString(docKey, docValue) .build() - val schemaWithMeta = new StructType(Seq( + val schemaWithMeta = new StructType(Array( schema("id"), schema("name").copy(metadata = metadata), schema("age"))) val personWithMeta = applySchema(person, schemaWithMeta) def validateMetadata(rdd: SchemaRDD): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index cf3a59e545..40fb8d5779 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 1806a1dd82..a0d54d17f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -20,9 +20,8 @@ package org.apache.spark.sql import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType -import org.apache.spark.sql.catalyst.types.UserDefinedType import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types._ @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 8afc3a9fb2..fdbb4282ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.api.java -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.beans.BeanProperty import org.scalatest.FunSuite @@ -26,6 +24,7 @@ import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.types.NullType // Implicits import scala.collection.JavaConversions._ @@ -78,10 +77,10 @@ class JavaSQLSuite extends FunSuite { schemaRDD.registerTempTable("people") val nullRDD = javaSqlCtx.sql("SELECT null FROM people") - val structFields = nullRDD.schema.getFields() + val structFields = nullRDD.schema.fields assert(structFields.size == 1) - assert(structFields(0).getDataType().isInstanceOf[NullType]) - assert(nullRDD.collect.head.row === Seq(null)) + assert(structFields(0).dataType === NullType) + assert(nullRDD.collect().head.row === Seq(null)) } test("all types in JavaBeans") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala deleted file mode 100644 index 62fe59dd34..0000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.api.java - -import org.scalatest.FunSuite - -import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField, StructType => SStructType} -import org.apache.spark.sql.types.util.DataTypeConversions._ - -class ScalaSideDataTypeConversionSuite extends FunSuite { - - def checkDataType(scalaDataType: SDataType) { - val javaDataType = asJavaDataType(scalaDataType) - val actual = asScalaDataType(javaDataType) - assert(scalaDataType === actual, s"Converted data type ${actual} " + - s"does not equal the expected data type ${scalaDataType}") - } - - test("convert data types") { - // Simple DataTypes. - checkDataType(org.apache.spark.sql.StringType) - checkDataType(org.apache.spark.sql.BinaryType) - checkDataType(org.apache.spark.sql.BooleanType) - checkDataType(org.apache.spark.sql.DateType) - checkDataType(org.apache.spark.sql.TimestampType) - checkDataType(org.apache.spark.sql.DecimalType.Unlimited) - checkDataType(org.apache.spark.sql.DoubleType) - checkDataType(org.apache.spark.sql.FloatType) - checkDataType(org.apache.spark.sql.ByteType) - checkDataType(org.apache.spark.sql.IntegerType) - checkDataType(org.apache.spark.sql.LongType) - checkDataType(org.apache.spark.sql.ShortType) - - // Simple ArrayType. - val simpleScalaArrayType = - org.apache.spark.sql.ArrayType(org.apache.spark.sql.StringType, true) - checkDataType(simpleScalaArrayType) - - // Simple MapType. - val simpleScalaMapType = - org.apache.spark.sql.MapType(org.apache.spark.sql.StringType, org.apache.spark.sql.LongType) - checkDataType(simpleScalaMapType) - - // Simple StructType. - val simpleScalaStructType = SStructType( - SStructField("a", org.apache.spark.sql.DecimalType.Unlimited, false) :: - SStructField("b", org.apache.spark.sql.BooleanType, true) :: - SStructField("c", org.apache.spark.sql.LongType, true) :: - SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil) - checkDataType(simpleScalaStructType) - - // Complex StructType. - val metadata = new MetadataBuilder() - .putString("name", "age") - .build() - val complexScalaStructType = SStructType( - SStructField("simpleArray", simpleScalaArrayType, true) :: - SStructField("simpleMap", simpleScalaMapType, true) :: - SStructField("simpleStruct", simpleScalaStructType, true) :: - SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: - SStructField("withMeta", org.apache.spark.sql.DoubleType, false, metadata) :: Nil) - checkDataType(complexScalaStructType) - - // Complex ArrayType. - val complexScalaArrayType = - org.apache.spark.sql.ArrayType(complexScalaStructType, true) - checkDataType(complexScalaArrayType) - - // Complex MapType. - val complexScalaMapType = - org.apache.spark.sql.MapType(complexScalaStructType, complexScalaArrayType, false) - checkDataType(complexScalaMapType) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index a9f0851f88..9be0b38e68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.types._ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 3f3f35d501..87e608a885 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -24,9 +24,9 @@ import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ class ColumnTypeSuite extends FunSuite with Logging { val DEFAULT_BUFFER_SIZE = 512 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index a1f21219ea..f941465fa3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.{DataType, NativeType} +import org.apache.spark.sql.types.{DataType, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int) = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 21906e3fdc..f95c895587 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.types.DataType class TestNullableColumnAccessor[T <: DataType, JvmType]( buffer: ByteBuffer, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index cb73f3da81..80bd5c9457 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.types._ class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 1cdb909146..c82d979935 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -22,9 +22,9 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.NativeType class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 73f31c0233..88011631ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.IntegralType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.IntegralType class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new IntColumnStats, INT, IntDelta) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 4ce2552112..08df1db375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.types.NativeType class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new NoopColumnStats, BOOLEAN) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 7db723d648..0b18b41192 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.columnar.compression -import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.types.NativeType class TestCompressibleColumnBuilder[T <: NativeType]( override val columnStats: ColumnStats, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index b09f1ac495..01c1ce2a61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,19 +17,19 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.types.decimal.Decimal -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} -import org.apache.spark.sql.{Row, SQLConf, QueryTest} +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.json.JsonRDD.{compatibleType, enforceCorrectType} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ - -import java.sql.{Date, Timestamp} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.decimal.Decimal +import org.apache.spark.sql.{QueryTest, Row, SQLConf} class JsonSuite extends QueryTest { - import TestJsonData._ + import org.apache.spark.sql.json.TestJsonData._ TestJsonData test("Type promotion") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 6ac67fcafe..973819aaa4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -21,8 +21,6 @@ import scala.collection.JavaConversions._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} import parquet.example.data.simple.SimpleGroup import parquet.example.data.{Group, GroupWriter} import parquet.hadoop.api.WriteSupport @@ -32,11 +30,13 @@ import parquet.hadoop.{ParquetFileWriter, ParquetWriter} import parquet.io.api.RecordConsumer import parquet.schema.{MessageType, MessageTypeParser} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.types.DecimalType import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types.DecimalType import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 0a92336a3c..fe781ec05f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -29,10 +29,10 @@ import parquet.io.api.Binary import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils case class TestRDDEntry(key: Int, value: String) @@ -911,20 +911,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) } - test("DataType string parser compatibility") { - val schema = StructType(List( - StructField("c1", IntegerType, false), - StructField("c2", BinaryType, false))) - - val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) - val fromJson = ParquetTypesConverter.convertFromString(schema.json) - - (fromCaseClassString, fromJson).zipped.foreach { (a, b) => - assert(a.name == b.name) - assert(a.dataType === b.dataType) - } - } - test("read/write fixed-length decimals") { for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { val tempDir = getTempFilePath("parquetTest").getCanonicalPath diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 34d61bf908..64274950b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.FunSuite import parquet.schema.MessageTypeParser import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.types.{BinaryType, IntegerType, StructField, StructType} import org.apache.spark.sql.test.TestSQLContext class ParquetSchemaSuite extends FunSuite with ParquetTest { @@ -148,12 +147,20 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { """.stripMargin) test("DataType string parser compatibility") { - val schema = StructType(List( - StructField("c1", IntegerType, false), - StructField("c2", BinaryType, true))) - - val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) - val fromJson = ParquetTypesConverter.convertFromString(schema.json) + // This is the generated string from previous versions of the Spark SQL, using the following: + // val schema = StructType(List( + // StructField("c1", IntegerType, false), + // StructField("c2", BinaryType, true))) + val caseClassString = + "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))" + + val jsonString = + """ + |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]} + """.stripMargin + + val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString) + val fromJson = ParquetTypesConverter.convertFromString(jsonString) (fromCaseClassString, fromJson).zipped.foreach { (a, b) => assert(a.name == b.name) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 939b3c0c66..390538d35a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.sources import scala.language.existentials import org.apache.spark.sql._ +import org.apache.spark.sql.types._ + class FilteredScanSource extends RelationProvider { override def createRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index fee2e22611..7900b3e894 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql._ +import org.apache.spark.sql.types._ class PrunedScanSource extends RelationProvider { override def createRelation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index a1d2468b25..382dddcdea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.sources import java.sql.{Timestamp, Date} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.types._ class DefaultSource extends SimpleScanSource |