-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17952][SQL] Nested Java beans support in createDataFrame #22527
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
ccea758
d8083cf
3fe63c8
3b2a431
e9b5a98
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
|
|
||
| package org.apache.spark.sql | ||
|
|
||
| import java.lang.reflect.Method | ||
| import java.util.Properties | ||
|
|
||
| import scala.collection.immutable | ||
|
|
@@ -1100,13 +1101,24 @@ object SQLContext { | |
| attrs: Seq[AttributeReference]): Iterator[InternalRow] = { | ||
| val extractors = | ||
| JavaTypeInference.getJavaBeanReadableProperties(beanClass).map(_.getReadMethod) | ||
| val methodsToConverts = extractors.zip(attrs).map { case (e, attr) => | ||
| (e, CatalystTypeConverters.createToCatalystConverter(attr.dataType)) | ||
| val methodsToTypes = extractors.zip(attrs).map { case (e, attr) => | ||
| (e, attr.dataType) | ||
| } | ||
| def invoke(element: Any)(tuple: (Method, DataType)): Any = tuple match { | ||
| case (e, structType: StructType) => | ||
| val value = e.invoke(element) | ||
| val nestedExtractors = JavaTypeInference.getJavaBeanReadableProperties(value.getClass) | ||
| .map(desc => desc.getName -> desc.getReadMethod) | ||
| .toMap | ||
| new GenericInternalRow(structType.map(nestedProperty => | ||
| invoke(value)(nestedExtractors(nestedProperty.name) -> nestedProperty.dataType) | ||
| ).toArray) | ||
|
||
| case (e, dataType) => | ||
| CatalystTypeConverters.createToCatalystConverter(dataType)(e.invoke(element)) | ||
| } | ||
| data.map { element => | ||
| new GenericInternalRow( | ||
| methodsToConverts.map { case (e, convert) => convert(e.invoke(element)) } | ||
| ): InternalRow | ||
| methodsToTypes.map(invoke(element))): InternalRow | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -134,6 +134,7 @@ public static class Bean implements Serializable { | |
| private Map<String, int[]> c = ImmutableMap.of("hello", new int[] { 1, 2 }); | ||
| private List<String> d = Arrays.asList("floppy", "disk"); | ||
| private BigInteger e = new BigInteger("1234567"); | ||
| private NestedBean f = new NestedBean(); | ||
|
|
||
| public double getA() { | ||
| return a; | ||
|
|
@@ -152,6 +153,18 @@ public List<String> getD() { | |
| } | ||
|
|
||
| public BigInteger getE() { return e; } | ||
|
|
||
| public NestedBean getF() { | ||
| return f; | ||
| } | ||
|
|
||
| public static class NestedBean implements Serializable { | ||
| private int a = 1; | ||
|
|
||
| public int getA() { | ||
| return a; | ||
| } | ||
| } | ||
| } | ||
|
|
||
| void validateDataFrameWithBeans(Bean bean, Dataset<Row> df) { | ||
|
|
@@ -171,7 +184,12 @@ void validateDataFrameWithBeans(Bean bean, Dataset<Row> df) { | |
| schema.apply("d")); | ||
| Assert.assertEquals(new StructField("e", DataTypes.createDecimalType(38,0), true, | ||
| Metadata.empty()), schema.apply("e")); | ||
| Row first = df.select("a", "b", "c", "d", "e").first(); | ||
| Assert.assertEquals(new StructField("f", | ||
| DataTypes.createStructType(Collections.singletonList(new StructField( | ||
| "a", IntegerType$.MODULE$, false, Metadata.empty()))), | ||
| true, Metadata.empty()), | ||
| schema.apply("f")); | ||
|
||
| Row first = df.select("a", "b", "c", "d", "e", "f").first(); | ||
| Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); | ||
| // Now Java lists and maps are converted to Scala Seq's and Map's. Once we get a Seq below, | ||
| // verify that it has the expected length, and contains expected elements. | ||
|
|
@@ -192,6 +210,8 @@ void validateDataFrameWithBeans(Bean bean, Dataset<Row> df) { | |
| } | ||
| // Java.math.BigInteger is equivalent to Spark Decimal(38,0) | ||
| Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); | ||
| Row nested = first.getStruct(5); | ||
| Assert.assertEquals(bean.getF().getA(), nested.getInt(0)); | ||
| } | ||
|
|
||
| @Test | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we create converters before
data.map { ... }instead of calculating converters for each row?I mean something like:
and then
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point. Thank you. Changed it in the latest commit