Skip to content

Commit a7683af

Browse files
committed
[SPARK-26346][BUILD][SQL] Upgrade Parquet to 1.11.1
### What changes were proposed in this pull request? This PR upgrade Parquet to 1.11.1. Parquet 1.11.1 new features: - [PARQUET-1201](https://issues.apache.org/jira/browse/PARQUET-1201) - Column indexes - [PARQUET-1253](https://issues.apache.org/jira/browse/PARQUET-1253) - Support for new logical type representation - [PARQUET-1388](https://issues.apache.org/jira/browse/PARQUET-1388) - Nanosecond precision time and timestamp - parquet-mr More details: https://github.com/apache/parquet-mr/blob/apache-parquet-1.11.1/CHANGES.md ### Why are the changes needed? Support column indexes to improve query performance. ### Does this PR introduce any user-facing change? No. ### How was this patch tested? Existing test. Closes #26804 from wangyum/SPARK-26346. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: Yuming Wang <yumwang@ebay.com>
1 parent bc41c5a commit a7683af

File tree

6 files changed

+32
-26
lines changed

6 files changed

+32
-26
lines changed

dev/deps/spark-deps-hadoop-2.7-hive-2.3

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -202,12 +202,12 @@ orc-shims/1.6.7//orc-shims-1.6.7.jar
202202
oro/2.0.8//oro-2.0.8.jar
203203
osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar
204204
paranamer/2.8//paranamer-2.8.jar
205-
parquet-column/1.10.1//parquet-column-1.10.1.jar
206-
parquet-common/1.10.1//parquet-common-1.10.1.jar
207-
parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar
208-
parquet-format/2.4.0//parquet-format-2.4.0.jar
209-
parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar
210-
parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar
205+
parquet-column/1.11.1//parquet-column-1.11.1.jar
206+
parquet-common/1.11.1//parquet-common-1.11.1.jar
207+
parquet-encoding/1.11.1//parquet-encoding-1.11.1.jar
208+
parquet-format-structures/1.11.1//parquet-format-structures-1.11.1.jar
209+
parquet-hadoop/1.11.1//parquet-hadoop-1.11.1.jar
210+
parquet-jackson/1.11.1//parquet-jackson-1.11.1.jar
211211
protobuf-java/2.5.0//protobuf-java-2.5.0.jar
212212
py4j/0.10.9.1//py4j-0.10.9.1.jar
213213
pyrolite/4.30//pyrolite-4.30.jar

dev/deps/spark-deps-hadoop-3.2-hive-2.3

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -172,12 +172,12 @@ orc-shims/1.6.7//orc-shims-1.6.7.jar
172172
oro/2.0.8//oro-2.0.8.jar
173173
osgi-resource-locator/1.0.3//osgi-resource-locator-1.0.3.jar
174174
paranamer/2.8//paranamer-2.8.jar
175-
parquet-column/1.10.1//parquet-column-1.10.1.jar
176-
parquet-common/1.10.1//parquet-common-1.10.1.jar
177-
parquet-encoding/1.10.1//parquet-encoding-1.10.1.jar
178-
parquet-format/2.4.0//parquet-format-2.4.0.jar
179-
parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar
180-
parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar
175+
parquet-column/1.11.1//parquet-column-1.11.1.jar
176+
parquet-common/1.11.1//parquet-common-1.11.1.jar
177+
parquet-encoding/1.11.1//parquet-encoding-1.11.1.jar
178+
parquet-format-structures/1.11.1//parquet-format-structures-1.11.1.jar
179+
parquet-hadoop/1.11.1//parquet-hadoop-1.11.1.jar
180+
parquet-jackson/1.11.1//parquet-jackson-1.11.1.jar
181181
protobuf-java/2.5.0//protobuf-java-2.5.0.jar
182182
py4j/0.10.9.1//py4j-0.10.9.1.jar
183183
pyrolite/4.30//pyrolite-4.30.jar

pom.xml

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -136,7 +136,7 @@
136136
<kafka.version>2.6.0</kafka.version>
137137
<!-- After 10.15.1.3, the minimum required version is JDK9 -->
138138
<derby.version>10.14.2.0</derby.version>
139-
<parquet.version>1.10.1</parquet.version>
139+
<parquet.version>1.11.1</parquet.version>
140140
<orc.version>1.6.7</orc.version>
141141
<jetty.version>9.4.34.v20201102</jetty.version>
142142
<jakartaservlet.version>4.0.3</jakartaservlet.version>
@@ -2290,6 +2290,10 @@
22902290
<groupId>commons-pool</groupId>
22912291
<artifactId>commons-pool</artifactId>
22922292
</exclusion>
2293+
<exclusion>
2294+
<groupId>javax.annotation</groupId>
2295+
<artifactId>javax.annotation-api</artifactId>
2296+
</exclusion>
22932297
</exclusions>
22942298
</dependency>
22952299
<dependency>

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -251,7 +251,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest {
251251
"""
252252
|message root {
253253
| optional group _1 (MAP) {
254-
| repeated group map (MAP_KEY_VALUE) {
254+
| repeated group key_value (MAP_KEY_VALUE) {
255255
| required int32 key;
256256
| optional binary value (UTF8);
257257
| }
@@ -267,7 +267,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest {
267267
"""
268268
|message root {
269269
| optional group _1 (MAP) {
270-
| repeated group map (MAP_KEY_VALUE) {
270+
| repeated group key_value (MAP_KEY_VALUE) {
271271
| required group key {
272272
| optional binary _1 (UTF8);
273273
| optional binary _2 (UTF8);
@@ -300,7 +300,7 @@ class ParquetSchemaInferenceSuite extends ParquetSchemaTest {
300300
"""
301301
|message root {
302302
| optional group _1 (MAP_KEY_VALUE) {
303-
| repeated group map {
303+
| repeated group key_value {
304304
| required int32 key;
305305
| optional group value {
306306
| optional binary _1 (UTF8);
@@ -740,7 +740,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
740740
nullable = true))),
741741
"""message root {
742742
| optional group f1 (MAP_KEY_VALUE) {
743-
| repeated group map {
743+
| repeated group key_value {
744744
| required int32 num;
745745
| required binary str (UTF8);
746746
| }
@@ -759,7 +759,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
759759
nullable = true))),
760760
"""message root {
761761
| optional group f1 (MAP) {
762-
| repeated group map (MAP_KEY_VALUE) {
762+
| repeated group key_value (MAP_KEY_VALUE) {
763763
| required int32 key;
764764
| required binary value (UTF8);
765765
| }
@@ -797,7 +797,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
797797
nullable = true))),
798798
"""message root {
799799
| optional group f1 (MAP_KEY_VALUE) {
800-
| repeated group map {
800+
| repeated group key_value {
801801
| required int32 num;
802802
| optional binary str (UTF8);
803803
| }
@@ -816,7 +816,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
816816
nullable = true))),
817817
"""message root {
818818
| optional group f1 (MAP) {
819-
| repeated group map (MAP_KEY_VALUE) {
819+
| repeated group key_value (MAP_KEY_VALUE) {
820820
| required int32 key;
821821
| optional binary value (UTF8);
822822
| }
@@ -857,7 +857,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
857857
nullable = true))),
858858
"""message root {
859859
| optional group f1 (MAP) {
860-
| repeated group map (MAP_KEY_VALUE) {
860+
| repeated group key_value (MAP_KEY_VALUE) {
861861
| required int32 key;
862862
| required binary value (UTF8);
863863
| }
@@ -893,7 +893,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
893893
nullable = true))),
894894
"""message root {
895895
| optional group f1 (MAP) {
896-
| repeated group map (MAP_KEY_VALUE) {
896+
| repeated group key_value (MAP_KEY_VALUE) {
897897
| required int32 key;
898898
| optional binary value (UTF8);
899899
| }
@@ -1447,7 +1447,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
14471447
parquetSchema =
14481448
"""message root {
14491449
| required group f0 (MAP) {
1450-
| repeated group map (MAP_KEY_VALUE) {
1450+
| repeated group key_value (MAP_KEY_VALUE) {
14511451
| required int32 key;
14521452
| required group value {
14531453
| required int32 value_f0;
@@ -1472,7 +1472,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
14721472
expectedSchema =
14731473
"""message root {
14741474
| required group f0 (MAP) {
1475-
| repeated group map (MAP_KEY_VALUE) {
1475+
| repeated group key_value (MAP_KEY_VALUE) {
14761476
| required int32 key;
14771477
| required group value {
14781478
| required int64 value_f1;

sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -214,7 +214,9 @@ class StreamSuite extends StreamTest {
214214
.start(outputDir.getAbsolutePath)
215215
try {
216216
query.processAllAvailable()
217-
val outputDf = spark.read.parquet(outputDir.getAbsolutePath).as[Long]
217+
// Parquet write page-level CRC checksums will change the file size and
218+
// affect the data order when reading these files. Please see PARQUET-1746 for details.
219+
val outputDf = spark.read.parquet(outputDir.getAbsolutePath).sort('a).as[Long]
218220
checkDataset[Long](outputDf, (0L to 10L).toArray: _*)
219221
} finally {
220222
query.stop()

sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1528,7 +1528,7 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto
15281528
Seq(tbl, ext_tbl).foreach { tblName =>
15291529
sql(s"INSERT INTO $tblName VALUES (1, 'a', '2019-12-13')")
15301530

1531-
val expectedSize = 601
1531+
val expectedSize = 651
15321532
// analyze table
15331533
sql(s"ANALYZE TABLE $tblName COMPUTE STATISTICS NOSCAN")
15341534
var tableStats = getTableStats(tblName)

0 commit comments

Comments
 (0)