From 4cc5baf57420d628f017fb9dd0fd4d3c2b5e2749 Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Wed, 22 Jan 2020 11:22:42 +0100 Subject: [PATCH 01/36] Prepare next development iteration --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 4 ++-- 22 files changed, 23 insertions(+), 23 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index dd20e9be9b..b7a94810e7 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index c53071da88..e7b8e6b6af 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 904e4e973e..ae16bfb3b1 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 08f9d47753..1ceb23a1f7 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 9c9d638416..e67f0fa105 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 379e81b4ec..d886951aee 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 30bd635117..cb0c1853b5 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index c4b944b3a2..a3684464f5 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index ac53aa6904..f1f7d9a26f 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index d0aa0ee85a..4fb990ba4d 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 694fbf547e..1126c8ef88 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 6e1950a536..4212ae00f6 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 5e61b31031..c11c56022c 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 3810886037..c12e2477f6 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index a86f6eee97..eede36938b 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index ac109b5eb9..a7251846af 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index e333ff733b..990d5e9d03 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 482dc07322..5f5f759a4e 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 9c762e0b48..96f8ecdf6d 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index a9e216dee4..51e4aa3c3a 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 3290b112a0..5343d25785 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.0 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 7445e1c0d4..30b65ab21e 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.0 + 1.12.1-SNAPSHOT pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - apache-parquet-1.12.0-rc4 + HEAD From 76f35944d5994817a1f60fac5af791e7f515d3cf Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Fri, 23 Apr 2021 19:05:34 +0200 Subject: [PATCH 02/36] PARQUET-2027: Fix calculating directory offset for merge (#896) (cherry picked from commit 2ce35c73746cf091ed223da150daefd323a9ad3a) --- .../org/apache/parquet/hadoop/Offsets.java | 8 ++- .../hadoop/TestParquetWriterAppendBlocks.java | 63 +++++++++++++++++- .../src/test/resources/test-append_1.parquet | Bin 0 -> 7375 bytes .../src/test/resources/test-append_2.parquet | Bin 0 -> 7374 bytes 4 files changed, 65 insertions(+), 6 deletions(-) create mode 100644 parquet-hadoop/src/test/resources/test-append_1.parquet create mode 100644 parquet-hadoop/src/test/resources/test-append_2.parquet diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/Offsets.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/Offsets.java index fa25943b82..2edc585c4b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/Offsets.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/Offsets.java @@ -55,7 +55,7 @@ public static Offsets getOffsets(SeekableInputStream input, ColumnChunkMetaData * (0 cannot be a valid offset because of the MAGIC bytes) * - The firstDataPageOffset might point to the dictionary page */ - dictionaryPageSize = readDictionaryPageSize(input, newChunkStart); + dictionaryPageSize = readDictionaryPageSize(input, chunk); } else { dictionaryPageSize = chunk.getFirstDataPageOffset() - chunk.getDictionaryPageOffset(); } @@ -68,12 +68,14 @@ public static Offsets getOffsets(SeekableInputStream input, ColumnChunkMetaData return new Offsets(firstDataPageOffset, dictionaryPageOffset); } - private static long readDictionaryPageSize(SeekableInputStream in, long pos) throws IOException { + private static long readDictionaryPageSize(SeekableInputStream in, ColumnChunkMetaData chunk) throws IOException { long origPos = -1; try { origPos = in.getPos(); + in.seek(chunk.getStartingPos()); + long headerStart = in.getPos(); PageHeader header = Util.readPageHeader(in); - long headerSize = in.getPos() - origPos; + long headerSize = in.getPos() - headerStart; return headerSize + header.getCompressed_page_size(); } finally { if (origPos != -1) { diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterAppendBlocks.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterAppendBlocks.java index bda5333523..82d48f411d 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterAppendBlocks.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriterAppendBlocks.java @@ -1,4 +1,4 @@ -/* +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, * software distributed under the License is distributed on an * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -38,6 +38,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; +import java.net.URISyntaxException; import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedList; @@ -68,6 +69,17 @@ public class TestParquetWriterAppendBlocks { public static final SimpleGroupFactory GROUP_FACTORY = new SimpleGroupFactory(FILE_SCHEMA); + private static final Path STATIC_FILE_1 = createPathFromCP("/test-append_1.parquet"); + private static final Path STATIC_FILE_2 = createPathFromCP("/test-append_2.parquet"); + + private static Path createPathFromCP(String path) { + try { + return new Path(TestParquetWriterAppendBlocks.class.getResource(path).toURI()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + public Path file1; public List file1content = new ArrayList(); public Path file2; @@ -134,6 +146,51 @@ public void testBasicBehavior() throws IOException { Assert.assertEquals("All records should be present", 0, expected.size()); } + /** + * This test is similar to {@link #testBasicBehavior()} only that it uses static files generated by a previous release + * (1.11.1). This test is to validate the fix of PARQUET-2027. + */ + @Test + public void testBasicBehaviorWithStaticFiles() throws IOException { + List expected = new ArrayList<>(); + readAll(STATIC_FILE_1, expected); + readAll(STATIC_FILE_2, expected); + + Path combinedFile = newTemp(); + ParquetFileWriter writer = new ParquetFileWriter( + CONF, FILE_SCHEMA, combinedFile); + writer.start(); + writer.appendFile(CONF, STATIC_FILE_1); + writer.appendFile(CONF, STATIC_FILE_2); + writer.end(EMPTY_METADATA); + + try (ParquetReader reader = ParquetReader + .builder(new GroupReadSupport(), combinedFile) + .build()) { + + for (Group expectedNext : expected) { + Group next = reader.read(); + // check each value; equals is not supported for simple records + Assert.assertEquals("Each id should match", + expectedNext.getInteger("id", 0), next.getInteger("id", 0)); + Assert.assertEquals("Each string should match", + expectedNext.getString("string", 0), next.getString("string", 0)); + } + Assert.assertNull("No extra records should be present", reader.read()); + } + + } + + private void readAll(Path file, List values) throws IOException { + try (ParquetReader reader = ParquetReader + .builder(new GroupReadSupport(), file) + .build()) { + for (Group g = reader.read(); g != null; g = reader.read()) { + values.add(g); + } + } + } + @Test public void testMergedMetadata() throws IOException { Path combinedFile = newTemp(); diff --git a/parquet-hadoop/src/test/resources/test-append_1.parquet b/parquet-hadoop/src/test/resources/test-append_1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a255f86ebcf8147712d9f07ed85631e5a738708a GIT binary patch literal 7375 zcma*scYICv|G@ExAV_Q8NQm00b<(qkqGE4BY}KAwAt5%cwnmI-)e5C*rB2ZnZkT{k_i3iT*3b=U5a+97>=hN+BFl8f8!x33vz(qa4bk0xF^sDkFkKR6!DwQ5DsY zg6en#HBb|^P#bkn7xj>eM^PUQ&=8H#7y{Dp7(_^rp+JQO9R^HTXoAPl6wS~aPoM?T z(GsoD8g0-P?a&?_&=H-`8C}p7-H?Iq=z*TdL@)G4AM`~(^v3`U#FH3=r|>jvIB?;? zM}TKA7+DyCp~yxKhG95zF#;no3ZpRwV=)fnF#*rwIZVX!cmXeB5+-8`rXmm1FdZ{6 z6Zx2hm+&%X;}y)oT+G9KEI+vQw zU?VnRGq&I@Y{fQg$J^L}o!EtUup94U5BB0cypIp?A@<=T?8nDAfP*-M!}tV8a1@{7 z7>?rvKEp|z!fAYtGdPQLIFAdsh)cMPEBFFeaShjT12^#{Zs99@jobJJcknIl;vT-k z_xJ%n;wSu!U+^n_!+rdYKkxv5;xGJ-fABy2iQiw-sltEb};2}JW zawv}qsEA6aj0h4@1xZLoRa8R?s^byVKuy#_ZPYr`7%*X>2_8pNG(&Sdffh(dOSD33v_V_6Lwj^UM|47GbU{~iLk7B|2YMnCz0ezd z&=>vC9|JHDPht?B!qc$fz=a1N0iMBNWMK$~A{#jvhT+J?2#mxijK&y@#W;+|1U!r9 zFcHt=1-yt!n2afyiaboibj-j^2F%R>x00mfxMR*m9u>`MSDPG4i zEXNA0#3~fx4Xnl*ti?L4$D7!Ijo5_E*n+pP72B{KZ(|2`Vi(@QZoG><*o*h@K0d&Q z*oTj>A0OiY4&o3F;}aagQGAMHIF1we3@334r|~(?;4IGJJTBlOF5xn+;0s*EHC)FH z+{Bl-g|F~6ZsQx=!MC`Jd-x9D;|KhRpYSt&!LRrY_whUazyth=zwkHy!T<0t!W(}v z#G*LjPy!`U3h^k7GAN4#JcNf)4&_k+6;TP55kVrVAPLE+ifTwfbv%L^sEJyrjXJ1{ zdPv2isE-C{h(>4(0cm&)A|%LAphANV112mq!Q*I(W@wHl&;sdbiB@QhHfW1>Xpau) zh)(E?F6fGG$Ut}WKu=_%7kZ-)`l28DV*m!?NeseMcp5ewxbWa3z%v+(EDXU=WFrT| zFdVrUfsq)6(HMiV7>DtgfM@X>CgOR#fEO_dlQ9KTk%wuRjv1JVe9Xd2cp0mf$rk#p_sxV2oA?s9@D;wsZG3|}_!f6@58vT?{D2?v6Mn`o_!Yn5 zK7Pj^cz{3g7yiaS_#gg7_@ciUVo@A%D1nkFg?N-k8I(l=9>T*Yhw`X^il~Ijh#(PF zkc4DZMKz?LIvzm{)I=@RMjg~eJ*47M)JFp}L?bkYfHXV?5fWr5P@zGG0TULQ;BhoX zGc?B&Xn}OJL@TsL8?;3`v_}VYL??7c7j#88WS~2GpeHiX3%$_?ebEp7F#rScBnIIr zJPjKTTzK#i;28`?7KUIbvXO&f7>-AoK$(Vwv z$ip;D#|+FwK4#%1yo}j+1#>VL^DrL^P=JM4gjcZ`OYj<&;&m*;a;(5gtU@8)z-p|) zTCBr*yon9ih)vjxEqDuCu?^etHg;eqcHtfD#=F>qy?77r;{$w%efS9b@i7kIAP(U$ zKEV+j#iuxi<2ZrOa1y6*8lU3~&f*--;{q?hDQZ;Z?!O?tM@FdR_UDGlG zLk`D@wr;ZuO|TW6ak}CNb|71VZ98njb_2WUx{~b}uI4d;!o^itHw9A-Od*g}(Y6%N z)lD~?PgN|_uw+;8bVCzV*Jq`^8wgC~o2sLWP7vL%B1^jJD3V~=h9Rg@Uf*htJGnpYOMmSFMO-r+EUvNw=sp_uGdB}?3 znLN?495oQd==uW9k~B#aY}s`gXW4@3da~d-f#PeLCmTjI4-7HTbdd=ZMPhxvYzU4n z>ztdT7*-I-q8S~hE1u3ddV=gYCVAyJj1!qOP&hhS*CfddudCXICX3;t8lh87lEzeQ zTadND<46_Ha-(rkeOEDc)~BmviRwAwjwD5}B~5iTO;O~cedxCA$v#tw9HgMCq9<6c zsS5$eqnnN+8+P%~Q{oNG48Sb-m~(M=*7T zB31)mHylw7v}l~^AGX4xIX8ponilc1T!Ub%oGs1G3!-rmHG{IZ6&4=N15Fi75>j9` zT?}MWp=f=9YDCTm%ijg&qI=A%@8R`UDd+#2+u8ins2Kl zpCXbn(fVLsM^z{=mqK7OnkAT`FAIh)iZzY)k#yB=2>6-0YO0dzv56*A&MlpB;|E_Bc06*OE-r zGTk7W_Y!?wvOJ>e5?xgyscEpbAV~r3!*RpM!t08jruq_vYgBR`w8z< zzO6;`KvWd6)Cq*ZWQeMAOK@D(qFtM&rhAfPsL}lfG#k^Bbb)A-2V|pQ1=I(3Ckn!F z>_Ca;3z@EHh7``{GoNSCH#|!fr0}NVdp7AGjk7Dut{hMwvQAS_Esy($tcLeNF>ome zMb?VqXBwuX)0!31oqF@aJP6ArFhxbzL~gl7<8*qDOjUSpct7%FiTIgekr=KLXrf`* z(Qy`IXi0`Z52TN)gv63u+9CDm7$Ut*mU%P&X4&v#%i+S~MQ_i?6}>%Q^2HW46Aok~ zm6~6_(?0b?@mk&M7OT{&UzJ5!i)Cr)zA^LGOt@MqHKk%!#`=_Q@hu8h&Gzny(C%_HT?#6=1=oXXl%Ft6C+!2?S+Y1T24dF;ont(GqQ{&L~5x&7OIKEF=>rux;! z9}5a=cbrym|NZIy*5YF(UzlI9_vwBc3g%Ah)PLoLEh)WIzCW@{tQ24B+l4dSB_|GM zeBJAXh0m&s7pPmRG~6avZC$zB7H!J3lpn4m&UjFB~s}@|EoRQujwGSYMNPcZ1wNVX0P=~zuPG<=fVDKdDXkznV8og zGx_g?{Lx$T{@B$rH+PP>f5uwtdbI_2cRqZiOpjA{M@;^B;<41+ytesgW@NT)vwBdU zNvZp~E^N?Y<=z9gHs+PNl#`v>cv$g*{8@dUY@0D7rl9<&na?{dCoOwWKCeWF83zuv z`!p>+zW@0hYc4GvwBlxuZ}ks`T|ZDE^+36j)i%b}s#PYYo1x!69ozawx^PKLAGqmM zkIrAYNuSj$93DHhXUTT)#}e`*eKH5sJ?d>s`fJv?4f|8-b)PjOyTgDUx2BdaS8rSP z@4N3@SUyF5IezTuhksgoWM7H-b9Zj|WarRgX{A>#THSq5@60yEDleIyy!zbW{Ck6A zf7)KJ&Xt(L%;jx5&8{&saovO+(my3MQ>$ED@6y}nAHB8h-p1fw!GvY2 zgCA}ztJ-+x!Z!+6joWcGFQ?S?H;=}YOS>C;qV=i;m$Ik-eDGN5HIGj3+C@xQadAR} zzEbL|lWvr1GEnN=xnG6DMq%YW!Ix9j+|o~P*_k%zU}n9u4LfOj`VCBKd*6w#)MY>! zuWI|P-`wrd`RwK|YLBl_eD&h?joVzB(rU-W#Pj}%Rtk{BQPwK*4vNlr;TG$^ttA^e^Po3<~Lo}23r^D_NWqj}98iLJ+bSy^68 zB!O>Ajv75OYuGdOVmhZVJn>|?&|1VZF;q^v|*BkL@Q$42sf7<98)oPD zNu~Vpc1~`#ACuHG*B&`yj6Yh)8Ci3zKXO#o@L@HjG)YMlg`Qp0GkUh~mRYmDGbStB ytI1tR4S2=Gs|Q|vh`LRqrM+?=^jtO2JyYbRCb#DX$&HC{807Gc6amNtH$^Z5Gx-|NL&e0Xl|bKlo}UDr84cAH*OG!i}3I{bL~_MGp_ zr$*0K;VX&J#7N|7*+}FXzQuR=9zWnm{DkZH8Nc8LZsJ$`hTriA{=_ZZ#$Wgw|KMLl z;vxsfDGIV5fWr5P@zGG0TUMPLn~zBeze8|$U+-Dh_+~l_UM3) z=!DMbg0AR>?&yJskd2<`h2H3czUYVk7=VEogu!?ik6;KM#ZcIA;KGBC0K+gGIT(SF z7=_UogR#iPIOJhGCSW2aVKS!RG34WMOvN-jfhRE?GcXggP=MK(gSnW8`B;F3ScJt` zf~T+)%di|PP>83o605KpMOcHiScmm^2G3#xHewStV+)?cR%}BtwqpmL$4VXt9T8s;|;utLpY2hIEqpn!*RTYw{Zd|aSEsL4$k0RoW*-M zhxc(FAK(H$#7FoT7jX%n;8R@2XSjmT@ddubSNIzL!#B8!Yxox5;d}gmAMq2e<7fPW z8@P#I@f&`}ANUiua2tQ&Z~TLQ5lNu_Q3hoZj|7xMc_iX4R6s>kLS@{IDyWKTsE!(_ zi71j#3&}`9ZPY<3>f#>MLmKL%0UDwa8Y3M|&=k$k94*ii0y1zfL`aaKK!pY!225DE z53P`i`_UQ?APa5qAljlG+M@$Hq7yo!3%a5kx}ygkLNW5F%{GB1fIln z%)m^{LIGxD4(4JW=3@aCVi6W&37*1IEW>iFKp~#SO02?a6k!e4Vjb4w89a*(*oaNo zj4gN$Td@tr*p3}|9y{>@c40S4um^kbBKF}W?8nPEfP;7iui`bljyLco4&gA4;3!IQ z49D>n-o^=>#3`J{J2-=PaTf349Nx!ye1Hr15Fg=VT*M`Of=_W7pWzBV#~1h#U*T)~ z58vP_uHjpJhwt$Ne#B3>j-T-hZr~<<#c%i>f8bBt!fpJ8zwrh3H09j~*2hkSo&>kJo5uMN(UCcPS z5VFw|z0ezd&=>vC9|JHDgD@Bm;}HzOqZkSs4qSNf5nvdGBL^ce5~DC0V=xxE7>7KJ z#{^8oBuvH>JcfKcj;WZ2C-5YuV+Lko778#Mb1)b4FdqxB5R0%FOYjtyVi}fW1q$&r zR$>)aqX=uT7VEGc&)`{Xz(#DsW^BQ8*oti^#&+z$^Vo?OunW6Uf<4%a7qJg7VLx8R z0UX3DconbVb-aN$aR`TT1V>SdV>pht@HS51Bu?Qp-oY8Xi?etS=kPwx;{#m4hxiB| z<03BM6MTxx_zYL@IljP`_zGX+fA|JhaSh+%JA98H@FRZ0b^MH9a055-D}KZ8_yd39 z7H;D&{EdI`FCyisf0RL4#3KRaP#%f63l&fil~5UXqYA2`8mglPY9fjx)Iu^+P#blS zin_Q5^^k`8Xn=-jgvLlm6EsCLG)D`xgn$g(3lS1zC{Uq6hXE57?n5hN;(oNo1IR)f zJczbvhxX`zj_8EW=z^~3hVJNrhmeh)=!M?sgTCm8{uqFP7=*!i7>{5G9>q}DaNxp& zj{w6k961<)kr;*17=y9M#W>_)JSJcwCSfwB;4$RmaZJTDJb@=M9WyW!vrvH9n1i{P zhxu55g;<2eSc0dp6w9z2D^Q51u@bAW8bw%xwOEJscm~g612$q4He(B(!&YoVF}7m| zp2tqSfL+*)670cVyoh~x3H$Lf4&WeO!K-);uj388i9~@t|6GdYAJ#3s9s=(<6K#GrmwRbJ<+s$!xhxPk_FY46~XpJUl4uCw#k;|Pw<^k$%Vk|*n$B*o&Qa&EF1zN-ZWi}WSIRBT(2wZOAEJkN4t`_)X{ zw*tiy6y4XkxTEqN93GRoqH9`4V94Qh9YuExlMu_27O+0g6C7e9NS>iZp7lO zSf*jgE(dRLzOK(W-wgyeuzi#B6rCV;UD4B2U($6}Z?O-Zq%lR^6C~C0!^rx!riSy0 znqhjrtq5A6*{n|s&qMZwz}Iy#kZs}{Tc09Jy6PyBK*F+LiA|G@f#A8e$ON9Jh*CJ8 zWIKkdd2CP#JVBLp@^O4lHihhs>}(vo)gYz z+lHzdOy$^Nz8GY-Y3sh=`9?TOW7|PE&UZY8bR>~fhx64fO)xFl5)50^Ws&@50pYlS zLo_W(7bK0Ssexw+Rv`HTt1x}jaO^;d?N{|(#ndHH&{eLcdX6tRrbN<68cC)p3h5Tk zCo_W?aAdx(vp(Gp&qHD7jxO7-Y-o}e+lOaqzQ}h{Z!UGNtD;~VA}?jd@D$6@UCFz% z5AxI?F0O5>)SGG14?LZ6;gm#Qmo@E9o{FXuc;UOg%03j!B`%^Q$f9Rzw&Xaj6RVHF z^4O_w2%2sXXHzl+TUQ9XuIavGIlkn?u4`(BEr+Y}REkq|6;-fRDvwlgWSWK{DzW?v zH>=W#1l{w)e!zZBQlK{4~jHwO@ggZB&x1Bf*p`Aw(ZbdY|80Q{&}{< z;nOWt*Ngo_9q9_mud)%t5viEi`gB{SY^Vy61gAfX^gq|sg@AI=O~;W9JGNiSB(O9~ z5gbP(&W0%qwn)F%0^d{=)vydFmZz3%do&S4AO~5WO6gjXOF9x?#}JuQj^(N5n@nei z*-5Kcbyud2SgvPskesg?+{sfC!u1T(6nxVp4{SnbTik4X)wGpBv{mm;otqkEz_i`-%xaVH?g7dg#E>n zCF;Wri_UPBKobqaj@>^b`h#S7oR-UJsS?RW=>pl}fXdkIsn9uimoTmzh;-+&@F@*{l7)8WV*9{WAM(8J#+A`1-k{KHJu9 zsMMx@!zD!>?peLHa_u>V%bImAoxgJBs`S|ti=zi-rsoYTcVWV)OZgo?n!di(-bL=v zN(Bctj%Z_V+Bw*$u(KfRV=?K_M)Tb7{R-j+)hns6Dt&e2V&mhFZJ&Lk{_&wNlw8Z6 zH@RQ&f+u#)?67;|oNXs(d{j7b@ugS#9BOnVx7)cD`}&u4&)h2P*L=v3Ut2djKcKoi zbVi>psp8I_S;2{(1$jL z`LfH@!5`~KY7MG&DPdT`;)>1UuUu@NaImP#hIJ3lE?E*JZY^m%_~x?d1v%YP+b$f} z^Kc8Z*`$oVM#5!d`t@5Y`%QVKe*Tg3rSwH#N>5e3+v}WGv(;M*N(=96eXQW=0?oYQjkq$rQt`Pi3olGxn_jp3+jE*FEzgcW zJUlKxH-BBWS$d{;(14=+4uj1lCyQ%tol}^yw{T!w-JbqE9TJ9FaV5VU-}e0M_i}LK^_q-(4 zTr@1B&-|qyt|&crF+b=~_49=-2NkLbwQDY(62EfZw5%@ON6ebvZmK!wNM6=N@rqs0 z^RzgB9A%UR1l>K(%D#Rq_4dJ60W?U(~gz-qLot*$wP*=`}VL_lP_2 z>8vhUr)%bvZ~bATr4 z(>Rh`CdZ3JEAgLl<0s_hj2YHA(k+$YNoT91q^7h?PfX<-8z-`I21dg#Co%WU*6E3v zclL5;V@dqYDcsqeS6rCtXMX0{KPsA(ba6y9Ba;4C>(po^CW}w`|BKz3@N$c!%Pn{} zQ;#(LPaFMjQH!Ls4xjPm!e4qoT=4^F~H@#YK|NjbJmZG`ZU3yqpPsUdHIL zoM&tFSa`X literal 0 HcmV?d00001 From 49f40a359cc317e0c83953e285446020c78da579 Mon Sep 17 00:00:00 2001 From: Xinli Shang Date: Mon, 16 Aug 2021 02:24:30 -0700 Subject: [PATCH 03/36] PARQUET-2064: Make Range public accessible in RowRanges (#921) --- .../internal/filter2/columnindex/RowRanges.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java index cf6a1cadd9..52a8e50926 100644 --- a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java +++ b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java @@ -37,7 +37,8 @@ * @see ColumnIndexFilter#calculateRowRanges(Filter, ColumnIndexStore, Set, long) */ public class RowRanges { - private static class Range { + // Make it public because some uppler layer application need to access it + public static class Range { // Returns the union of the two ranges or null if there are elements between them. private static Range union(Range left, Range right) { @@ -63,8 +64,8 @@ private static Range intersection(Range left, Range right) { return null; } - final long from; - final long to; + public final long from; + public final long to; // Creates a range of [from, to] (from and to are inclusive; empty ranges are not valid) Range(long from, long to) { @@ -299,6 +300,10 @@ public boolean isOverlapping(long from, long to) { (r1, r2) -> r1.isBefore(r2) ? -1 : r1.isAfter(r2) ? 1 : 0) >= 0; } + public List getRanges() { + return Collections.unmodifiableList(ranges); + } + @Override public String toString() { return ranges.toString(); From e8c9acd56e1fd0d27e709c6b9fe9a466df155a15 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 19 Apr 2021 01:22:39 -0700 Subject: [PATCH 04/36] PARQUET-2022: ZstdDecompressorStream should close `zstdInputStream` (#889) --- .../parquet/hadoop/codec/ZstdDecompressorStream.java | 9 +++++++++ .../org/apache/parquet/hadoop/TestZstandardCodec.java | 9 ++++----- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java index a505e7bde0..a24612f2ee 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/ZstdDecompressorStream.java @@ -50,4 +50,13 @@ public int read() throws IOException { public void resetState() throws IOException { // no-opt, doesn't apply to ZSTD } + + @Override + public void close() throws IOException { + try { + zstdInputStream.close(); + } finally { + super.close(); + } + } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java index c0d98266f7..b6bab26785 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestZstandardCodec.java @@ -79,8 +79,8 @@ private void testZstd(ZstandardCodec codec, int dataSize) throws IOException { byte[] data = new byte[dataSize]; (new Random()).nextBytes(data); BytesInput compressedData = compress(codec, BytesInput.from(data)); - BytesInput decompressedData = decompress(codec, compressedData, data.length); - Assert.assertArrayEquals(data, decompressedData.toByteArray()); + byte[] decompressedData = decompress(codec, compressedData, data.length); + Assert.assertArrayEquals(data, decompressedData); } private BytesInput compress(ZstandardCodec codec, BytesInput bytes) throws IOException { @@ -91,10 +91,9 @@ private BytesInput compress(ZstandardCodec codec, BytesInput bytes) throws IOExc return BytesInput.from(compressedOutBuffer); } - private BytesInput decompress(ZstandardCodec codec, BytesInput bytes, int uncompressedSize) throws IOException { - BytesInput decompressed; + private byte[] decompress(ZstandardCodec codec, BytesInput bytes, int uncompressedSize) throws IOException { InputStream is = codec.createInputStream(bytes.toInputStream(), null); - decompressed = BytesInput.from(is, uncompressedSize); + byte[] decompressed = BytesInput.from(is, uncompressedSize).toByteArray(); is.close(); return decompressed; } From 7597c74c55c02980021eb9e8c5345ea0d86fa62a Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 26 May 2021 00:43:22 -0700 Subject: [PATCH 05/36] PARQUET-2052: Integer overflow when writing huge binary using dictionary encoding (#910) --- .../values/dictionary/DictionaryValuesWriter.java | 6 +++--- .../column/values/dictionary/TestDictionary.java | 15 +++++++++++++++ pom.xml | 4 ++++ 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java index 2999f3cdc4..c4a985224f 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java @@ -81,7 +81,7 @@ public abstract class DictionaryValuesWriter extends ValuesWriter implements Req protected boolean dictionaryTooBig; /* current size in bytes the dictionary will take once serialized */ - protected int dictionaryByteSize; + protected long dictionaryByteSize; /* size in bytes of the dictionary at the end of last dictionary encoded page (in case the current page falls back to PLAIN) */ protected int lastUsedDictionaryByteSize; @@ -173,7 +173,7 @@ public BytesInput getBytes() { BytesInput bytes = concat(BytesInput.from(bytesHeader), rleEncodedBytes); // remember size of dictionary when we last wrote a page lastUsedDictionarySize = getDictionarySize(); - lastUsedDictionaryByteSize = dictionaryByteSize; + lastUsedDictionaryByteSize = Math.toIntExact(dictionaryByteSize); return bytes; } catch (IOException e) { throw new ParquetEncodingException("could not encode the values", e); @@ -249,7 +249,7 @@ public void writeBytes(Binary v) { id = binaryDictionaryContent.size(); binaryDictionaryContent.put(v.copy(), id); // length as int (4 bytes) + actual bytes - dictionaryByteSize += 4 + v.length(); + dictionaryByteSize += 4L + v.length(); } encodedValues.add(id); } diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java index 2783b696d5..174fad8918 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java @@ -53,6 +53,7 @@ import org.apache.parquet.column.values.plain.PlainValuesWriter; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.mockito.Mockito; public class TestDictionary { @@ -171,6 +172,20 @@ public void testBinaryDictionaryFallBack() throws IOException { assertEquals(0, cw.getBufferedSize()); } + @Test + public void testBinaryDictionaryIntegerOverflow() { + Binary mock = Mockito.mock(Binary.class); + Mockito.when(mock.length()).thenReturn(Integer.MAX_VALUE - 1); + // make the writer happy + Mockito.when(mock.copy()).thenReturn(Binary.fromString(" world")); + + final ValuesWriter cw = newPlainBinaryDictionaryValuesWriter(100, 100); + cw.writeBytes(Binary.fromString("hello")); + cw.writeBytes(mock); + + assertEquals(PLAIN, cw.getEncoding()); + } + @Test public void testBinaryDictionaryChangedValues() throws IOException { int COUNT = 100; diff --git a/pom.xml b/pom.xml index 30b65ab21e..8916ccc85a 100644 --- a/pom.xml +++ b/pom.xml @@ -506,6 +506,10 @@ ${shade.prefix} + + org.apache.parquet.column.values.dictionary.DictionaryValuesWriter#dictionaryByteSize From aa132b3787b20170f3554cfeac214884cfecb820 Mon Sep 17 00:00:00 2001 From: Edward Wright Date: Fri, 11 Jun 2021 09:23:48 +0100 Subject: [PATCH 06/36] PARQUET-1633: Fix integer overflow (#902) Unit test: - Updated ParquetWriter to support setting row group size in long - Removed Xmx settings in the pom to allow more memory for the tests Co-authored-by: Gabor Szadovszky --- .../parquet/hadoop/ParquetFileReader.java | 16 +- .../apache/parquet/hadoop/ParquetWriter.java | 16 +- .../parquet/hadoop/TestLargeColumnChunk.java | 142 ++++++++++++++++++ pom.xml | 3 +- 4 files changed, 165 insertions(+), 12 deletions(-) create mode 100644 parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLargeColumnChunk.java diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 791f9ef188..95f93ab809 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -916,7 +916,7 @@ public PageReadStore readNextRowGroup() throws IOException { currentParts = new ConsecutivePartList(startingPos); allParts.add(currentParts); } - currentParts.addChunk(new ChunkDescriptor(columnDescriptor, mc, startingPos, (int)mc.getTotalSize())); + currentParts.addChunk(new ChunkDescriptor(columnDescriptor, mc, startingPos, mc.getTotalSize())); } } // actually read all the chunks @@ -994,7 +994,7 @@ public PageReadStore readNextFilteredRowGroup() throws IOException { allParts.add(currentParts); } ChunkDescriptor chunkDescriptor = new ChunkDescriptor(columnDescriptor, mc, startingPos, - (int) range.getLength()); + range.getLength()); currentParts.addChunk(chunkDescriptor); builder.setOffsetIndex(chunkDescriptor, filteredOffsetIndex); } @@ -1613,7 +1613,7 @@ private static class ChunkDescriptor { private final ColumnDescriptor col; private final ColumnChunkMetaData metadata; private final long fileOffset; - private final int size; + private final long size; /** * @param col column this chunk is part of @@ -1625,7 +1625,7 @@ private ChunkDescriptor( ColumnDescriptor col, ColumnChunkMetaData metadata, long fileOffset, - int size) { + long size) { super(); this.col = col; this.metadata = metadata; @@ -1657,8 +1657,8 @@ public boolean equals(Object obj) { private class ConsecutivePartList { private final long offset; - private int length; - private final List chunks = new ArrayList(); + private long length; + private final List chunks = new ArrayList<>(); /** * @param offset where the first chunk starts @@ -1686,8 +1686,8 @@ public void readAll(SeekableInputStream f, ChunkListBuilder builder) throws IOEx List result = new ArrayList(chunks.size()); f.seek(offset); - int fullAllocations = length / options.getMaxAllocationSize(); - int lastAllocationSize = length % options.getMaxAllocationSize(); + int fullAllocations = Math.toIntExact(length / options.getMaxAllocationSize()); + int lastAllocationSize = Math.toIntExact(length % options.getMaxAllocationSize()); int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0); List buffers = new ArrayList<>(numAllocations); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java index e1afaca994..9a71987c6b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java @@ -269,7 +269,7 @@ public ParquetWriter(Path file, Configuration conf, WriteSupport writeSupport ParquetFileWriter.Mode mode, WriteSupport writeSupport, CompressionCodecName compressionCodecName, - int rowGroupSize, + long rowGroupSize, boolean validating, Configuration conf, int maxPaddingSize, @@ -355,7 +355,7 @@ public abstract static class Builder> { private Configuration conf = new Configuration(); private ParquetFileWriter.Mode mode; private CompressionCodecName codecName = DEFAULT_COMPRESSION_CODEC_NAME; - private int rowGroupSize = DEFAULT_BLOCK_SIZE; + private long rowGroupSize = DEFAULT_BLOCK_SIZE; private int maxPaddingSize = MAX_PADDING_SIZE_DEFAULT; private boolean enableValidation = DEFAULT_IS_VALIDATING_ENABLED; private ParquetProperties.Builder encodingPropsBuilder = @@ -432,8 +432,20 @@ public SELF withEncryption (FileEncryptionProperties encryptionProperties) { * * @param rowGroupSize an integer size in bytes * @return this builder for method chaining. + * @deprecated Use {@link #withRowGroupSize(long)} instead */ + @Deprecated public SELF withRowGroupSize(int rowGroupSize) { + return withRowGroupSize((long) rowGroupSize); + } + + /** + * Set the Parquet format row group size used by the constructed writer. + * + * @param rowGroupSize an integer size in bytes + * @return this builder for method chaining. + */ + public SELF withRowGroupSize(long rowGroupSize) { this.rowGroupSize = rowGroupSize; return self(); } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLargeColumnChunk.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLargeColumnChunk.java new file mode 100644 index 0000000000..90015f57e1 --- /dev/null +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestLargeColumnChunk.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.hadoop; + +import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn; +import static org.apache.parquet.filter2.predicate.FilterApi.eq; +import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; +import static org.apache.parquet.hadoop.metadata.CompressionCodecName.UNCOMPRESSED; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; +import static org.apache.parquet.schema.Types.buildMessage; +import static org.apache.parquet.schema.Types.required; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.io.IOException; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.GroupFactory; +import org.apache.parquet.example.data.simple.SimpleGroupFactory; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.MessageType; +import org.junit.*; +import org.junit.rules.TemporaryFolder; + +/** + * This test is to test parquet-mr working with potential int overflows (when the sizes are greater than + * Integer.MAX_VALUE). The test requires ~3GB memory so it is likely to fail in the CI environment, so these + * tests are flagged to be ignored. + */ +@Ignore +public class TestLargeColumnChunk { + private static final MessageType SCHEMA = buildMessage().addFields( + required(INT64).named("id"), + required(BINARY).named("data")) + .named("schema"); + private static final int DATA_SIZE = 256; + // Ensure that the size of the column chunk would overflow an int + private static final int ROW_COUNT = Integer.MAX_VALUE / DATA_SIZE + 1000; + private static final long RANDOM_SEED = 42; + private static final int ID_INDEX = SCHEMA.getFieldIndex("id"); + private static final int DATA_INDEX = SCHEMA.getFieldIndex("data"); + + private static final long ID_OF_FILTERED_DATA = ROW_COUNT / 2; + private static Binary VALUE_IN_DATA; + private static Binary VALUE_NOT_IN_DATA; + private static Path file; + + @ClassRule + public static TemporaryFolder folder = new TemporaryFolder(); + + @BeforeClass + public static void createFile() throws IOException { + file = new Path(folder.newFile().getAbsolutePath()); + + GroupFactory factory = new SimpleGroupFactory(SCHEMA); + Random random = new Random(RANDOM_SEED); + Configuration conf = new Configuration(); + GroupWriteSupport.setSchema(SCHEMA, conf); + try (ParquetWriter writer = ExampleParquetWriter + .builder(HadoopOutputFile.fromPath(file, conf)) + .withWriteMode(OVERWRITE) + .withConf(conf) + .withCompressionCodec(UNCOMPRESSED) + .withRowGroupSize(4L * 1024 * 1024 * 1024) // 4G to ensure all data goes to one row group + .withBloomFilterEnabled(true) + .build()) { + for (long id = 0; id < ROW_COUNT; ++id) { + Group group = factory.newGroup(); + group.add(ID_INDEX, id); + Binary data = nextBinary(random); + group.add(DATA_INDEX, data); + writer.write(group); + if (id == ID_OF_FILTERED_DATA) { + VALUE_IN_DATA = data; + } + } + } + VALUE_NOT_IN_DATA = nextBinary(random); + } + + private static Binary nextBinary(Random random) { + byte[] bytes = new byte[DATA_SIZE]; + random.nextBytes(bytes); + return Binary.fromConstantByteArray(bytes); + } + + @Test + public void validateAllData() throws IOException { + Random random = new Random(RANDOM_SEED); + try (ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), file).build()) { + for (long id = 0; id < ROW_COUNT; ++id) { + Group group = reader.read(); + assertEquals(id, group.getLong(ID_INDEX, 0)); + assertEquals(nextBinary(random), group.getBinary(DATA_INDEX, 0)); + } + assertNull("No more record should be read", reader.read()); + } + } + + @Test + public void validateFiltering() throws IOException { + try (ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), file) + .withFilter(FilterCompat.get(eq(binaryColumn("data"), VALUE_IN_DATA))) + .build()) { + Group group = reader.read(); + assertEquals(ID_OF_FILTERED_DATA, group.getLong(ID_INDEX, 0)); + assertEquals(VALUE_IN_DATA, group.getBinary(DATA_INDEX, 0)); + assertNull("No more record should be read", reader.read()); + } + try (ParquetReader reader = ParquetReader.builder(new GroupReadSupport(), file) + .withFilter(FilterCompat.get(eq(binaryColumn("data"), VALUE_NOT_IN_DATA))) + .build()) { + assertNull("No record should be read", reader.read()); + } + } +} diff --git a/pom.xml b/pom.xml index 8916ccc85a..9f60ac9327 100644 --- a/pom.xml +++ b/pom.xml @@ -114,7 +114,6 @@ 1.8 - -Xmx512m INFO @@ -602,7 +601,7 @@ ci-test WARN - -Xmx512m -XX:MaxJavaStackTraceDepth=10 + -XX:MaxJavaStackTraceDepth=10 From 387e138cfba07a48d0a977bb5c69aa622f65f446 Mon Sep 17 00:00:00 2001 From: Kai Jiang Date: Tue, 22 Jun 2021 00:52:43 -0700 Subject: [PATCH 07/36] PARQUET-2054: fix TCP leaking when calling ParquetFileWriter.appendFile (#913) * use try-with-resource statement for ParquetFileReader to call close explicitly --- .../cli/commands/CheckParquet251Command.java | 6 +- .../parquet/cli/commands/SchemaCommand.java | 7 +- .../cli/commands/ShowDictionaryCommand.java | 91 +++++----- .../cli/commands/ShowPagesCommand.java | 90 +++++----- .../parquet/hadoop/ParquetFileWriter.java | 4 +- .../parquet/hadoop/TestParquetFileWriter.java | 167 +++++++++--------- .../parquet/hadoop/TestParquetWriter.java | 15 +- .../hadoop/TestReadWriteEncodingStats.java | 49 ++--- 8 files changed, 220 insertions(+), 209 deletions(-) diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/CheckParquet251Command.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/CheckParquet251Command.java index fbeebdfba6..d7aa82dcfd 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/CheckParquet251Command.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/CheckParquet251Command.java @@ -108,10 +108,8 @@ public boolean apply(@Nullable ColumnDescriptor input) { })); // now check to see if the data is actually corrupt - ParquetFileReader reader = new ParquetFileReader(getConf(), - fakeMeta, path, footer.getBlocks(), columns); - - try { + try (ParquetFileReader reader = new ParquetFileReader(getConf(), + fakeMeta, path, footer.getBlocks(), columns)) { PageStatsValidator validator = new PageStatsValidator(); for (PageReadStore pages = reader.readNextRowGroup(); pages != null; pages = reader.readNextRowGroup()) { diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/SchemaCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/SchemaCommand.java index ca29dd0268..988ab0f40f 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/SchemaCommand.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/SchemaCommand.java @@ -119,9 +119,10 @@ private String getParquetSchema(String source) throws IOException { switch (format) { case PARQUET: - return new ParquetFileReader( - getConf(), qualifiedPath(source), ParquetMetadataConverter.NO_FILTER) - .getFileMetaData().getSchema().toString(); + try (ParquetFileReader reader = new ParquetFileReader( + getConf(), qualifiedPath(source), ParquetMetadataConverter.NO_FILTER)) { + return reader.getFileMetaData().getSchema().toString(); + } default: throw new IllegalArgumentException(String.format( "Could not get a Parquet schema for format %s: %s", format, source)); diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowDictionaryCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowDictionaryCommand.java index 20a694ff7f..7a167ed635 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowDictionaryCommand.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowDictionaryCommand.java @@ -64,56 +64,57 @@ public int run() throws IOException { String source = targets.get(0); - ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source)); - MessageType schema = reader.getFileMetaData().getSchema(); - ColumnDescriptor descriptor = Util.descriptor(column, schema); - PrimitiveType type = Util.primitive(column, schema); - Preconditions.checkNotNull(type); - - DictionaryPageReadStore dictionaryReader; - int rowGroup = 0; - while ((dictionaryReader = reader.getNextDictionaryReader()) != null) { - DictionaryPage page = dictionaryReader.readDictionaryPage(descriptor); - - Dictionary dict = page.getEncoding().initDictionary(descriptor, page); - - console.info("\nRow group {} dictionary for \"{}\":", rowGroup, column, page.getCompressedSize()); - for (int i = 0; i <= dict.getMaxId(); i += 1) { - switch(type.getPrimitiveTypeName()) { - case BINARY: - if (type.getLogicalTypeAnnotation() instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) { + MessageType schema = reader.getFileMetaData().getSchema(); + ColumnDescriptor descriptor = Util.descriptor(column, schema); + PrimitiveType type = Util.primitive(column, schema); + Preconditions.checkNotNull(type); + + DictionaryPageReadStore dictionaryReader; + int rowGroup = 0; + while ((dictionaryReader = reader.getNextDictionaryReader()) != null) { + DictionaryPage page = dictionaryReader.readDictionaryPage(descriptor); + + Dictionary dict = page.getEncoding().initDictionary(descriptor, page); + + console.info("\nRow group {} dictionary for \"{}\":", rowGroup, column, page.getCompressedSize()); + for (int i = 0; i <= dict.getMaxId(); i += 1) { + switch(type.getPrimitiveTypeName()) { + case BINARY: + if (type.getLogicalTypeAnnotation() instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + console.info("{}: {}", String.format("%6d", i), + Util.humanReadable(dict.decodeToBinary(i).toStringUsingUTF8(), 70)); + } else { + console.info("{}: {}", String.format("%6d", i), + Util.humanReadable(dict.decodeToBinary(i).getBytesUnsafe(), 70)); + } + break; + case INT32: console.info("{}: {}", String.format("%6d", i), - Util.humanReadable(dict.decodeToBinary(i).toStringUsingUTF8(), 70)); - } else { + dict.decodeToInt(i)); + break; + case INT64: console.info("{}: {}", String.format("%6d", i), - Util.humanReadable(dict.decodeToBinary(i).getBytesUnsafe(), 70)); - } - break; - case INT32: - console.info("{}: {}", String.format("%6d", i), - dict.decodeToInt(i)); - break; - case INT64: - console.info("{}: {}", String.format("%6d", i), - dict.decodeToLong(i)); - break; - case FLOAT: - console.info("{}: {}", String.format("%6d", i), - dict.decodeToFloat(i)); - break; - case DOUBLE: - console.info("{}: {}", String.format("%6d", i), - dict.decodeToDouble(i)); - break; - default: - throw new IllegalArgumentException( - "Unknown dictionary type: " + type.getPrimitiveTypeName()); + dict.decodeToLong(i)); + break; + case FLOAT: + console.info("{}: {}", String.format("%6d", i), + dict.decodeToFloat(i)); + break; + case DOUBLE: + console.info("{}: {}", String.format("%6d", i), + dict.decodeToDouble(i)); + break; + default: + throw new IllegalArgumentException( + "Unknown dictionary type: " + type.getPrimitiveTypeName()); + } } - } - reader.skipNextRowGroup(); + reader.skipNextRowGroup(); - rowGroup += 1; + rowGroup += 1; + } } console.info(""); diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowPagesCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowPagesCommand.java index 58321064e3..bf030ac606 100644 --- a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowPagesCommand.java +++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowPagesCommand.java @@ -75,57 +75,57 @@ public int run() throws IOException { "Cannot process multiple Parquet files."); String source = targets.get(0); - ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source)); - - MessageType schema = reader.getFileMetaData().getSchema(); - Map columns = Maps.newLinkedHashMap(); - if (this.columns == null || this.columns.isEmpty()) { - for (ColumnDescriptor descriptor : schema.getColumns()) { - columns.put(descriptor, primitive(schema, descriptor.getPath())); - } - } else { - for (String column : this.columns) { - columns.put(descriptor(column, schema), primitive(column, schema)); - } - } - - CompressionCodecName codec = reader.getRowGroups().get(0).getColumns().get(0).getCodec(); - // accumulate formatted lines to print by column - Map> formatted = Maps.newLinkedHashMap(); - PageFormatter formatter = new PageFormatter(); - PageReadStore pageStore; - int rowGroupNum = 0; - while ((pageStore = reader.readNextRowGroup()) != null) { - for (ColumnDescriptor descriptor : columns.keySet()) { - List lines = formatted.get(columnName(descriptor)); - if (lines == null) { - lines = Lists.newArrayList(); - formatted.put(columnName(descriptor), lines); + try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) { + MessageType schema = reader.getFileMetaData().getSchema(); + Map columns = Maps.newLinkedHashMap(); + if (this.columns == null || this.columns.isEmpty()) { + for (ColumnDescriptor descriptor : schema.getColumns()) { + columns.put(descriptor, primitive(schema, descriptor.getPath())); } - - formatter.setContext(rowGroupNum, columns.get(descriptor), codec); - PageReader pages = pageStore.getPageReader(descriptor); - - DictionaryPage dict = pages.readDictionaryPage(); - if (dict != null) { - lines.add(formatter.format(dict)); + } else { + for (String column : this.columns) { + columns.put(descriptor(column, schema), primitive(column, schema)); } - DataPage page; - while ((page = pages.readPage()) != null) { - lines.add(formatter.format(page)); + } + + CompressionCodecName codec = reader.getRowGroups().get(0).getColumns().get(0).getCodec(); + // accumulate formatted lines to print by column + Map> formatted = Maps.newLinkedHashMap(); + PageFormatter formatter = new PageFormatter(); + PageReadStore pageStore; + int rowGroupNum = 0; + while ((pageStore = reader.readNextRowGroup()) != null) { + for (ColumnDescriptor descriptor : columns.keySet()) { + List lines = formatted.get(columnName(descriptor)); + if (lines == null) { + lines = Lists.newArrayList(); + formatted.put(columnName(descriptor), lines); + } + + formatter.setContext(rowGroupNum, columns.get(descriptor), codec); + PageReader pages = pageStore.getPageReader(descriptor); + + DictionaryPage dict = pages.readDictionaryPage(); + if (dict != null) { + lines.add(formatter.format(dict)); + } + DataPage page; + while ((page = pages.readPage()) != null) { + lines.add(formatter.format(page)); + } } + rowGroupNum += 1; } - rowGroupNum += 1; - } - // TODO: Show total column size and overall size per value in the column summary line - for (String columnName : formatted.keySet()) { - console.info(String.format("\nColumn: %s\n%s", columnName, new TextStringBuilder(80).appendPadding(80, '-'))); - console.info(formatter.getHeader()); - for (String line : formatted.get(columnName)) { - console.info(line); + // TODO: Show total column size and overall size per value in the column summary line + for (String columnName : formatted.keySet()) { + console.info(String.format("\nColumn: %s\n%s", columnName, new TextStringBuilder(80).appendPadding(80, '-'))); + console.info(formatter.getHeader()); + for (String line : formatted.get(columnName)) { + console.info(line); + } + console.info(""); } - console.info(""); } return 0; diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java index a246a52c73..afbdf7637e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java @@ -887,7 +887,9 @@ public void endBlock() throws IOException { */ @Deprecated public void appendFile(Configuration conf, Path file) throws IOException { - ParquetFileReader.open(conf, file).appendTo(this); + try (ParquetFileReader reader = ParquetFileReader.open(conf, file)) { + reader.appendTo(this); + } } public void appendFile(InputFile file) throws IOException { diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java index 73ef70e462..5b8c5ed1b4 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java @@ -204,35 +204,37 @@ public void testWriteRead() throws Exception { assertEquals(expectedEncoding,rowGroup.getColumns().get(0).getEncodings()); { // read first block of col #1 - ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, - Arrays.asList(rowGroup), Arrays.asList(SCHEMA.getColumnDescription(PATH1))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - assertNull(r.readNextRowGroup()); + try (ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, + Arrays.asList(rowGroup), Arrays.asList(SCHEMA.getColumnDescription(PATH1)))) { + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + assertNull(r.readNextRowGroup()); + } } { // read all blocks of col #1 and #2 - ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, - readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + try (ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, + readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)))) { - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); - pages = r.readNextRowGroup(); - assertEquals(4, pages.getRowCount()); + pages = r.readNextRowGroup(); + assertEquals(4, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); - validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); + validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); + validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); - assertNull(r.readNextRowGroup()); + assertNull(r.readNextRowGroup()); + } } PrintFooter.main(new String[] {path.toString()}); } @@ -281,12 +283,14 @@ public void testBloomFilterWriteRead() throws Exception { w.endBlock(); w.end(new HashMap<>()); ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, path); - ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, - Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(schema.getColumnDescription(colPath))); - BloomFilterReader bloomFilterReader = r.getBloomFilterDataReader(readFooter.getBlocks().get(0)); - BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(readFooter.getBlocks().get(0).getColumns().get(0)); - assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("hello")))); - assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("world")))); + + try (ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, + Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(schema.getColumnDescription(colPath)))) { + BloomFilterReader bloomFilterReader = r.getBloomFilterDataReader(readFooter.getBlocks().get(0)); + BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(readFooter.getBlocks().get(0).getColumns().get(0)); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("hello")))); + assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("world")))); + } } @Test @@ -340,16 +344,16 @@ public void testWriteReadDataPageV2() throws Exception { expectedEncoding.add(PLAIN); assertEquals(expectedEncoding, readFooter.getBlocks().get(0).getColumns().get(0).getEncodings()); - ParquetFileReader reader = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, - readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); - - PageReadStore pages = reader.readNextRowGroup(); - assertEquals(14, pages.getRowCount()); - validateV2Page(SCHEMA, pages, PATH1, 3, 4, 1, repLevels.toByteArray(), defLevels.toByteArray(), data.toByteArray(), 12); - validateV2Page(SCHEMA, pages, PATH1, 3, 3, 0, repLevels.toByteArray(), defLevels.toByteArray(),data.toByteArray(), 12); - validateV2Page(SCHEMA, pages, PATH2, 3, 5, 2, repLevels.toByteArray(), defLevels.toByteArray(), data2.toByteArray(), 12); - validateV2Page(SCHEMA, pages, PATH2, 2, 2, 0, repLevels.toByteArray(), defLevels.toByteArray(), data2.toByteArray(), 12); - assertNull(reader.readNextRowGroup()); + try (ParquetFileReader reader = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, + readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)))) { + PageReadStore pages = reader.readNextRowGroup(); + assertEquals(14, pages.getRowCount()); + validateV2Page(SCHEMA, pages, PATH1, 3, 4, 1, repLevels.toByteArray(), defLevels.toByteArray(), data.toByteArray(), 12); + validateV2Page(SCHEMA, pages, PATH1, 3, 3, 0, repLevels.toByteArray(), defLevels.toByteArray(), data.toByteArray(), 12); + validateV2Page(SCHEMA, pages, PATH2, 3, 5, 2, repLevels.toByteArray(), defLevels.toByteArray(), data2.toByteArray(), 12); + validateV2Page(SCHEMA, pages, PATH2, 2, 2, 0, repLevels.toByteArray(), defLevels.toByteArray(), data2.toByteArray(), 12); + assertNull(reader.readNextRowGroup()); + } } @Test @@ -426,35 +430,37 @@ public void testAlignmentWithPadding() throws Exception { 120, readFooter.getBlocks().get(1).getStartingPos()); { // read first block of col #1 - ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, - Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - assertNull(r.readNextRowGroup()); + try (ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, + Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1)))) { + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + assertNull(r.readNextRowGroup()); + } } { // read all blocks of col #1 and #2 - ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, - readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + try (ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, + readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)))) { - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); - pages = r.readNextRowGroup(); - assertEquals(4, pages.getRowCount()); + pages = r.readNextRowGroup(); + assertEquals(4, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); - validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); + validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); + validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); - assertNull(r.readNextRowGroup()); + assertNull(r.readNextRowGroup()); + } } PrintFooter.main(new String[] {path.toString()}); } @@ -533,35 +539,36 @@ public void testAlignmentWithNoPaddingNeeded() throws Exception { 109, readFooter.getBlocks().get(1).getStartingPos()); { // read first block of col #1 - ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, - Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - assertNull(r.readNextRowGroup()); + try (ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, + Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(SCHEMA.getColumnDescription(PATH1)))) { + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + assertNull(r.readNextRowGroup()); + } } { // read all blocks of col #1 and #2 - ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, - readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); + try (ParquetFileReader r = new ParquetFileReader(conf, readFooter.getFileMetaData(), path, + readFooter.getBlocks(), Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2)))) { + PageReadStore pages = r.readNextRowGroup(); + assertEquals(3, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); + validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); + validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(SCHEMA, pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH1, 3, BytesInput.from(BYTES1)); - validateContains(SCHEMA, pages, PATH2, 2, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 3, BytesInput.from(BYTES2)); - validateContains(SCHEMA, pages, PATH2, 1, BytesInput.from(BYTES2)); + pages = r.readNextRowGroup(); + assertEquals(4, pages.getRowCount()); - pages = r.readNextRowGroup(); - assertEquals(4, pages.getRowCount()); + validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); + validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); - validateContains(SCHEMA, pages, PATH1, 7, BytesInput.from(BYTES3)); - validateContains(SCHEMA, pages, PATH2, 8, BytesInput.from(BYTES4)); - - assertNull(r.readNextRowGroup()); + assertNull(r.readNextRowGroup()); + } } PrintFooter.main(new String[] {path.toString()}); } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java index de53e96264..673d5503c7 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java @@ -269,14 +269,15 @@ public void testParquetFileWithBloomFilter() throws IOException { } } - ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration())); - BlockMetaData blockMetaData = reader.getFooter().getBlocks().get(0); - BloomFilter bloomFilter = reader.getBloomFilterDataReader(blockMetaData) - .readBloomFilter(blockMetaData.getColumns().get(0)); + try (ParquetFileReader reader = ParquetFileReader.open(HadoopInputFile.fromPath(path, new Configuration()))) { + BlockMetaData blockMetaData = reader.getFooter().getBlocks().get(0); + BloomFilter bloomFilter = reader.getBloomFilterDataReader(blockMetaData) + .readBloomFilter(blockMetaData.getColumns().get(0)); - for (String name: testNames) { - assertTrue(bloomFilter.findHash( - LongHashFunction.xx(0).hashBytes(Binary.fromString(name).toByteBuffer()))); + for (String name : testNames) { + assertTrue(bloomFilter.findHash( + LongHashFunction.xx(0).hashBytes(Binary.fromString(name).toByteBuffer()))); + } } } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestReadWriteEncodingStats.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestReadWriteEncodingStats.java index 69e11c14e9..fdb7c8677d 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestReadWriteEncodingStats.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestReadWriteEncodingStats.java @@ -93,29 +93,30 @@ public void testReadWrite() throws Exception { writeData(writer); writer.close(); - ParquetFileReader reader = ParquetFileReader.open(CONF, path); - assertEquals("Should have one row group", 1, reader.getRowGroups().size()); - BlockMetaData rowGroup = reader.getRowGroups().get(0); - - ColumnChunkMetaData dictColumn = rowGroup.getColumns().get(0); - EncodingStats dictStats = dictColumn.getEncodingStats(); - assertNotNull("Dict column should have non-null encoding stats", dictStats); - assertTrue("Dict column should have a dict page", dictStats.hasDictionaryPages()); - assertTrue("Dict column should have dict-encoded pages", dictStats.hasDictionaryEncodedPages()); - assertFalse("Dict column should not have non-dict pages", dictStats.hasNonDictionaryEncodedPages()); - - ColumnChunkMetaData plainColumn = rowGroup.getColumns().get(1); - EncodingStats plainStats = plainColumn.getEncodingStats(); - assertNotNull("Plain column should have non-null encoding stats", plainStats); - assertFalse("Plain column should not have a dict page", plainStats.hasDictionaryPages()); - assertFalse("Plain column should not have dict-encoded pages", plainStats.hasDictionaryEncodedPages()); - assertTrue("Plain column should have non-dict pages", plainStats.hasNonDictionaryEncodedPages()); - - ColumnChunkMetaData fallbackColumn = rowGroup.getColumns().get(2); - EncodingStats fallbackStats = fallbackColumn.getEncodingStats(); - assertNotNull("Fallback column should have non-null encoding stats", fallbackStats); - assertTrue("Fallback column should have a dict page", fallbackStats.hasDictionaryPages()); - assertTrue("Fallback column should have dict-encoded pages", fallbackStats.hasDictionaryEncodedPages()); - assertTrue("Fallback column should have non-dict pages", fallbackStats.hasNonDictionaryEncodedPages()); + try (ParquetFileReader reader = ParquetFileReader.open(CONF, path)) { + assertEquals("Should have one row group", 1, reader.getRowGroups().size()); + BlockMetaData rowGroup = reader.getRowGroups().get(0); + + ColumnChunkMetaData dictColumn = rowGroup.getColumns().get(0); + EncodingStats dictStats = dictColumn.getEncodingStats(); + assertNotNull("Dict column should have non-null encoding stats", dictStats); + assertTrue("Dict column should have a dict page", dictStats.hasDictionaryPages()); + assertTrue("Dict column should have dict-encoded pages", dictStats.hasDictionaryEncodedPages()); + assertFalse("Dict column should not have non-dict pages", dictStats.hasNonDictionaryEncodedPages()); + + ColumnChunkMetaData plainColumn = rowGroup.getColumns().get(1); + EncodingStats plainStats = plainColumn.getEncodingStats(); + assertNotNull("Plain column should have non-null encoding stats", plainStats); + assertFalse("Plain column should not have a dict page", plainStats.hasDictionaryPages()); + assertFalse("Plain column should not have dict-encoded pages", plainStats.hasDictionaryEncodedPages()); + assertTrue("Plain column should have non-dict pages", plainStats.hasNonDictionaryEncodedPages()); + + ColumnChunkMetaData fallbackColumn = rowGroup.getColumns().get(2); + EncodingStats fallbackStats = fallbackColumn.getEncodingStats(); + assertNotNull("Fallback column should have non-null encoding stats", fallbackStats); + assertTrue("Fallback column should have a dict page", fallbackStats.hasDictionaryPages()); + assertTrue("Fallback column should have dict-encoded pages", fallbackStats.hasDictionaryEncodedPages()); + assertTrue("Fallback column should have non-dict pages", fallbackStats.hasNonDictionaryEncodedPages()); + } } } From 413d1dd4552dba5a7d66647747d17ef573d201c4 Mon Sep 17 00:00:00 2001 From: belugabehr <12578579+belugabehr@users.noreply.github.com> Date: Mon, 9 Aug 2021 11:21:04 -0400 Subject: [PATCH 08/36] PARQUET-2072: Do Not Determine Both Min/Max for Binary Stats (#920) --- .../parquet/column/statistics/BinaryStatistics.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java b/parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java index 6cd5395c7f..7715c079c6 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java @@ -54,9 +54,13 @@ private BinaryStatistics(BinaryStatistics other) { @Override public void updateStats(Binary value) { if (!this.hasNonNullValue()) { - initializeStats(value, value); - } else { - updateStats(value, value); + min = value.copy(); + max = value.copy(); + this.markAsNotEmpty(); + } else if (comparator().compare(min, value) > 0) { + min = value.copy(); + } else if (comparator().compare(max, value) < 0) { + max = value.copy(); } } From 6e72dd463aed5eeabb87090cb200695755cd6a92 Mon Sep 17 00:00:00 2001 From: Jiang Yang <37145547+Ted-Jiang@users.noreply.github.com> Date: Mon, 16 Aug 2021 17:23:04 +0800 Subject: [PATCH 09/36] PARQUET-2073: Fix estimate remaining row count in ColumnWriteStoreBase. (#922) --- .../org/apache/parquet/column/impl/ColumnWriteStoreBase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java index a5e7836b3a..8cfdace8de 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java @@ -237,10 +237,11 @@ private void sizeCheck() { } else { rowCountForNextRowCountCheck = min(rowCountForNextRowCountCheck, writer.getRowsWrittenSoFar() + pageRowCountLimit); } + //estimate remaining row count by previous input for next row count check long rowsToFillPage = usedMem == 0 ? props.getMaxRowCountForPageSizeCheck() - : (long) rows / usedMem * remainingMem; + : rows * remainingMem / usedMem; if (rowsToFillPage < minRecordToWait) { minRecordToWait = rowsToFillPage; } From 615d769afdbab96d01a7e91e4d4b406f83b4e2d6 Mon Sep 17 00:00:00 2001 From: loudongfeng Date: Fri, 10 Sep 2021 02:17:04 +0800 Subject: [PATCH 10/36] =?UTF-8?q?PARQUET-2078:=20Failed=20to=20read=20parq?= =?UTF-8?q?uet=20file=20after=20writing=20with=20the=20same=20=E2=80=A6=20?= =?UTF-8?q?(#925)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * PARQUET-2078 Failed to read parquet file after writing with the same parquet version * PARQUET-2078 Failed to read parquet file after writing with the same parquet version Read path fix that make usage of this information: RowGroup[n].file_offset = RowGroup[n-1].file_offset + RowGroup[n-1].total_compressed_size * PARQUET-2078 Failed to read parquet file after writing with the same parquet version addressing review comments: more check on writer side. * PARQUET-2078 Failed to read parquet file after writing with the same parquet version taking alignment padding and sumarry file into account * PARQUET-2078 Failed to read parquet file after writing with the same parquet version only throw exception when: 1.footer(first column of block meta) encrypted and 2.file_offset corrupted * PARQUET-2078 Failed to read parquet file after writing with the same parquet version only check firstColumnChunk.isSetMeta_data() for the first block * PARQUET-2078 Failed to read parquet file after writing with the same parquet version address review comments: empty lines * PARQUET-2078 Failed to read parquet file after writing with the same parquet version check first rowgroup's file_offset too(SPARK-36696) * PARQUET-2078 Failed to read parquet file after writing with the same parquet version Using Preconditions.checkState instead of assert in write path remove summary file footers case check in read path(which will never happen) * PARQUET-2078 Failed to read parquet file after writing with the same parquet version more special case for first row group --- .../io/InvalidFileOffsetException.java | 29 ++++++ .../converter/ParquetMetadataConverter.java | 94 +++++++++++++++++-- .../parquet/hadoop/ParquetFileWriter.java | 1 + .../parquet/hadoop/TestParquetFileWriter.java | 83 ++++++++++++++++ 4 files changed, 200 insertions(+), 7 deletions(-) create mode 100644 parquet-column/src/main/java/org/apache/parquet/io/InvalidFileOffsetException.java diff --git a/parquet-column/src/main/java/org/apache/parquet/io/InvalidFileOffsetException.java b/parquet-column/src/main/java/org/apache/parquet/io/InvalidFileOffsetException.java new file mode 100644 index 0000000000..9c0cbe3999 --- /dev/null +++ b/parquet-column/src/main/java/org/apache/parquet/io/InvalidFileOffsetException.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.parquet.io; + +import org.apache.parquet.ParquetRuntimeException; + +public class InvalidFileOffsetException extends ParquetRuntimeException { + private static final long serialVersionUID = 1L; + + public InvalidFileOffsetException(String message) { + super(message); + } +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java index 3a10b1c895..3c6e32cfde 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java @@ -45,6 +45,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.parquet.CorruptStatistics; import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.Preconditions; import org.apache.parquet.bytes.ByteBufferInputStream; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.column.statistics.BinaryStatistics; @@ -119,10 +120,10 @@ import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder; import org.apache.parquet.internal.column.columnindex.OffsetIndexBuilder; import org.apache.parquet.internal.hadoop.metadata.IndexReference; +import org.apache.parquet.io.InvalidFileOffsetException; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.ColumnOrder.ColumnOrderName; -import org.apache.parquet.schema.LogicalTypeAnnotation.LogicalTypeAnnotationVisitor; import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; @@ -201,8 +202,22 @@ public FileMetaData toParquetMetadata(int currentVersion, ParquetMetadata parque List blocks = parquetMetadata.getBlocks(); List rowGroups = new ArrayList(); long numRows = 0; + long preBlockStartPos = 0; + long preBlockCompressedSize = 0; for (BlockMetaData block : blocks) { numRows += block.getRowCount(); + long blockStartPos = block.getStartingPos(); + // first block + if (blockStartPos == 4) { + preBlockStartPos = 0; + preBlockCompressedSize = 0; + } + if (preBlockStartPos != 0) { + Preconditions.checkState(blockStartPos >= preBlockStartPos + preBlockCompressedSize, + "Invalid block starting position:" + blockStartPos); + } + preBlockStartPos = blockStartPos; + preBlockCompressedSize = block.getCompressedSize(); addRowGroup(parquetMetadata, rowGroups, block, fileEncryptor); } FileMetaData fileMetaData = new FileMetaData( @@ -1226,14 +1241,36 @@ public ParquetMetadata readParquetMetadata(InputStream from) throws IOException static FileMetaData filterFileMetaDataByMidpoint(FileMetaData metaData, RangeMetadataFilter filter) { List rowGroups = metaData.getRow_groups(); List newRowGroups = new ArrayList(); + long preStartIndex = 0; + long preCompressedSize = 0; + boolean firstColumnWithMetadata = true; + if (rowGroups != null && rowGroups.size() > 0) { + firstColumnWithMetadata = rowGroups.get(0).getColumns().get(0).isSetMeta_data(); + } for (RowGroup rowGroup : rowGroups) { long totalSize = 0; long startIndex; + ColumnChunk columnChunk = rowGroup.getColumns().get(0); + if (firstColumnWithMetadata) { + startIndex = getOffset(columnChunk); + } else { + assert rowGroup.isSetFile_offset(); + assert rowGroup.isSetTotal_compressed_size(); - if (rowGroup.isSetFile_offset()) { + //the file_offset of first block always holds the truth, while other blocks don't : + //see PARQUET-2078 for details startIndex = rowGroup.getFile_offset(); - } else { - startIndex = getOffset(rowGroup.getColumns().get(0)); + if (invalidFileOffset(startIndex, preStartIndex, preCompressedSize)) { + //first row group's offset is always 4 + if (preStartIndex == 0) { + startIndex = 4; + } else { + // use minStartIndex(imprecise in case of padding, but good enough for filtering) + startIndex = preStartIndex + preCompressedSize; + } + } + preStartIndex = startIndex; + preCompressedSize = rowGroup.getTotal_compressed_size(); } if (rowGroup.isSetTotal_compressed_size()) { @@ -1254,16 +1291,59 @@ static FileMetaData filterFileMetaDataByMidpoint(FileMetaData metaData, RangeMet return metaData; } + private static boolean invalidFileOffset(long startIndex, long preStartIndex, long preCompressedSize) { + boolean invalid = false; + assert preStartIndex <= startIndex; + // checking the first rowGroup + if (preStartIndex == 0 && startIndex != 4) { + invalid = true; + return invalid; + } + + //calculate start index for other blocks + long minStartIndex = preStartIndex + preCompressedSize; + if (startIndex < minStartIndex) { + // a bad offset detected, try first column's offset + // can not use minStartIndex in case of padding + invalid = true; + } + + return invalid; + } + // Visible for testing static FileMetaData filterFileMetaDataByStart(FileMetaData metaData, OffsetMetadataFilter filter) { List rowGroups = metaData.getRow_groups(); List newRowGroups = new ArrayList(); + long preStartIndex = 0; + long preCompressedSize = 0; + boolean firstColumnWithMetadata = true; + if (rowGroups != null && rowGroups.size() > 0) { + firstColumnWithMetadata = rowGroups.get(0).getColumns().get(0).isSetMeta_data(); + } for (RowGroup rowGroup : rowGroups) { long startIndex; - if (rowGroup.isSetFile_offset()) { - startIndex = rowGroup.getFile_offset(); + ColumnChunk columnChunk = rowGroup.getColumns().get(0); + if (firstColumnWithMetadata) { + startIndex = getOffset(columnChunk); } else { - startIndex = getOffset(rowGroup.getColumns().get(0)); + assert rowGroup.isSetFile_offset(); + assert rowGroup.isSetTotal_compressed_size(); + + //the file_offset of first block always holds the truth, while other blocks don't : + //see PARQUET-2078 for details + startIndex = rowGroup.getFile_offset(); + if (invalidFileOffset(startIndex, preStartIndex, preCompressedSize)) { + //first row group's offset is always 4 + if (preStartIndex == 0) { + startIndex = 4; + } else { + throw new InvalidFileOffsetException("corrupted RowGroup.file_offset found, " + + "please use file range instead of block offset for split."); + } + } + preStartIndex = startIndex; + preCompressedSize = rowGroup.getTotal_compressed_size(); } if (filter.contains(startIndex)) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java index afbdf7637e..2e5d55c28c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java @@ -852,6 +852,7 @@ public void endColumn() throws IOException { this.currentBlock.setTotalByteSize(currentBlock.getTotalByteSize() + uncompressedLength); this.uncompressedLength = 0; this.compressedLength = 0; + this.currentChunkDictionaryPageOffset = 0; columnIndexBuilder = null; offsetIndexBuilder = null; } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java index 5b8c5ed1b4..8dcbf4acf4 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java @@ -23,6 +23,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Version; import org.apache.parquet.bytes.BytesUtils; @@ -30,6 +34,8 @@ import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; import org.apache.parquet.column.values.bloomfilter.BloomFilter; import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.util.ContextUtil; import org.apache.parquet.io.ParquetEncodingException; import org.junit.Assume; import org.junit.Rule; @@ -67,6 +73,7 @@ import static org.apache.parquet.CorruptStatistics.shouldIgnoreStatistics; import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; +import static org.apache.parquet.hadoop.ParquetInputFormat.READ_SUPPORT_CLASS; import static org.junit.Assert.*; import static org.apache.parquet.column.Encoding.BIT_PACKED; import static org.apache.parquet.column.Encoding.PLAIN; @@ -75,6 +82,7 @@ import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; import static org.apache.parquet.schema.Type.Repetition.*; import static org.apache.parquet.hadoop.TestUtils.enforceEmptyDir; +import static org.junit.Assert.assertEquals; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroup; @@ -239,6 +247,81 @@ public void testWriteRead() throws Exception { PrintFooter.main(new String[] {path.toString()}); } + @Test + public void testWriteReadWithRecordReader() throws Exception { + File testFile = temp.newFile(); + testFile.delete(); + + Path path = new Path(testFile.toURI()); + Configuration configuration = new Configuration(); + + ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); + w.start(); + w.startBlock(3); + w.startColumn(C1, 5, CODEC); + w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.startColumn(C2, 6, CODEC); + long c2Starts = w.getPos(); + w.writeDictionaryPage(new DictionaryPage(BytesInput.from(BYTES2), 4, RLE_DICTIONARY)); + long c2p1Starts = w.getPos(); + w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + long c2Ends = w.getPos(); + w.endBlock(); + w.startBlock(4); + w.startColumn(C1, 7, CODEC); + long c1Bock2Starts = w.getPos(); + long c1p1Bock2Starts = w.getPos(); + w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + long c1Block2Ends = w.getPos(); + w.startColumn(C2, 8, CODEC); + w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap()); + + ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, path); + assertEquals("footer: "+ readFooter, 2, readFooter.getBlocks().size()); + BlockMetaData rowGroup = readFooter.getBlocks().get(0); + assertEquals(c2Ends - c2Starts, rowGroup.getColumns().get(1).getTotalSize()); + + assertEquals(0, rowGroup.getColumns().get(0).getDictionaryPageOffset()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getStartingPos()); + assertEquals(c2Starts, rowGroup.getColumns().get(1).getDictionaryPageOffset()); + assertEquals(c2p1Starts, rowGroup.getColumns().get(1).getFirstDataPageOffset()); + + BlockMetaData rowGroup2 = readFooter.getBlocks().get(1); + assertEquals(0, rowGroup2.getColumns().get(0).getDictionaryPageOffset()); + assertEquals(c1Bock2Starts, rowGroup2.getColumns().get(0).getStartingPos()); + assertEquals(c1p1Bock2Starts, rowGroup2.getColumns().get(0).getFirstDataPageOffset()); + assertEquals(c1Block2Ends - c1Bock2Starts, rowGroup2.getColumns().get(0).getTotalSize()); + + HashSet expectedEncoding=new HashSet(); + expectedEncoding.add(PLAIN); + expectedEncoding.add(BIT_PACKED); + assertEquals(expectedEncoding,rowGroup.getColumns().get(0).getEncodings()); + + ParquetInputSplit split = new ParquetInputSplit(path, 0, w.getPos(),null, + readFooter.getBlocks(), SCHEMA.toString(), + readFooter.getFileMetaData().getSchema().toString(), + readFooter.getFileMetaData().getKeyValueMetaData(), + null); + ParquetInputFormat input = new ParquetInputFormat(); + configuration.set(READ_SUPPORT_CLASS, GroupReadSupport.class.getName()); + TaskAttemptID taskAttemptID = TaskAttemptID.forName("attempt_0_1_m_1_1"); + TaskAttemptContext taskContext = ContextUtil.newTaskAttemptContext(configuration, taskAttemptID); + RecordReader reader = input.createRecordReader(split, taskContext); + assertTrue(reader instanceof ParquetRecordReader); + //RowGroup.file_offset is checked here + reader.initialize(split, taskContext); + reader.close(); + } + @Test public void testWriteEmptyBlock() throws Exception { File testFile = temp.newFile(); From 0d6a49e94e9c3e9efc61d72dc69a69d509c9e4a9 Mon Sep 17 00:00:00 2001 From: Xinli Shang Date: Fri, 10 Sep 2021 09:19:10 -0700 Subject: [PATCH 11/36] Update CHANGES.md for 1.12.1 (#931) --- CHANGES.md | 73 ++++++++++++++++++++++++++++++++++-------------------- 1 file changed, 46 insertions(+), 27 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 7785db5486..9d752ed924 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -19,6 +19,25 @@ # Parquet # +### Version 1.12.1 ### + +Release Notes - Parquet - Version 1.12.1 + +#### Bug + +* [PARQUET-1633](https://issues.apache.org/jira/browse/PARQUET-1633) - Fix integer overflow +* [PARQUET-2022](https://issues.apache.org/jira/browse/PARQUET-2022) - ZstdDecompressorStream should close zstdInputStream +* [PARQUET-2027](https://issues.apache.org/jira/browse/PARQUET-2027) - Fix calculating directory offset for merge +* [PARQUET-2052](https://issues.apache.org/jira/browse/PARQUET-2052) - Integer overflow when writing huge binary using dictionary encoding +* [PARQUET-2054](https://issues.apache.org/jira/browse/PARQUET-2054) - fix TCP leaking when calling ParquetFileWriter.appendFile +* [PARQUET-2072](https://issues.apache.org/jira/browse/PARQUET-2072) - Do Not Determine Both Min/Max for Binary Stats +* [PARQUET-2073](https://issues.apache.org/jira/browse/PARQUET-2073) - Fix estimate remaining row count in ColumnWriteStoreBase. +* [PARQUET-2078](https://issues.apache.org/jira/browse/PARQUET-2078) - Failed to read parquet file after writing with the same parquet version + +#### Improvement + +* [PARQUET-2064](https://issues.apache.org/jira/browse/PARQUET-2064) - Make Range public accessible in RowRanges + ### Version 1.12.0 ### Release Notes - Parquet - Version 1.12.0 @@ -739,7 +758,7 @@ Release Notes - Parquet - Version 1.10.0 * ISSUE [346](https://github.com/Parquet/parquet-mr/pull/346): stop using strings and b64 for compressed input splits * ISSUE [345](https://github.com/Parquet/parquet-mr/pull/345): set cascading version to 2.5.3 * ISSUE [342](https://github.com/Parquet/parquet-mr/pull/342): compress kv pairs in ParquetInputSplits - + ### Version 1.4.0 ### * ISSUE [333](https://github.com/Parquet/parquet-mr/pull/333): Compress schemas in split * ISSUE [329](https://github.com/Parquet/parquet-mr/pull/329): fix filesystem resolution @@ -879,37 +898,37 @@ Release Notes - Parquet - Version 1.10.0 * ISSUE 159: Counter for mapred * ISSUE 156: Fix site * ISSUE 153: Fix projection required field - + ### Version 1.1.1 ### * ISSUE 150: add thrift validation on read ### Version 1.1.0 ### -* ISSUE 149: changing default block size to 128mb -* ISSUE 146: Fix and add unit tests for Hive nested types -* ISSUE 145: add getStatistics method to parquetloader -* ISSUE 144: Map key fields should allow other types than strings -* ISSUE 143: Fix empty encoding col metadata -* ISSUE 142: Fix total size row group -* ISSUE 141: add parquet counters for benchmark -* ISSUE 140: Implemented partial schema for GroupReadSupport -* ISSUE 138: fix bug of wrong column metadata size -* ISSUE 137: ParquetMetadataConverter bug -* ISSUE 133: Update plugin versions for maven aether migration - fixes #125 -* ISSUE 130: Schema validation should not validate the root element's name -* ISSUE 127: Adding dictionary encoding for non string types.. #99 -* ISSUE 125: Unable to build -* ISSUE 124: Fix Short and Byte types in Hive SerDe. -* ISSUE 123: Fix Snappy compressor in parquet-hadoop. -* ISSUE 120: Fix RLE bug with partial literal groups at end of stream. -* ISSUE 118: Refactor column reader -* ISSUE 115: Map key fields should allow other types than strings -* ISSUE 103: Map key fields should allow other types than strings -* ISSUE 99: Dictionary encoding for non string types (float double int long boolean) -* ISSUE 47: Add tests for parquet-scrooge and parquet-cascading +* ISSUE 149: changing default block size to 128mb +* ISSUE 146: Fix and add unit tests for Hive nested types +* ISSUE 145: add getStatistics method to parquetloader +* ISSUE 144: Map key fields should allow other types than strings +* ISSUE 143: Fix empty encoding col metadata +* ISSUE 142: Fix total size row group +* ISSUE 141: add parquet counters for benchmark +* ISSUE 140: Implemented partial schema for GroupReadSupport +* ISSUE 138: fix bug of wrong column metadata size +* ISSUE 137: ParquetMetadataConverter bug +* ISSUE 133: Update plugin versions for maven aether migration - fixes #125 +* ISSUE 130: Schema validation should not validate the root element's name +* ISSUE 127: Adding dictionary encoding for non string types.. #99 +* ISSUE 125: Unable to build +* ISSUE 124: Fix Short and Byte types in Hive SerDe. +* ISSUE 123: Fix Snappy compressor in parquet-hadoop. +* ISSUE 120: Fix RLE bug with partial literal groups at end of stream. +* ISSUE 118: Refactor column reader +* ISSUE 115: Map key fields should allow other types than strings +* ISSUE 103: Map key fields should allow other types than strings +* ISSUE 99: Dictionary encoding for non string types (float double int long boolean) +* ISSUE 47: Add tests for parquet-scrooge and parquet-cascading ### Version 1.0.1 ### -* ISSUE 126: Unit tests for parquet cascading -* ISSUE 121: fix wrong RecordConverter for ParquetTBaseScheme -* ISSUE 119: fix compatibility with thrift remove unused dependency +* ISSUE 126: Unit tests for parquet cascading +* ISSUE 121: fix wrong RecordConverter for ParquetTBaseScheme +* ISSUE 119: fix compatibility with thrift remove unused dependency ### Version 1.0.0 ### From f5595fedc36524bab58e0d36b34be26eaa4177e6 Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Mon, 16 Aug 2021 11:19:36 +0200 Subject: [PATCH 12/36] PARQUET-2043: Fail for undeclared dependencies (#916) The purpose of this change is to fail the build if some classes are used from not direct dependencies. Only classes from direct dependencies shall be used. Also fixed some references that broke this rule. --- parquet-arrow/pom.xml | 20 ------ parquet-avro/pom.xml | 24 ++++--- parquet-benchmarks/pom.xml | 43 +++++++++---- parquet-cli/pom.xml | 62 ++++++++++++++++++- parquet-column/pom.xml | 10 +++ parquet-common/pom.xml | 6 -- parquet-encoding/pom.xml | 6 ++ parquet-format-structures/pom.xml | 12 +--- parquet-hadoop-bundle/pom.xml | 15 +++++ parquet-hadoop/pom.xml | 50 +++++++++++++-- .../SchemaControlEncryptionTest.java | 15 +++-- parquet-jackson/pom.xml | 15 +++++ parquet-pig-bundle/pom.xml | 15 +++++ parquet-pig/pom.xml | 32 +++++++--- parquet-protobuf/pom.xml | 16 ++++- parquet-scala/pom.xml | 6 ++ parquet-thrift/pom.xml | 42 +++++++++---- pom.xml | 55 ++++++++++++++++ 18 files changed, 353 insertions(+), 91 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index b7a94810e7..3ff10913dd 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -42,26 +42,6 @@ arrow-vector ${arrow.version} - - org.apache.arrow - arrow-memory - ${arrow.version} - - - org.apache.arrow - arrow-format - ${arrow.version} - - - org.apache.parquet - parquet-common - ${project.version} - - - org.apache.parquet - parquet-encoding - ${project.version} - org.apache.parquet parquet-column diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index e7b8e6b6af..3b851eacfa 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -45,7 +45,7 @@ org.apache.parquet - parquet-format-structures + parquet-common ${project.version} @@ -53,6 +53,11 @@ avro ${avro.version} + + org.slf4j + slf4j-api + ${slf4j.version} + it.unimi.dsi fastutil @@ -61,14 +66,17 @@ org.apache.hadoop hadoop-client - ${hadoop.version} provided - - - org.slf4j - slf4j-log4j12 - - + + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + org.apache.hadoop + hadoop-common + provided com.google.guava diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index ae16bfb3b1..dd8c2ee3bb 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -37,26 +37,24 @@ + + org.apache.parquet + parquet-hadoop + ${project.version} + org.apache.parquet - parquet-encoding + parquet-column ${project.version} - org.apache.parquet - parquet-hadoop - ${project.version} + org.apache.parquet + parquet-common + ${project.version} org.apache.hadoop hadoop-client - ${hadoop.version} - - - org.slf4j - slf4j-log4j12 - - org.openjdk.jmh @@ -79,6 +77,11 @@ fastutil ${fastutil.version} + + org.slf4j + slf4j-api + ${slf4j.version} + @@ -124,6 +127,24 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + true + + + + diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index d886951aee..fb416d6e5f 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -38,6 +38,26 @@ parquet-avro ${project.version} + + org.apache.parquet + parquet-format-structures + ${project.version} + + + org.apache.parquet + parquet-common + ${project.version} + + + org.apache.parquet + parquet-column + ${project.version} + + + org.apache.parquet + parquet-hadoop + ${project.version} + org.apache.avro avro @@ -47,6 +67,7 @@ com.github.luben zstd-jni ${zstd-jni.version} + runtime org.slf4j @@ -67,12 +88,23 @@ org.apache.parquet parquet-jackson ${project.version} + runtime ${jackson.groupId} jackson-databind ${jackson-databind.version} + + ${jackson.groupId} + jackson-core + ${jackson.version} + + + ${jackson.groupId} + jackson-annotations + ${jackson.version} + com.beust jcommander @@ -93,7 +125,35 @@ org.apache.hadoop hadoop-client - ${hadoop.version} + provided + + + org.apache.hadoop + hadoop-common + provided + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + provided + + + log4j + log4j + 1.2.17 + provided + + + commons-io + commons-io + 2.4 + provided + + + commons-logging + commons-logging + 1.1.3 provided diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index cb0c1853b5..d00a7ced92 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -70,6 +70,16 @@ zero-allocation-hashing ${net.openhft.version} + + org.apache.yetus + audience-annotations + ${yetus.audience-annotations.version} + + + org.slf4j + slf4j-api + ${slf4j.version} + com.carrotsearch diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index a3684464f5..b0bc410367 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -61,12 +61,6 @@ ${slf4j.version} test - - - org.apache.yetus - audience-annotations - 0.12.0 - diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index f1f7d9a26f..1452a68957 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -42,6 +42,12 @@ ${project.version} + + org.slf4j + slf4j-api + ${slf4j.version} + + org.slf4j slf4j-simple diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 4fb990ba4d..395a8cdaea 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -47,7 +47,7 @@ unpack - generate-sources + initialize unpack @@ -153,21 +153,11 @@ - - org.slf4j - slf4j-api - ${slf4j.version} - org.apache.thrift libthrift ${format.thrift.version} - - javax.annotation - javax.annotation-api - 1.3.2 - diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 4212ae00f6..7d408d89aa 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -84,6 +84,21 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + true + + + + diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index c11c56022c..8c14b73ff1 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -46,6 +46,11 @@ parquet-format-structures ${project.version} + + org.apache.parquet + parquet-common + ${project.version} + org.apache.parquet parquet-column @@ -56,19 +61,29 @@ org.apache.hadoop hadoop-client + provided + + + org.apache.hadoop + hadoop-common + provided + + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + org.apache.hadoop + hadoop-annotations ${hadoop.version} provided - - - org.slf4j - slf4j-log4j12 - - org.apache.parquet parquet-jackson ${project.version} + runtime ${jackson.groupId} @@ -127,6 +142,28 @@ 4.6.0 test + + org.slf4j + slf4j-api + ${slf4j.version} + + + commons-io + commons-io + 2.4 + test + + + net.openhft + zero-allocation-hashing + ${net.openhft.version} + test + + + org.apache.yetus + audience-annotations + ${yetus.audience-annotations.version} + @@ -159,6 +196,7 @@ com.github.rdblue brotli-codec ${brotli-codec.version} + runtime true diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java index 17fda97d19..862ae672c6 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java @@ -19,8 +19,6 @@ package org.apache.parquet.crypto.propertiesfactory; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.column.ColumnDescriptor; @@ -28,7 +26,6 @@ import org.apache.parquet.crypto.ParquetCipher; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroup; -import org.apache.parquet.format.EncryptionAlgorithm; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.api.WriteSupport; @@ -39,6 +36,8 @@ import org.apache.parquet.schema.PrimitiveType; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.file.Files; @@ -58,13 +57,13 @@ public class SchemaControlEncryptionTest { - private final static Log LOG = LogFactory.getLog(SchemaControlEncryptionTest.class); + private final static Logger LOG = LoggerFactory.getLogger(SchemaControlEncryptionTest.class); private final static int numRecord = 1000; private Random rnd = new Random(5); - + // In the test We use a map to tell WriteSupport which columns to be encrypted with what key. In real use cases, people - // can find whatever easy way to do so basing on how do they get these information, for example people can choose to - // store in HMS, or other metastore. + // can find whatever easy way to do so basing on how do they get these information, for example people can choose to + // store in HMS, or other metastore. private Map> cryptoMetadata = new HashMap<>(); private Map testData = new HashMap<>(); @@ -122,7 +121,7 @@ private void runTest(Configuration conf ) throws Exception { encryptParquetFile(file, conf); decryptParquetFileAndValid(file, conf); } - + private void markEncryptColumns() { Map ageMetadata = new HashMap<>(); ageMetadata.put("columnKeyMetaData", "age_key_id"); diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index c12e2477f6..16d6e4f7a9 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -90,6 +90,21 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + true + + + + diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index eede36938b..a10b4eb57a 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -74,6 +74,21 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + true + + + + diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index a7251846af..ef05c7b9b4 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -48,9 +48,15 @@ org.apache.parquet - parquet-format-structures + parquet-common ${project.version} + + org.apache.parquet + parquet-jackson + ${project.version} + runtime + org.apache.pig pig @@ -65,20 +71,27 @@ provided - org.apache.parquet - parquet-jackson - ${project.version} + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + provided - ${jackson.groupId} - jackson-core - ${jackson.version} + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided ${jackson.groupId} jackson-databind ${jackson-databind.version} + + ${jackson.groupId} + jackson-annotations + ${jackson.version} + org.apache.parquet parquet-column @@ -110,6 +123,11 @@ ${slf4j.version} test + + org.slf4j + slf4j-api + ${slf4j.version} + diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 990d5e9d03..2e82b4979f 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -87,9 +87,23 @@ org.apache.hadoop hadoop-client - ${hadoop.version} provided + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + org.apache.hadoop + hadoop-common + provided + + + org.slf4j + slf4j-api + ${slf4j.version} + org.slf4j slf4j-simple diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 5f5f759a4e..9786b10621 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -64,6 +64,12 @@ 3.0.1 test + + org.scalactic + scalactic_${scala.binary.version} + 3.0.1 + test + org.slf4j slf4j-simple diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 51e4aa3c3a..f0d8e77bba 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -47,17 +47,25 @@ parquet-hadoop ${project.version} + + org.apache.parquet + parquet-common + ${project.version} + org.apache.hadoop hadoop-client - ${hadoop.version} provided - - - org.slf4j - slf4j-log4j12 - - + + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + org.apache.hadoop + hadoop-common + provided com.twitter.elephantbird @@ -87,16 +95,22 @@ org.apache.parquet parquet-jackson ${project.version} + runtime ${jackson.groupId} - jackson-core - ${jackson.version} + jackson-databind + ${jackson-databind.version} ${jackson.groupId} - jackson-databind - ${jackson-databind.version} + jackson-annotations + ${jackson.version} + + + com.google.guava + guava + ${guava.version} org.apache.parquet @@ -135,6 +149,11 @@ ${thrift.version} provided + + org.slf4j + slf4j-api + ${slf4j.version} + org.slf4j slf4j-simple @@ -155,7 +174,6 @@ test-jar test - diff --git a/pom.xml b/pom.xml index 9f60ac9327..f28f7d7718 100644 --- a/pom.xml +++ b/pom.xml @@ -106,12 +106,14 @@ 0.9 3.1.0 1.6.0 + 0.13.0 2.3 1.72 1.4.9-1 1.8 + 3.0.2 INFO @@ -156,6 +158,44 @@ + + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.slf4j + slf4j-log4j12 + + + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + org.slf4j + slf4j-log4j12 + + + + + + @@ -521,6 +561,21 @@ + + org.apache.maven.plugins + maven-dependency-plugin + + + + analyze-only + + + true + true + + + + From 261e320329d719ef3c7f05ff2c3feb788d6ffd4f Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Fri, 10 Sep 2021 19:33:58 -0700 Subject: [PATCH 13/36] Remove unused dependency & Fix warning message --- parquet-cli/pom.xml | 5 ----- parquet-hadoop/pom.xml | 15 ++++++++++----- pom.xml | 4 ---- 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index fb416d6e5f..4a3c0eb80c 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -100,11 +100,6 @@ jackson-core ${jackson.version} - - ${jackson.groupId} - jackson-annotations - ${jackson.version} - com.beust jcommander diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 8c14b73ff1..e9b7b27e72 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -85,11 +85,6 @@ ${project.version} runtime - - ${jackson.groupId} - jackson-core - ${jackson.version} - ${jackson.groupId} jackson-databind @@ -164,6 +159,16 @@ audience-annotations ${yetus.audience-annotations.version} + + org.codehaus.jackson + jackson-core-asl + 1.9.13 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.13 + diff --git a/pom.xml b/pom.xml index f28f7d7718..f0b6a5f69b 100644 --- a/pom.xml +++ b/pom.xml @@ -123,8 +123,6 @@ parquet-arrow parquet-avro parquet-benchmarks - parquet-cascading-deprecated - parquet-cascading3-deprecated parquet-cli parquet-column parquet-common @@ -137,10 +135,8 @@ parquet-pig-bundle parquet-protobuf parquet-scala - parquet-scrooge-deprecated parquet-thrift parquet-hadoop-bundle - parquet-tools-deprecated From d1dccf6e680d86e94ce97005f5ac51848ba6d794 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Sat, 11 Sep 2021 09:23:03 -0700 Subject: [PATCH 14/36] [maven-release-plugin] prepare release apache-parquet-1.12.1-rc0 --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- pom.xml | 4 ++-- 18 files changed, 19 insertions(+), 19 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 3ff10913dd..79c0949dd5 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 3b851eacfa..b4d5eeab79 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index dd8c2ee3bb..71d42247ea 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 4a3c0eb80c..a8b92f7791 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index d00a7ced92..70523bf467 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index b0bc410367..03be915968 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 1452a68957..3e356103db 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 395a8cdaea..d3ac9f222c 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 1126c8ef88..c6a9fc4606 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 7d408d89aa..6043aaf8ec 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index e9b7b27e72..e8bbd65a4e 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 16d6e4f7a9..3fea2e49d7 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index a10b4eb57a..974e1b62c6 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index ef05c7b9b4..77564f4d00 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 2e82b4979f..bc7bc323d3 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 9786b10621..767effaa75 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index f0d8e77bba..2a7b871e5d 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/pom.xml b/pom.xml index f0b6a5f69b..19a6a5d077 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.1-SNAPSHOT + 1.12.1 pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - HEAD + apache-parquet-1.12.1-rc0 From 9455f48cc622e688b3301ec368726beadc28dac9 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Sat, 11 Sep 2021 09:23:20 -0700 Subject: [PATCH 15/36] [maven-release-plugin] prepare for next development iteration --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- pom.xml | 4 ++-- 18 files changed, 19 insertions(+), 19 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 79c0949dd5..3ff10913dd 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index b4d5eeab79..3b851eacfa 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 71d42247ea..dd8c2ee3bb 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index a8b92f7791..4a3c0eb80c 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 70523bf467..d00a7ced92 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 03be915968..b0bc410367 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 3e356103db..1452a68957 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index d3ac9f222c..395a8cdaea 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index c6a9fc4606..1126c8ef88 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 6043aaf8ec..7d408d89aa 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index e8bbd65a4e..e9b7b27e72 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 3fea2e49d7..16d6e4f7a9 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 974e1b62c6..a10b4eb57a 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index 77564f4d00..ef05c7b9b4 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index bc7bc323d3..2e82b4979f 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 767effaa75..9786b10621 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 2a7b871e5d..f0d8e77bba 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 19a6a5d077..f0b6a5f69b 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.1 + 1.12.1-SNAPSHOT pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - apache-parquet-1.12.1-rc0 + HEAD From afd490456881cac870be80bcdd2bcaca6ae338f7 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Mon, 13 Sep 2021 08:44:47 -0700 Subject: [PATCH 16/36] Revert "Remove unused dependency & Fix warning message" This reverts commit 261e320329d719ef3c7f05ff2c3feb788d6ffd4f. --- parquet-cli/pom.xml | 5 +++++ parquet-hadoop/pom.xml | 15 +++++---------- pom.xml | 4 ++++ 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 4a3c0eb80c..fb416d6e5f 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -100,6 +100,11 @@ jackson-core ${jackson.version} + + ${jackson.groupId} + jackson-annotations + ${jackson.version} + com.beust jcommander diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index e9b7b27e72..8c14b73ff1 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -85,6 +85,11 @@ ${project.version} runtime + + ${jackson.groupId} + jackson-core + ${jackson.version} + ${jackson.groupId} jackson-databind @@ -159,16 +164,6 @@ audience-annotations ${yetus.audience-annotations.version} - - org.codehaus.jackson - jackson-core-asl - 1.9.13 - - - org.codehaus.jackson - jackson-mapper-asl - 1.9.13 - diff --git a/pom.xml b/pom.xml index f0b6a5f69b..f28f7d7718 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,8 @@ parquet-arrow parquet-avro parquet-benchmarks + parquet-cascading-deprecated + parquet-cascading3-deprecated parquet-cli parquet-column parquet-common @@ -135,8 +137,10 @@ parquet-pig-bundle parquet-protobuf parquet-scala + parquet-scrooge-deprecated parquet-thrift parquet-hadoop-bundle + parquet-tools-deprecated From 64544c1ad5469373a2000b7f533414be6f815485 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Mon, 13 Sep 2021 09:16:34 -0700 Subject: [PATCH 17/36] Revert "PARQUET-2043: Fail for undeclared dependencies (#916)" --- parquet-arrow/pom.xml | 20 ++++++ parquet-avro/pom.xml | 24 +++---- parquet-benchmarks/pom.xml | 43 ++++--------- parquet-cli/pom.xml | 62 +------------------ parquet-column/pom.xml | 10 --- parquet-common/pom.xml | 6 ++ parquet-encoding/pom.xml | 6 -- parquet-format-structures/pom.xml | 12 +++- parquet-hadoop-bundle/pom.xml | 15 ----- parquet-hadoop/pom.xml | 50 ++------------- .../SchemaControlEncryptionTest.java | 15 ++--- parquet-jackson/pom.xml | 15 ----- parquet-pig-bundle/pom.xml | 15 ----- parquet-pig/pom.xml | 32 +++------- parquet-protobuf/pom.xml | 16 +---- parquet-scala/pom.xml | 6 -- parquet-thrift/pom.xml | 42 ++++--------- pom.xml | 55 ---------------- 18 files changed, 91 insertions(+), 353 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 3ff10913dd..b7a94810e7 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -42,6 +42,26 @@ arrow-vector ${arrow.version} + + org.apache.arrow + arrow-memory + ${arrow.version} + + + org.apache.arrow + arrow-format + ${arrow.version} + + + org.apache.parquet + parquet-common + ${project.version} + + + org.apache.parquet + parquet-encoding + ${project.version} + org.apache.parquet parquet-column diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 3b851eacfa..e7b8e6b6af 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -45,7 +45,7 @@ org.apache.parquet - parquet-common + parquet-format-structures ${project.version} @@ -53,11 +53,6 @@ avro ${avro.version} - - org.slf4j - slf4j-api - ${slf4j.version} - it.unimi.dsi fastutil @@ -66,17 +61,14 @@ org.apache.hadoop hadoop-client + ${hadoop.version} provided - - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - org.apache.hadoop - hadoop-common - provided + + + org.slf4j + slf4j-log4j12 + + com.google.guava diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index dd8c2ee3bb..ae16bfb3b1 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -37,24 +37,26 @@ - - org.apache.parquet - parquet-hadoop - ${project.version} - org.apache.parquet - parquet-column + parquet-encoding ${project.version} - org.apache.parquet - parquet-common - ${project.version} + org.apache.parquet + parquet-hadoop + ${project.version} org.apache.hadoop hadoop-client + ${hadoop.version} + + + org.slf4j + slf4j-log4j12 + + org.openjdk.jmh @@ -77,11 +79,6 @@ fastutil ${fastutil.version} - - org.slf4j - slf4j-api - ${slf4j.version} - @@ -127,24 +124,6 @@ - - - org.apache.maven.plugins - maven-dependency-plugin - - - - analyze-only - - - true - - - - diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index fb416d6e5f..d886951aee 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -38,26 +38,6 @@ parquet-avro ${project.version} - - org.apache.parquet - parquet-format-structures - ${project.version} - - - org.apache.parquet - parquet-common - ${project.version} - - - org.apache.parquet - parquet-column - ${project.version} - - - org.apache.parquet - parquet-hadoop - ${project.version} - org.apache.avro avro @@ -67,7 +47,6 @@ com.github.luben zstd-jni ${zstd-jni.version} - runtime org.slf4j @@ -88,23 +67,12 @@ org.apache.parquet parquet-jackson ${project.version} - runtime ${jackson.groupId} jackson-databind ${jackson-databind.version} - - ${jackson.groupId} - jackson-core - ${jackson.version} - - - ${jackson.groupId} - jackson-annotations - ${jackson.version} - com.beust jcommander @@ -125,35 +93,7 @@ org.apache.hadoop hadoop-client - provided - - - org.apache.hadoop - hadoop-common - provided - - - com.google.code.findbugs - jsr305 - ${jsr305.version} - provided - - - log4j - log4j - 1.2.17 - provided - - - commons-io - commons-io - 2.4 - provided - - - commons-logging - commons-logging - 1.1.3 + ${hadoop.version} provided diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index d00a7ced92..cb0c1853b5 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -70,16 +70,6 @@ zero-allocation-hashing ${net.openhft.version} - - org.apache.yetus - audience-annotations - ${yetus.audience-annotations.version} - - - org.slf4j - slf4j-api - ${slf4j.version} - com.carrotsearch diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index b0bc410367..a3684464f5 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -61,6 +61,12 @@ ${slf4j.version} test + + + org.apache.yetus + audience-annotations + 0.12.0 + diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 1452a68957..f1f7d9a26f 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -42,12 +42,6 @@ ${project.version} - - org.slf4j - slf4j-api - ${slf4j.version} - - org.slf4j slf4j-simple diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 395a8cdaea..4fb990ba4d 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -47,7 +47,7 @@ unpack - initialize + generate-sources unpack @@ -153,11 +153,21 @@ + + org.slf4j + slf4j-api + ${slf4j.version} + org.apache.thrift libthrift ${format.thrift.version} + + javax.annotation + javax.annotation-api + 1.3.2 + diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 7d408d89aa..4212ae00f6 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -84,21 +84,6 @@ - - - org.apache.maven.plugins - maven-dependency-plugin - - - - analyze-only - - - true - - - - diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 8c14b73ff1..c11c56022c 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -46,11 +46,6 @@ parquet-format-structures ${project.version} - - org.apache.parquet - parquet-common - ${project.version} - org.apache.parquet parquet-column @@ -61,29 +56,19 @@ org.apache.hadoop hadoop-client - provided - - - org.apache.hadoop - hadoop-common - provided - - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - org.apache.hadoop - hadoop-annotations ${hadoop.version} provided + + + org.slf4j + slf4j-log4j12 + + org.apache.parquet parquet-jackson ${project.version} - runtime ${jackson.groupId} @@ -142,28 +127,6 @@ 4.6.0 test - - org.slf4j - slf4j-api - ${slf4j.version} - - - commons-io - commons-io - 2.4 - test - - - net.openhft - zero-allocation-hashing - ${net.openhft.version} - test - - - org.apache.yetus - audience-annotations - ${yetus.audience-annotations.version} - @@ -196,7 +159,6 @@ com.github.rdblue brotli-codec ${brotli-codec.version} - runtime true diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java index 862ae672c6..17fda97d19 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/propertiesfactory/SchemaControlEncryptionTest.java @@ -19,6 +19,8 @@ package org.apache.parquet.crypto.propertiesfactory; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.column.ColumnDescriptor; @@ -26,6 +28,7 @@ import org.apache.parquet.crypto.ParquetCipher; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.format.EncryptionAlgorithm; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.api.WriteSupport; @@ -36,8 +39,6 @@ import org.apache.parquet.schema.PrimitiveType; import org.junit.Before; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.nio.file.Files; @@ -57,13 +58,13 @@ public class SchemaControlEncryptionTest { - private final static Logger LOG = LoggerFactory.getLogger(SchemaControlEncryptionTest.class); + private final static Log LOG = LogFactory.getLog(SchemaControlEncryptionTest.class); private final static int numRecord = 1000; private Random rnd = new Random(5); - + // In the test We use a map to tell WriteSupport which columns to be encrypted with what key. In real use cases, people - // can find whatever easy way to do so basing on how do they get these information, for example people can choose to - // store in HMS, or other metastore. + // can find whatever easy way to do so basing on how do they get these information, for example people can choose to + // store in HMS, or other metastore. private Map> cryptoMetadata = new HashMap<>(); private Map testData = new HashMap<>(); @@ -121,7 +122,7 @@ private void runTest(Configuration conf ) throws Exception { encryptParquetFile(file, conf); decryptParquetFileAndValid(file, conf); } - + private void markEncryptColumns() { Map ageMetadata = new HashMap<>(); ageMetadata.put("columnKeyMetaData", "age_key_id"); diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 16d6e4f7a9..c12e2477f6 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -90,21 +90,6 @@ - - - org.apache.maven.plugins - maven-dependency-plugin - - - - analyze-only - - - true - - - - diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index a10b4eb57a..eede36938b 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -74,21 +74,6 @@ - - - org.apache.maven.plugins - maven-dependency-plugin - - - - analyze-only - - - true - - - - diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index ef05c7b9b4..a7251846af 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -48,15 +48,9 @@ org.apache.parquet - parquet-common + parquet-format-structures ${project.version} - - org.apache.parquet - parquet-jackson - ${project.version} - runtime - org.apache.pig pig @@ -71,27 +65,20 @@ provided - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - provided + org.apache.parquet + parquet-jackson + ${project.version} - org.apache.hadoop - hadoop-common - ${hadoop.version} - provided + ${jackson.groupId} + jackson-core + ${jackson.version} ${jackson.groupId} jackson-databind ${jackson-databind.version} - - ${jackson.groupId} - jackson-annotations - ${jackson.version} - org.apache.parquet parquet-column @@ -123,11 +110,6 @@ ${slf4j.version} test - - org.slf4j - slf4j-api - ${slf4j.version} - diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 2e82b4979f..990d5e9d03 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -87,23 +87,9 @@ org.apache.hadoop hadoop-client + ${hadoop.version} provided - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - org.apache.hadoop - hadoop-common - provided - - - org.slf4j - slf4j-api - ${slf4j.version} - org.slf4j slf4j-simple diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 9786b10621..5f5f759a4e 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -64,12 +64,6 @@ 3.0.1 test - - org.scalactic - scalactic_${scala.binary.version} - 3.0.1 - test - org.slf4j slf4j-simple diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index f0d8e77bba..51e4aa3c3a 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -47,25 +47,17 @@ parquet-hadoop ${project.version} - - org.apache.parquet - parquet-common - ${project.version} - org.apache.hadoop hadoop-client + ${hadoop.version} provided - - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - org.apache.hadoop - hadoop-common - provided + + + org.slf4j + slf4j-log4j12 + + com.twitter.elephantbird @@ -95,22 +87,16 @@ org.apache.parquet parquet-jackson ${project.version} - runtime ${jackson.groupId} - jackson-databind - ${jackson-databind.version} - - - ${jackson.groupId} - jackson-annotations + jackson-core ${jackson.version} - com.google.guava - guava - ${guava.version} + ${jackson.groupId} + jackson-databind + ${jackson-databind.version} org.apache.parquet @@ -149,11 +135,6 @@ ${thrift.version} provided - - org.slf4j - slf4j-api - ${slf4j.version} - org.slf4j slf4j-simple @@ -174,6 +155,7 @@ test-jar test + diff --git a/pom.xml b/pom.xml index f28f7d7718..9f60ac9327 100644 --- a/pom.xml +++ b/pom.xml @@ -106,14 +106,12 @@ 0.9 3.1.0 1.6.0 - 0.13.0 2.3 1.72 1.4.9-1 1.8 - 3.0.2 INFO @@ -158,44 +156,6 @@ - - - - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - - - org.slf4j - slf4j-log4j12 - - - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - - - org.slf4j - slf4j-log4j12 - - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - org.slf4j - slf4j-log4j12 - - - - - - @@ -561,21 +521,6 @@ - - org.apache.maven.plugins - maven-dependency-plugin - - - - analyze-only - - - true - true - - - - From 2a5c06c58fa987f85aa22170be14d927d5ff6e7d Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Mon, 13 Sep 2021 10:28:24 -0700 Subject: [PATCH 18/36] [maven-release-plugin] prepare release apache-parquet-1.12.1-rc1 --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 4 ++-- 22 files changed, 23 insertions(+), 23 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index b7a94810e7..1c82c72f1b 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index e7b8e6b6af..d239ab57f5 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index ae16bfb3b1..c850f9c18a 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 1ceb23a1f7..cd329e2cd4 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index e67f0fa105..c804f4bedb 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index d886951aee..a91376c1e9 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index cb0c1853b5..0c26ed8eb4 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index a3684464f5..220c4c077b 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index f1f7d9a26f..75ae733a90 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 4fb990ba4d..2cae7310db 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 1126c8ef88..c6a9fc4606 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 4212ae00f6..90b5962b1b 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index c11c56022c..396ac23f47 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index c12e2477f6..b6f08a76e6 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index eede36938b..f2a2e743d4 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index a7251846af..b457c8efce 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 990d5e9d03..00a0604311 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 5f5f759a4e..01089455ff 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 96f8ecdf6d..2d5391c8ef 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 51e4aa3c3a..3e2a6b54fb 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 5343d25785..a8d79fb9c1 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.1 4.0.0 diff --git a/pom.xml b/pom.xml index 9f60ac9327..605e9fe35d 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.1-SNAPSHOT + 1.12.1 pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - HEAD + apache-parquet-1.12.1-rc1 From 16d7608ff44e77c192b36b106d38d07358fae9d2 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Mon, 13 Sep 2021 10:29:33 -0700 Subject: [PATCH 19/36] [maven-release-plugin] prepare for next development iteration --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 4 ++-- 22 files changed, 23 insertions(+), 23 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 1c82c72f1b..b7a94810e7 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index d239ab57f5..e7b8e6b6af 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index c850f9c18a..ae16bfb3b1 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index cd329e2cd4..1ceb23a1f7 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index c804f4bedb..e67f0fa105 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index a91376c1e9..d886951aee 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 0c26ed8eb4..cb0c1853b5 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 220c4c077b..a3684464f5 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 75ae733a90..f1f7d9a26f 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 2cae7310db..4fb990ba4d 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index c6a9fc4606..1126c8ef88 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 90b5962b1b..4212ae00f6 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 396ac23f47..c11c56022c 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index b6f08a76e6..c12e2477f6 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index f2a2e743d4..eede36938b 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index b457c8efce..a7251846af 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 00a0604311..990d5e9d03 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 01089455ff..5f5f759a4e 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 2d5391c8ef..96f8ecdf6d 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 3e2a6b54fb..51e4aa3c3a 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index a8d79fb9c1..5343d25785 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1 + 1.12.1-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 605e9fe35d..9f60ac9327 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.1 + 1.12.1-SNAPSHOT pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - apache-parquet-1.12.1-rc1 + HEAD From ef6274f8606e6e6a6ffb24a2c11566cc1974c7a4 Mon Sep 17 00:00:00 2001 From: Xinli shang Date: Tue, 14 Sep 2021 21:35:01 -0700 Subject: [PATCH 20/36] [maven-release-plugin] prepare for next development iteration --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 6 +++--- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 22 files changed, 24 insertions(+), 24 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index b7a94810e7..b96cebb7c2 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index e7b8e6b6af..882963dcae 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index ae16bfb3b1..041e1af85f 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 1ceb23a1f7..482759e28d 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index e67f0fa105..3447d644e9 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index d886951aee..dda16a00f6 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index cb0c1853b5..41da0a3f3e 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index a3684464f5..4fa9a56c32 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index f1f7d9a26f..8a4b52ecfb 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 4fb990ba4d..f3c789c56b 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 1126c8ef88..a9aaae878f 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 4212ae00f6..d9deb2115c 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index c11c56022c..a2be765e8f 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index c12e2477f6..33bbf97644 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index eede36938b..670ca7a5b1 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index a7251846af..8cd2a83ac9 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 990d5e9d03..ac047ac177 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 5f5f759a4e..5f788a1b40 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 @@ -46,13 +46,13 @@ https://scala-tools.org/repo-releases - + org.apache.parquet parquet-column ${project.version} - + org.scala-lang scala-library diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 96f8ecdf6d..788130dbe8 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 51e4aa3c3a..68f55109f1 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 5343d25785..d31c0998e9 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index 9f60ac9327..2ba0784132 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.1-SNAPSHOT + 1.12.2-SNAPSHOT pom Apache Parquet MR From afb64b8566204776081cfc56997cf8432b9491b4 Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Thu, 30 Sep 2021 10:07:06 +0200 Subject: [PATCH 21/36] PARQUET-2094: Handle negative values in page headers (#933) (cherry picked from commit 1695d92cc07288713a9f2230f3aac61e2dc6a8e4) --- .../InvalidParquetMetadataException.java | 30 +++++++++++++ .../parquet/format/MetadataValidator.java | 44 +++++++++++++++++++ .../java/org/apache/parquet/format/Util.java | 2 +- .../org/apache/parquet/format/TestUtil.java | 20 ++++++++- 4 files changed, 94 insertions(+), 2 deletions(-) create mode 100644 parquet-format-structures/src/main/java/org/apache/parquet/format/InvalidParquetMetadataException.java create mode 100644 parquet-format-structures/src/main/java/org/apache/parquet/format/MetadataValidator.java diff --git a/parquet-format-structures/src/main/java/org/apache/parquet/format/InvalidParquetMetadataException.java b/parquet-format-structures/src/main/java/org/apache/parquet/format/InvalidParquetMetadataException.java new file mode 100644 index 0000000000..c0852bbbd2 --- /dev/null +++ b/parquet-format-structures/src/main/java/org/apache/parquet/format/InvalidParquetMetadataException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.format; + +/** + * A specific RuntimeException thrown when invalid values are found in the Parquet file metadata (including the footer, + * page header etc.). + */ +public class InvalidParquetMetadataException extends RuntimeException { + InvalidParquetMetadataException(String message) { + super(message); + } +} diff --git a/parquet-format-structures/src/main/java/org/apache/parquet/format/MetadataValidator.java b/parquet-format-structures/src/main/java/org/apache/parquet/format/MetadataValidator.java new file mode 100644 index 0000000000..b3738ec48f --- /dev/null +++ b/parquet-format-structures/src/main/java/org/apache/parquet/format/MetadataValidator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.format; + +/** + * Utility class to validate different types of Parquet metadata (e.g. footer, page headers etc.). + */ +public class MetadataValidator { + + static PageHeader validate(PageHeader pageHeader) { + int compressed_page_size = pageHeader.getCompressed_page_size(); + validateValue(compressed_page_size >= 0, + String.format("Compressed page size must not be negative but was: %s", compressed_page_size)); + return pageHeader; + } + + private static void validateValue(boolean valid, String message) { + if (!valid) { + throw new InvalidParquetMetadataException(message); + } + } + + private MetadataValidator() { + // Private constructor to prevent instantiation + } + +} diff --git a/parquet-format-structures/src/main/java/org/apache/parquet/format/Util.java b/parquet-format-structures/src/main/java/org/apache/parquet/format/Util.java index 36aaf7494e..cb62325f99 100644 --- a/parquet-format-structures/src/main/java/org/apache/parquet/format/Util.java +++ b/parquet-format-structures/src/main/java/org/apache/parquet/format/Util.java @@ -129,7 +129,7 @@ public static PageHeader readPageHeader(InputStream from) throws IOException { public static PageHeader readPageHeader(InputStream from, BlockCipher.Decryptor decryptor, byte[] AAD) throws IOException { - return read(from, new PageHeader(), decryptor, AAD); + return MetadataValidator.validate(read(from, new PageHeader(), decryptor, AAD)); } public static void writeFileMetaData(org.apache.parquet.format.FileMetaData fileMetadata, diff --git a/parquet-format-structures/src/test/java/org/apache/parquet/format/TestUtil.java b/parquet-format-structures/src/test/java/org/apache/parquet/format/TestUtil.java index 1adf0998fb..685e2514b0 100644 --- a/parquet-format-structures/src/test/java/org/apache/parquet/format/TestUtil.java +++ b/parquet-format-structures/src/test/java/org/apache/parquet/format/TestUtil.java @@ -23,13 +23,16 @@ import static junit.framework.Assert.assertNull; import static org.apache.parquet.format.Util.readFileMetaData; import static org.apache.parquet.format.Util.writeFileMetaData; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.IOException; import org.junit.Test; - import org.apache.parquet.format.Util.DefaultFileMetaDataConsumer; + public class TestUtil { @Test @@ -77,6 +80,21 @@ public void testReadFileMetadata() throws Exception { assertEquals(md, md6); } + @Test + public void testInvalidPageHeader() throws IOException { + PageHeader ph = new PageHeader(PageType.DATA_PAGE, 100, -50); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Util.writePageHeader(ph, out); + + try { + Util.readPageHeader(in(out)); + fail("Expected exception but did not thrown"); + } catch (InvalidParquetMetadataException e) { + assertTrue("Exception message does not contain the expected parts", + e.getMessage().contains("Compressed page size")); + } + } + private ByteArrayInputStream in(ByteArrayOutputStream baos) { return new ByteArrayInputStream(baos.toByteArray()); } From a1131890565083cfd3e77dad74e1a7786b3c54fd Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Thu, 30 Sep 2021 11:06:37 +0200 Subject: [PATCH 22/36] Update CHANGES.md for 1.12.2 --- CHANGES.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index 9d752ed924..98873a4794 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -19,6 +19,14 @@ # Parquet # +### Version 1.12.2 ### + +Release Notes - Parquet - Version 1.12.2 + +#### Bug + +* [PARQUET-2094](https://issues.apache.org/jira/browse/PARQUET-2094) - Handle negative values in page headers + ### Version 1.12.1 ### Release Notes - Parquet - Version 1.12.1 From 77e30c8093386ec52c3cfa6c34b7ef3321322c94 Mon Sep 17 00:00:00 2001 From: Gabor Szadovszky Date: Thu, 30 Sep 2021 14:10:36 +0200 Subject: [PATCH 23/36] [maven-release-plugin] prepare release apache-parquet-1.12.2-rc0 --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 4 ++-- 22 files changed, 23 insertions(+), 23 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index b96cebb7c2..03c3afb0d0 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 882963dcae..4b79363641 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 041e1af85f..3e695a9c57 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 482759e28d..bcf16ad866 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 3447d644e9..1cfcbb765d 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index dda16a00f6..3446acadfb 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 41da0a3f3e..829d72bc9b 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 4fa9a56c32..0b3b08e697 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 8a4b52ecfb..ed818244b6 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index f3c789c56b..f35f22111a 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index a9aaae878f..d0c54dd324 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index d9deb2115c..9b922b198d 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index a2be765e8f..7b5bb484ce 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 33bbf97644..d0ec6da5eb 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 670ca7a5b1..b2c7473261 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index 8cd2a83ac9..72a9ef9581 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index ac047ac177..0b70f2ddc2 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 5f788a1b40..3f7a92ab0c 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 788130dbe8..52f99a0c9b 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 68f55109f1..25b3de8d62 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index d31c0998e9..0a43eca05a 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-SNAPSHOT + 1.12.2 4.0.0 diff --git a/pom.xml b/pom.xml index 2ba0784132..0970853e6c 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-SNAPSHOT + 1.12.2 pom Apache Parquet MR @@ -20,7 +20,7 @@ scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git scm:git:git@github.com:apache/parquet-mr.git - HEAD + apache-parquet-1.12.2-rc0 From c17fc98294cdff91565e960ed65dc6d1eba22aa3 Mon Sep 17 00:00:00 2001 From: Jiatao Tao <245915794@qq.com> Date: Tue, 14 May 2019 17:29:31 +0800 Subject: [PATCH 24/36] reduce ByteBuffer allocation --- .../hadoop/codec/SnappyDecompressor.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java index 2e0c558930..55f31ee366 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/codec/SnappyDecompressor.java @@ -1,4 +1,4 @@ -/* +/* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -6,9 +6,9 @@ * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, * software distributed under the License is distributed on an * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY @@ -53,7 +53,7 @@ public synchronized int decompress(byte[] buffer, int off, int len) throws IOExc if (inputBuffer.position() == 0 && !outputBuffer.hasRemaining()) { return 0; } - + if (!outputBuffer.hasRemaining()) { inputBuffer.rewind(); Preconditions.checkArgument(inputBuffer.position() == 0, "Invalid position of 0."); @@ -79,7 +79,7 @@ public synchronized int decompress(byte[] buffer, int off, int len) throws IOExc // Return compressed output up to 'len' int numBytes = Math.min(len, outputBuffer.remaining()); outputBuffer.get(buffer, off, numBytes); - return numBytes; + return numBytes; } /** @@ -101,15 +101,17 @@ public synchronized void setInput(byte[] buffer, int off, int len) { SnappyUtil.validateBuffer(buffer, off, len); if (inputBuffer.capacity() - inputBuffer.position() < len) { - final ByteBuffer newBuffer = ByteBuffer.allocateDirect(inputBuffer.position() + len); + int maxSize = Math.max(inputBuffer.position() * 2 , inputBuffer.position() + len); + ByteBuffer newBuffer = ByteBuffer.allocateDirect(maxSize); inputBuffer.rewind(); newBuffer.put(inputBuffer); final ByteBuffer oldBuffer = inputBuffer; inputBuffer = newBuffer; CleanUtil.cleanDirectBuffer(oldBuffer); - } else { - inputBuffer.limit(inputBuffer.position() + len); } + + inputBuffer.limit(inputBuffer.position() + len); + inputBuffer.put(buffer, off, len); } @@ -150,7 +152,7 @@ public boolean needsDictionary() { @Override public void setDictionary(byte[] b, int off, int len) { - // No-op + // No-op } } //class SnappyDecompressor From bf398e3d36250c6d8feabba9f9df5e474fe08573 Mon Sep 17 00:00:00 2001 From: "mingming.ge@kyligence.io" Date: Tue, 20 Aug 2019 18:16:36 +0800 Subject: [PATCH 25/36] add single record size check --- .../hadoop/InternalParquetRecordWriter.java | 27 ++++++++++--------- .../parquet/hadoop/api/WriteSupport.java | 2 ++ 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java index d2e505babc..ecb2cf54a8 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/InternalParquetRecordWriter.java @@ -65,7 +65,7 @@ class InternalParquetRecordWriter { private ColumnChunkPageWriteStore pageStore; private BloomFilterWriteStore bloomFilterWriteStore; private RecordConsumer recordConsumer; - + private InternalFileEncryptor fileEncryptor; private int rowGroupOrdinal; @@ -78,14 +78,14 @@ class InternalParquetRecordWriter { * @param compressor the codec used to compress */ public InternalParquetRecordWriter( - ParquetFileWriter parquetFileWriter, - WriteSupport writeSupport, - MessageType schema, - Map extraMetaData, - long rowGroupSize, - BytesCompressor compressor, - boolean validating, - ParquetProperties props) { + ParquetFileWriter parquetFileWriter, + WriteSupport writeSupport, + MessageType schema, + Map extraMetaData, + long rowGroupSize, + BytesCompressor compressor, + boolean validating, + ParquetProperties props) { this.parquetFileWriter = parquetFileWriter; this.writeSupport = Objects.requireNonNull(writeSupport, "writeSupport cannot be null"); this.schema = schema; @@ -148,7 +148,8 @@ public long getDataSize() { } private void checkBlockSizeReached() throws IOException { - if (recordCount >= recordCountForNextMemCheck) { // checking the memory size is relatively expensive, so let's not do it for every record. + if (recordCount >= recordCountForNextMemCheck || writeSupport.needCheckRowSize) { // checking the memory size is relatively expensive, so let's not do it for every record. + writeSupport.needCheckRowSize = false; long memSize = columnStore.getBufferedSize(); long recordSize = memSize / recordCount; // flush the row group if it is within ~2 records of the limit @@ -172,7 +173,7 @@ private void checkBlockSizeReached() throws IOException { } private void flushRowGroupToStore() - throws IOException { + throws IOException { recordConsumer.flush(); LOG.debug("Flushing mem columnStore to file. allocated memory: {}", columnStore.getAllocatedSize()); if (columnStore.getAllocatedSize() > (3 * rowGroupSizeThreshold)) { @@ -187,8 +188,8 @@ private void flushRowGroupToStore() recordCount = 0; parquetFileWriter.endBlock(); this.nextRowGroupSize = Math.min( - parquetFileWriter.getNextRowGroupSize(), - rowGroupSizeThreshold); + parquetFileWriter.getNextRowGroupSize(), + rowGroupSizeThreshold); } columnStore = null; diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java index 9549d5f492..f959a81771 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/api/WriteSupport.java @@ -36,6 +36,8 @@ */ abstract public class WriteSupport { + public boolean needCheckRowSize = false; + /** * information to be persisted in the file */ From d9333f6668ef5b76ad447f51c1dea5c907138d07 Mon Sep 17 00:00:00 2001 From: "yiming.xu" <100650920@qq.com> Date: Fri, 8 Nov 2019 11:22:09 +0800 Subject: [PATCH 26/36] optimize snappy decompress --- .../apache/parquet/hadoop/CodecFactory.java | 2 +- .../hadoop/ColumnChunkPageReadStore.java | 37 ++++++++++++------- 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java index f0e7af35c5..513e71295c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/CodecFactory.java @@ -89,7 +89,7 @@ public static CodecFactory createDirectCodecFactory(Configuration config, ByteBu class HeapBytesDecompressor extends BytesDecompressor { - private final CompressionCodec codec; + final CompressionCodec codec; private final Decompressor decompressor; HeapBytesDecompressor(CompressionCodecName codecName) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java index 3d1bafe0a5..fc17edcb5e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java @@ -40,11 +40,13 @@ import org.apache.parquet.crypto.AesCipher; import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.hadoop.codec.SnappyCodec; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.apache.parquet.internal.filter2.columnindex.RowRanges; import org.apache.parquet.io.ParquetDecodingException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.xerial.snappy.Snappy; /** * TODO: should this actually be called RowGroupImpl or something? @@ -72,14 +74,14 @@ static final class ColumnChunkPageReader implements PageReader { private final OffsetIndex offsetIndex; private final long rowCount; private int pageIndex = 0; - + private final BlockCipher.Decryptor blockDecryptor; private final byte[] dataPageAAD; private final byte[] dictionaryPageAAD; ColumnChunkPageReader(BytesInputDecompressor decompressor, List compressedPages, DictionaryPage compressedDictionaryPage, OffsetIndex offsetIndex, long rowCount, - BlockCipher.Decryptor blockDecryptor, byte[] fileAAD, + BlockCipher.Decryptor blockDecryptor, byte[] fileAAD, int rowGroupOrdinal, int columnOrdinal) { this.decompressor = decompressor; this.compressedPages = new ArrayDeque(compressedPages); @@ -91,9 +93,9 @@ static final class ColumnChunkPageReader implements PageReader { this.valueCount = count; this.offsetIndex = offsetIndex; this.rowCount = rowCount; - + this.blockDecryptor = blockDecryptor; - + if (null != blockDecryptor) { dataPageAAD = AesCipher.createModuleAAD(fileAAD, ModuleType.DataPage, rowGroupOrdinal, columnOrdinal, 0); dictionaryPageAAD = AesCipher.createModuleAAD(fileAAD, ModuleType.DictionaryPage, rowGroupOrdinal, columnOrdinal, -1); @@ -102,12 +104,12 @@ static final class ColumnChunkPageReader implements PageReader { dictionaryPageAAD = null; } } - + private int getPageOrdinal(int currentPageIndex) { if (null == offsetIndex) { return currentPageIndex; } - + return offsetIndex.getPageOrdinal(currentPageIndex); } @@ -123,11 +125,11 @@ public DataPage readPage() { return null; } final int currentPageIndex = pageIndex++; - + if (null != blockDecryptor) { AesCipher.quickUpdatePageAAD(dataPageAAD, getPageOrdinal(currentPageIndex)); } - + return compressedPage.accept(new DataPage.Visitor() { @Override public DataPage visit(DataPageV1 dataPageV1) { @@ -136,8 +138,17 @@ public DataPage visit(DataPageV1 dataPageV1) { if (null != blockDecryptor) { bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD)); } - BytesInput decompressed = decompressor.decompress(bytes, dataPageV1.getUncompressedSize()); - + + BytesInput decompressed; + if (decompressor instanceof CodecFactory.HeapBytesDecompressor &&(((CodecFactory.HeapBytesDecompressor) decompressor).codec instanceof SnappyCodec)) { + byte[] bytesArray = bytes.toByteArray(); + byte[] decompressedBytes = new byte[dataPageV1.getUncompressedSize()]; + Snappy.uncompress(bytesArray, 0, bytesArray.length, decompressedBytes, 0); + decompressed = BytesInput.from(decompressedBytes); + } else { + decompressed = decompressor.decompress(dataPageV1.getBytes(), dataPageV1.getUncompressedSize()); + } + final DataPageV1 decompressedPage; if (offsetIndex == null) { decompressedPage = new DataPageV1( @@ -176,7 +187,7 @@ public DataPage visit(DataPageV2 dataPageV2) { return dataPageV2; } BytesInput pageBytes = dataPageV2.getData(); - + if (null != blockDecryptor) { try { pageBytes = BytesInput.from(blockDecryptor.decrypt(pageBytes.toByteArray(), dataPageAAD)); @@ -195,7 +206,7 @@ public DataPage visit(DataPageV2 dataPageV2) { throw new ParquetDecodingException("could not decompress page", e); } } - + if (offsetIndex == null) { return DataPageV2.uncompressed( dataPageV2.getRowCount(), @@ -218,7 +229,7 @@ public DataPage visit(DataPageV2 dataPageV2) { pageBytes, dataPageV2.getStatistics()); } - } + } }); } From 19aec9dc6dd0af3f9ed2c795b36e01b6355d5903 Mon Sep 17 00:00:00 2001 From: "yiming.xu" <100650920@qq.com> Date: Fri, 8 Nov 2019 11:30:26 +0800 Subject: [PATCH 27/36] maven maven-source-plugin --- pom.xml | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/pom.xml b/pom.xml index 0970853e6c..ffb9d75c28 100644 --- a/pom.xml +++ b/pom.xml @@ -375,6 +375,23 @@ ${maven.compiler.target} + + org.apache.maven.plugins + maven-source-plugin + 3.0.1 + + true + + + + create-source-jar + + jar-no-fork + test-jar-no-fork + + + + org.apache.maven.plugins From d9a2d1cfce18f581892304e408239857cd737c31 Mon Sep 17 00:00:00 2001 From: xifeng yang Date: Wed, 22 Dec 2021 16:36:01 +0800 Subject: [PATCH 28/36] update version to 1.12.2-kylin-r1 --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 22 files changed, 22 insertions(+), 22 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 03c3afb0d0..61de316353 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 4b79363641..ef1fcfef28 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 3e695a9c57..816f3a3c0d 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index bcf16ad866..61c40bdc6d 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 1cfcbb765d..b0410011ff 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 3446acadfb..828bb18e42 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 829d72bc9b..c544982a4d 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 0b3b08e697..e611c03870 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index ed818244b6..14e8e623a9 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index f35f22111a..c88b67f305 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index d0c54dd324..a279ccac4a 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 9b922b198d..9e531e9633 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 7b5bb484ce..5458bf509c 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index d0ec6da5eb..aa4cfe65fa 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index b2c7473261..1fd675c160 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index 72a9ef9581..3bb1bcd72c 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 0b70f2ddc2..8bc1f0514c 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 3f7a92ab0c..caf59aaa75 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 52f99a0c9b..8913816b38 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 25b3de8d62..a8c25e554a 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 0a43eca05a..f9b933b49e 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2 + 1.12.2-kylin-r1 4.0.0 diff --git a/pom.xml b/pom.xml index ffb9d75c28..cbaae179d9 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2 + 1.12.2-kylin-r1 pom Apache Parquet MR From bd9e6fe1dc08578d8849e750a9dfd3a53f0daf46 Mon Sep 17 00:00:00 2001 From: Haifeng Wang Date: Tue, 10 Dec 2019 16:23:52 +0800 Subject: [PATCH 29/36] Revert "mior, add local nexus" (#15) This reverts commit 0f6fc7f030d9e3a3ae877e8b3bf32ab9103a93cb. --- pom.xml | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cbaae179d9..b0dd5b0281 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,18 @@ test - + + + user-snapshots + User Project SNAPSHOTS + http://repository.kyligence.io:8081/repository/maven-snapshots/ + + + user-releases + User Project Release + http://repository.kyligence.io:8081/repository/maven-releases/ + + From c7154bdaab4ec7d9a3326415e6c25d365fc690cb Mon Sep 17 00:00:00 2001 From: ChenliangLu <31469905+yabola@users.noreply.github.com> Date: Fri, 3 Mar 2023 10:23:21 +0800 Subject: [PATCH 30/36] KE-40948 Add RowGroup filters info (#66) KE-40948 Add rowGroup filters info --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- .../bloomfilterlevel/BloomFilterImpl.java | 27 ++++++++- .../parquet/filter2/compat/QueryMetrics.java | 59 +++++++++++++++++++ .../filter2/compat/RowGroupFilter.java | 45 ++++++++++++-- .../parquet/hadoop/ParquetFileReader.java | 5 +- .../parquet/hadoop/ParquetFileWriter.java | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 24 ++++++-- 27 files changed, 169 insertions(+), 35 deletions(-) create mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 61de316353..3fe44873d3 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index ef1fcfef28..4158450958 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 816f3a3c0d..1c4c02bebe 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 61c40bdc6d..3b80fc8467 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index b0410011ff..5d45f2ebee 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 828bb18e42..2445c78765 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index c544982a4d..3ee1d0de3d 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index e611c03870..ae5b9ff1e9 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 14e8e623a9..f0008df206 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index c88b67f305..1a777ac8dc 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index a279ccac4a..2f0cc9ef0f 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 9e531e9633..6bfde2a874 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 5458bf509c..7a924f9f1a 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java index d98416445f..c5e8d521af 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java @@ -19,9 +19,13 @@ package org.apache.parquet.filter2.bloomfilterlevel; +import static org.apache.parquet.Preconditions.checkNotNull; + import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,8 +37,6 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import static org.apache.parquet.Preconditions.checkNotNull; - public class BloomFilterImpl implements FilterPredicate.Visitor{ private static final Logger LOG = LoggerFactory.getLogger(BloomFilterImpl.class); private static final boolean BLOCK_MIGHT_MATCH = false; @@ -42,12 +44,27 @@ public class BloomFilterImpl implements FilterPredicate.Visitor{ private final Map columns = new HashMap(); + public static boolean canDropWithInfo(FilterPredicate pred, List columns, + BloomFilterReader bloomFilterReader, AtomicInteger bloomInfo) { + checkNotNull(pred, "pred"); + checkNotNull(columns, "columns"); + return pred.accept(new BloomFilterImpl(columns, bloomFilterReader, bloomInfo)); + } + public static boolean canDrop(FilterPredicate pred, List columns, BloomFilterReader bloomFilterReader) { checkNotNull(pred, "pred"); checkNotNull(columns, "columns"); return pred.accept(new BloomFilterImpl(columns, bloomFilterReader)); } + private BloomFilterImpl(List columnsList, BloomFilterReader bloomFilterReader, AtomicInteger bloomInfo) { + for (ColumnChunkMetaData chunk : columnsList) { + columns.put(chunk.getPath(), chunk); + } + this.bloomFilterReader = bloomFilterReader; + this.bloomInfo = bloomInfo; + } + private BloomFilterImpl(List columnsList, BloomFilterReader bloomFilterReader) { for (ColumnChunkMetaData chunk : columnsList) { columns.put(chunk.getPath(), chunk); @@ -56,6 +73,8 @@ private BloomFilterImpl(List columnsList, BloomFilterReader this.bloomFilterReader = bloomFilterReader; } + private AtomicInteger bloomInfo = new AtomicInteger(0); + private BloomFilterReader bloomFilterReader; private ColumnChunkMetaData getColumnChunk(ColumnPath columnPath) { @@ -82,6 +101,10 @@ public > Boolean visit(Operators.Eq eq) { try { BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(meta); + if (bloomFilter != null) { + // use bloom + bloomInfo.set(1); + } if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { return BLOCK_CANNOT_MATCH; } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java new file mode 100644 index 0000000000..ac6d90b616 --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.filter2.compat; + +public class QueryMetrics { + private String skipBloomFilter = ""; + private long skipBloomBlocks = 0; + private long skipBloomRows = 0; + private long totalBloomBlocks = 0; + + public String getSkipBloomFilter() { + return skipBloomFilter; + } + + public void setSkipBloomFilter(String skipBloomFilter) { + this.skipBloomFilter = skipBloomFilter; + } + + public long getSkipBloomBlocks() { + return skipBloomBlocks; + } + + public void setSkipBloomBlocks(long skipBloomBlocks) { + this.skipBloomBlocks = skipBloomBlocks; + } + + public long getSkipBloomRows() { + return skipBloomRows; + } + + public void setSkipBloomRows(long skipBloomRows) { + this.skipBloomRows = skipBloomRows; + } + + public long getTotalBloomBlocks() { + return totalBloomBlocks; + } + + public void setTotalBloomBlocks(long totalBloomBlocks) { + this.totalBloomBlocks = totalBloomBlocks; + } +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java index 73ff9aae67..77e23ed3e0 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java @@ -22,6 +22,10 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.parquet.filter2.bloomfilterlevel.BloomFilterImpl; import org.apache.parquet.filter2.compat.FilterCompat.Filter; @@ -29,13 +33,12 @@ import org.apache.parquet.filter2.compat.FilterCompat.Visitor; import org.apache.parquet.filter2.dictionarylevel.DictionaryFilter; import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; import org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator; import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.schema.MessageType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Given a {@link Filter} applies it to a list of BlockMetaData (row groups) @@ -43,10 +46,14 @@ * no filtering will be performed. */ public class RowGroupFilter implements Visitor> { + + public static Logger LOGGER = LoggerFactory.getLogger(RowGroupFilter.class); + private final List blocks; private final MessageType schema; private final List levels; private final ParquetFileReader reader; + private QueryMetrics queryMetrics = new QueryMetrics(); public enum FilterLevel { STATISTICS, @@ -72,6 +79,12 @@ public static List filterRowGroups(List levels, Filt return filter.accept(new RowGroupFilter(levels, blocks, reader)); } + public static List filterRowGroups(List levels, Filter filter, List blocks, + ParquetFileReader reader, QueryMetrics queryMetrics) { + Objects.requireNonNull(filter, "filter cannot be null"); + return filter.accept(new RowGroupFilter(levels, blocks, reader, queryMetrics)); + } + @Deprecated private RowGroupFilter(List blocks, MessageType schema) { this.blocks = Objects.requireNonNull(blocks, "blocks cannnot be null"); @@ -87,6 +100,15 @@ private RowGroupFilter(List levels, List blocks, Par this.levels = levels; } + private RowGroupFilter(List levels, List blocks, ParquetFileReader reader, + QueryMetrics queryMetrics) { + this.blocks = Objects.requireNonNull(blocks, "blocks cannnot be null"); + this.reader = Objects.requireNonNull(reader, "reader cannnot be null"); + this.schema = reader.getFileMetaData().getSchema(); + this.levels = levels; + this.queryMetrics = queryMetrics; + } + @Override public List visit(FilterCompat.FilterPredicateCompat filterPredicateCompat) { FilterPredicate filterPredicate = filterPredicateCompat.getFilterPredicate(); @@ -95,7 +117,7 @@ public List visit(FilterCompat.FilterPredicateCompat filterPredic SchemaCompatibilityValidator.validate(filterPredicate, schema); List filteredBlocks = new ArrayList(); - + long start = System.currentTimeMillis(); for (BlockMetaData block : blocks) { boolean drop = false; @@ -108,14 +130,27 @@ public List visit(FilterCompat.FilterPredicateCompat filterPredic } if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { - drop = BloomFilterImpl.canDrop(filterPredicate, block.getColumns(), reader.getBloomFilterDataReader(block)); + AtomicInteger bloomInfo = new AtomicInteger(0); + drop = BloomFilterImpl.canDropWithInfo(filterPredicate, block.getColumns(), + reader.getBloomFilterDataReader(block), bloomInfo); + if (bloomInfo.get() != 0) { + this.queryMetrics.setTotalBloomBlocks(this.queryMetrics.getTotalBloomBlocks() + 1); + if (drop) { + this.queryMetrics.setSkipBloomFilter(this.queryMetrics.getSkipBloomFilter() + filterPredicateCompat); + this.queryMetrics.setSkipBloomRows(this.queryMetrics.getSkipBloomRows() + block.getRowCount()); + this.queryMetrics.setSkipBloomBlocks(this.queryMetrics.getSkipBloomBlocks() + 1); + } + } } if(!drop) { filteredBlocks.add(block); } } - + long end = System.currentTimeMillis(); + if ((end - start) > 100) { + LOGGER.warn("Reading RowGroupFilter costs much time : " + (end - start)); + } return filteredBlocks; } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 95f93ab809..18d3ebc1ea 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -82,6 +82,7 @@ import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.filter2.compat.FilterCompat; import org.apache.parquet.filter2.compat.RowGroupFilter; +import org.apache.parquet.filter2.compat.QueryMetrics; import org.apache.parquet.format.BlockCipher; import org.apache.parquet.format.BloomFilterHeader; import org.apache.parquet.format.DataPageHeader; @@ -130,6 +131,8 @@ public class ParquetFileReader implements Closeable { private final CRC32 crc; + public QueryMetrics queryMetrics = new QueryMetrics(); + /** * for files provided, check if there's a summary file. * If a summary file is found it is used otherwise the file footer is used. @@ -867,7 +870,7 @@ private List filterRowGroups(List blocks) throws I if (options.useBloomFilter()) { levels.add(BLOOMFILTER); } - return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this); + return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this, queryMetrics); } return blocks; diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java index 2e5d55c28c..e926072327 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java @@ -793,7 +793,7 @@ void writeColumnChunk(ColumnDescriptor descriptor, // write bloom filter if one of data pages is not dictionary encoded boolean isWriteBloomFilter = false; for (Encoding encoding : dataEncodings) { - if (encoding != Encoding.RLE_DICTIONARY) { + if (encoding != Encoding.PLAIN_DICTIONARY && encoding != Encoding.RLE_DICTIONARY) { isWriteBloomFilter = true; break; } diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index aa4cfe65fa..6525c83ff7 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 1fd675c160..7779972fec 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index 3bb1bcd72c..ea898c8ab0 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 8bc1f0514c..01444170d0 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index caf59aaa75..d887bacf56 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 8913816b38..e78c5f41b1 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index a8c25e554a..4438715fb0 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index f9b933b49e..1fc3380ef8 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 4.0.0 diff --git a/pom.xml b/pom.xml index b0dd5b0281..c53b98e1e8 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-kylin-r1 + 1.12.2-kylin-r3 pom Apache Parquet MR @@ -155,16 +155,30 @@ test + + + + + + + + + + + + + + - user-snapshots + open User Project SNAPSHOTS - http://repository.kyligence.io:8081/repository/maven-snapshots/ + https://repository.kyligence.io/repository/maven-snapshots/ - user-releases + open User Project Release - http://repository.kyligence.io:8081/repository/maven-releases/ + https://repository.kyligence.io/repository/maven-releases/ From 26fc72e472bf11a551d2e2b79e2bb27e8266323d Mon Sep 17 00:00:00 2001 From: ChenliangLu <31469905+yabola@users.noreply.github.com> Date: Thu, 20 Apr 2023 15:36:31 +0800 Subject: [PATCH 31/36] PARQUET-2254 Support building dynamic bloom filter that adapts to the data (#67) --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- .../parquet/column/ParquetProperties.java | 35 +- .../parquet/column/impl/ColumnWriterBase.java | 10 +- .../bloomfilter/DynamicBlockBloomFilter.java | 321 ++++++++++++++++++ .../TestBlockSplitBloomFilter.java | 32 ++ parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- .../parquet/hadoop/ParquetFileReader.java | 18 +- .../parquet/hadoop/ParquetFileWriter.java | 15 +- .../parquet/hadoop/ParquetOutputFormat.java | 16 +- .../apache/parquet/hadoop/ParquetWriter.java | 23 ++ .../parquet/hadoop/TestBloomFiltering.java | 174 +++++++--- .../TestDynamicBlockBloomFiltering.java | 78 +++++ parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 32 files changed, 676 insertions(+), 90 deletions(-) create mode 100644 parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/DynamicBlockBloomFilter.java create mode 100644 parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDynamicBlockBloomFiltering.java diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 3fe44873d3..d033b5e200 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 4158450958..d46df7c967 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 1c4c02bebe..4c5ba16560 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 3b80fc8467..f1db451b3a 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 5d45f2ebee..dbb0b44ddf 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 2445c78765..700a6673f9 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 3ee1d0de3d..eb9bb6fb4d 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java index 7e55ebcdb9..1ca1217a17 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java @@ -18,6 +18,8 @@ */ package org.apache.parquet.column; +import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt; + import java.util.Objects; import java.util.OptionalLong; @@ -37,8 +39,6 @@ import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter; import org.apache.parquet.schema.MessageType; -import static org.apache.parquet.bytes.BytesUtils.getWidthFromMaxInt; - /** * This class represents all the configurable Parquet properties. */ @@ -57,6 +57,8 @@ public class ParquetProperties { public static final int DEFAULT_PAGE_ROW_COUNT_LIMIT = 20_000; public static final int DEFAULT_MAX_BLOOM_FILTER_BYTES = 1024 * 1024; public static final boolean DEFAULT_BLOOM_FILTER_ENABLED = false; + public static final boolean DEFAULT_DYNAMIC_BLOOM_FILTER_ENABLED = true; + public static final int DEFAULT_BLOOM_FILTER_CANDIDATE_SIZE = 3; public static final boolean DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED = true; @@ -102,6 +104,8 @@ public static WriterVersion fromString(String name) { private final ColumnProperty bloomFilterNDVs; private final int maxBloomFilterBytes; private final ColumnProperty bloomFilterEnabled; + private final ColumnProperty dynamicBloomFilterEnabled; + private final ColumnProperty bloomFilterCandidateSize; private final int pageRowCountLimit; private final boolean pageWriteChecksumEnabled; private final boolean enableByteStreamSplit; @@ -124,6 +128,8 @@ private ParquetProperties(Builder builder) { this.bloomFilterNDVs = builder.bloomFilterNDVs.build(); this.bloomFilterEnabled = builder.bloomFilterEnabled.build(); this.maxBloomFilterBytes = builder.maxBloomFilterBytes; + this.dynamicBloomFilterEnabled = builder.dynamicBloomFilterEnabled.build(); + this.bloomFilterCandidateSize = builder.bloomFilterCandidateSize.build(); this.pageRowCountLimit = builder.pageRowCountLimit; this.pageWriteChecksumEnabled = builder.pageWriteChecksumEnabled; this.enableByteStreamSplit = builder.enableByteStreamSplit; @@ -266,6 +272,14 @@ public int getMaxBloomFilterBytes() { return maxBloomFilterBytes; } + public boolean getDynamicBloomFilterEnabled(ColumnDescriptor column) { + return dynamicBloomFilterEnabled.getValue(column); + } + + public int getBloomFilterCandidateSize(ColumnDescriptor column) { + return bloomFilterCandidateSize.getValue(column); + } + public static Builder builder() { return new Builder(); } @@ -306,6 +320,8 @@ public static class Builder { private int statisticsTruncateLength = DEFAULT_STATISTICS_TRUNCATE_LENGTH; private final ColumnProperty.Builder bloomFilterNDVs; private int maxBloomFilterBytes = DEFAULT_MAX_BLOOM_FILTER_BYTES; + private final ColumnProperty.Builder dynamicBloomFilterEnabled; + private final ColumnProperty.Builder bloomFilterCandidateSize; private final ColumnProperty.Builder bloomFilterEnabled; private int pageRowCountLimit = DEFAULT_PAGE_ROW_COUNT_LIMIT; private boolean pageWriteChecksumEnabled = DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED; @@ -315,6 +331,8 @@ private Builder() { enableDict = ColumnProperty.builder().withDefaultValue(DEFAULT_IS_DICTIONARY_ENABLED); bloomFilterEnabled = ColumnProperty.builder().withDefaultValue(DEFAULT_BLOOM_FILTER_ENABLED); bloomFilterNDVs = ColumnProperty.builder().withDefaultValue(null); + dynamicBloomFilterEnabled = ColumnProperty.builder().withDefaultValue(DEFAULT_DYNAMIC_BLOOM_FILTER_ENABLED); + bloomFilterCandidateSize = ColumnProperty.builder().withDefaultValue(DEFAULT_BLOOM_FILTER_CANDIDATE_SIZE); } private Builder(ParquetProperties toCopy) { @@ -331,6 +349,8 @@ private Builder(ParquetProperties toCopy) { this.pageWriteChecksumEnabled = toCopy.pageWriteChecksumEnabled; this.bloomFilterNDVs = ColumnProperty.builder(toCopy.bloomFilterNDVs); this.bloomFilterEnabled = ColumnProperty.builder(toCopy.bloomFilterEnabled); + this.dynamicBloomFilterEnabled = ColumnProperty.builder(toCopy.dynamicBloomFilterEnabled); + this.bloomFilterCandidateSize = ColumnProperty.builder(toCopy.bloomFilterCandidateSize); this.maxBloomFilterBytes = toCopy.maxBloomFilterBytes; this.enableByteStreamSplit = toCopy.enableByteStreamSplit; } @@ -484,6 +504,17 @@ public Builder withBloomFilterEnabled(boolean enabled) { return this; } + public Builder withDynamicBloomFilterEnabled(String columnPath, boolean enabled) { + this.dynamicBloomFilterEnabled.withDefaultValue(enabled); + return this; + } + + public Builder withBloomFilterCandidateSize(String columnPath, int size) { + Preconditions.checkArgument(size > 0, "Invalid candidate size for column \"%s\": %d", columnPath, size); + this.bloomFilterCandidateSize.withDefaultValue(size); + return this; + } + /** * Enable or disable the bloom filter for the specified column. * One may either disable bloom filters for all columns by invoking {@link #withBloomFilterEnabled(boolean)} with a diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java index fce085fe50..1049e7d22e 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java @@ -31,11 +31,11 @@ import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; import org.apache.parquet.column.values.bloomfilter.BloomFilter; import org.apache.parquet.column.values.bloomfilter.BloomFilterWriter; +import org.apache.parquet.column.values.bloomfilter.DynamicBlockBloomFilter; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.api.Binary; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.parquet.bytes.BytesInput; /** * Base implementation for {@link ColumnWriter} to be extended to specialize for V1 and V2 pages. @@ -95,7 +95,13 @@ abstract class ColumnWriterBase implements ColumnWriter { int optimalNumOfBits = BlockSplitBloomFilter.optimalNumOfBits(ndv.getAsLong(), BlockSplitBloomFilter.DEFAULT_FPP); this.bloomFilter = new BlockSplitBloomFilter(optimalNumOfBits / 8, maxBloomFilterSize); } else { - this.bloomFilter = new BlockSplitBloomFilter(maxBloomFilterSize); + boolean useDynamicBloomFilter = props.getDynamicBloomFilterEnabled(path); + int candidateSize = props.getBloomFilterCandidateSize(path); + if(useDynamicBloomFilter) { + this.bloomFilter = new DynamicBlockBloomFilter(maxBloomFilterSize, candidateSize, BlockSplitBloomFilter.DEFAULT_FPP, path); + } else { + this.bloomFilter = new BlockSplitBloomFilter(maxBloomFilterSize); + } } } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/DynamicBlockBloomFilter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/DynamicBlockBloomFilter.java new file mode 100644 index 0000000000..aaca591e7f --- /dev/null +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/DynamicBlockBloomFilter.java @@ -0,0 +1,321 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.column.values.bloomfilter; + +import static org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter.LOWER_BOUND_BYTES; +import static org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter.UPPER_BOUND_BYTES; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Objects; +import java.util.TreeSet; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; + +/** + * `DynamicBlockBloomFilter` contains multiple `BlockSplitBloomFilter` as candidates and inserts values in + * the candidates at the same time. + * The purpose of this is to finally generate a bloom filter with the optimal bit size according to the number + * of real data distinct values. Use the largest bloom filter as an approximate deduplication counter, and then + * remove bad bloom filter candidate during data insertion. + */ +public class DynamicBlockBloomFilter implements BloomFilter { + + private static final Logger LOG = LoggerFactory.getLogger(DynamicBlockBloomFilter.class); + + // multiple candidates, inserting data at the same time. If the deduplication value is greater than the + // expected NDV of candidate, it will be removed and finally choose the smallest candidate to write out. + private final TreeSet candidates = new TreeSet<>(); + + // the largest among candidates and used as an approximate deduplication counter + private BloomFilterCandidate maxCandidate; + + // the accumulator of the number of distinct values that have been inserted so far + private int distinctValueCounter = 0; + + // indicates that the bloom filter candidate has been written out and new data should be no longer allowed to be inserted + private boolean finalized = false; + + private int maximumBytes = UPPER_BOUND_BYTES; + private int minimumBytes = LOWER_BOUND_BYTES; + // the hash strategy used in this bloom filter. + private final HashStrategy hashStrategy; + // the column to build bloom filter + private ColumnDescriptor column; + + public DynamicBlockBloomFilter(int numBytes, int candidatesNum, double fpp, ColumnDescriptor column) { + this(numBytes, LOWER_BOUND_BYTES, UPPER_BOUND_BYTES, HashStrategy.XXH64, fpp, candidatesNum, column); + } + + public DynamicBlockBloomFilter(int numBytes, int maximumBytes, int candidatesNum, double fpp, ColumnDescriptor column) { + this(numBytes, LOWER_BOUND_BYTES, maximumBytes, HashStrategy.XXH64, fpp, candidatesNum, column); + } + + public DynamicBlockBloomFilter(int numBytes, int minimumBytes, int maximumBytes, HashStrategy hashStrategy, + double fpp, int candidatesNum, ColumnDescriptor column) { + if (minimumBytes > maximumBytes) { + throw new IllegalArgumentException("the minimum bytes should be less or equal than maximum bytes"); + } + + if (minimumBytes > LOWER_BOUND_BYTES && minimumBytes < UPPER_BOUND_BYTES) { + this.minimumBytes = minimumBytes; + } + + if (maximumBytes > LOWER_BOUND_BYTES && maximumBytes < UPPER_BOUND_BYTES) { + this.maximumBytes = maximumBytes; + } + this.column = column; + switch (hashStrategy) { + case XXH64: + this.hashStrategy = hashStrategy; + break; + default: + throw new RuntimeException("Unsupported hash strategy"); + } + initCandidates(numBytes, candidatesNum, fpp); + } + + /** + * Given the maximum acceptable bytes size of bloom filter, generate candidates according + * to the max expected distinct values. The size of the candidate bytes needs to be a + * square number of 2. Therefore, set the candidate size according to `maxBytes` of `1/2`, `1/4`, `1/8`, etc. + * + * @param maxBytes the maximum acceptable bit size + * @param candidatesNum the number of candidates + * @param fpp the false positive probability + */ + private void initCandidates(int maxBytes, int candidatesNum, double fpp) { + int candidateByteSize = calculateTwoPowerSize(maxBytes); + for (int i = 1; i <= candidatesNum; i++) { + int candidateExpectedNDV = expectedNDV(candidateByteSize, fpp); + // `candidateByteSize` is too small, just drop it + if (candidateExpectedNDV <= 0) { + break; + } + BloomFilterCandidate candidate = + new BloomFilterCandidate(candidateExpectedNDV, candidateByteSize, minimumBytes, maximumBytes, hashStrategy); + candidates.add(candidate); + candidateByteSize = calculateTwoPowerSize(candidateByteSize / 2); + } + maxCandidate = candidates.last(); + } + + /** + * According to the size of bytes, calculate the expected number of distinct values. + * The expected number result may be slightly smaller than what `numBytes` can support. + * + * @param numBytes the bytes size + * @param fpp the false positive probability + * @return the expected number of distinct values + */ + private int expectedNDV(int numBytes, double fpp) { + int expectedNDV = 0; + int optimalBytes = 0; + int step = 500; + while (optimalBytes < numBytes) { + expectedNDV += step; + optimalBytes = BlockSplitBloomFilter.optimalNumOfBits(expectedNDV, fpp); + } + // make sure it is slightly smaller than what `numBytes` can support + expectedNDV -= step; + + // numBytes is too small, 256 Bytes -> 1 NDV + if (expectedNDV <= 0 && numBytes >= 256) { + expectedNDV = 1; + } else if (expectedNDV <= 0) { + expectedNDV = 0; + } + return expectedNDV; + } + + /** + * BloomFilter bitsets size should be power of 2, see [[BlockSplitBloomFilter#initBitset]] + * + * @param numBytes the bytes size + * @return power of 2 less than or equal to numBytes + */ + private int calculateTwoPowerSize(int numBytes) { + if (numBytes < minimumBytes) { + numBytes = minimumBytes; + } + // Get power of 2 less than or equal to numBytes if it is not power of 2. + if ((numBytes & (numBytes - 1)) != 0) { + numBytes = Integer.highestOneBit(numBytes) >> 1; + } + if (numBytes > maximumBytes || numBytes < 0) { + numBytes = maximumBytes; + } + if (numBytes < minimumBytes) { + numBytes = minimumBytes; + } + return numBytes; + } + + /** + * Used at the end of the insertion, select the candidate of the smallest size. + * At least one of the largest candidates will be kept when inserting data. + * + * @return the smallest candidate + */ + protected BloomFilterCandidate optimalCandidate() { + return candidates.stream() + .min(BloomFilterCandidate::compareTo).get(); + } + + protected TreeSet getCandidates() { + return candidates; + } + + @Override + public void writeTo(OutputStream out) throws IOException { + finalized = true; + BloomFilterCandidate optimalBloomFilter = optimalCandidate(); + optimalBloomFilter.bloomFilter.writeTo(out); + String columnName = column != null && column.getPath() != null ? Arrays.toString(column.getPath()) : "unknown"; + LOG.info("The number of distinct values in {} is approximately {}, the optimal bloom filter NDV is {}, byte size is {}.", + columnName, distinctValueCounter, optimalBloomFilter.getExpectedNDV(), + optimalBloomFilter.bloomFilter.getBitsetSize()); + } + + /** + * Insert an element to the multiple bloom filter candidates and remove the bad candidate + * if the number of distinct values exceeds its expected size. + * + * @param hash the hash result of element. + */ + @Override + public void insertHash(long hash) { + Preconditions.checkArgument(!finalized, + "Dynamic bloom filter insertion has been mark as finalized, no more data is allowed!"); + if (!maxCandidate.bloomFilter.findHash(hash)) { + distinctValueCounter++; + } + // distinct values exceed the expected size, remove the bad bloom filter (leave at least the max bloom filter candidate) + candidates.removeIf(candidate -> candidate.getExpectedNDV() < distinctValueCounter && candidate != maxCandidate); + candidates.forEach(candidate -> candidate.getBloomFilter().insertHash(hash)); + } + + @Override + public int getBitsetSize() { + return optimalCandidate().getBloomFilter().getBitsetSize(); + } + + @Override + public boolean findHash(long hash) { + return maxCandidate.bloomFilter.findHash(hash); + } + + @Override + public long hash(Object value) { + return maxCandidate.bloomFilter.hash(value); + } + + @Override + public HashStrategy getHashStrategy() { + return maxCandidate.bloomFilter.getHashStrategy(); + } + + @Override + public Algorithm getAlgorithm() { + return maxCandidate.bloomFilter.getAlgorithm(); + } + + @Override + public Compression getCompression() { + return maxCandidate.bloomFilter.getCompression(); + } + + @Override + public long hash(int value) { + return maxCandidate.bloomFilter.hash(value); + } + + @Override + public long hash(long value) { + return maxCandidate.bloomFilter.hash(value); + } + + @Override + public long hash(double value) { + return maxCandidate.bloomFilter.hash(value); + } + + @Override + public long hash(float value) { + return maxCandidate.bloomFilter.hash(value); + } + + @Override + public long hash(Binary value) { + return maxCandidate.bloomFilter.hash(value); + } + + protected class BloomFilterCandidate implements Comparable { + private BlockSplitBloomFilter bloomFilter; + private int expectedNDV; + + public BloomFilterCandidate(int expectedNDV, int candidateBytes, + int minimumBytes, int maximumBytes, HashStrategy hashStrategy) { + this.bloomFilter = new BlockSplitBloomFilter(candidateBytes, minimumBytes, maximumBytes, hashStrategy); + this.expectedNDV = expectedNDV; + } + + public BlockSplitBloomFilter getBloomFilter() { + return bloomFilter; + } + + public void setBloomFilter(BlockSplitBloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + public int getExpectedNDV() { + return expectedNDV; + } + + public void setExpectedNDV(int expectedNDV) { + this.expectedNDV = expectedNDV; + } + + @Override + public int compareTo(BloomFilterCandidate o) { + return this.bloomFilter.getBitsetSize() - o.bloomFilter.getBitsetSize(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + BloomFilterCandidate that = (BloomFilterCandidate) o; + return expectedNDV == that.expectedNDV && + Objects.equals(bloomFilter, that.bloomFilter); + } + + @Override + public int hashCode() { + return Objects.hash(bloomFilter, expectedNDV); + } + } +} diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/bloomfilter/TestBlockSplitBloomFilter.java b/parquet-column/src/test/java/org/apache/parquet/column/values/bloomfilter/TestBlockSplitBloomFilter.java index 6adabe79d8..9403200995 100644 --- a/parquet-column/src/test/java/org/apache/parquet/column/values/bloomfilter/TestBlockSplitBloomFilter.java +++ b/parquet-column/src/test/java/org/apache/parquet/column/values/bloomfilter/TestBlockSplitBloomFilter.java @@ -144,6 +144,38 @@ public void testBloomFilterFPPAccuracy() { assertTrue(exist < totalCount * (FPP * 1.2)); } + @Test + public void testDynamicBloomFilter() { + int maxBloomFilterSize = 1024 * 1024; + int candidateSize = 10; + DynamicBlockBloomFilter dynamicBloomFilter = new DynamicBlockBloomFilter(maxBloomFilterSize, + candidateSize, 0.01, null); + + assertEquals(candidateSize, dynamicBloomFilter.getCandidates().size()); + + Set existedValue = new HashSet<>(); + while (existedValue.size() < 10000) { + String str = RandomStringUtils.randomAlphabetic(1, 64); + dynamicBloomFilter.insertHash(dynamicBloomFilter.hash(Binary.fromString(str))); + existedValue.add(str); + } + // removed some small bloom filter + assertEquals(4, dynamicBloomFilter.getCandidates().size()); + BlockSplitBloomFilter optimalCandidate = dynamicBloomFilter.optimalCandidate().getBloomFilter(); + for (String value : existedValue) { + assertTrue(optimalCandidate.findHash(optimalCandidate.hash(Binary.fromString(value)))); + } + + int maxCandidateNDV = dynamicBloomFilter.getCandidates().last().getExpectedNDV(); + while (existedValue.size() < maxCandidateNDV + 1) { + String str = RandomStringUtils.randomAlphabetic(1, 64); + dynamicBloomFilter.insertHash(dynamicBloomFilter.hash(Binary.fromString(str))); + existedValue.add(str); + } + // the number of distinct value exceeds the maximum candidate's expected NDV, so only the maximum candidate is kept + assertEquals(1, dynamicBloomFilter.getCandidates().size()); + } + @Test public void testEquals() { final String[] words = {"hello", "parquet", "bloom", "filter"}; diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index ae5b9ff1e9..7ab59886ef 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index f0008df206..1c2ef36be9 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 1a777ac8dc..1709f85863 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 2f0cc9ef0f..da0dfd8c3c 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 6bfde2a874..929bebca1c 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 7a924f9f1a..ae520f3739 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 18d3ebc1ea..e63d934b46 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -60,6 +60,10 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.yetus.audience.InterfaceAudience.Private; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.bytes.ByteBufferInputStream; @@ -81,8 +85,8 @@ import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.filter2.compat.FilterCompat; -import org.apache.parquet.filter2.compat.RowGroupFilter; import org.apache.parquet.filter2.compat.QueryMetrics; +import org.apache.parquet.filter2.compat.RowGroupFilter; import org.apache.parquet.format.BlockCipher; import org.apache.parquet.format.BloomFilterHeader; import org.apache.parquet.format.DataPageHeader; @@ -114,9 +118,6 @@ import org.apache.parquet.io.SeekableInputStream; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.apache.yetus.audience.InterfaceAudience.Private; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Internal implementation of the Parquet file reader as a block container @@ -771,9 +772,11 @@ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer) } public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOException { + long start = System.currentTimeMillis(); this.converter = new ParquetMetadataConverter(options); this.file = file; this.f = file.newStream(); + long openStreamEndTime = System.currentTimeMillis(); this.options = options; try { this.footer = readFooter(file, options, f, converter); @@ -783,6 +786,7 @@ public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOEx f.close(); throw e; } + long footerReadEndtime = System.currentTimeMillis(); this.fileMetaData = footer.getFileMetaData(); this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! if (null != fileDecryptor && fileDecryptor.plaintextFile()) { @@ -796,6 +800,12 @@ public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOEx paths.put(ColumnPath.get(col.getPath()), col); } this.crc = options.usePageChecksumVerification() ? new CRC32() : null; + long filterRowGroupEndTime = System.currentTimeMillis(); + if ((filterRowGroupEndTime - start) > 50) { + LOG.info("open stream costs {} ms, read footer costs {} ms, filter rowGroups cost {} ms", + (openStreamEndTime - start), (footerReadEndtime - openStreamEndTime), + (filterRowGroupEndTime - footerReadEndtime)); + } } private static List listWithNulls(int size) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java index e926072327..8f425bda6b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileWriter.java @@ -42,6 +42,8 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.parquet.Preconditions; import org.apache.parquet.Version; @@ -62,19 +64,19 @@ import org.apache.parquet.crypto.ModuleCipherFactory; import org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; -import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.format.BlockCipher; import org.apache.parquet.format.Util; import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.hadoop.metadata.CompressionCodecName; -import org.apache.parquet.hadoop.metadata.StrictKeyValueMetadataMergeStrategy; import org.apache.parquet.hadoop.metadata.FileMetaData; import org.apache.parquet.hadoop.metadata.GlobalMetaData; import org.apache.parquet.hadoop.metadata.KeyValueMetadataMergeStrategy; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.metadata.StrictKeyValueMetadataMergeStrategy; import org.apache.parquet.hadoop.util.HadoopOutputFile; import org.apache.parquet.hadoop.util.HadoopStreams; import org.apache.parquet.internal.column.columnindex.ColumnIndex; @@ -84,14 +86,12 @@ import org.apache.parquet.internal.hadoop.metadata.IndexReference; import org.apache.parquet.io.InputFile; import org.apache.parquet.io.OutputFile; -import org.apache.parquet.io.SeekableInputStream; import org.apache.parquet.io.ParquetEncodingException; import org.apache.parquet.io.PositionOutputStream; +import org.apache.parquet.io.SeekableInputStream; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.TypeUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Internal implementation of the Parquet file writer as a block container @@ -800,6 +800,9 @@ void writeColumnChunk(ColumnDescriptor descriptor, } if (isWriteBloomFilter) { currentBloomFilters.put(String.join(".", descriptor.getPath()), bloomFilter); + } else { + LOG.info("skip write bloom filter because column is encoded as dictionary, column path {}", + descriptor.getPath()); } } LOG.debug("{}: write data pages", out.getPos()); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java index 6be27e47bd..7d4d7e10f3 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java @@ -153,6 +153,8 @@ public static enum JobSummaryLevel { public static final String BLOOM_FILTER_MAX_BYTES = "parquet.bloom.filter.max.bytes"; public static final String PAGE_ROW_COUNT_LIMIT = "parquet.page.row.count.limit"; public static final String PAGE_WRITE_CHECKSUM_ENABLED = "parquet.page.write-checksum.enabled"; + public static final String DYNAMIC_BLOOM_FILTER_ENABLED = "parquet.bloom.filter.dynamic.enabled"; + public static final String BLOOM_FILTER_CANDIDATE_SIZE = "parquet.bloom.filter.candidate.size"; public static JobSummaryLevel getJobSummaryLevel(Configuration conf) { String level = conf.get(JOB_SUMMARY_LEVEL); @@ -226,10 +228,12 @@ public static int getBloomFilterMaxBytes(Configuration conf) { public static boolean getBloomFilterEnabled(Configuration conf) { return conf.getBoolean(BLOOM_FILTER_ENABLED, DEFAULT_BLOOM_FILTER_ENABLED); } + public static int getBlockSize(JobContext jobContext) { return getBlockSize(getConfiguration(jobContext)); } + public static int getPageSize(JobContext jobContext) { return getPageSize(getConfiguration(jobContext)); } @@ -458,6 +462,14 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp .withColumnConfig(BLOOM_FILTER_ENABLED, key -> conf.getBoolean(key, false), propsBuilder::withBloomFilterEnabled) .withColumnConfig(BLOOM_FILTER_EXPECTED_NDV, key -> conf.getLong(key, -1L), propsBuilder::withBloomFilterNDV) + .withColumnConfig( + DYNAMIC_BLOOM_FILTER_ENABLED, + key -> conf.getBoolean(key, ParquetProperties.DEFAULT_DYNAMIC_BLOOM_FILTER_ENABLED), + propsBuilder::withDynamicBloomFilterEnabled) + .withColumnConfig( + BLOOM_FILTER_CANDIDATE_SIZE, + key -> conf.getInt(key, ParquetProperties.DEFAULT_BLOOM_FILTER_CANDIDATE_SIZE), + propsBuilder::withBloomFilterCandidateSize) .parseConfig(conf); ParquetProperties props = propsBuilder.build(); @@ -474,9 +486,9 @@ public RecordWriter getRecordWriter(Configuration conf, Path file, Comp } WriteContext fileWriteContext = writeSupport.init(conf); - + FileEncryptionProperties encryptionProperties = createEncryptionProperties(conf, file, fileWriteContext); - + ParquetFileWriter w = new ParquetFileWriter(HadoopOutputFile.fromPath(file, conf), fileWriteContext.getSchema(), mode, blockSize, maxPaddingSize, props.getColumnIndexTruncateLength(), props.getStatisticsTruncateLength(), props.getPageWriteChecksumEnabled(), encryptionProperties); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java index 9a71987c6b..18dc2f2e8b 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java @@ -627,6 +627,29 @@ public SELF withBloomFilterEnabled(String columnPath, boolean enabled) { return self(); } + /** + * When NDV (number of distinct values) for a specified column is not set, whether to use + * `DynamicBloomFilter` to automatically adjust the BloomFilter size according to `parquet.bloom.filter.max.bytes` + * + * @param columnPath the path of the column (dot-string) + * @param enabled whether to write bloom filter for the column + */ + public SELF withDynamicBloomFilterEnabled(String columnPath, boolean enabled) { + encodingPropsBuilder.withDynamicBloomFilterEnabled(columnPath, enabled); + return self(); + } + + /** + * When `DynamicBloomFilter` is enabled, set how many bloomFilters to split as candidates. + * + * @param columnPath the path of the column (dot-string) + * @param size the candidate size + */ + public SELF withBloomFilterCandidateSize(String columnPath, int size) { + encodingPropsBuilder.withBloomFilterCandidateSize(columnPath, size); + return self(); + } + /** * Set a property that will be available to the read path. For writers that use a Hadoop * configuration, this is the recommended way to add configuration values. diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java index 4ebe15aecf..b51093d46f 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java @@ -19,27 +19,14 @@ package org.apache.parquet.hadoop; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.column.ParquetProperties; -import org.apache.parquet.crypto.ColumnEncryptionProperties; -import org.apache.parquet.crypto.DecryptionKeyRetrieverMock; -import org.apache.parquet.crypto.FileDecryptionProperties; -import org.apache.parquet.crypto.FileEncryptionProperties; -import org.apache.parquet.filter2.compat.FilterCompat; -import org.apache.parquet.filter2.predicate.FilterPredicate; -import org.apache.parquet.filter2.recordlevel.PhoneBookWriter; -import org.apache.parquet.hadoop.example.ExampleParquetWriter; -import org.apache.parquet.hadoop.example.GroupReadSupport; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.io.api.Binary; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn; +import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn; +import static org.apache.parquet.filter2.predicate.FilterApi.eq; +import static org.apache.parquet.filter2.predicate.FilterApi.longColumn; +import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.nio.file.Files; @@ -48,17 +35,41 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.parquet.filter2.predicate.FilterApi.*; -import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE; -import static org.junit.Assert.*; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.crypto.ColumnEncryptionProperties; +import org.apache.parquet.crypto.DecryptionKeyRetrieverMock; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.crypto.FileEncryptionProperties; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.recordlevel.PhoneBookWriter; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; +import org.apache.parquet.io.api.Binary; @RunWith(Parameterized.class) public class TestBloomFiltering { @@ -148,6 +159,22 @@ private static List generateNames(int rowCount) { return list; } + protected static List generateDictionaryData(int rowCount) { + List users = new ArrayList<>(); + List names = new ArrayList<>(); + for (int i = 0; i < rowCount / 5; i++) { + names.add("miller"); + names.add("anderson"); + names.add("thomas"); + names.add("chenLiang"); + names.add("len"); + } + for (int i = 0; i < rowCount; ++i) { + users.add(new PhoneBookWriter.User(i, names.get(i), generatePhoneNumbers(), generateLocation(i, rowCount))); + } + return users; + } + private static List generatePhoneNumbers() { int length = RANDOM.nextInt(5) - 1; if (length < 0) { @@ -178,18 +205,8 @@ private static PhoneBookWriter.Location generateLocation(int id, int rowCount) { } private List readUsers(FilterPredicate filter, boolean useOtherFiltering, - boolean useBloomFilter) throws IOException { - FileDecryptionProperties fileDecryptionProperties = null; - if (isEncrypted) { - DecryptionKeyRetrieverMock decryptionKeyRetrieverMock = new DecryptionKeyRetrieverMock() - .putKey(FOOTER_ENCRYPTION_KEY_ID, FOOTER_ENCRYPTION_KEY) - .putKey(COLUMN_ENCRYPTION_KEY1_ID, COLUMN_ENCRYPTION_KEY1) - .putKey(COLUMN_ENCRYPTION_KEY2_ID, COLUMN_ENCRYPTION_KEY2); - - fileDecryptionProperties = FileDecryptionProperties.builder() - .withKeyRetriever(decryptionKeyRetrieverMock) - .build(); - } + boolean useBloomFilter) throws IOException { + FileDecryptionProperties fileDecryptionProperties = getFileDecryptionProperties(); return PhoneBookWriter.readUsers(ParquetReader.builder(new GroupReadSupport(), file) .withFilter(FilterCompat.get(filter)) @@ -201,6 +218,20 @@ private List readUsers(FilterPredicate filter, boolean use .useColumnIndexFilter(useOtherFiltering)); } + public FileDecryptionProperties getFileDecryptionProperties() { + if (!isEncrypted) { + return null; + } + DecryptionKeyRetrieverMock decryptionKeyRetrieverMock = new DecryptionKeyRetrieverMock() + .putKey(FOOTER_ENCRYPTION_KEY_ID, FOOTER_ENCRYPTION_KEY) + .putKey(COLUMN_ENCRYPTION_KEY1_ID, COLUMN_ENCRYPTION_KEY1) + .putKey(COLUMN_ENCRYPTION_KEY2_ID, COLUMN_ENCRYPTION_KEY2); + + return FileDecryptionProperties.builder() + .withKeyRetriever(decryptionKeyRetrieverMock) + .build(); + } + // Assumes that both lists are in the same order private static void assertContains(Stream expected, List actual) { Iterator expIt = expected.iterator(); @@ -237,7 +268,7 @@ private void assertCorrectFiltering(Predicate expectedFilt assertEquals(DATA.stream().filter(expectedFilter).collect(Collectors.toList()), result); } - private static FileEncryptionProperties getFileEncryptionProperties() { + protected static FileEncryptionProperties getFileEncryptionProperties() { ColumnEncryptionProperties columnProperties1 = ColumnEncryptionProperties .builder("id") .withKey(COLUMN_ENCRYPTION_KEY1) @@ -262,35 +293,55 @@ private static FileEncryptionProperties getFileEncryptionProperties() { return encryptionProperties; } - private static void writePhoneBookToFile(Path file, - ParquetProperties.WriterVersion parquetVersion, - FileEncryptionProperties encryptionProperties) throws IOException { + protected static void writePhoneBookToFile(Path file, + ParquetProperties.WriterVersion parquetVersion, + FileEncryptionProperties encryptionProperties, + boolean useDynamicBloomFilter) throws IOException { int pageSize = DATA.size() / 100; // Ensure that several pages will be created int rowGroupSize = pageSize * 4; // Ensure that there are more row-groups created - PhoneBookWriter.write(ExampleParquetWriter.builder(file) - .withWriteMode(OVERWRITE) - .withRowGroupSize(rowGroupSize) - .withPageSize(pageSize) + ExampleParquetWriter.Builder writeBuilder = ExampleParquetWriter.builder(file) + .withWriteMode(OVERWRITE) + .withRowGroupSize(rowGroupSize) + .withPageSize(pageSize) + .withEncryption(encryptionProperties) + .withWriterVersion(parquetVersion); + if (useDynamicBloomFilter) { + writeBuilder + .withBloomFilterEnabled("location.lat", true) + .withBloomFilterCandidateSize("location.lat", 10) + .withBloomFilterEnabled("name", true) + .withBloomFilterCandidateSize("name", 10) + .withBloomFilterEnabled("id", true) + .withBloomFilterCandidateSize("id", 10); + } else { + writeBuilder .withBloomFilterNDV("location.lat", 10000L) .withBloomFilterNDV("name", 10000L) - .withBloomFilterNDV("id", 10000L) - .withEncryption(encryptionProperties) - .withWriterVersion(parquetVersion), - DATA); + .withBloomFilterNDV("id", 10000L); + } + PhoneBookWriter.write(writeBuilder, DATA); } private static void deleteFile(Path file) throws IOException { file.getFileSystem(new Configuration()).delete(file, false); } + public Path getFile() { + return file; + } + @BeforeClass public static void createFiles() throws IOException { - writePhoneBookToFile(FILE_V1, ParquetProperties.WriterVersion.PARQUET_1_0, null); - writePhoneBookToFile(FILE_V2, ParquetProperties.WriterVersion.PARQUET_2_0, null); + createFiles(false); + } + + public static void createFiles(boolean useDynamicBloomFilter) throws IOException { + writePhoneBookToFile(FILE_V1, ParquetProperties.WriterVersion.PARQUET_1_0, null, useDynamicBloomFilter); + writePhoneBookToFile(FILE_V2, ParquetProperties.WriterVersion.PARQUET_2_0, null, useDynamicBloomFilter); FileEncryptionProperties encryptionProperties = getFileEncryptionProperties(); - writePhoneBookToFile(FILE_V1_E, ParquetProperties.WriterVersion.PARQUET_1_0, encryptionProperties); - writePhoneBookToFile(FILE_V2_E, ParquetProperties.WriterVersion.PARQUET_2_0, encryptionProperties); + writePhoneBookToFile(FILE_V1_E, ParquetProperties.WriterVersion.PARQUET_1_0, encryptionProperties, useDynamicBloomFilter); + writePhoneBookToFile(FILE_V2_E, ParquetProperties.WriterVersion.PARQUET_2_0, encryptionProperties, useDynamicBloomFilter); } @AfterClass @@ -322,4 +373,23 @@ record -> { }, eq(doubleColumn("location.lat"), 99.9)); } + + @Test + public void checkBloomFilterSize() throws IOException { + FileDecryptionProperties fileDecryptionProperties = getFileDecryptionProperties(); + final ParquetReadOptions readOptions = ParquetReadOptions.builder().withDecryption(fileDecryptionProperties).build(); + InputFile inputFile = HadoopInputFile.fromPath(getFile(), new Configuration()); + try (ParquetFileReader fileReader = ParquetFileReader.open(inputFile, readOptions)) { + fileReader.getRowGroups().forEach(block -> { + BloomFilterReader bloomFilterReader = fileReader.getBloomFilterDataReader(block); + block.getColumns().stream() + .filter(column -> column.getBloomFilterOffset() > 0) + .forEach(column -> { + int bitsetSize = bloomFilterReader.readBloomFilter(column).getBitsetSize(); + // when setting nvd to a fixed value 10000L, bitsetSize will always be 16384 + assertEquals(16384, bitsetSize); + }); + }); + } + } } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDynamicBlockBloomFiltering.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDynamicBlockBloomFiltering.java new file mode 100644 index 0000000000..1f9f5fc1cc --- /dev/null +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDynamicBlockBloomFiltering.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.hadoop; + +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.crypto.FileDecryptionProperties; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.io.InputFile; + +public class TestDynamicBlockBloomFiltering extends TestBloomFiltering { + + @BeforeClass + public static void createFiles() throws IOException { + createFiles(true); + } + + public TestDynamicBlockBloomFiltering(Path file, boolean isEncrypted) { + super(file, isEncrypted); + } + + @Test + public void testSimpleFiltering() throws IOException { + super.testSimpleFiltering(); + } + + @Test + public void testNestedFiltering() throws IOException { + super.testNestedFiltering(); + } + + @Test + public void checkBloomFilterSize() throws IOException { + FileDecryptionProperties fileDecryptionProperties = getFileDecryptionProperties(); + final ParquetReadOptions readOptions = ParquetReadOptions.builder().withDecryption(fileDecryptionProperties).build(); + InputFile inputFile = HadoopInputFile.fromPath(getFile(), new Configuration()); + try (ParquetFileReader fileReader = ParquetFileReader.open(inputFile, readOptions)) { + fileReader.getRowGroups().forEach(block -> { + BloomFilterReader bloomFilterReader = fileReader.getBloomFilterDataReader(block); + block.getColumns().stream() + .filter(column -> column.getBloomFilterOffset() > 0) + .forEach(column -> { + int bitsetSize = bloomFilterReader.readBloomFilter(column).getBitsetSize(); + // set 10 candidates: + // [bitSize=8192, expectedNVD=500], [bitSize=16384, expectedNVD=1500] , [bitSize=32768, expectedNVD=3000], + // [bitSize=65536, expectedNVD=6500] ........ + // number of distinct values is less than 100, so the bitSize should be less than 131073. + assertTrue(bitsetSize <= 8192); + }); + }); + } + } +} diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 6525c83ff7..7ec8e8a54e 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 7779972fec..0bf63be30f 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index ea898c8ab0..bf48b28dcc 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 01444170d0..f1144e5230 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index d887bacf56..37ff4cdb50 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index e78c5f41b1..5682a6df17 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index 4438715fb0..ac2b219996 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 1fc3380ef8..48611db8a4 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 4.0.0 diff --git a/pom.xml b/pom.xml index c53b98e1e8..b1f1130b0d 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-kylin-r3 + 1.12.2-kylin-r4 pom Apache Parquet MR From 894234b32cd40d91542957e2237864c354858370 Mon Sep 17 00:00:00 2001 From: Yaguang Jia Date: Fri, 28 Apr 2023 10:14:28 +0800 Subject: [PATCH 32/36] KE-40433 add page index filter log and release 1.12.2-kylin-r5 (#68) * KE-40433 add page index filter log * KE-40433 release 1.12.2-kylin-r5 --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- .../parquet/filter2/compat/QueryMetrics.java | 33 +++++++++++++++++++ .../parquet/hadoop/ParquetFileReader.java | 1 + parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 24 files changed, 56 insertions(+), 22 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index d033b5e200..3b59ae57ff 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index d46df7c967..58ab779b66 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 4c5ba16560..1920a17230 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index f1db451b3a..7ba330216b 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index dbb0b44ddf..432d98db17 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 700a6673f9..5863476640 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index eb9bb6fb4d..02f861ff79 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 7ab59886ef..849ac5af82 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 1c2ef36be9..52daf8e58a 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index 1709f85863..ab7489f71d 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index da0dfd8c3c..67fd36b5cd 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 929bebca1c..99b7d41a3a 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index ae520f3739..b73654b6cf 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java index ac6d90b616..ed9a547b3f 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/QueryMetrics.java @@ -24,6 +24,9 @@ public class QueryMetrics { private long skipBloomBlocks = 0; private long skipBloomRows = 0; private long totalBloomBlocks = 0; + private long totalPagesCount = 0; + private long filteredPagesCount = 0; + private long afterFilterPagesCount = 0; public String getSkipBloomFilter() { return skipBloomFilter; @@ -56,4 +59,34 @@ public long getTotalBloomBlocks() { public void setTotalBloomBlocks(long totalBloomBlocks) { this.totalBloomBlocks = totalBloomBlocks; } + + public long getTotalPagesCount() { + return totalPagesCount; + } + + public void setTotalPagesCount(long totalPagesCount) { + this.totalPagesCount = totalPagesCount; + } + + public long getFilteredPagesCount() { + return filteredPagesCount; + } + + public void setFilteredPagesCount(long filteredPagesCount) { + this.filteredPagesCount = filteredPagesCount; + } + + public long getAfterFilterPagesCount() { + return afterFilterPagesCount; + } + + public void setAfterFilterPagesCount(long afterFilterPagesCount) { + this.afterFilterPagesCount = afterFilterPagesCount; + } + + public void logParquetPageFilter(long total, long afterFilter) { + totalPagesCount += total; + filteredPagesCount += (total - afterFilter); + afterFilterPagesCount += afterFilter; + } } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index e63d934b46..8fe871982d 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -998,6 +998,7 @@ public PageReadStore readNextFilteredRowGroup() throws IOException { OffsetIndex filteredOffsetIndex = filterOffsetIndex(offsetIndex, rowRanges, block.getRowCount()); + this.queryMetrics.logParquetPageFilter(offsetIndex.getPageCount(), filteredOffsetIndex.getPageCount()); for (OffsetRange range : calculateOffsetRanges(filteredOffsetIndex, mc, offsetIndex.getOffset(0))) { BenchmarkCounter.incrementTotalBytes(range.getLength()); long startingPos = range.getOffset(); diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 7ec8e8a54e..394afaebbc 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 0bf63be30f..a701362192 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index bf48b28dcc..b761eed2fc 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index f1144e5230..6cb6abd485 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index 37ff4cdb50..e7bb40f9e2 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 5682a6df17..3f16505389 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index ac2b219996..e2d2affdb1 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 48611db8a4..11f30fe7e4 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 4.0.0 diff --git a/pom.xml b/pom.xml index b1f1130b0d..496667b965 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-kylin-r4 + 1.12.2-kylin-r5 pom Apache Parquet MR From 1f2ac5443dde561e4dd3eabcdca80c68eac6af56 Mon Sep 17 00:00:00 2001 From: Yaguang Jia Date: Tue, 9 May 2023 10:34:17 +0800 Subject: [PATCH 33/36] change distributionManagement (#70) --- pom.xml | 45 +++++++++++++++++++++++---------------------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 496667b965..ffc5b79a25 100644 --- a/pom.xml +++ b/pom.xml @@ -155,32 +155,33 @@ test - - - - - - - - - - - - - - - - open - User Project SNAPSHOTS - https://repository.kyligence.io/repository/maven-snapshots/ - - open - User Project Release - https://repository.kyligence.io/repository/maven-releases/ + ${repository.id} + ${repository.url} + ${repository.name} + default + + ${repository.id.snapshots} + ${repository.url.snapshots} + ${repository.name.snapshots} + default + + + + + + + + + + + + + + From 6e2172cae734b83618be767cffdcbef8d59741b1 Mon Sep 17 00:00:00 2001 From: "chenliang.lu" Date: Thu, 18 May 2023 15:53:11 +0800 Subject: [PATCH 34/36] KE-41399 Avoid parquet footer reads twice in vectorized reader --- .../parquet/hadoop/ParquetFileReader.java | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 8fe871982d..e1bf68be2c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -663,12 +663,12 @@ public static ParquetFileReader open(InputFile file, ParquetReadOptions options) protected final SeekableInputStream f; private final InputFile file; - private final ParquetReadOptions options; + private ParquetReadOptions options; private final Map paths = new HashMap<>(); private final FileMetaData fileMetaData; // may be null - private final List blocks; - private final List blockIndexStores; - private final List blockRowRanges; + private List blocks; + private List blockIndexStores; + private List blockRowRanges; // not final. in some cases, this may be lazily loaded for backward-compat. private ParquetMetadata footer; @@ -754,17 +754,8 @@ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer) this.f = this.file.newStream(); this.fileMetaData = footer.getFileMetaData(); this.fileDecryptor = fileMetaData.getFileDecryptor(); - if (null == fileDecryptor) { - this.options = HadoopReadOptions.builder(conf).build(); - } else { - this.options = HadoopReadOptions.builder(conf) - .withDecryption(fileDecryptor.getDecryptionProperties()) - .build(); - } this.footer = footer; - this.blocks = filterRowGroups(footer.getBlocks()); - this.blockIndexStores = listWithNulls(this.blocks.size()); - this.blockRowRanges = listWithNulls(this.blocks.size()); + resetBlocks(conf); for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { paths.put(ColumnPath.get(col.getPath()), col); } @@ -808,6 +799,19 @@ public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOEx } } + public void resetBlocks(Configuration conf) throws IOException { + if (null == fileDecryptor) { + this.options = HadoopReadOptions.builder(conf).build(); + } else { + this.options = HadoopReadOptions.builder(conf) + .withDecryption(fileDecryptor.getDecryptionProperties()) + .build(); + } + this.blocks = filterRowGroups(footer.getBlocks()); + this.blockIndexStores = listWithNulls(this.blocks.size()); + this.blockRowRanges = listWithNulls(this.blocks.size()); + } + private static List listWithNulls(int size) { return Stream.generate(() -> (T) null).limit(size).collect(Collectors.toCollection(ArrayList::new)); } From b65961475d1e55135be4e9e8511f38a832b7649f Mon Sep 17 00:00:00 2001 From: "chenliang.lu" Date: Thu, 18 May 2023 15:54:22 +0800 Subject: [PATCH 35/36] KE-41399 Avoid parquet footer reads twice in vectorized reader --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 22 files changed, 22 insertions(+), 22 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 3b59ae57ff..8f198e7ba8 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index 58ab779b66..b7a05e517e 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 1920a17230..69b90d608e 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 7ba330216b..6dea8473b9 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 432d98db17..2f4f4cd679 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 5863476640..089c6f691d 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 02f861ff79..1ce0087799 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index 849ac5af82..c7e5d0b1ba 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index 52daf8e58a..a0439d1eba 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index ab7489f71d..fed449a2a0 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 67fd36b5cd..320c9c980c 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index 99b7d41a3a..aa32b3093f 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index b73654b6cf..1b17f540db 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 394afaebbc..840613adac 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index a701362192..261e499b32 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index b761eed2fc..d06dbcc873 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 6cb6abd485..519dbfebe0 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index e7bb40f9e2..e2cdeceebf 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 3f16505389..1a82030d86 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index e2d2affdb1..e3ac125c2d 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 11f30fe7e4..01897b632a 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT 4.0.0 diff --git a/pom.xml b/pom.xml index ffc5b79a25..bc4cf8a904 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-kylin-r5 + 1.12.2-kylin-r5-SNAPSHOT pom Apache Parquet MR From 8f6047bfbb2c2ddec15fc8da8436a37637a37bc9 Mon Sep 17 00:00:00 2001 From: "chenliang.lu" Date: Mon, 22 May 2023 08:52:25 +0800 Subject: [PATCH 36/36] KE-41399 Avoid parquet footer reads twice in vectorized reader --- parquet-arrow/pom.xml | 2 +- parquet-avro/pom.xml | 2 +- parquet-benchmarks/pom.xml | 2 +- parquet-cascading-deprecated/pom.xml | 2 +- parquet-cascading3-deprecated/pom.xml | 2 +- parquet-cli/pom.xml | 2 +- parquet-column/pom.xml | 2 +- parquet-common/pom.xml | 2 +- parquet-encoding/pom.xml | 2 +- parquet-format-structures/pom.xml | 2 +- parquet-generator/pom.xml | 2 +- parquet-hadoop-bundle/pom.xml | 2 +- parquet-hadoop/pom.xml | 2 +- parquet-jackson/pom.xml | 2 +- parquet-pig-bundle/pom.xml | 2 +- parquet-pig/pom.xml | 2 +- parquet-protobuf/pom.xml | 2 +- parquet-scala/pom.xml | 2 +- parquet-scrooge-deprecated/pom.xml | 2 +- parquet-thrift/pom.xml | 2 +- parquet-tools-deprecated/pom.xml | 2 +- pom.xml | 2 +- 22 files changed, 22 insertions(+), 22 deletions(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index 8f198e7ba8..7d780a0a68 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-avro/pom.xml b/parquet-avro/pom.xml index b7a05e517e..7f77557a7d 100644 --- a/parquet-avro/pom.xml +++ b/parquet-avro/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-benchmarks/pom.xml b/parquet-benchmarks/pom.xml index 69b90d608e..97121e22c9 100644 --- a/parquet-benchmarks/pom.xml +++ b/parquet-benchmarks/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-cascading-deprecated/pom.xml b/parquet-cascading-deprecated/pom.xml index 6dea8473b9..00f92ec00b 100644 --- a/parquet-cascading-deprecated/pom.xml +++ b/parquet-cascading-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-cascading3-deprecated/pom.xml b/parquet-cascading3-deprecated/pom.xml index 2f4f4cd679..64c28a97f6 100644 --- a/parquet-cascading3-deprecated/pom.xml +++ b/parquet-cascading3-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-cli/pom.xml b/parquet-cli/pom.xml index 089c6f691d..d40a320e48 100644 --- a/parquet-cli/pom.xml +++ b/parquet-cli/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-column/pom.xml b/parquet-column/pom.xml index 1ce0087799..eed7a7504b 100644 --- a/parquet-column/pom.xml +++ b/parquet-column/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-common/pom.xml b/parquet-common/pom.xml index c7e5d0b1ba..30c5934b76 100644 --- a/parquet-common/pom.xml +++ b/parquet-common/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-encoding/pom.xml b/parquet-encoding/pom.xml index a0439d1eba..6bee29a8bd 100644 --- a/parquet-encoding/pom.xml +++ b/parquet-encoding/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-format-structures/pom.xml b/parquet-format-structures/pom.xml index fed449a2a0..cf422964ed 100644 --- a/parquet-format-structures/pom.xml +++ b/parquet-format-structures/pom.xml @@ -24,7 +24,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 parquet-format-structures diff --git a/parquet-generator/pom.xml b/parquet-generator/pom.xml index 320c9c980c..61ba66b8dc 100644 --- a/parquet-generator/pom.xml +++ b/parquet-generator/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-hadoop-bundle/pom.xml b/parquet-hadoop-bundle/pom.xml index aa32b3093f..8efb3318e1 100644 --- a/parquet-hadoop-bundle/pom.xml +++ b/parquet-hadoop-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml index 1b17f540db..1c53f21d14 100644 --- a/parquet-hadoop/pom.xml +++ b/parquet-hadoop/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-jackson/pom.xml b/parquet-jackson/pom.xml index 840613adac..6e07c27396 100644 --- a/parquet-jackson/pom.xml +++ b/parquet-jackson/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-pig-bundle/pom.xml b/parquet-pig-bundle/pom.xml index 261e499b32..a39ab12a44 100644 --- a/parquet-pig-bundle/pom.xml +++ b/parquet-pig-bundle/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-pig/pom.xml b/parquet-pig/pom.xml index d06dbcc873..724df97a1b 100644 --- a/parquet-pig/pom.xml +++ b/parquet-pig/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-protobuf/pom.xml b/parquet-protobuf/pom.xml index 519dbfebe0..e3793aee15 100644 --- a/parquet-protobuf/pom.xml +++ b/parquet-protobuf/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-scala/pom.xml b/parquet-scala/pom.xml index e2cdeceebf..7f3bddc640 100644 --- a/parquet-scala/pom.xml +++ b/parquet-scala/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-scrooge-deprecated/pom.xml b/parquet-scrooge-deprecated/pom.xml index 1a82030d86..8c96f2054c 100644 --- a/parquet-scrooge-deprecated/pom.xml +++ b/parquet-scrooge-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-thrift/pom.xml b/parquet-thrift/pom.xml index e3ac125c2d..e30315a655 100644 --- a/parquet-thrift/pom.xml +++ b/parquet-thrift/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/parquet-tools-deprecated/pom.xml b/parquet-tools-deprecated/pom.xml index 01897b632a..b741cc76d4 100644 --- a/parquet-tools-deprecated/pom.xml +++ b/parquet-tools-deprecated/pom.xml @@ -21,7 +21,7 @@ org.apache.parquet parquet ../pom.xml - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 4.0.0 diff --git a/pom.xml b/pom.xml index bc4cf8a904..5b22962209 100644 --- a/pom.xml +++ b/pom.xml @@ -9,7 +9,7 @@ org.apache.parquet parquet - 1.12.2-kylin-r5-SNAPSHOT + 1.12.2-kylin-r6 pom Apache Parquet MR