-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[MINOR][SQL][DOC] Correct parquet nullability documentation #22759
[MINOR][SQL][DOC] Correct parquet nullability documentation #22759
Conversation
@dima-asana it looks like this was on purpose: 2f38378 CC @gatorsmile |
LGTM Could you do us a favor to add the test cases for ensuring that the generated parquet files have a correct nullability value? |
docs/sql-programming-guide.md
Outdated
@@ -706,7 +706,7 @@ data across a fixed number of buckets and can be used when a number of unique va | |||
|
|||
[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. | |||
Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema | |||
of the original data. When writing Parquet files, all columns are automatically converted to be nullable for | |||
of the original data. When reading Parquet files, all columns are automatically converted to be nullable for |
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.
This file has been re-orged . Could you merge the latest master?
@dima-asana can you rebase, and add a simple test case or else find one that does demonstrate the behavior here? |
Ping @dima-asana to rebase or close |
0c6ae56
to
acede3e
Compare
done, sorry for the delay |
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.
It seems fine, I just had a few trivial suggestions.
@@ -542,6 +551,35 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be | |||
} | |||
} | |||
|
|||
test("parquet - column nullability -- write only") { | |||
val schema = StructType( | |||
StructField("cl1", IntegerType, nullable = false) :: |
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.
Nit: could we indent these at the same level?
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.
fixed
new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "cl2") | ||
) | ||
|
||
assert (expectedParquetSchema == parquetSchema) |
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.
Nit: I think ideally you use the ===
test operator, so that failures generated a better message
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.
fixed
f.close | ||
|
||
// the write keeps nullable info from the schema | ||
val expectedParquetSchema: Seq[PrimitiveType] = Seq( |
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.
Also really doesn't matter, but you can simplify the code by omitting types like this, etc.
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.
fixed
Parquet files appear to have nullability info when being written. However, they do lose nullability info while read
acede3e
to
ac138ff
Compare
Test build #4459 has finished for PR 22759 at commit
|
## What changes were proposed in this pull request? Parquet files appear to have nullability info when being written, not being read. ## How was this patch tested? Some test code: (running spark 2.3, but the relevant code in DataSource looks identical on master) case class NullTest(bo: Boolean, opbol: Option[Boolean]) val testDf = spark.createDataFrame(Seq(NullTest(true, Some(false)))) defined class NullTest testDf: org.apache.spark.sql.DataFrame = [bo: boolean, opbol: boolean] testDf.write.parquet("s3://asana-stats/tmp_dima/parquet_check_schema") spark.read.parquet("s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet4").printSchema() root |-- bo: boolean (nullable = true) |-- opbol: boolean (nullable = true) Meanwhile, the parquet file formed does have nullable info: []batchprod-report000:/tmp/dimakamalov-batch$ aws s3 ls s3://asana-stats/tmp_dima/parquet_check_schema/ 2018-10-17 21:03:52 0 _SUCCESS 2018-10-17 21:03:50 504 part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet []batchprod-report000:/tmp/dimakamalov-batch$ aws s3 cp s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet . download: s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet to ./part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet []batchprod-report000:/tmp/dimakamalov-batch$ java -jar parquet-tools-1.8.2.jar schema part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet message spark_schema { required boolean bo; optional boolean opbol; } Closes apache#22759 from dima-asana/dima-asana-nullable-parquet-doc. Authored-by: dima-asana <42555784+dima-asana@users.noreply.github.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
What changes were proposed in this pull request?
Parquet files appear to have nullability info when being written, not being read.
How was this patch tested?
Some test code: (running spark 2.3, but the relevant code in DataSource looks identical on master)
case class NullTest(bo: Boolean, opbol: Option[Boolean])
val testDf = spark.createDataFrame(Seq(NullTest(true, Some(false))))
defined class NullTest
testDf: org.apache.spark.sql.DataFrame = [bo: boolean, opbol: boolean]
testDf.write.parquet("s3://asana-stats/tmp_dima/parquet_check_schema")
spark.read.parquet("s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet4").printSchema()
root
|-- bo: boolean (nullable = true)
|-- opbol: boolean (nullable = true)
Meanwhile, the parquet file formed does have nullable info:
[]batch@prod-report000:/tmp/dimakamalov-batch$ aws s3 ls s3://asana-stats/tmp_dima/parquet_check_schema/
2018-10-17 21:03:52 0 _SUCCESS
2018-10-17 21:03:50 504 part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batch@prod-report000:/tmp/dimakamalov-batch$ aws s3 cp s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet .
download: s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet to ./part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batch@prod-report000:/tmp/dimakamalov-batch$ java -jar parquet-tools-1.8.2.jar schema part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
message spark_schema {
required boolean bo;
optional boolean opbol;
}