From 73a4430af6c40f8eb246ad4911eb6d103c9a2abe Mon Sep 17 00:00:00 2001 From: Fei Wang Date: Thu, 31 Oct 2024 01:50:22 -0700 Subject: [PATCH 1/4] PARQUET-3031: Support to transfer input stream when building ParquetFileReader (#3030) --- .../parquet/hadoop/ParquetFileReader.java | 36 +++++++++++++++++-- 1 file changed, 34 insertions(+), 2 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 1d8cce3d8c..785f145b20 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 @@ -710,6 +710,20 @@ public static ParquetFileReader open(InputFile file, ParquetReadOptions options) return new ParquetFileReader(file, options); } + /** + * Open a {@link InputFile file} with {@link ParquetReadOptions options}. + * + * @param file an input file + * @param options parquet read options + * @param f the input stream for the file + * @return an open ParquetFileReader + * @throws IOException if there is an error while opening the file + */ + public static ParquetFileReader open(InputFile file, ParquetReadOptions options, SeekableInputStream f) + throws IOException { + return new ParquetFileReader(file, options, f); + } + protected final SeekableInputStream f; private final InputFile file; private final ParquetReadOptions options; @@ -863,9 +877,23 @@ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer) */ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer, ParquetReadOptions options) throws IOException { + this(conf, file, footer, options, HadoopInputFile.fromPath(file, conf).newStream()); + } + + /** + * @param conf the Hadoop Configuration + * @param file Path to a parquet file + * @param footer a {@link ParquetMetadata} footer already read from the file + * @param options {@link ParquetReadOptions} + * @param f a {@link SeekableInputStream} for the parquet file + * @throws IOException if the file can not be opened + */ + public ParquetFileReader( + Configuration conf, Path file, ParquetMetadata footer, ParquetReadOptions options, SeekableInputStream f) + throws IOException { this.converter = new ParquetMetadataConverter(conf); this.file = HadoopInputFile.fromPath(file, conf); - this.f = this.file.newStream(); + this.f = f; this.fileMetaData = footer.getFileMetaData(); this.fileDecryptor = fileMetaData.getFileDecryptor(); this.options = options; @@ -894,9 +922,13 @@ public ParquetFileReader(Configuration conf, Path file, ParquetMetadata footer, } public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOException { + this(file, options, file.newStream()); + } + + public ParquetFileReader(InputFile file, ParquetReadOptions options, SeekableInputStream f) throws IOException { this.converter = new ParquetMetadataConverter(options); this.file = file; - this.f = file.newStream(); + this.f = f; this.options = options; try { this.footer = readFooter(file, options, f, converter); From d2128afda4ba53667e95128f9de50518b555c96d Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Fri, 1 Nov 2024 11:12:34 +0800 Subject: [PATCH 2/4] GH-3029: Fix EncryptionPropertiesHelper not to use java.nio.file.Path (#3038) --- .../apache/parquet/hadoop/EncryptionPropertiesHelper.java | 8 +++----- .../java/org/apache/parquet/hadoop/ParquetWriter.java | 5 +---- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java index 8d4f6f7ca5..090f29d302 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/EncryptionPropertiesHelper.java @@ -18,27 +18,25 @@ */ package org.apache.parquet.hadoop; -import java.net.URI; -import java.nio.file.Path; import org.apache.hadoop.conf.Configuration; import org.apache.parquet.conf.ParquetConfiguration; import org.apache.parquet.crypto.EncryptionPropertiesFactory; import org.apache.parquet.crypto.FileEncryptionProperties; import org.apache.parquet.hadoop.api.WriteSupport; import org.apache.parquet.hadoop.util.ConfigurationUtil; +import org.apache.parquet.io.OutputFile; final class EncryptionPropertiesHelper { static FileEncryptionProperties createEncryptionProperties( - ParquetConfiguration fileParquetConfig, Path tempFilePath, WriteSupport.WriteContext fileWriteContext) { + ParquetConfiguration fileParquetConfig, OutputFile file, WriteSupport.WriteContext fileWriteContext) { EncryptionPropertiesFactory cryptoFactory = EncryptionPropertiesFactory.loadFactory(fileParquetConfig); if (null == cryptoFactory) { return null; } Configuration hadoopConf = ConfigurationUtil.createHadoopConfiguration(fileParquetConfig); - URI path = tempFilePath == null ? null : tempFilePath.toUri(); return cryptoFactory.getFileEncryptionProperties( - hadoopConf, path == null ? null : new org.apache.hadoop.fs.Path(path), fileWriteContext); + hadoopConf, file == null ? null : new org.apache.hadoop.fs.Path(file.getPath()), fileWriteContext); } static FileEncryptionProperties createEncryptionProperties( 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 60867aa93a..4fa6d96be4 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 @@ -20,7 +20,6 @@ import java.io.Closeable; import java.io.IOException; -import java.nio.file.Paths; import java.util.HashMap; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -387,9 +386,7 @@ public ParquetWriter(Path file, Configuration conf, WriteSupport writeSupport // encryptionProperties could be built from the implementation of EncryptionPropertiesFactory when it is // attached. if (encryptionProperties == null) { - String path = file == null ? null : file.getPath(); - encryptionProperties = EncryptionPropertiesHelper.createEncryptionProperties( - conf, path == null ? null : Paths.get(path), writeContext); + encryptionProperties = EncryptionPropertiesHelper.createEncryptionProperties(conf, file, writeContext); } ParquetFileWriter fileWriter = new ParquetFileWriter( From 72520d2b9f4982009203675e7c1d3ae2723eda9a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 09:25:59 +0100 Subject: [PATCH 3/4] Bump net.openhft:zero-allocation-hashing from 0.16 to 0.26ea0 (#3024) Bumps [net.openhft:zero-allocation-hashing](https://github.com/OpenHFT/Zero-Allocation-Hashing) from 0.16 to 0.26ea0. - [Release notes](https://github.com/OpenHFT/Zero-Allocation-Hashing/releases) - [Commits](https://github.com/OpenHFT/Zero-Allocation-Hashing/commits) --- updated-dependencies: - dependency-name: net.openhft:zero-allocation-hashing dependency-type: direct:development ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a16c3454c2..2b735ab791 100644 --- a/pom.xml +++ b/pom.xml @@ -96,7 +96,7 @@ 0.1.1 1.10.19 2.0.9 - 0.16 + 0.26ea0 1.6.0 From bdc93464cbd882270a714826ce05476aab9d8cf0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 22:37:11 +0800 Subject: [PATCH 4/4] Bump org.apache.arrow:arrow-vector from 16.1.0 to 17.0.0 (#2980) * Bump org.apache.arrow:arrow-vector from 16.1.0 to 17.0.0 Bumps [org.apache.arrow:arrow-vector](https://github.com/apache/arrow) from 16.1.0 to 17.0.0. - [Release notes](https://github.com/apache/arrow/releases) - [Commits](https://github.com/apache/arrow/compare/r-16.1.0...go/v17.0.0) --- updated-dependencies: - dependency-name: org.apache.arrow:arrow-vector dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] * fix ci --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Gang Wu --- parquet-arrow/pom.xml | 2 +- .../parquet/arrow/schema/SchemaConverter.java | 30 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/parquet-arrow/pom.xml b/parquet-arrow/pom.xml index a80fe3da22..f4d59fba9a 100644 --- a/parquet-arrow/pom.xml +++ b/parquet-arrow/pom.xml @@ -33,7 +33,7 @@ https://parquet.apache.org - 16.1.0 + 17.0.0 diff --git a/parquet-arrow/src/main/java/org/apache/parquet/arrow/schema/SchemaConverter.java b/parquet-arrow/src/main/java/org/apache/parquet/arrow/schema/SchemaConverter.java index c17f1465d5..532fa7c53a 100644 --- a/parquet-arrow/src/main/java/org/apache/parquet/arrow/schema/SchemaConverter.java +++ b/parquet-arrow/src/main/java/org/apache/parquet/arrow/schema/SchemaConverter.java @@ -174,6 +174,11 @@ public TypeMapping visit(org.apache.arrow.vector.types.pojo.ArrowType.FixedSizeL return createListTypeMapping(); } + @Override + public TypeMapping visit(ArrowType.ListView type) { + return createListTypeMapping(); + } + private ListTypeMapping createListTypeMapping() { if (children.size() != 1) { throw new IllegalArgumentException("list fields must have exactly one child: " + field); @@ -249,11 +254,21 @@ public TypeMapping visit(ArrowType.LargeUtf8 largeUtf8) { return primitive(BINARY, stringType()); } + @Override + public TypeMapping visit(ArrowType.Utf8View type) { + return primitive(BINARY, stringType()); + } + @Override public TypeMapping visit(Binary type) { return primitive(BINARY); } + @Override + public TypeMapping visit(ArrowType.BinaryView type) { + return primitive(BINARY); + } + @Override public TypeMapping visit(ArrowType.LargeBinary largeBinary) { return primitive(BINARY); @@ -749,6 +764,11 @@ public TypeMapping visit(org.apache.arrow.vector.types.pojo.ArrowType.FixedSizeL return createListTypeMapping(type); } + @Override + public TypeMapping visit(ArrowType.ListView type) { + return createListTypeMapping(type); + } + private TypeMapping createListTypeMapping(ArrowType.ComplexType type) { if (arrowField.getChildren().size() != 1) { throw new IllegalArgumentException("Invalid list type: " + type); @@ -818,6 +838,11 @@ public TypeMapping visit(ArrowType.LargeUtf8 largeUtf8) { return primitive(); } + @Override + public TypeMapping visit(ArrowType.Utf8View type) { + return primitive(); + } + @Override public TypeMapping visit(Binary type) { return primitive(); @@ -828,6 +853,11 @@ public TypeMapping visit(ArrowType.LargeBinary largeBinary) { return primitive(); } + @Override + public TypeMapping visit(ArrowType.BinaryView type) { + return primitive(); + } + @Override public TypeMapping visit(Bool type) { return primitive();