Skip to content
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

[SPARK-16472][SQL] Force user specified schema to the nullable one #14124

Closed
wants to merge 1 commit into from

Conversation

HyukjinKwon
Copy link
Member

@HyukjinKwon HyukjinKwon commented Jul 10, 2016

What changes were proposed in this pull request?

This PR proposes to force the user-specified schema to the nullable one, as Spark SQL can't validate it.

How was this patch tested?

Unit tests adde in FileStreamSourceSuite.scala and DataFrameReaderWriterSuite.scala.

@HyukjinKwon
Copy link
Member Author

Hi @gatorsmile and @marmbrus, I saw the discussion and found you are related with this one. Could you please review this?

@@ -879,6 +879,19 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes
}
}
}

test("Check if fields in the schema are nullable") {
Copy link
Member Author

@HyukjinKwon HyukjinKwon Jul 10, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one is forcing the schema as nullable but it has no tests. So the tests were added.

@SparkQA
Copy link

SparkQA commented Jul 10, 2016

Test build #62049 has finished for PR 14124 at commit a917678.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 10, 2016

Test build #62053 has finished for PR 14124 at commit adae8de.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jul 10, 2016

Test build #62054 has finished for PR 14124 at commit 3980681.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

gatorsmile commented Jul 10, 2016

val rdd = spark.sparkContext.makeRDD(Seq("{\"a\" : 1}", "{\"a\" : null}"))
val schema = StructType(StructField("a", IntegerType, nullable = false) :: Nil)
val df = spark.read.schema(schema).json(rdd)
df.show()

When user-specified schemas are not nullable and the data contains null, the null value in the result becomes 0. This looks like a bug, right?

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 10, 2016

Ah, yes it seems a bug to me. I thought it throws an exception in that case or works fine after thid PR. I will test this before/after this PR. Thanks!

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 11, 2016

Oh, I see, before this patch

+---+
|  a|
+---+
|  1|
|  0|
+---+

after this patch

+----+
|   a|
+----+
|   1|
|null|
+----+

FYI, currently (before this patch) the code with StringType below

val rdd = spark.sparkContext.makeRDD(Seq("{\"a\" : 1}", "{\"a\" : null}"))
val schema = StructType(StructField("a", StringType, nullable = false) :: Nil)
val df = spark.read.schema(schema).json(rdd)
df.show()

is being failed with the exception below:

Error while decoding: java.lang.NullPointerException
createexternalrow(input[0, string, false].toString, StructField(a,StringType,false))
+- input[0, string, false].toString
   +- input[0, string, false]

java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
createexternalrow(input[0, string, false].toString, StructField(a,StringType,false))
+- input[0, string, false].toString
   +- input[0, string, false]

    at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:292)
    at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$15.apply(Dataset.scala:2218)
    at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1$$anonfun$apply$15.apply(Dataset.scala:2218)
    at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
    at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
    at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
    at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
...

It seems an unexpected behaviour anyway (I think at least we should fix the error message). I will submit a patch if this one is decided not worth being added. Thanks @gatorsmile again!

@gatorsmile
Copy link
Member

@HyukjinKwon No matter whether this PR is merged or not, I still think we should fix the above issue. Silent conversion does not look good to me.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 11, 2016

@gatorsmile I am a bit confused if we are allowed to read JSON (via json(jsonRDD: RDD[String]) API) with schema having fields set false in nullable.
If it is meant to be not allowed, this PR will prevents the case above.

But, yea, I think I agree that it is a potential problem anyway (even if the case above is not allowed.)

@viirya
Copy link
Member

viirya commented Jul 11, 2016

@HyukjinKwon Your patch solves this inconsistency by forcing schema as nullable at all. However, looks like the parquet case is for compatibility, is this the same for json? If no, why we want to do this?

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 11, 2016

@viirya Thanks for your comment! Actually, that's what I want to have some feedback about from @marmbrus .

It seems forcing to a nullable schema all is already happening, (see here) for all data sources implementing FileFormat when you read data via read.format(...).save(...) API (but not for structured streaming and another API for json, which this PR deals with).

So, actually, the purpose of this PR is, to make all read APIs consistent. The reason to make them consistent in a way that the schema is forced as nullable, is what he said in the mailing list.

Sure, but a traditional RDBMS has the opportunity to do validation before
loading data in. Thats not really an option when you are reading random
files from S3. This is why Hive and many other systems in this space treat
all columns as nullable.

Actually, apparently, Parquet also reads and writes the schema with nullability correctly if we get rid of asNullable in here (I tested this by a roundtrip in writing and reading before) but it seems that's prevented as a safeguard due to (I assume) the reason above.

@marmbrus Do you mind if I ask to clarify here please?

I think we might have to deal with this as a datasource-specific problem.

@HyukjinKwon HyukjinKwon changed the title [SPARK-16472][SQL] Inconsistent nullability in schema after being read in SQL API [SPARK-16472][SQL] Inconsistent nullability in schema after being read by data sources implementing FileFormat Jul 12, 2016
@HyukjinKwon
Copy link
Member Author

Could you take a look please @marmbrus ?

@HyukjinKwon
Copy link
Member Author

gentle ping @marmbrus

@marmbrus
Copy link
Contributor

@cloud-fan

@cloud-fan
Copy link
Contributor

What will happen if the given schema is wrong? It seems weird that we allow users to provide schema while reading the data, but without validating it.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 27, 2016

Thanks for feedback @cloud-fan !

If the user-given schema is wrong, it is handled differently for each datasource.

Should we disallow specifying schemas for these (maybe ORC and Parquet)?

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 27, 2016

BTW, actually, this PR is not only about user-given schema.

Currently, it always reads data into dataframe by datasources based on FileFormat ignoring nullability in schema (for both user-given schema and inferred/read schema).

However, this does not happen when reading for streaming by the datasources (FileFormat) (and another JSON api).

So, this PR tries to make them all ignore the nullability in schema to be consistent.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Jul 28, 2016

@cloud-fan If nullability should be not ignored, then I can fix this PR to make them consistent to not ignoring it (and of course I will try to identify the related problems). In this case, I will work on what @gatorsmile pointed out in #14124 (comment) about JSON (and will check the other data sources as well).

I will follow your decision.

To cut all the comments above short, (for other reviewers),

  • The purpose of this PR is whether it should force all schema to nullable schema or not.
  • Forcing this as nullable one is already happening with normal reading and writing for data sources based on FileFormat but not for structured streaming and json(rdd: RDD[String]) API.
  • This is for both inferred/read schema and user-given schema.

@SparkQA
Copy link

SparkQA commented Aug 19, 2016

Test build #64066 has finished for PR 14124 at commit 079aae2.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Aug 30, 2016

Test build #64631 has finished for PR 14124 at commit ffacb55.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 14, 2016

Test build #65360 has finished for PR 14124 at commit f6be52b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 22, 2016

Test build #65747 has finished for PR 14124 at commit 0bc06c6.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 16, 2016

Test build #67029 has finished for PR 14124 at commit 3f153a3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

cloud-fan commented Nov 10, 2016

Sorry for the delay. After thinking it again, I think it doesn't make sense to allow users to specify the nullability when reading a data source, as Spark SQL can't validate it. How about we turn schema to nullable in DataFrameReader.schema?

@HyukjinKwon
Copy link
Member Author

Thanks @cloud-fan, sure, that sounds great.

@HyukjinKwon
Copy link
Member Author

Oh wait, @cloud-fan, it seems, at least, Parquet files could possibly be written with not nullable fields. So, reading it without user-specified schema might also cause the inconsistency between the schema read from structured streaming and the one read from filed sources.

If you are not sure of this, I am fine with turning the schema into nullable in DataFrameReader.schema for now. Let me rebase this one first.

@HyukjinKwon
Copy link
Member Author

Actually, nvm. I think handling this in DataFrameReader.schema will deal with most of general cases.

@HyukjinKwon HyukjinKwon changed the title [SPARK-16472][SQL] Inconsistent nullability in schema after being read by data sources implementing FileFormat [SPARK-16472][SQL] Force user specified schema to the nullable one Nov 10, 2016
@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68474 has finished for PR 14124 at commit 7306937.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 10, 2016

Test build #68477 has finished for PR 14124 at commit d240c0d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 5, 2017

Test build #70922 has finished for PR 14124 at commit 1abaf1b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

cc @liancheng , what do you think about the nullability change?

@kiszk
Copy link
Member

kiszk commented Mar 20, 2017

#17293 added data validation using schema information for Parquet Reader, as @gatorsmile suggested in https://www.mail-archive.com/[email protected]/msg39233.html.

@HyukjinKwon
Copy link
Member Author

Let me rather close this for a while. I will reopen if it looks worth. I think the same subject could be discussed in @kiszk 's PR.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants