Skip to content

Commit 991f860

Browse files
committed
Move "asJavaDataType" and "asScalaDataType" to DataTypeConversions.scala.
1 parent 1cb35fe commit 991f860

File tree

7 files changed

+162
-44
lines changed

7 files changed

+162
-44
lines changed

sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -19,19 +19,18 @@ package org.apache.spark.sql.api.java
1919

2020
import java.beans.Introspector
2121

22-
import scala.collection.JavaConverters._
23-
2422
import org.apache.hadoop.conf.Configuration
2523

2624
import org.apache.spark.annotation.{DeveloperApi, Experimental}
2725
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
28-
import org.apache.spark.sql.api.java.types.{DataType => JDataType, StructType => JStructType}
29-
import org.apache.spark.sql.api.java.types.{StructField => JStructField}
26+
import org.apache.spark.sql.api.java.types.{StructType => JStructType}
3027
import org.apache.spark.sql.json.JsonRDD
3128
import org.apache.spark.sql._
3229
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow}
3330
import org.apache.spark.sql.parquet.ParquetRelation
3431
import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan}
32+
import org.apache.spark.sql.types.util.DataTypeConversions
33+
import DataTypeConversions.asScalaDataType;
3534
import org.apache.spark.util.Utils
3635

3736
/**
@@ -107,7 +106,7 @@ class JavaSQLContext(val sqlContext: SQLContext) {
107106
@DeveloperApi
108107
def applySchema(rowRDD: JavaRDD[Row], schema: JStructType): JavaSchemaRDD = {
109108
val scalaRowRDD = rowRDD.rdd.map(r => r.row)
110-
val scalaSchema = sqlContext.asScalaDataType(schema).asInstanceOf[StructType]
109+
val scalaSchema = asScalaDataType(schema).asInstanceOf[StructType]
111110
val logicalPlan = SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))
112111
new JavaSchemaRDD(sqlContext, logicalPlan)
113112
}
@@ -156,7 +155,7 @@ class JavaSQLContext(val sqlContext: SQLContext) {
156155
@Experimental
157156
def jsonRDD(json: JavaRDD[String], schema: JStructType): JavaSchemaRDD = {
158157
val appliedScalaSchema =
159-
Option(sqlContext.asScalaDataType(schema)).getOrElse(
158+
Option(asScalaDataType(schema)).getOrElse(
160159
JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[StructType]
161160
val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema)
162161
val logicalPlan = SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))

sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,10 @@ import org.apache.spark.Partitioner
2323
import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
2424
import org.apache.spark.api.java.function.{Function => JFunction}
2525
import org.apache.spark.sql.api.java.types.StructType
26+
import org.apache.spark.sql.types.util.DataTypeConversions
2627
import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike}
2728
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
29+
import DataTypeConversions._
2830
import org.apache.spark.rdd.RDD
2931
import org.apache.spark.storage.StorageLevel
3032

@@ -56,7 +58,7 @@ class JavaSchemaRDD(
5658

5759
/** Returns the schema of this JavaSchemaRDD (represented by a StructType). */
5860
def schema: StructType =
59-
sqlContext.asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType]
61+
asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType]
6062

6163
// =======================================================================
6264
// Base RDD functions that do NOT change schema

sql/core/src/main/scala/org/apache/spark/sql/package.scala

Lines changed: 13 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,10 @@
1717

1818
package org.apache.spark
1919

20+
import scala.collection.JavaConverters._
21+
2022
import org.apache.spark.annotation.DeveloperApi
23+
import org.apache.spark.sql.api.java.types.{DataType => JDataType, StructField => JStructField}
2124

2225
/**
2326
* Allows the execution of relational queries, including those expressed in SQL using Spark.
@@ -243,8 +246,7 @@ package object sql {
243246
* The data type representing `Seq`s.
244247
* An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and
245248
* `containsNull: Boolean`. The field of `elementType` is used to specify the type of
246-
* array elements. The field of `containsNull` is used to specify if the array can have
247-
* any `null` value.
249+
* array elements. The field of `containsNull` is used to specify if the array has `null` valus.
248250
*
249251
* @group dataType
250252
*/
@@ -271,10 +273,11 @@ package object sql {
271273
/**
272274
* :: DeveloperApi ::
273275
*
274-
* The data type representing `Map`s. A [[MapType]] object comprises two fields,
275-
* `keyType: [[DataType]]` and `valueType: [[DataType]]`.
276+
* The data type representing `Map`s. A [[MapType]] object comprises three fields,
277+
* `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`.
276278
* The field of `keyType` is used to specify the type of keys in the map.
277279
* The field of `valueType` is used to specify the type of values in the map.
280+
* The field of `valueContainsNull` is used to specify if values of this map has `null` values.
278281
*
279282
* @group dataType
280283
*/
@@ -284,10 +287,15 @@ package object sql {
284287
/**
285288
* :: DeveloperApi ::
286289
*
287-
* A [[MapType]] can be constructed by
290+
* A [[MapType]] object can be constructed with two ways,
291+
* {{{
292+
* MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean)
293+
* }}} and
288294
* {{{
289295
* MapType(keyType: DataType, valueType: DataType)
290296
* }}}
297+
* For `MapType(keyType: DataType, valueType: DataType)`,
298+
* the field of `valueContainsNull` is set to `true`.
291299
*
292300
* @group dataType
293301
*/
Lines changed: 124 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,124 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.types.util
19+
20+
import org.apache.spark.sql._
21+
import org.apache.spark.sql.api.java.types.{DataType => JDataType, StructField => JStructField}
22+
23+
import scala.collection.JavaConverters._
24+
25+
protected[sql] object DataTypeConversions {
26+
27+
/**
28+
* Returns the equivalent StructField in Scala for the given StructField in Java.
29+
*/
30+
def asJavaStructField(scalaStructField: StructField): JStructField = {
31+
org.apache.spark.sql.api.java.types.DataType.createStructField(
32+
scalaStructField.name,
33+
asJavaDataType(scalaStructField.dataType),
34+
scalaStructField.nullable)
35+
}
36+
37+
/**
38+
* Returns the equivalent DataType in Java for the given DataType in Scala.
39+
*/
40+
def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match {
41+
case StringType =>
42+
org.apache.spark.sql.api.java.types.DataType.StringType
43+
case BinaryType =>
44+
org.apache.spark.sql.api.java.types.DataType.BinaryType
45+
case BooleanType =>
46+
org.apache.spark.sql.api.java.types.DataType.BooleanType
47+
case TimestampType =>
48+
org.apache.spark.sql.api.java.types.DataType.TimestampType
49+
case DecimalType =>
50+
org.apache.spark.sql.api.java.types.DataType.DecimalType
51+
case DoubleType =>
52+
org.apache.spark.sql.api.java.types.DataType.DoubleType
53+
case FloatType =>
54+
org.apache.spark.sql.api.java.types.DataType.FloatType
55+
case ByteType =>
56+
org.apache.spark.sql.api.java.types.DataType.ByteType
57+
case IntegerType =>
58+
org.apache.spark.sql.api.java.types.DataType.IntegerType
59+
case LongType =>
60+
org.apache.spark.sql.api.java.types.DataType.LongType
61+
case ShortType =>
62+
org.apache.spark.sql.api.java.types.DataType.ShortType
63+
64+
case arrayType: ArrayType =>
65+
org.apache.spark.sql.api.java.types.DataType.createArrayType(
66+
asJavaDataType(arrayType.elementType), arrayType.containsNull)
67+
case mapType: MapType =>
68+
org.apache.spark.sql.api.java.types.DataType.createMapType(
69+
asJavaDataType(mapType.keyType),
70+
asJavaDataType(mapType.valueType),
71+
mapType.valueContainsNull)
72+
case structType: StructType =>
73+
org.apache.spark.sql.api.java.types.DataType.createStructType(
74+
structType.fields.map(asJavaStructField).asJava)
75+
}
76+
77+
/**
78+
* Returns the equivalent StructField in Scala for the given StructField in Java.
79+
*/
80+
def asScalaStructField(javaStructField: JStructField): StructField = {
81+
StructField(
82+
javaStructField.getName,
83+
asScalaDataType(javaStructField.getDataType),
84+
javaStructField.isNullable)
85+
}
86+
87+
/**
88+
* Returns the equivalent DataType in Scala for the given DataType in Java.
89+
*/
90+
def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match {
91+
case stringType: org.apache.spark.sql.api.java.types.StringType =>
92+
StringType
93+
case binaryType: org.apache.spark.sql.api.java.types.BinaryType =>
94+
BinaryType
95+
case booleanType: org.apache.spark.sql.api.java.types.BooleanType =>
96+
BooleanType
97+
case timestampType: org.apache.spark.sql.api.java.types.TimestampType =>
98+
TimestampType
99+
case decimalType: org.apache.spark.sql.api.java.types.DecimalType =>
100+
DecimalType
101+
case doubleType: org.apache.spark.sql.api.java.types.DoubleType =>
102+
DoubleType
103+
case floatType: org.apache.spark.sql.api.java.types.FloatType =>
104+
FloatType
105+
case byteType: org.apache.spark.sql.api.java.types.ByteType =>
106+
ByteType
107+
case integerType: org.apache.spark.sql.api.java.types.IntegerType =>
108+
IntegerType
109+
case longType: org.apache.spark.sql.api.java.types.LongType =>
110+
LongType
111+
case shortType: org.apache.spark.sql.api.java.types.ShortType =>
112+
ShortType
113+
114+
case arrayType: org.apache.spark.sql.api.java.types.ArrayType =>
115+
ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull)
116+
case mapType: org.apache.spark.sql.api.java.types.MapType =>
117+
MapType(
118+
asScalaDataType(mapType.getKeyType),
119+
asScalaDataType(mapType.getValueType),
120+
mapType.isValueContainsNull)
121+
case structType: org.apache.spark.sql.api.java.types.StructType =>
122+
StructType(structType.getFields.map(asScalaStructField))
123+
}
124+
}

sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java

Lines changed: 4 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -21,40 +21,20 @@
2121
import java.util.ArrayList;
2222

2323
import org.junit.Assert;
24-
import org.junit.After;
25-
import org.junit.Before;
26-
import org.junit.Rule;
2724
import org.junit.Test;
2825

29-
import org.apache.spark.api.java.JavaSparkContext;
26+
import org.apache.spark.sql.types.util.DataTypeConversions;
3027
import org.apache.spark.sql.api.java.types.DataType;
3128
import org.apache.spark.sql.api.java.types.StructField;
32-
import org.apache.spark.sql.test.TestSQLContext;
33-
import org.junit.rules.ExpectedException;
3429

3530
public class JavaSideDataTypeConversionSuite {
36-
private transient JavaSparkContext javaCtx;
37-
private transient JavaSQLContext javaSqlCtx;
38-
3931
public void checkDataType(DataType javaDataType) {
4032
org.apache.spark.sql.catalyst.types.DataType scalaDataType =
41-
javaSqlCtx.sqlContext().asScalaDataType(javaDataType);
42-
DataType actual = javaSqlCtx.sqlContext().asJavaDataType(scalaDataType);
33+
DataTypeConversions.asScalaDataType(javaDataType);
34+
DataType actual = DataTypeConversions.asJavaDataType(scalaDataType);
4335
Assert.assertEquals(javaDataType, actual);
4436
}
4537

46-
@Before
47-
public void setUp() {
48-
javaCtx = new JavaSparkContext(TestSQLContext.sparkContext());
49-
javaSqlCtx = new JavaSQLContext(javaCtx);
50-
}
51-
52-
@After
53-
public void tearDown() {
54-
javaCtx.stop();
55-
javaCtx = null;
56-
}
57-
5838
@Test
5939
public void createDataTypes() {
6040
// Simple DataTypes.
@@ -102,7 +82,7 @@ public void createDataTypes() {
10282

10383
// Complex MapType.
10484
DataType complexJavaMapType =
105-
DataType.createMapType(complexJavaStructType, complexJavaArrayType);
85+
DataType.createMapType(complexJavaStructType, complexJavaArrayType, false);
10686
checkDataType(complexJavaMapType);
10787
}
10888

sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,12 @@ class DataTypeSuite extends FunSuite {
2727
assert(ArrayType(StringType, false) === array)
2828
}
2929

30+
test("construct an MapType") {
31+
val map = MapType(StringType, IntegerType)
32+
33+
assert(MapType(StringType, IntegerType, true) === map)
34+
}
35+
3036
test("extract fields from a StructType") {
3137
val struct = StructType(
3238
StructField("a", IntegerType, true) ::

sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala

Lines changed: 7 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -17,18 +17,17 @@
1717

1818
package org.apache.spark.sql.api.java
1919

20-
import org.apache.spark.api.java.JavaSparkContext
21-
import org.apache.spark.sql._
22-
import org.apache.spark.sql.test.TestSQLContext
20+
import org.apache.spark.sql.types.util.DataTypeConversions
2321
import org.scalatest.FunSuite
2422

23+
import org.apache.spark.sql._
24+
import DataTypeConversions._
25+
2526
class ScalaSideDataTypeConversionSuite extends FunSuite {
26-
val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext)
27-
val javaSqlCtx = new JavaSQLContext(javaCtx)
2827

2928
def checkDataType(scalaDataType: DataType) {
30-
val javaDataType = javaSqlCtx.sqlContext.asJavaDataType(scalaDataType)
31-
val actual = javaSqlCtx.sqlContext.asScalaDataType(javaDataType)
29+
val javaDataType = asJavaDataType(scalaDataType)
30+
val actual = asScalaDataType(javaDataType)
3231
assert(scalaDataType === actual, s"Converted data type ${actual} " +
3332
s"does not equal the expected data type ${scalaDataType}")
3433
}
@@ -76,7 +75,7 @@ class ScalaSideDataTypeConversionSuite extends FunSuite {
7675
checkDataType(complexScalaArrayType)
7776

7877
// Complex MapType.
79-
val complexScalaMapType = MapType(complexScalaStructType, complexScalaArrayType)
78+
val complexScalaMapType = MapType(complexScalaStructType, complexScalaArrayType, false)
8079
checkDataType(complexScalaMapType)
8180
}
8281
}

0 commit comments

Comments
 (0)