You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by dima-asana <gi...@git.apache.org> on 2018/10/17 21:14:35 UTC
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
GitHub user dima-asana opened a pull request:
https://github.com/apache/spark/pull/22759
[MINOR][SQL][DOC] Correct parquet nullability documentation
## 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;
}
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/dima-asana/spark dima-asana-nullable-parquet-doc
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/22759.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #22759
----
commit 0c6ae5621e0fee76f836bdc6ec504685b7983a75
Author: dima-asana <42...@...>
Date: 2018-10-17T21:08:50Z
Correct parquet nullability documentation
Parquet files appear to have nullability info when being written, not being read. Some test code:
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;
}
----
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:
https://github.com/apache/spark/pull/22759
Can one of the admins verify this patch?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:
https://github.com/apache/spark/pull/22759
@dima-asana can you rebase, and add a simple test case or else find one that does demonstrate the behavior here?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r232441937
--- Diff: docs/sql-programming-guide.md ---
@@ -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
--- End diff --
This file has been re-org . Could you merge the latest master?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239475203
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
+ StructField("cl2", IntegerType, nullable = true) :: Nil)
+ val row = Row(3, 4)
+ val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema)
+
+ withTempPath { dir =>
+ val path = dir.getAbsolutePath
+ df.write.mode("overwrite").parquet(path)
+ val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0)
+
+ val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration())
+ val f = ParquetFileReader.open(hadoopInputFile)
+ val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala
+ .map(_.getPrimitiveType)
+ f.close
+
+ // the write keeps nullable info from the schema
+ val expectedParquetSchema: Seq[PrimitiveType] = Seq(
+ new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "cl1"),
+ new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "cl2")
+ )
+
+ assert (expectedParquetSchema == parquetSchema)
--- End diff --
Nit: I think ideally you use the `===` test operator, so that failures generated a better message
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239475332
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
+ StructField("cl2", IntegerType, nullable = true) :: Nil)
+ val row = Row(3, 4)
+ val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema)
+
+ withTempPath { dir =>
+ val path = dir.getAbsolutePath
+ df.write.mode("overwrite").parquet(path)
+ val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0)
+
+ val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration())
+ val f = ParquetFileReader.open(hadoopInputFile)
+ val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala
+ .map(_.getPrimitiveType)
+ f.close
+
+ // the write keeps nullable info from the schema
+ val expectedParquetSchema: Seq[PrimitiveType] = Seq(
--- End diff --
Also really doesn't matter, but you can simplify the code by omitting types like this, etc.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:
https://github.com/apache/spark/pull/22759
Can one of the admins verify this patch?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:
https://github.com/apache/spark/pull/22759
Ping @dima-asana to rebase or close
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:
https://github.com/apache/spark/pull/22759
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:
https://github.com/apache/spark/pull/22759
@dima-asana it looks like this was on purpose: https://github.com/apache/spark/commit/2f38378856fb56bdd9be7ccedf56427e81701f4e CC @gatorsmile
I agree though, doesn't appear that it's actually forced to be nullable on write.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:
https://github.com/apache/spark/pull/22759
LGTM
Could you do us a favor to add the test cases for ensuring that the generated parquet files have a correct nullability value?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:
https://github.com/apache/spark/pull/22759
**[Test build #4459 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4459/testReport)** for PR 22759 at commit [`ac138ff`](https://github.com/apache/spark/commit/ac138ff47bbc33a630d4bb598b9435d1dc653a9e).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by dima-asana <gi...@git.apache.org>.
Github user dima-asana commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239577173
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
+ StructField("cl2", IntegerType, nullable = true) :: Nil)
+ val row = Row(3, 4)
+ val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema)
+
+ withTempPath { dir =>
+ val path = dir.getAbsolutePath
+ df.write.mode("overwrite").parquet(path)
+ val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0)
+
+ val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration())
+ val f = ParquetFileReader.open(hadoopInputFile)
+ val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala
+ .map(_.getPrimitiveType)
+ f.close
+
+ // the write keeps nullable info from the schema
+ val expectedParquetSchema: Seq[PrimitiveType] = Seq(
--- End diff --
fixed
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by dima-asana <gi...@git.apache.org>.
Github user dima-asana commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239577117
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
--- End diff --
fixed
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by dima-asana <gi...@git.apache.org>.
Github user dima-asana commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239577230
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
+ StructField("cl2", IntegerType, nullable = true) :: Nil)
+ val row = Row(3, 4)
+ val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema)
+
+ withTempPath { dir =>
+ val path = dir.getAbsolutePath
+ df.write.mode("overwrite").parquet(path)
+ val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0)
+
+ val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration())
+ val f = ParquetFileReader.open(hadoopInputFile)
+ val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala
+ .map(_.getPrimitiveType)
+ f.close
+
+ // the write keeps nullable info from the schema
+ val expectedParquetSchema: Seq[PrimitiveType] = Seq(
+ new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "cl1"),
+ new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "cl2")
+ )
+
+ assert (expectedParquetSchema == parquetSchema)
--- End diff --
fixed
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark pull request #22759: [MINOR][SQL][DOC] Correct parquet nullability doc...
Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:
https://github.com/apache/spark/pull/22759#discussion_r239474962
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ---
@@ -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) ::
--- End diff --
Nit: could we indent these at the same level?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:
https://github.com/apache/spark/pull/22759
**[Test build #4459 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4459/testReport)** for PR 22759 at commit [`ac138ff`](https://github.com/apache/spark/commit/ac138ff47bbc33a630d4bb598b9435d1dc653a9e).
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by dima-asana <gi...@git.apache.org>.
Github user dima-asana commented on the issue:
https://github.com/apache/spark/pull/22759
> @dima-asana can you rebase, and add a simple test case or else find one that does demonstrate the behavior here?
done, sorry for the delay
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org
[GitHub] spark issue #22759: [MINOR][SQL][DOC] Correct parquet nullability documentat...
Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:
https://github.com/apache/spark/pull/22759
Can one of the admins verify this patch?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org