[jira] [Commented] (PARQUET-2181) parquet-cli fails at supporting parquet-protobuf generated files
[ https://issues.apache.org/jira/browse/PARQUET-2181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17601415#comment-17601415 ] Timothy Miller commented on PARQUET-2181: - Is this related to PARQUET-2069? Or maybe PARQUET-1681? > parquet-cli fails at supporting parquet-protobuf generated files > > > Key: PARQUET-2181 > URL: https://issues.apache.org/jira/browse/PARQUET-2181 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: J Y >Priority: Critical > Attachments: sample-depth-1.tgz, samples.tgz > > > i generated a parquet file using a protobuf with this proto definition: > {code:java} > message IndexPath { > // Index of item in path. > repeated int32 index = 1; > } > message SomeEvent { > // truncated/obfuscated wrapper > optional IndexPath client_position = 1; > } > {code} > this gets translated to the following parquet schema using the new compliant > schema for lists: > {code:java} > message SomeEvent { > optional group client_position = 1 { > optional group index (LIST) = 1 { > repeated group list { > required int32 element; > } > } > } > } > {code} > this causes parquet-cli cat to barf on a file containing these events: > {quote}java.lang.RuntimeException: Failed on record 0 > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) > Caused by: java.lang.ClassCastException: required int32 element is not a group > at org.apache.parquet.schema.Type.asGroupType(Type.java:248) > at > org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:284) > at > org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:228) > at > org.apache.parquet.avro.AvroRecordConverter.access$100(AvroRecordConverter.java:74) > at > org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter$ElementConverter.(AvroRecordConverter.java:539) > at > org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter.(AvroRecordConverter.java:489) > at > org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:293) > at > org.apache.parquet.avro.AvroRecordConverter.(AvroRecordConverter.java:137) > at > org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:284) > at > org.apache.parquet.avro.AvroRecordConverter.(AvroRecordConverter.java:137) > at > org.apache.parquet.avro.AvroRecordConverter.(AvroRecordConverter.java:91) > at > org.apache.parquet.avro.AvroRecordMaterializer.(AvroRecordMaterializer.java:33) > at > org.apache.parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:142) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:190) > at > org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:166) > at > org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135) > at > org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) > at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) > at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) > ... 3 more > {quote} > using the old parquet-tools binary to cat this file works fine. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2171) Implement vectored IO in parquet file format
[ https://issues.apache.org/jira/browse/PARQUET-2171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17579770#comment-17579770 ] Timothy Miller commented on PARQUET-2171: - The parquet reader has two phases of reading. One does the raw I/O and decompression. Someone is working on an asynchronous implementation of this, which should help a lot. The second phase works on the output of that, providing higher-level data types. My PRs improve on this by eliminating LittleEndianInputStream, which was super inefficient, plus some other improvements in the most critical paths. All of these improvements are incremental, of course, and we're happy to get contributions that improve on this further. > Implement vectored IO in parquet file format > > > Key: PARQUET-2171 > URL: https://issues.apache.org/jira/browse/PARQUET-2171 > Project: Parquet > Issue Type: New Feature > Components: parquet-mr >Reporter: Mukund Thakur >Priority: Major > > We recently added a new feature called vectored IO in Hadoop for improving > read performance for seek heavy readers. Spark Jobs and others which uses > parquet will greatly benefit from this api. Details can be found here > [https://github.com/apache/hadoop/commit/e1842b2a749d79cbdc15c524515b9eda64c339d5] > https://issues.apache.org/jira/browse/HADOOP-18103 > https://issues.apache.org/jira/browse/HADOOP-11867 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2171) Implement vectored IO in parquet file format
[ https://issues.apache.org/jira/browse/PARQUET-2171?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17578470#comment-17578470 ] Timothy Miller commented on PARQUET-2171: - This might synergize well with the bulk I/O features I've been adding to ParquetMR. Some of the initial work is already in some PRs, and the rest of the plan can be found at [https://docs.google.com/document/d/1fBGpF_LgtfaeHnPD5CFEIpA2Ga_lTITmFdFIcO9Af-g/edit?usp=sharing] I determined what to optimize from profiling, and I have run experiments on the new implementation. I glanced through your Hadoop commits, and I noticed that you use ByteBuffer a lot. I have found ByteBuffer to impose a nontrivial amount of overhead, and you might want to consider providing array-based methods as well. > Implement vectored IO in parquet file format > > > Key: PARQUET-2171 > URL: https://issues.apache.org/jira/browse/PARQUET-2171 > Project: Parquet > Issue Type: New Feature > Components: parquet-mr >Reporter: Mukund Thakur >Priority: Major > > We recently added a new feature called vectored IO in Hadoop for improving > read performance for seek heavy readers. Spark Jobs and others which uses > parquet will greatly benefit from this api. Details can be found here > [https://github.com/apache/hadoop/commit/e1842b2a749d79cbdc15c524515b9eda64c339d5] > https://issues.apache.org/jira/browse/HADOOP-18103 > https://issues.apache.org/jira/browse/HADOOP-11867 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2126) Thread safety bug in CodecFactory
[ https://issues.apache.org/jira/browse/PARQUET-2126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17573826#comment-17573826 ] Timothy Miller commented on PARQUET-2126: - Thanks, Steve. I really like your suggestion. I think we should already hold off on this PR because I'm pretty sure the double-pooling is a problem. So I'll see about looking into this and maybe implement something completely different. > Thread safety bug in CodecFactory > - > > Key: PARQUET-2126 > URL: https://issues.apache.org/jira/browse/PARQUET-2126 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: James Turton >Priority: Major > > The code for returning Compressor objects to the caller goes to some lengths > to achieve thread safety, including keeping Codec objects in an Apache > Commons pool that has thread-safe borrow semantics. This is all undone by > the BytesCompressor and BytesDecompressor Maps in > org.apache.parquet.hadoop.CodecFactory which end up caching single compressor > and decompressor instances due to code in CodecFactory@getCompressor and > CodecFactory@getDecompressor. When the caller runs multiple threads, those > threads end up sharing compressor and decompressor instances. > For compressors based on Xerial Snappy this bug has no effect because that > library is itself thread safe. But when BuiltInGzipCompressor from Hadoop is > selected for the CompressionCodecName.GZIP case, serious problems ensue. > That class is not thread safe and sharing one instance of it between threads > produces both silent data corruption and JVM crashes. > To fix this situation, parquet-mr should stop caching single compressor and > decompressor instances. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17555142#comment-17555142 ] Timothy Miller commented on PARQUET-2159: - If this is already being generated at runtime, then it sounds to me like you might be able to generate different things depending on the compiler's source version setting. > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17555099#comment-17555099 ] Timothy Miller commented on PARQUET-2159: - I frequently wish Java had a preprocessor like C++ that would solve your problem. Currently, we have to build ParquetMR with Java 8, and plenty of things that depend on it (like Trino and Presto) use Java 11 at the latest. There are some solutions involving runtime loading of class files (e.g. [https://stackoverflow.com/questions/4526113/java-conditional-compilation-how-to-prevent-code-chunks-from-being-compiled),] but there's already enough weirdness in the ParquetMR build process (e.g. compile-time generated code that makes debugging a huge pain) that I hesitate to suggest making it even more challenging. Where actually is the code coming from that you're working on? Is it part of ParquetMR or some other library? I seem to recall that when debugging Trino, the code you're working on has to go through the decompiler in IntelliJ. So if it's already external in some way, then it probably wouldn't hurt to make it just a bit more dynamic, where ParquetMR loads different versions of the bit-packing library depending on the Java version. > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17554652#comment-17554652 ] Timothy Miller commented on PARQUET-2159: - Could you add a link to the PR, please? > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2153) Cannot read schema from parquet file
[ https://issues.apache.org/jira/browse/PARQUET-2153?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17553635#comment-17553635 ] Timothy Miller commented on PARQUET-2153: - Is this related to anything fixed by [https://github.com/apache/parquet-mr/pull/957] ? > Cannot read schema from parquet file > > > Key: PARQUET-2153 > URL: https://issues.apache.org/jira/browse/PARQUET-2153 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.3 >Reporter: Nils Broman >Priority: Major > > I'm trying to generate a Avro schema from a parquet file. I get the issue > both when using > [https://github.com/benwatson528/intellij-avro-parquet-plugin] in Intellij as > well as when I'm using my own implementation of generating a schema. > The parquet file contains nested entries and arrays: \{"a": a, "b": [{"c": > c}]} > I get the following error message: > {code:java} > org.apache.avro.SchemaParseException: Can't redefine: element > at org.apache.avro.Schema$Names.put(Schema.java:1547) > at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:810) > at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:972) > at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:1239) > at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:1000) > at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:984) > at org.apache.avro.Schema$ArraySchema.toJson(Schema.java:1134) > at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:1239) > at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:1000) > at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:984) > at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:1239) > at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:1000) > at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:984) > at org.apache.avro.Schema.toString(Schema.java:424) > at org.apache.avro.Schema.toString(Schema.java:396) > at > uk.co.hadoopathome.intellij.viewer.fileformat.ParquetFileReader.getSchema(ParquetFileReader.java:65) > at > uk.co.hadoopathome.intellij.viewer.FileViewerToolWindow$2.doInBackground(FileViewerToolWindow.java:196) > at > uk.co.hadoopathome.intellij.viewer.FileViewerToolWindow$2.doInBackground(FileViewerToolWindow.java:184) > at java.desktop/javax.swing.SwingWorker$1.call(SwingWorker.java:304) > at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264) > at java.desktop/javax.swing.SwingWorker.run(SwingWorker.java:343) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:829) {code} > It appears to be an issue similar to > [PARQUET-1441|https://issues.apache.org/jira/browse/PARQUET-1441] or > [PARQUET-1409|https://issues.apache.org/jira/browse/PARQUET-1409]. > Or could it possibly be something wrong in my parquet file? -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2143) parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file access
[ https://issues.apache.org/jira/browse/PARQUET-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17542998#comment-17542998 ] Timothy Miller commented on PARQUET-2143: - I tested PR #957 on this, and it doesn't fix the problem. So they're unrelated. This is just a mismatch between some duplicated code in the CLI and other code elsewhere. > parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file > access > > > Key: PARQUET-2143 > URL: https://issues.apache.org/jira/browse/PARQUET-2143 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8, Hadoop 3.2.3 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > cd target > hadoop jar parquet-cli-1.13.0-SNAPSHOT-runtime.jar > org.apache.parquet.cli.Main cat {noformat} > Results: > {noformat} > Unknown error > java.lang.RuntimeException: Failed on record 0 > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.apache.hadoop.util.RunJar.run(RunJar.java:323) > at org.apache.hadoop.util.RunJar.main(RunJar.java:236) > Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value > at 0 in block -1 in file file:/home/theosib/guid.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) > at org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) > at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) > at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) > ... 9 more > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:140) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:230) > ... 15 more{noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2143) parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file access
[ https://issues.apache.org/jira/browse/PARQUET-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17540143#comment-17540143 ] Timothy Miller commented on PARQUET-2143: - There's a slight chance that [https://github.com/apache/parquet-mr/pull/957] might fix this. I'll have to try this out later. > parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file > access > > > Key: PARQUET-2143 > URL: https://issues.apache.org/jira/browse/PARQUET-2143 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8, Hadoop 3.2.3 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > cd target > hadoop jar parquet-cli-1.13.0-SNAPSHOT-runtime.jar > org.apache.parquet.cli.Main cat {noformat} > Results: > {noformat} > Unknown error > java.lang.RuntimeException: Failed on record 0 > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.apache.hadoop.util.RunJar.run(RunJar.java:323) > at org.apache.hadoop.util.RunJar.main(RunJar.java:236) > Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value > at 0 in block -1 in file file:/home/theosib/guid.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) > at org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) > at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) > at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) > ... 9 more > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:140) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:230) > ... 15 more{noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17540138#comment-17540138 ] Timothy Miller commented on PARQUET-2140: - There's a slight chance that [https://github.com/apache/parquet-mr/pull/957] could fix this. I'll have to try to remember to test that out later. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root > { required binary Message (STRING); required fixed_len_byte_array(16) > Number (UUID); } > +Values as read by parquet-cpp:+ > — Values — > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > {code:java} > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > {code} > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) > { Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\\{"name":"Message","type":"string"} > ,\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17537604#comment-17537604 ] Timothy Miller commented on PARQUET-2069: - Well, I tried modifying prepareForRead to just reconstruct the avro schema always from the parquet schema, but that caused another test to fail, which is org.apache.parquet.avro.TestGenericLogicalTypes. So in the end what I decided to do was try using the avro schema, but if that throws an exception, it falls back to conversion and tried again. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17536902#comment-17536902 ] Timothy Miller commented on PARQUET-2069: - Yup. If I force prepareForRead() to ignore the avro schema in the metadata, then the modified.parquet file parses just fine, even without the code in the PR. What do y'all say to us ditching entirely the attempt to salvage the avro schema from the metadata? That would make a bunch of reported parsing problems go away. It seems like the file writer is modifying the file schema in a way that becomes incompatible with the avro schema it attempts to save. If we want to go with the new format, we should stop trying to use the avro schema. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17536894#comment-17536894 ] Timothy Miller commented on PARQUET-2069: - So, where does the avro schema come from in the first place? This is in org.apache.parquet.avro.AvroReadSupport.prepareForRead(). This IF test passes: {code:java} if (keyValueMetaData.get(AVRO_SCHEMA_METADATA_KEY) != null){code} Using a parquet metadata analysis tool I wrote, we can see where this is coming from: {code:java} 5: List(KeyValue key_value_metadata): 1: Struct(KeyValue key_value_metadata): 1: string(key) = Binary("parquet.avro.schema") 2: string(value) = Binary("{"type":"record","name":"spark_schema","fields":[{"name":"destination_addresses","type":["null",{"type":"array","items":{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}}],"default":null},{"name":"origin_addresses","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list2","fields":[{"name":"element","type":["null","string"],"default":null}]}}],"default":null},{"name":"rows","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}]}}],"default":null},{"name":"status","type":["null","string"],"default":null}]}"){code} The parquet schema comes from the thrift data structures at the end of the parquet file. The decode of that is so lengthy that I decided to put it on pastebin, here: [https://pastebin.com/cRGPkSwH|[https://pastebin.com/cRGPkSwH].] I'll try to put these side-by-side later and see if I can make sense of them. But I'm guessing that the writer messed up and made the two schemas not compatible. If I hack prepareForRead() to ignore the avro schema, everything parses just fine. Right now I'm rebuilding the whole thing where the fix provided in the PR is backed out. I'll post another comment to let you know how that goes. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Comment Edited] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17536888#comment-17536888 ] Timothy Miller edited comment on PARQUET-2069 at 5/13/22 9:02 PM: -- I managed to probe this just a bit. No idea why this diverges from the behavior when I use the simple reader program I wrote ([https://github.com/theosib-amazon/parquet-mr-minreader]), despite using the exact same API calls. Specifically, I'm trying to read the modified.parquet that's attached to this bug report. The exception is thrown by org.apache.parquet.avro.AvroRecordConverter.getAvroField(), where it's trying to look up an element from a parquetSchema in the avroSchema. The caller is org.apache.parquet.avro.AvroRecordConverter.AvroRecordConverter() (the constructor), where we can find a loop over parquetSchema.getFields(). The parquet schema it's trying to use is this: {noformat} optional group element { optional group elements (LIST) { repeated group array { optional group element { optional group distance { optional binary text (STRING); optional int64 value; } optional group duration { optional binary text (STRING); optional int64 value; } optional binary status (STRING); } } } }{noformat} The avro schema is this: {code:java} {"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}],"aliases":["array"]}{code} There is indeed something in the avro schema called "elements". However, examining the avroSchema itself that is being searched from getAvroField, the Schema class's fieldMap only contains entries for "rows", "destination_address", "origin_address", and "status". It seems like these two schemas have very little to do with each other and somehow the wrong kind of thing got passed down into this code. was (Author: JIRAUSER287471): I managed to probe this just a bit. No idea why this diverges from the behavior when I use the simple reader program I wrote (https://github.com/theosib-amazon/parquet-mr-minreader), despite using the exact same API calls. Specifically, I'm trying to read the modified.parquet that's attached to this bug report. The exception is thrown by org.apache.parquet.avro.AvroRecordConverter.getAvroField(), where it's trying to look up an element from a parquetSchema in the avroSchema. The caller is org.apache.parquet.avro.AvroRecordConverter.AvroRecordConverter() (the constructor), where we can find a loop over parquetSchema.getFields(). The parquet schema it's trying to use is this: {noformat} optional group element { optional group elements (LIST) { repeated group array { optional group element { optional group distance { optional binary text (STRING); optional int64 value; } optional group duration { optional binary text (STRING); optional int64 value; } optional binary status (STRING); } } } }{noformat} The avro schema is this: {noformat} {"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}],"aliases":["array"]}{noformat}
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17536888#comment-17536888 ] Timothy Miller commented on PARQUET-2069: - I managed to probe this just a bit. No idea why this diverges from the behavior when I use the simple reader program I wrote (https://github.com/theosib-amazon/parquet-mr-minreader), despite using the exact same API calls. Specifically, I'm trying to read the modified.parquet that's attached to this bug report. The exception is thrown by org.apache.parquet.avro.AvroRecordConverter.getAvroField(), where it's trying to look up an element from a parquetSchema in the avroSchema. The caller is org.apache.parquet.avro.AvroRecordConverter.AvroRecordConverter() (the constructor), where we can find a loop over parquetSchema.getFields(). The parquet schema it's trying to use is this: {noformat} optional group element { optional group elements (LIST) { repeated group array { optional group element { optional group distance { optional binary text (STRING); optional int64 value; } optional group duration { optional binary text (STRING); optional int64 value; } optional binary status (STRING); } } } }{noformat} The avro schema is this: {noformat} {"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}],"aliases":["array"]}{noformat} There is indeed something in the avro schema called "elements". However, examining the avroSchema itself that is being searched from getAvroField, the Schema class's fieldMap only contains entries for "rows", "destination_address", "origin_address", and "status". It seems like these two schemas have very little to do with each other and somehow the wrong kind of thing got passed down into this code. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Created] (PARQUET-2147) Can't run ParquetMR test in IDEs
Timothy Miller created PARQUET-2147: --- Summary: Can't run ParquetMR test in IDEs Key: PARQUET-2147 URL: https://issues.apache.org/jira/browse/PARQUET-2147 Project: Parquet Issue Type: Bug Components: parquet-testing Affects Versions: 1.12.2, 1.11.2, 1.10.2, 1.13.0 Environment: In my case, IntelliJ IDEA. Reporter: Timothy Miller ParquetMR contains a suite of self-tests. When one of those self-tests fails, it would be nice to be able to pull up the test in an IDE like IntelliJ. Then we can use the debugger and track down what the problem is. Unfortunately, this is impossible due to some unfortunate design choices. If I open a test source file in IntelliJ and tell the IDE to run it, the first errors I get is a bunch of errors in org.apache.parquet.VersionTest, which evidently cannot find a bunch of symbols. If I hack VersionTest to no longer depend on those symbols, the next thing that happens is this error: {code:java} /home/theosib/packages/PARQUET-2069/parquet-mr/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java:43:46 java: cannot find symbol symbol: class IncrementallyUpdatedFilterPredicateBuilder location: package org.apache.parquet.filter2.recordlevel{code} Evidently, IncrementallyUpdatedFilterPredicateBuilder doesn't exist. Instead, it is auto-generated during the maven build process, but the code is stashed away somewhere where IntelliJ can't find it, and this is the end of the road. If we want more people to be able to contribute to ParquetMR, I think we should work towards making it easier to debug. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2147) Can't run ParquetMR tests in IDEs
[ https://issues.apache.org/jira/browse/PARQUET-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2147: Summary: Can't run ParquetMR tests in IDEs (was: Can't run ParquetMR test in IDEs) > Can't run ParquetMR tests in IDEs > - > > Key: PARQUET-2147 > URL: https://issues.apache.org/jira/browse/PARQUET-2147 > Project: Parquet > Issue Type: Bug > Components: parquet-testing >Affects Versions: 1.10.2, 1.13.0, 1.11.2, 1.12.2 > Environment: In my case, IntelliJ IDEA. >Reporter: Timothy Miller >Priority: Blocker > > ParquetMR contains a suite of self-tests. When one of those self-tests fails, > it would be nice to be able to pull up the test in an IDE like IntelliJ. Then > we can use the debugger and track down what the problem is. Unfortunately, > this is impossible due to some unfortunate design choices. > If I open a test source file in IntelliJ and tell the IDE to run it, the > first errors I get is a bunch of errors in org.apache.parquet.VersionTest, > which evidently cannot find a bunch of symbols. If I hack VersionTest to no > longer depend on those symbols, the next thing that happens is this error: > {code:java} > /home/theosib/packages/PARQUET-2069/parquet-mr/parquet-column/src/main/java/org/apache/parquet/io/MessageColumnIO.java:43:46 > java: cannot find symbol > symbol: class IncrementallyUpdatedFilterPredicateBuilder > location: package org.apache.parquet.filter2.recordlevel{code} > Evidently, IncrementallyUpdatedFilterPredicateBuilder doesn't exist. Instead, > it is auto-generated during the maven build process, but the code is stashed > away somewhere where IntelliJ can't find it, and this is the end of the road. > If we want more people to be able to contribute to ParquetMR, I think we > should work towards making it easier to debug. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528393#comment-17528393 ] Timothy Miller commented on PARQUET-2140: - I got this information from a combination of System.out.println and stepping through the code with a debugger. For your UUID field, this code in AvroSchemaConverter.java was used: @Override public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { return Schema.create(Schema.Type.STRING); } else { int size = parquetType.asPrimitiveType().getTypeLength(); return Schema.createFixed(parquetType.getName(), null, null, size); } } I put a breakpoint here, and the debugger stopped here, and I was able to check the top of the annotation. On 4/26/22, 2:36 PM, "Balaji K (Jira)" wrote: CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can confirm the sender and know the content is safe. [ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Balaji K mentioned you on PARQUET-2140 -- Thank you for sharing the details and for the min repro, [~theosib-amazon] – much appreciated. One more question, if I may - you mention "The UUID from the file is indeed a UUIDLogicalTypeAnnotation" – would you know if this is correct? i.e., would parquet-mr encode guids with the same annotation? also, how did you notice this annotation being present - by stepping through code or does parquet-tools/cli show this information? many thanks. As for parquet-cli, I suppose we'll have to wait for someone with more knowledge on these two codebases to weigh in on how to fix this. > Key: PARQUET-2140 > View Online: https://issues.apache.org/jira/browse/PARQUET-2140 > Add Comment: https://issues.apache.org/jira/browse/PARQUET-2140#add-comment Hint: You can mention someone in an issue description or comment by typing "@" in front of their username. -- This message was sent by Atlassian Jira (v8.20.7#820007) > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root > { required binary Message (STRING); required fixed_len_byte_array(16) > Number (UUID); } > +Values as read by parquet-cpp:+ > — Values — > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > {code:java} > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > {code} > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528263#comment-17528263 ] Timothy Miller commented on PARQUET-2140: - Never mind on that PR. It breaks other things. If instead of using parquet-cli I use my own Java code ([https://github.com/theosib-amazon/parquet-mr-minreader]), what used to work now break with the following error: {noformat} Caused by: java.lang.UnsupportedOperationException: fromFixed is not supported for uuid{noformat} It seems that there are different parts of ParquetMR that duplicate functionality, but they're out of sync. When adding UUID support for reading files, the author finished the job for the API, but that left the CLI code in a broken state. Figuring out how these two codebases are out of sync and resolving them is going to require a higher level perspective on this project than what I have right now. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528248#comment-17528248 ] Timothy Miller commented on PARQUET-2140: - I'm still working on this, but the problem appears to be in org.apache.parquet.avro.AvroSchemaConverter.convertField(). There's an anonymous inner class, where this method is defined: {code:java} @Override public Schema convertFIXED_LEN_BYTE_ARRAY(PrimitiveTypeName primitiveTypeName) { if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { return Schema.create(Schema.Type.STRING); } else { int size = parquetType.asPrimitiveType().getTypeLength(); return Schema.createFixed(parquetType.getName(), null, null, size); } }{code} The UUID from the file is indeed a UUIDLogicalTypeAnnotation, so this is specifying that the internal Avro schema to convert to should be STRING, which is elsewhere rejected as incompatible. It looks like someone was attempting to add UUID support but didn't finish the job. If I edit this code so that it doesn't do that, then everything works. I don't get a UUID printed all pretty, but there's no exception. This is the output I get: {noformat} {"Message": "First record", "Number": [-41, 48, -44, -37, -38, 57, -87, 67, -90, 116, 7, 79, 44, -29, 50, 17]}{noformat} I have no idea how to fix this properly, since org.apache.parquet.io.ColumnIOFactory.visit() insists on strict type checking. So I'm going to create a PR to just make this not crash, in the hopes that someone who understands this better might provide some feedback. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528175#comment-17528175 ] Timothy Miller commented on PARQUET-2140: - Here's my minimal parquet reader that I've been using: [https://github.com/theosib-amazon/parquet-mr-minreader] > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2104) parquet-cli broken in master
[ https://issues.apache.org/jira/browse/PARQUET-2104?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528167#comment-17528167 ] Timothy Miller commented on PARQUET-2104: - As I mentioned in https://issues.apache.org/jira/browse/PARQUET-2142, there's a workaround for this. There are duplicate methods in target/parquet-cli-1.13.0-SNAPSHOT-runtime.jar that should be picked up from the dependencies. You can run without hadoop and exclude the runtime by specifying just target/parquet-cli-1.13.0-SNAPSHOT.jar and the deps. > parquet-cli broken in master > > > Key: PARQUET-2104 > URL: https://issues.apache.org/jira/browse/PARQUET-2104 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.12.2 > Environment: ubuntu 18.04 and ubuntu 20.04 >Reporter: Balaji K >Priority: Major > > Creating a Jira per this thread: > [https://lists.apache.org/thread/k233838g010lvbp81s99floqjmm7nnvs] > # clone parquet-mr and build the repo locally > # run parquet-cli without Hadoop (according to this ReadMe > <[https://github.com/apache/parquet-mr/tree/master/parquet-cli#running-without-hadoop]> > ) > # try a command that deserializes data such as cat or head > # observe NoSuchMethodError being thrown > *Error stack:* ~/repos/parquet-mr/parquet-cli$ parquet cat > ../../testdata/dictionaryEncodingSample.parquet WARNING: An illegal > reflective access operation has occurred .. > Exception in thread "main" java.lang.NoSuchMethodError: > 'org.apache.avro.Schema > org.apache.parquet.avro.AvroSchemaConverter.convert(org.apache.parquet.schema.MessageType)' > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) at > org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) at > org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) at > org.apache.parquet.cli.Main.run(Main.java:157) at > org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at > org.apache.parquet.cli.Main.main(Main.java:187) -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Comment Edited] (PARQUET-2142) parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527556#comment-17527556 ] Timothy Miller edited comment on PARQUET-2142 at 4/26/22 1:19 PM: -- I added -verbose:class to the java command line, and I found this in the output: {noformat} [Loaded org.apache.parquet.avro.AvroSchemaConverter from file:/local/home/theosib/packages/PARQUET-2140/parquet-mr/parquet-cli/target/parquet-cli-1.13.0-SNAPSHOT-runtime.jar] {noformat} This isn't correct. It should be loading AvroSchemaConverter from target/dependency/parquet-avro-1.13.0-SNAPSHOT.jar. Evidently the instructions are wrong. I changed the run command to this: {noformat} java -cp 'target/parquet-cli-1.13.0-SNAPSHOT.jar:target/dependency/*' org.apache.parquet.cli.Main cat ~/guid.parquet{noformat} And the NoSuchMethodError went away. Now I get the Unknown error and java.lang.RuntimeException that we see in PARQUET-2143. was (Author: JIRAUSER287471): I added -verbose:class to the java command line, and I found this in the output: {noformat} [Loaded org.apache.parquet.avro.AvroSchemaConverter from file:/local/home/theosib/packages/PARQUET-2140/parquet-mr/parquet-cli/target/parquet-cli-1.13.0-SNAPSHOT-runtime.jar] {noformat} This isn't correct. It should be loading AvroSchemaConverter from target/dependency/parquet-avro-1.13.0-SNAPSHOT.jar. Evidently the instructions are wrong. I changed the run command to this: {noformat} java -cp 'target/parquet-cli-1.13.0-SNAPSHOT.jar:target/dependency/*:target/dependency/parquet-avro-1.13.0-SNAPSHOT.jar' org.apache.parquet.cli.Main cat ~/guid.parquet{noformat} And the NoSuchMethodError went away. Now I get the Unknown error and java.lang.RuntimeException that we see in PARQUET-2143. > parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet > file access command > > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528162#comment-17528162 ] Timothy Miller commented on PARQUET-2140: - I'm going to get back onto this today, so I'll probably upload my test code later today. What's really strange is how inconsistent parquet-mr can be with regard to what it can read. If I write my own program to use the API or use parquet-tools from the command line, I can parse your file. If I use parquet-cli, there's a load of files I can't parse. It's almost as though parquet-mr has duplicate code in different places that's out of sync. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2098) Add more methods into interface of BlockCipher
[ https://issues.apache.org/jira/browse/PARQUET-2098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17528160#comment-17528160 ] Timothy Miller commented on PARQUET-2098: - I don't personally have a use case. I was just looking through the Jira for things I could help with. > Add more methods into interface of BlockCipher > -- > > Key: PARQUET-2098 > URL: https://issues.apache.org/jira/browse/PARQUET-2098 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Reporter: Xinli Shang >Assignee: Xinli Shang >Priority: Major > > Currently BlockCipher interface has methods without letting caller to specify > length/offset. In some use cases like Presto, it is needed to pass in a byte > array and the data to be encrypted only occupys partially of the array. So > we need to add a new methods something like below for decrypt. Similar > methods might be needed for encrypt. > byte[] decrypt(byte[] ciphertext, int cipherTextOffset, int cipherTextLength, > byte[] aad); -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527592#comment-17527592 ] Timothy Miller commented on PARQUET-2140: - The error is caused in org.apache.parquet.io.ColumnIOFactory.visit(), where it sees that FIXED_LEN_BYTE_ARRAY in the file is not compatible with STRING, which is the type that it wants. It might be that if strictTypeChecking were off, this would work, but I haven't figured out how to disable it yet. If I force struct type checking off, it doesn't fail, but the results are kinds weird: {noformat} {"Message": "First record", "Number": "�0���9�C�t\u0007O,�2\u0011"}{noformat} I don't think this is the right way to handle this, though. Like why is it looking for a STRING in the first place? I can't find anything in the file that should indicate that this field ought to be interpreted as a string. It's always a FIXED_LEN_BYTE_ARRAY. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID >
[jira] [Comment Edited] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527559#comment-17527559 ] Timothy Miller edited comment on PARQUET-2140 at 4/25/22 3:50 PM: -- I just realized that I managed to duplicate this in PARQUET-2143, so PARQUET-2143 should probably be resolved as a duplicate of this one. Now, if I can just figure out how to run this is a debugger... EDIT: This is interesting. The guid.parquet file isn't the only one I have that parquet-cli can't parse. I also can't parse either of the files attached to PARQUET-2069. Both of those throw this error: {noformat} Unknown error java.lang.RuntimeException: Failed on record 0 at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) at org.apache.parquet.cli.Main.run(Main.java:157) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.parquet.cli.Main.main(Main.java:187) Caused by: java.lang.ClassCastException: optional binary element (STRING) is not a group ...{noformat} was (Author: JIRAUSER287471): I just realized that I managed to duplicate this in PARQUET-2143, so PARQUET-2143 should probably be resolved as a duplicate of this one. Now, if I can just figure out how to run this is a debugger... > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527559#comment-17527559 ] Timothy Miller commented on PARQUET-2140: - I just realized that I managed to duplicate this in PARQUET-2143, so PARQUET-2143 should probably be resolved as a duplicate of this one. Now, if I can just figure out how to run this is a debugger... > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2143) parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file access
[ https://issues.apache.org/jira/browse/PARQUET-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527558#comment-17527558 ] Timothy Miller commented on PARQUET-2143: - I didn't notice this earlier, but the ParquetDecodingException is the same as what's shown in PARQUET-2140. We may want to resolve PARQUET-2143 as a duplicate of that. > parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file > access > > > Key: PARQUET-2143 > URL: https://issues.apache.org/jira/browse/PARQUET-2143 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8, Hadoop 3.2.3 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > cd target > hadoop jar parquet-cli-1.13.0-SNAPSHOT-runtime.jar > org.apache.parquet.cli.Main cat {noformat} > Results: > {noformat} > Unknown error > java.lang.RuntimeException: Failed on record 0 > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.apache.hadoop.util.RunJar.run(RunJar.java:323) > at org.apache.hadoop.util.RunJar.main(RunJar.java:236) > Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value > at 0 in block -1 in file file:/home/theosib/guid.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) > at org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) > at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) > at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) > ... 9 more > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:140) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:230) > ... 15 more{noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2142) parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527556#comment-17527556 ] Timothy Miller commented on PARQUET-2142: - I added -verbose:class to the java command line, and I found this in the output: {noformat} [Loaded org.apache.parquet.avro.AvroSchemaConverter from file:/local/home/theosib/packages/PARQUET-2140/parquet-mr/parquet-cli/target/parquet-cli-1.13.0-SNAPSHOT-runtime.jar] {noformat} This isn't correct. It should be loading AvroSchemaConverter from target/dependency/parquet-avro-1.13.0-SNAPSHOT.jar. Evidently the instructions are wrong. I changed the run command to this: {noformat} java -cp 'target/parquet-cli-1.13.0-SNAPSHOT.jar:target/dependency/*:target/dependency/parquet-avro-1.13.0-SNAPSHOT.jar' org.apache.parquet.cli.Main cat ~/guid.parquet{noformat} And the NoSuchMethodError went away. Now I get the Unknown error and java.lang.RuntimeException that we see in PARQUET-2143. > parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet > file access command > > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2143) parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file access
[ https://issues.apache.org/jira/browse/PARQUET-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2143: Summary: parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file access (was: parquet-cli with hadoop throws java.lang.RuntimeException) > parquet-cli with hadoop throws java.lang.RuntimeException on any parquet file > access > > > Key: PARQUET-2143 > URL: https://issues.apache.org/jira/browse/PARQUET-2143 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8, Hadoop 3.2.3 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > cd target > hadoop jar parquet-cli-1.13.0-SNAPSHOT-runtime.jar > org.apache.parquet.cli.Main cat {noformat} > Results: > {noformat} > Unknown error > java.lang.RuntimeException: Failed on record 0 > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.apache.hadoop.util.RunJar.run(RunJar.java:323) > at org.apache.hadoop.util.RunJar.main(RunJar.java:236) > Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value > at 0 in block -1 in file file:/home/theosib/guid.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) > at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) > at org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) > at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) > at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) > ... 9 more > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:140) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:230) > ... 15 more{noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527519#comment-17527519 ] Timothy Miller commented on PARQUET-2140: - I've been trying to reproduce this with parquet-cli, but that tool is majorly broken. I'm filing PARQUET-2142 and PARQUET-2143 on this. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Created] (PARQUET-2143) parquet-cli with hadoop throws java.lang.RuntimeException
Timothy Miller created PARQUET-2143: --- Summary: parquet-cli with hadoop throws java.lang.RuntimeException Key: PARQUET-2143 URL: https://issues.apache.org/jira/browse/PARQUET-2143 Project: Parquet Issue Type: Bug Components: parquet-cli Affects Versions: 1.13.0 Environment: Amazon Linux 2, Java 8, Hadoop 3.2.3 Reporter: Timothy Miller I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. {noformat} git clone cd parquet-mr mvn clean install -DskipTests cd parquet-cli mvn clean install -DskipTests cd target hadoop jar parquet-cli-1.13.0-SNAPSHOT-runtime.jar org.apache.parquet.cli.Main cat {noformat} Results: {noformat} Unknown error java.lang.RuntimeException: Failed on record 0 at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86) at org.apache.parquet.cli.Main.run(Main.java:157) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.parquet.cli.Main.main(Main.java:187) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.util.RunJar.run(RunJar.java:323) at org.apache.hadoop.util.RunJar.main(RunJar.java:236) Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in block -1 in file file:/home/theosib/guid.parquet at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264) at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132) at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136) at org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363) at org.apache.parquet.cli.BaseCommand$1$1.(BaseCommand.java:344) at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342) at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73) ... 9 more Caused by: org.apache.parquet.io.ParquetDecodingException: The requested schema is not compatible with the file schema. incompatible types: required binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) at org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) at org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) at org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) at org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) at org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) at org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:140) at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:230) ... 15 more{noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2142) parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2142: Summary: parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet file access command (was: parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command) > parquet-cli without hadoop throws java.lang.NoSuchMethodError on any parquet > file access command > > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2142: Priority: Blocker (was: Major) > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Blocker > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2142: Description: I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. Steps to reproduce: {noformat} git clone cd parquet-mr mvn clean install -DskipTests cd parquet-cli mvn clean install -DskipTests mvn dependency:copy-dependencies java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat {noformat} Results: {noformat} Exception in thread "main" java.lang.NoSuchMethodError: org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) at org.apache.parquet.cli.Main.run(Main.java:157) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.parquet.cli.Main.main(Main.java:187){noformat} was: I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. Steps to reproduce: git clone cd parquet-mr mvn clean install -DskipTests cd parquet-cli mvn clean install -DskipTests mvn dependency:copy-dependencies java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat Results: Exception in thread "main" java.lang.NoSuchMethodError: org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) at org.apache.parquet.cli.Main.run(Main.java:157) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.parquet.cli.Main.main(Main.java:187) > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Major > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142 ] Timothy Miller deleted comment on PARQUET-2142: - was (Author: JIRAUSER287471): Jira swallowed the asterisks from the "java -cp" command, but you can see them in the README.md file. {noformat} java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main{noformat} > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Major > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > {noformat} > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > {noformat} > Results: > {noformat} > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187){noformat} -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527511#comment-17527511 ] Timothy Miller commented on PARQUET-2142: - Jira swallowed the asterisks from the "java -cp" command, but you can see them in the README.md file. I'll see if I can get it to work here "preformatted": {{java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main}} > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Major > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > > Results: > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Comment Edited] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527511#comment-17527511 ] Timothy Miller edited comment on PARQUET-2142 at 4/25/22 1:55 PM: -- Jira swallowed the asterisks from the "java -cp" command, but you can see them in the README.md file. {noformat} java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main{noformat} was (Author: JIRAUSER287471): Jira swallowed the asterisks from the "java -cp" command, but you can see them in the README.md file. I'll see if I can get it to work here "preformatted": {{java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main}} > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Major > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > > Results: > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Updated] (PARQUET-2142) parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command
[ https://issues.apache.org/jira/browse/PARQUET-2142?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2142: Component/s: parquet-cli Affects Version/s: 1.13.0 Description: I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. Steps to reproduce: git clone cd parquet-mr mvn clean install -DskipTests cd parquet-cli mvn clean install -DskipTests mvn dependency:copy-dependencies java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat Results: Exception in thread "main" java.lang.NoSuchMethodError: org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) at org.apache.parquet.cli.Main.run(Main.java:157) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.parquet.cli.Main.main(Main.java:187) was: git clone [parquet-mr repo] Environment: Amazon Linux 2, Java 8 Summary: parquet-cli throws java.lang.NoSuchMethodError on any parquet file access command (was: parquet-cli throws ) > parquet-cli throws java.lang.NoSuchMethodError on any parquet file access > command > - > > Key: PARQUET-2142 > URL: https://issues.apache.org/jira/browse/PARQUET-2142 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Affects Versions: 1.13.0 > Environment: Amazon Linux 2, Java 8 >Reporter: Timothy Miller >Priority: Major > > I can't do even basic things with parquet-cli from 1.13.0-SNAPSHOT. > Steps to reproduce: > git clone > cd parquet-mr > mvn clean install -DskipTests > cd parquet-cli > mvn clean install -DskipTests > mvn dependency:copy-dependencies > java -cp 'target/*:target/dependency/*' org.apache.parquet.cli.Main cat > > Results: > Exception in thread "main" java.lang.NoSuchMethodError: > org.apache.parquet.avro.AvroSchemaConverter.convert(Lorg/apache/parquet/schema/MessageType;)Lorg/apache/avro/Schema; > at org.apache.parquet.cli.util.Schemas.fromParquet(Schemas.java:89) > at org.apache.parquet.cli.BaseCommand.getAvroSchema(BaseCommand.java:405) > at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:66) > at org.apache.parquet.cli.Main.run(Main.java:157) > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) > at org.apache.parquet.cli.Main.main(Main.java:187) -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Created] (PARQUET-2142) parquet-cli throws
Timothy Miller created PARQUET-2142: --- Summary: parquet-cli throws Key: PARQUET-2142 URL: https://issues.apache.org/jira/browse/PARQUET-2142 Project: Parquet Issue Type: Bug Reporter: Timothy Miller git clone [parquet-mr repo] -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17527480#comment-17527480 ] Timothy Miller commented on PARQUET-2140: - I can't reproduce this bug with parquet-tools or 1.12.0 or 1.13.0-SNAPSHOT. I have a program that does nothing but read the schema and records out of a parquet file. Here's what I get: {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - RecordReader initialized will read a total of 1 records.}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - at row 0. reading next block}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - block read in memory in 25 ms. row count = 1}} {{{}} {{ "type" : "record",}} {{ "name" : "root",}} {{ "fields" : [ {}} {{ "name" : "Message",}} {{ "type" : "string"}} {{ }, {}} {{ "name" : "Number",}} {{ "type" : {}} {{ "type" : "string",}} {{ "logicalType" : "uuid"}} {{ }}} {{ } ]}} {{}}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - RecordReader initialized will read a total of 1 records.}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - at row 0. reading next block}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - block read in memory in 2 ms. row count = 1}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - RecordReader initialized will read a total of 1 records.}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - at row 0. reading next block}} {{[main] INFO org.apache.parquet.hadoop.InternalParquetRecordReader - block read in memory in 1 ms. row count = 1}} {{num_records=1}} {{Retrieved 1 records}} {{{"Message": "First record", "Number": d730d4db-da39-a943-a674-074f2ce33211}}} {{{}Process finished with exit code 0{}}}{{{}{}}} I'll see about uploading my test program to github so you can try it. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > >
[jira] [Commented] (PARQUET-2140) parquet-cli unable to read UUID values
[ https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526736#comment-17526736 ] Timothy Miller commented on PARQUET-2140: - I'll look at this on Monday. Have you tried writing the same parquet data from the Java implementation? Then we can compare the two files at a binary level. There are some things that are legal that the Java implementation doesn't handle so well. It may be that your C# is legal but that for the Java code to understand it, you have to do some slightly different legal thing. > parquet-cli unable to read UUID values > -- > > Key: PARQUET-2140 > URL: https://issues.apache.org/jira/browse/PARQUET-2140 > Project: Parquet > Issue Type: Bug > Components: parquet-cli >Reporter: Balaji K >Priority: Minor > Attachments: guid.parquet > > > I am finding that parquet-cli throws when trying to read UUID values. > Attached to this bug report is a parquet file with 2 columns, message encoded > as byte-array and number encoded as fixed length byte array (UUID). This file > was written by my .net implementation of parquet specification. The file has > one row worth of data and is readable by parquet-cpp. > +Schema as read by parquet-cli:+ > message root { > required binary Message (STRING); > required fixed_len_byte_array(16) Number (UUID); > } > +Values as read by parquet-cpp:+ > --- Values --- > Message |Number | > First record |215 48 212 219 218 57 169 67 166 116 7 79 44 > 227 50 17 | > > +Here is the exception stack from parquet-cli when trying to read uuid > values:+ > Caused by: org.apache.parquet.io.ParquetDecodingException: The requested > schema is not compatible with the file schema. incompatible types: required > binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93) > at > org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83) > at > org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57) > at org.apache.parquet.schema.MessageType.accept(MessageType.java:55) > at > org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135) > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225) > > I debugged parquet-cli code and found that parquet-cli is trying to project > the UUID as a string and later on that throws as these types are not > compatible? > > +Source code references:+ > At AvroReadSupport.java, line 97 > > String requestedProjectionString = > configuration.get(AVRO_REQUESTED_PROJECTION); > if (requestedProjectionString != null) { > Schema avroRequestedProjection = new > Schema.Parser().parse(requestedProjectionString); > projection = new > AvroSchemaConverter(configuration).convert(avroRequestedProjection); > } > > > +Debugger values for+ > requestedProjectionString= > {"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]} > [Note that `Number` now has a type of `string` and a logicalType of `uuid`] > > At ColumnIOFactory.java line 93 > > incompatibleSchema(primitiveType, currentRequestedType); > > +Debugger values for+ > primitiveType = required fixed_len_byte_array(16) Number (UUID) > currentRequestedType = required binary Number (STRING) > > and this will throw. > > If I skip over the projection code in AvroReadSupport, parquet-cli is able to > read my file. > I am not sure if the bug is in parquet-cli or parquet-mr or in the library I > used to encode this file. The fact that parquet-cpp is able to read it gives > me some confidence to say that the problem is either in parquet-cli or > parquet-mr. > Please point me in the right direction if I could verify this UUID > roundtripping purely from parquet-mr itself in form of an unit-test. Happy to > contribute tests or fix if needed. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2098) Add more methods into interface of BlockCipher
[ https://issues.apache.org/jira/browse/PARQUET-2098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526699#comment-17526699 ] Timothy Miller commented on PARQUET-2098: - [~gershinsky] Did you ever get around to this? If not, I'd be happy to see if I can do it. Please let me know. > Add more methods into interface of BlockCipher > -- > > Key: PARQUET-2098 > URL: https://issues.apache.org/jira/browse/PARQUET-2098 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Reporter: Xinli Shang >Assignee: Xinli Shang >Priority: Major > > Currently BlockCipher interface has methods without letting caller to specify > length/offset. In some use cases like Presto, it is needed to pass in a byte > array and the data to be encrypted only occupys partially of the array. So > we need to add a new methods something like below for decrypt. Similar > methods might be needed for encrypt. > byte[] decrypt(byte[] ciphertext, int cipherTextOffset, int cipherTextLength, > byte[] aad); -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2125) ParquetFileReader has a currentBlock information in a private field
[ https://issues.apache.org/jira/browse/PARQUET-2125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526698#comment-17526698 ] Timothy Miller commented on PARQUET-2125: - Precisely how meaningful is it to provide this information? How is the user going to know how to interpret the number? If you can provide a good justification for this, I can submit a PR that implements this. > ParquetFileReader has a currentBlock information in a private field > --- > > Key: PARQUET-2125 > URL: https://issues.apache.org/jira/browse/PARQUET-2125 > Project: Parquet > Issue Type: Wish > Components: parquet-avro >Affects Versions: 1.8.1 >Reporter: Tanuja Dubey >Priority: Major > > The currentBlock variable is a metric information which can be useful to know > which block the current record is being read from. If this variable has a > getter, it would be possible to skip over a certain blocks. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-1928) Interpret Parquet INT96 type as FIXED[12] AVRO Schema
[ https://issues.apache.org/jira/browse/PARQUET-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526575#comment-17526575 ] Timothy Miller commented on PARQUET-1928: - It looks like the change was already merged. [~iamnitish] if you're running into a problem similar to this, it may be a different bug. Can you provide a minimal test case and a parquet file so that we can more easily reproduce and investigate this problem? Thanks. > Interpret Parquet INT96 type as FIXED[12] AVRO Schema > - > > Key: PARQUET-1928 > URL: https://issues.apache.org/jira/browse/PARQUET-1928 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.11.0 >Reporter: Anant Damle >Assignee: Anant Damle >Priority: Minor > Labels: patch > Fix For: 1.12.0 > > Attachments: Screen Shot 2022-04-22 at 12.02.12 PM.png > > > Reading Parquet files in Apache Beam using ParquetIO uses `AvroParquetReader` > causing it to throw `IllegalArgumentException("INT96 not implemented and is > deprecated")` > Customers have large datasets which can't be reprocessed again to convert > into a supported type. An easier approach would be to convert into a byte > array of 12 bytes, that can then be interpreted by the developer in any way > they want to interpret it. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-1928) Interpret Parquet INT96 type as FIXED[12] AVRO Schema
[ https://issues.apache.org/jira/browse/PARQUET-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526573#comment-17526573 ] Timothy Miller commented on PARQUET-1928: - Is there a reason why patches such as this are not merged? Do the maintainers want more evaluation of the consequences of the change? I'd be happy to help. > Interpret Parquet INT96 type as FIXED[12] AVRO Schema > - > > Key: PARQUET-1928 > URL: https://issues.apache.org/jira/browse/PARQUET-1928 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.11.0 >Reporter: Anant Damle >Assignee: Anant Damle >Priority: Minor > Labels: patch > Fix For: 1.12.0 > > Attachments: Screen Shot 2022-04-22 at 12.02.12 PM.png > > > Reading Parquet files in Apache Beam using ParquetIO uses `AvroParquetReader` > causing it to throw `IllegalArgumentException("INT96 not implemented and is > deprecated")` > Customers have large datasets which can't be reprocessed again to convert > into a supported type. An easier approach would be to convert into a byte > array of 12 bytes, that can then be interpreted by the developer in any way > they want to interpret it. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2126) Thread safety bug in CodecFactory
[ https://issues.apache.org/jira/browse/PARQUET-2126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17526497#comment-17526497 ] Timothy Miller commented on PARQUET-2126: - Alright. You have a point. If the maintainers want me to delete that stuff, they can let me know, and I'll go ahead and do it. > Thread safety bug in CodecFactory > - > > Key: PARQUET-2126 > URL: https://issues.apache.org/jira/browse/PARQUET-2126 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: James Turton >Priority: Major > > The code for returning Compressor objects to the caller goes to some lengths > to achieve thread safety, including keeping Codec objects in an Apache > Commons pool that has thread-safe borrow semantics. This is all undone by > the BytesCompressor and BytesDecompressor Maps in > org.apache.parquet.hadoop.CodecFactory which end up caching single compressor > and decompressor instances due to code in CodecFactory@getCompressor and > CodecFactory@getDecompressor. When the caller runs multiple threads, those > threads end up sharing compressor and decompressor instances. > For compressors based on Xerial Snappy this bug has no effect because that > library is itself thread safe. But when BuiltInGzipCompressor from Hadoop is > selected for the CompressionCodecName.GZIP case, serious problems ensue. > That class is not thread safe and sharing one instance of it between threads > produces both silent data corruption and JVM crashes. > To fix this situation, parquet-mr should stop caching single compressor and > decompressor instances. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-1681) Avro's isElementType() change breaks the reading of some parquet(1.8.1) files
[ https://issues.apache.org/jira/browse/PARQUET-1681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525795#comment-17525795 ] Timothy Miller commented on PARQUET-1681: - Have a look at my further analysis of PARQUET-2069. I don't think 1681 will be fixed by what I did to fix 2069, but the problem seems to be the same KIND of problem, happening in the same place. Basically, isElementType is returning the wrong thing because there are types that Avro should think are compatible, but it has not been properly informed about that. In the case of 2069, "list" and "array" were being considered incompatible, so I fixed it by hacking isElementType to inform Avro of that compatibility. It may be that we need to completely rethink isElementType to be smarter about detecting reader/writer compatibility. In the case of 1681, evidently "phones_items" is some kind of list/array type, so it should test positive as compatible with another list/array type simply by virtue of them both containing multiple members. > Avro's isElementType() change breaks the reading of some parquet(1.8.1) files > - > > Key: PARQUET-1681 > URL: https://issues.apache.org/jira/browse/PARQUET-1681 > Project: Parquet > Issue Type: Improvement > Components: parquet-avro >Affects Versions: 1.10.0, 1.9.1, 1.11.0 >Reporter: Xinli Shang >Assignee: Xinli Shang >Priority: Critical > > When using the Avro schema below to write a parquet(1.8.1) file and then read > back by using parquet 1.10.1 without passing any schema, the reading throws > an exception "XXX is not a group" . Reading through parquet 1.8.1 is fine. > { > "name": "phones", > "type": [ > "null", > { > "type": "array", > "items": { > "type": "record", > "name": "phones_items", > "fields": [ > > { "name": "phone_number", > "type": [ "null", > "string" ], "default": null > } > ] > } > } > ], > "default": null > } > The code to read is as below > val reader = > AvroParquetReader._builder_[SomeRecordType](parquetPath).withConf(*new* > Configuration).build() > reader.read() > PARQUET-651 changed the method isElementType() by relying on Avro's > checkReaderWriterCompatibility() to check the compatibility. However, > checkReaderWriterCompatibility() consider the ParquetSchema and the > AvroSchema(converted from File schema) as not compatible(the name in avro > schema is ‘phones_items’, but the name is ‘array’ in Parquet schema, hence > not compatible) . Hence return false and caused the “phone_number” field in > the above schema to be considered as group type which is not true. Then the > exception throws as .asGroupType(). > I didn’t try writing via parquet 1.10.1 would reproduce the same problem or > not. But it could because the translation of Avro schema to Parquet schema is > not changed(didn’t verify yet). > I hesitate to revert PARQUET-651 because it solved several problems. I would > like to hear the community's thoughts on it. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Comment Edited] (PARQUET-2139) Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files
[ https://issues.apache.org/jira/browse/PARQUET-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525272#comment-17525272 ] Timothy Miller edited comment on PARQUET-2139 at 4/20/22 8:37 PM: -- I just noticed that the file_offset field in ColumnChunk is "required." So there's a few possible mistakes: # The description of the metadata structures is wrong and this really is supposed to be a pointer to the data page (PageHeader), or # There's a completely separate bug where the parquet writer fails to store an extra copy of the ColumnMetaData in the file right before the PageHeader, or # The offset should point to where the unique copy of ColumnMetaData is already going to be found in the file footer, although that seems like it would be really hard to calculate. In any case, there's an inconsistency where the metadata definition specifies an offset to ColumnMetaData, where instead a PageHeader is placed. I'm going to go check out the reader and see what it does with this field. My guess is that it doesn't use the field at all, which is why this discrepancy is never a problem. was (Author: JIRAUSER287471): I just noticed that the file_offset field in ColumnChunk is "required." So there's a few possible mistakes: # The description of the metadata structures is wrong and this really is supposed to be a pointer to the data page (PageHeader), or # There's a completely separate bug where the parquet writer files to store an extra copy of the ColumnMetaData in the file right before the PageHeader, or # The offset should point to where the unique copy of ColumnMetaData is already going to be found in the file footer, although that seems like it would be really hard to calculate. In any case, there's an inconsistency where the metadata definition specifies an offset to ColumnMetaData, where instead a PageHeader is placed. I'm going to go check out the reader and see what it does with this field. My guess is that it doesn't use the field at all, which is why this discrepancy is never a problem. > Bogus file offset for ColumnMetaData written to ColumnChunk metadata of > single parquet files > > > Key: PARQUET-2139 > URL: https://issues.apache.org/jira/browse/PARQUET-2139 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: Timothy Miller >Priority: Major > > In an effort to understand the parquet format better, I've so far written my > own Thrift parser, and upon examining the output, I noticed something > peculiar. > To begin with, check out the definition for ColumnChunk here: > [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift] > You'll notice that if there's an element 2 in the struct, this is supposed to > be a file offset to where a redundant copy of the ColumnMetaData. > Next, have a look at the file called "modified.parquet" attached to > https://issues.apache.org/jira/browse/PARQUET-2069. When I dump the metadata > at the end of the file, I get this: > {{Struct(FileMetaData):}} > {{ 1: i32(version) = I32(1)}} > {{ 2: List(SchemaElement schema):}} > {{ ... > 3: i64(num_rows) = I64(1) > 4: List(RowGroup row_groups): > 1: Struct(RowGroup row_groups): > 1: List(ColumnChunk columns): > 1: Struct(ColumnChunk columns): > 2: i64(file_offset) = I64(4) > 3: Struct(ColumnMetaData meta_data): > 1: Type(type) = I32(6) = BYTE_ARRAY > 2: List(Encoding encodings): > 1: Encoding(encodings) = I32(0) = PLAIN > 2: Encoding(encodings) = I32(3) = RLE > 3: List(string path_in_schema): > 1: string(path_in_schema) = > Binary("destination_addresses") > 2: string(path_in_schema) = Binary("array") > 3: string(path_in_schema) = Binary("element") > 4: CompressionCodec(codec) = I32(0) = UNCOMPRESSED > 5: i64(num_values) = I64(6) > 6: i64(total_uncompressed_size) = I64(197) > 7: i64(total_compressed_size) = I64(197) > 9: i64(data_page_offset) = I64(4) > }} > As you can see, element 2 of the ColumnChunk indicates that there is another > copy of the ColumnMetaData at offset 4 of the file. But then we see that > element 9 of the ColumnMetaData shown above indicates that the data page > offset is ALSO 4, where we should find a Thrift encoding of a PageHeader > structure. Obviously, both structures can't be in the same place, and in fact > a PageHeader is what is located there. >
[jira] [Commented] (PARQUET-2139) Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files
[ https://issues.apache.org/jira/browse/PARQUET-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525272#comment-17525272 ] Timothy Miller commented on PARQUET-2139: - I just noticed that the file_offset field in ColumnChunk is "required." So there's a few possible mistakes: # The description of the metadata structures is wrong and this really is supposed to be a pointer to the data page (PageHeader), or # There's a completely separate bug where the parquet writer files to store an extra copy of the ColumnMetaData in the file right before the PageHeader, or # The offset should point to where the unique copy of ColumnMetaData is already going to be found in the file footer, although that seems like it would be really hard to calculate. In any case, there's an inconsistency where the metadata definition specifies an offset to ColumnMetaData, where instead a PageHeader is placed. I'm going to go check out the reader and see what it does with this field. My guess is that it doesn't use the field at all, which is why this discrepancy is never a problem. > Bogus file offset for ColumnMetaData written to ColumnChunk metadata of > single parquet files > > > Key: PARQUET-2139 > URL: https://issues.apache.org/jira/browse/PARQUET-2139 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: Timothy Miller >Priority: Major > > In an effort to understand the parquet format better, I've so far written my > own Thrift parser, and upon examining the output, I noticed something > peculiar. > To begin with, check out the definition for ColumnChunk here: > [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift] > You'll notice that if there's an element 2 in the struct, this is supposed to > be a file offset to where a redundant copy of the ColumnMetaData. > Next, have a look at the file called "modified.parquet" attached to > https://issues.apache.org/jira/browse/PARQUET-2069. When I dump the metadata > at the end of the file, I get this: > {{Struct(FileMetaData):}} > {{ 1: i32(version) = I32(1)}} > {{ 2: List(SchemaElement schema):}} > {{ ... > 3: i64(num_rows) = I64(1) > 4: List(RowGroup row_groups): > 1: Struct(RowGroup row_groups): > 1: List(ColumnChunk columns): > 1: Struct(ColumnChunk columns): > 2: i64(file_offset) = I64(4) > 3: Struct(ColumnMetaData meta_data): > 1: Type(type) = I32(6) = BYTE_ARRAY > 2: List(Encoding encodings): > 1: Encoding(encodings) = I32(0) = PLAIN > 2: Encoding(encodings) = I32(3) = RLE > 3: List(string path_in_schema): > 1: string(path_in_schema) = > Binary("destination_addresses") > 2: string(path_in_schema) = Binary("array") > 3: string(path_in_schema) = Binary("element") > 4: CompressionCodec(codec) = I32(0) = UNCOMPRESSED > 5: i64(num_values) = I64(6) > 6: i64(total_uncompressed_size) = I64(197) > 7: i64(total_compressed_size) = I64(197) > 9: i64(data_page_offset) = I64(4) > }} > As you can see, element 2 of the ColumnChunk indicates that there is another > copy of the ColumnMetaData at offset 4 of the file. But then we see that > element 9 of the ColumnMetaData shown above indicates that the data page > offset is ALSO 4, where we should find a Thrift encoding of a PageHeader > structure. Obviously, both structures can't be in the same place, and in fact > a PageHeader is what is located there. > Based on what I'm seeing here, I believe that element 2 of ColumnChunk should > be omitted entirely in this scenario, so as to not falsely indicate that > there would be another copy of the ColumnMetadata in this location in the > file where indeed something else is present. > It may take me a while to locate the offending code, but I thought I'd go > ahead and point this out before I set off to investigate. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2139) Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files
[ https://issues.apache.org/jira/browse/PARQUET-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525264#comment-17525264 ] Timothy Miller commented on PARQUET-2139: - I've noticed a few places that could be at fault here. I'm looking at 1.13.0-SNAPSHOT, for reference. The first is at line 513 of org.apache.parquet.format.converter.ParquetMetadataConverter.addRowGroup(), where we find: {{ ColumnChunk columnChunk = new ColumnChunk(columnMetaData.getFirstDataPageOffset()); // verify this is the right offset}} I'm pretty sure that this is the wrong thing to put here. The same thing (columnMetaData.getFirstDataPageOffset()) is used further down when constructing the ColumnMetaData, and since that works properly, this value is evidently the location of the PageHeader, not an extra copy of the ColumnMetaData. If no extra copy of the ColumnMetaData has been specified, then the default constructor should be called. In fact, it may be that we should ALWAYS call the default constructor here, since I cannot find any place in the code where a pointer to a redundant copy of the ColumnMetaData can even be specified. Secondly, I notice that at line 1179 of org.apache.parquet.format.ColumnChunk.write(), the FILE_OFFSET_FIELD_DESC field is always written unconditionally to the thrift encoder: {{ oprot.writeFieldBegin(FILE_OFFSET_FIELD_DESC);}} {{ oprot.writeI64(struct.file_offset);}} {{ oprot.writeFieldEnd();}} This should check and only write the field if it's nonzero. > Bogus file offset for ColumnMetaData written to ColumnChunk metadata of > single parquet files > > > Key: PARQUET-2139 > URL: https://issues.apache.org/jira/browse/PARQUET-2139 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: Timothy Miller >Priority: Major > > In an effort to understand the parquet format better, I've so far written my > own Thrift parser, and upon examining the output, I noticed something > peculiar. > To begin with, check out the definition for ColumnChunk here: > [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift] > You'll notice that if there's an element 2 in the struct, this is supposed to > be a file offset to where a redundant copy of the ColumnMetaData. > Next, have a look at the file called "modified.parquet" attached to > https://issues.apache.org/jira/browse/PARQUET-2069. When I dump the metadata > at the end of the file, I get this: > {{Struct(FileMetaData):}} > {{ 1: i32(version) = I32(1)}} > {{ 2: List(SchemaElement schema):}} > {{ ... > 3: i64(num_rows) = I64(1) > 4: List(RowGroup row_groups): > 1: Struct(RowGroup row_groups): > 1: List(ColumnChunk columns): > 1: Struct(ColumnChunk columns): > 2: i64(file_offset) = I64(4) > 3: Struct(ColumnMetaData meta_data): > 1: Type(type) = I32(6) = BYTE_ARRAY > 2: List(Encoding encodings): > 1: Encoding(encodings) = I32(0) = PLAIN > 2: Encoding(encodings) = I32(3) = RLE > 3: List(string path_in_schema): > 1: string(path_in_schema) = > Binary("destination_addresses") > 2: string(path_in_schema) = Binary("array") > 3: string(path_in_schema) = Binary("element") > 4: CompressionCodec(codec) = I32(0) = UNCOMPRESSED > 5: i64(num_values) = I64(6) > 6: i64(total_uncompressed_size) = I64(197) > 7: i64(total_compressed_size) = I64(197) > 9: i64(data_page_offset) = I64(4) > }} > As you can see, element 2 of the ColumnChunk indicates that there is another > copy of the ColumnMetaData at offset 4 of the file. But then we see that > element 9 of the ColumnMetaData shown above indicates that the data page > offset is ALSO 4, where we should find a Thrift encoding of a PageHeader > structure. Obviously, both structures can't be in the same place, and in fact > a PageHeader is what is located there. > Based on what I'm seeing here, I believe that element 2 of ColumnChunk should > be omitted entirely in this scenario, so as to not falsely indicate that > there would be another copy of the ColumnMetadata in this location in the > file where indeed something else is present. > It may take me a while to locate the offending code, but I thought I'd go > ahead and point this out before I set off to investigate. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2139) Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files
[ https://issues.apache.org/jira/browse/PARQUET-2139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17525229#comment-17525229 ] Timothy Miller commented on PARQUET-2139: - Of course, I'll be embarrassed if this turns out to just be a bug in my own thrift parser, but everything seems to line up so far. > Bogus file offset for ColumnMetaData written to ColumnChunk metadata of > single parquet files > > > Key: PARQUET-2139 > URL: https://issues.apache.org/jira/browse/PARQUET-2139 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: Timothy Miller >Priority: Major > > In an effort to understand the parquet format better, I've so far written my > own Thrift parser, and upon examining the output, I noticed something > peculiar. > To begin with, check out the definition for ColumnChunk here: > [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift] > You'll notice that if there's an element 2 in the struct, this is supposed to > be a file offset to where a redundant copy of the ColumnMetaData. > Next, have a look at the file called "modified.parquet" attached to > https://issues.apache.org/jira/browse/PARQUET-2069. When I dump the metadata > at the end of the file, I get this: > {{Struct(FileMetaData):}} > {{ 1: i32(version) = I32(1)}} > {{ 2: List(SchemaElement schema):}} > {{ ... > 3: i64(num_rows) = I64(1) > 4: List(RowGroup row_groups): > 1: Struct(RowGroup row_groups): > 1: List(ColumnChunk columns): > 1: Struct(ColumnChunk columns): > 2: i64(file_offset) = I64(4) > 3: Struct(ColumnMetaData meta_data): > 1: Type(type) = I32(6) = BYTE_ARRAY > 2: List(Encoding encodings): > 1: Encoding(encodings) = I32(0) = PLAIN > 2: Encoding(encodings) = I32(3) = RLE > 3: List(string path_in_schema): > 1: string(path_in_schema) = > Binary("destination_addresses") > 2: string(path_in_schema) = Binary("array") > 3: string(path_in_schema) = Binary("element") > 4: CompressionCodec(codec) = I32(0) = UNCOMPRESSED > 5: i64(num_values) = I64(6) > 6: i64(total_uncompressed_size) = I64(197) > 7: i64(total_compressed_size) = I64(197) > 9: i64(data_page_offset) = I64(4) > }} > As you can see, element 2 of the ColumnChunk indicates that there is another > copy of the ColumnMetaData at offset 4 of the file. But then we see that > element 9 of the ColumnMetaData shown above indicates that the data page > offset is ALSO 4, where we should find a Thrift encoding of a PageHeader > structure. Obviously, both structures can't be in the same place, and in fact > a PageHeader is what is located there. > Based on what I'm seeing here, I believe that element 2 of ColumnChunk should > be omitted entirely in this scenario, so as to not falsely indicate that > there would be another copy of the ColumnMetadata in this location in the > file where indeed something else is present. > It may take me a while to locate the offending code, but I thought I'd go > ahead and point this out before I set off to investigate. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Created] (PARQUET-2139) Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files
Timothy Miller created PARQUET-2139: --- Summary: Bogus file offset for ColumnMetaData written to ColumnChunk metadata of single parquet files Key: PARQUET-2139 URL: https://issues.apache.org/jira/browse/PARQUET-2139 Project: Parquet Issue Type: Bug Components: parquet-mr Affects Versions: 1.12.2 Reporter: Timothy Miller In an effort to understand the parquet format better, I've so far written my own Thrift parser, and upon examining the output, I noticed something peculiar. To begin with, check out the definition for ColumnChunk here: [https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift] You'll notice that if there's an element 2 in the struct, this is supposed to be a file offset to where a redundant copy of the ColumnMetaData. Next, have a look at the file called "modified.parquet" attached to https://issues.apache.org/jira/browse/PARQUET-2069. When I dump the metadata at the end of the file, I get this: {{Struct(FileMetaData):}} {{ 1: i32(version) = I32(1)}} {{ 2: List(SchemaElement schema):}} {{ ... 3: i64(num_rows) = I64(1) 4: List(RowGroup row_groups): 1: Struct(RowGroup row_groups): 1: List(ColumnChunk columns): 1: Struct(ColumnChunk columns): 2: i64(file_offset) = I64(4) 3: Struct(ColumnMetaData meta_data): 1: Type(type) = I32(6) = BYTE_ARRAY 2: List(Encoding encodings): 1: Encoding(encodings) = I32(0) = PLAIN 2: Encoding(encodings) = I32(3) = RLE 3: List(string path_in_schema): 1: string(path_in_schema) = Binary("destination_addresses") 2: string(path_in_schema) = Binary("array") 3: string(path_in_schema) = Binary("element") 4: CompressionCodec(codec) = I32(0) = UNCOMPRESSED 5: i64(num_values) = I64(6) 6: i64(total_uncompressed_size) = I64(197) 7: i64(total_compressed_size) = I64(197) 9: i64(data_page_offset) = I64(4) }} As you can see, element 2 of the ColumnChunk indicates that there is another copy of the ColumnMetaData at offset 4 of the file. But then we see that element 9 of the ColumnMetaData shown above indicates that the data page offset is ALSO 4, where we should find a Thrift encoding of a PageHeader structure. Obviously, both structures can't be in the same place, and in fact a PageHeader is what is located there. Based on what I'm seeing here, I believe that element 2 of ColumnChunk should be omitted entirely in this scenario, so as to not falsely indicate that there would be another copy of the ColumnMetadata in this location in the file where indeed something else is present. It may take me a while to locate the offending code, but I thought I'd go ahead and point this out before I set off to investigate. -- This message was sent by Atlassian Jira (v8.20.7#820007)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17522406#comment-17522406 ] Timothy Miller commented on PARQUET-2069: - I found a fix for the problem. This is going to look like an ugly hack-fix, but there doesn't seem to be any other place to put this, since the schema being unwound here isn't constructed in ParquetMR but is rather just some JSON that's been parsed out of the parquet file. The solution is this. In AvroRecordConverter.AvroRecordConverter.isElementType, just before this call to checkReaderWriterCompatibility, I made this change: {{+ if (elementSchema.getName().equals("list")) elementSchema.addAlias("array", "");}} {{+ if (elementSchema.getName().equals("array")) elementSchema.addAlias("list", "");}} {{ }}{{{}if (checkReaderWriterCompatibility(elementSchema, schemaFromRepeated){}}}{{{}{}}}{{{}{}}} The following is the PR I created to fix this: https://github.com/apache/parquet-mr/pull/957 If this is the wrong way to do it, maybe it'll get the attention of someone who understands this better than I do. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17522347#comment-17522347 ] Timothy Miller commented on PARQUET-2069: - This appears to occur due to the reader and writer schemas having a name mismatch on the record I mentioned earlier. The problem appears to be either in Avro or in how Avro is used. When comparing these structures for compatibility, there this method in org.apache.avro.SchemaCompatibility is being used: {{ public static boolean schemaNameEquals(final Schema reader, final Schema writer) {}} {{ if (objectsEqual(reader.getName(), writer.getName())) {}} {{ return true;}} {{ }}} {{ // Apply reader aliases:}} {{ return reader.getAliases().contains(writer.getFullName());}} {{ }}} Evidently, when the reader schema was set up, no alias was added that would allow "list" and "array" to match here, and this causes the failure. This method is called by checkSchemaNames, which is called by calculateCompatibility, which is called by getCompatibility, which is called by another getCompatibility, which is called from checkReaderWriterCompatibility, which is called from org.apache.parquet.avro.AvroRecordConverter.isElementType. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2133) Support Int8 and Int16 as basic type
[ https://issues.apache.org/jira/browse/PARQUET-2133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519690#comment-17519690 ] Timothy Miller commented on PARQUET-2133: - Have you started working on implementing this? What is your progress. I'd be happy to work with you on it. > Support Int8 and Int16 as basic type > > > Key: PARQUET-2133 > URL: https://issues.apache.org/jira/browse/PARQUET-2133 > Project: Parquet > Issue Type: Improvement > Components: parquet-format >Reporter: Jackey Lee >Priority: Major > > Int8 and Int16 are not supported as basic in previos version. Using 4 bytes > to store int8 seems not a good idea, which means requiring more storage and > read and write very slow. Besides, it is not friendly with regular computing > format, such as velox, arrow, vector and so on. > With Int8 and Int16 supported, we can get less storage and better performance > on reading and writing. As for forward compatible, we can use version in > FileMetaData to choose how to read parquet data. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-1681) Avro's isElementType() change breaks the reading of some parquet(1.8.1) files
[ https://issues.apache.org/jira/browse/PARQUET-1681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519683#comment-17519683 ] Timothy Miller commented on PARQUET-1681: - Is this related to PARQUET-2069? It looks like it might be. > Avro's isElementType() change breaks the reading of some parquet(1.8.1) files > - > > Key: PARQUET-1681 > URL: https://issues.apache.org/jira/browse/PARQUET-1681 > Project: Parquet > Issue Type: Improvement > Components: parquet-avro >Affects Versions: 1.10.0, 1.9.1, 1.11.0 >Reporter: Xinli Shang >Assignee: Xinli Shang >Priority: Critical > > When using the Avro schema below to write a parquet(1.8.1) file and then read > back by using parquet 1.10.1 without passing any schema, the reading throws > an exception "XXX is not a group" . Reading through parquet 1.8.1 is fine. > { > "name": "phones", > "type": [ > "null", > { > "type": "array", > "items": { > "type": "record", > "name": "phones_items", > "fields": [ > > { "name": "phone_number", > "type": [ "null", > "string" ], "default": null > } > ] > } > } > ], > "default": null > } > The code to read is as below > val reader = > AvroParquetReader._builder_[SomeRecordType](parquetPath).withConf(*new* > Configuration).build() > reader.read() > PARQUET-651 changed the method isElementType() by relying on Avro's > checkReaderWriterCompatibility() to check the compatibility. However, > checkReaderWriterCompatibility() consider the ParquetSchema and the > AvroSchema(converted from File schema) as not compatible(the name in avro > schema is ‘phones_items’, but the name is ‘array’ in Parquet schema, hence > not compatible) . Hence return false and caused the “phone_number” field in > the above schema to be considered as group type which is not true. Then the > exception throws as .asGroupType(). > I didn’t try writing via parquet 1.10.1 would reproduce the same problem or > not. But it could because the translation of Avro schema to Parquet schema is > not changed(didn’t verify yet). > I hesitate to revert PARQUET-651 because it solved several problems. I would > like to hear the community's thoughts on it. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519231#comment-17519231 ] Timothy Miller commented on PARQUET-2069: - With the original file, the debug message says this instead: [main] DEBUG org.apache.avro.SchemaCompatibility - Checking compatibility of reader \{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]} with writer \{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]} What I can't figure out is why the the reader record name is "list" in the modified case, since all the lists in the original were converted to arrays. So it should also be an array. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519218#comment-17519218 ] Timothy Miller commented on PARQUET-2069: - Here's a log message that shows why it's failing: {{[main] DEBUG org.apache.avro.SchemaCompatibility - Checking compatibility of }} {{reader \{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]} with }} {{writer \{"type":"record","name":"array","fields":[{"name":"element","type":["null","string"],"default":null}]}}} When {{AvroRecordConverter.newConverter(Schema schema, Type type, GenericData model, Class knownClass, ParentValueContainer setter)}} encounters an ARRAY type, it calls {{{}AvroRecordConverter.AvroCollectionConverter{}}}, which calls {{{}AvroRecordConverter.isElementType{}}}, which calls {{{}SchemaCompatibility.checkReaderWriterCompatibility{}}}. The type that returns is INCOMPATIBLE, and this is because the record name for the reader is "list" while the record name for the writer is "array", and these are considered incompatible. Either some change has to be made to the compatibility check to allow array and list types to be considered compatible, or the writer schema has to be computed differently. The latter appears to be done in {{{}AvroSchemaConverter.convertFields{}}}, which is called from {{{}AvroRecordConverter.isElementType{}}}, like this: {{ Schema schemaFromRepeated = CONVERTER.convert(repeatedType.asGroupType());}} > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17519019#comment-17519019 ] Timothy Miller commented on PARQUET-2069: - Based on the fact that the option is named "old" list structure, I'm going to guess that that's been deprecated in favor of a newer array format. That shouldn't necessarily cause trouble to the user, and it certainly should not throw an exception. Since I've already started on this, I may at least try to figure out how to make it not crash. The exception is thrown as a result of code in {{AvroRecordConverter.newConverter(Schema schema, Type type, GenericData model, Class knownClass, ParentValueContainer setter).}} The schema is indicating that the type is {{{}RECORD{}}}, while the the {{type}} variable indicates that the type is a string. Gotta figure out how this inconsistency is happening. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Comment Edited] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17518968#comment-17518968 ] Timothy Miller edited comment on PARQUET-2069 at 4/7/22 3:53 PM: - An initial look at this suggests that the writer is corrupting the data somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) harmless changes from "list" to "array", but there's also two examples of data being corrupted. See the screenshot. There, you can see that the string "Washington, DC, USA" is coming out of the modified file as "oX/xgBpN04At7pWhlc6/7kX0nIdGGp4EJCY2guUZ/vY=". It's probably worse than that, but the command line tool is doing its best to make sense of a corrupted file. was (Author: JIRAUSER287471): An initial look at this suggests that the writer is corrupting the data somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) harmless changes from "list" to "array", but there's also two examples of data being corrupted. See the screenshot. > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Updated] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timothy Miller updated PARQUET-2069: Attachment: parquet-diff.png > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet, parquet-diff.png > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Comment Edited] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17518968#comment-17518968 ] Timothy Miller edited comment on PARQUET-2069 at 4/7/22 3:50 PM: - An initial look at this suggests that the writer is corrupting the data somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) harmless changes from "list" to "array", but there's also two examples of data being corrupted. See the screenshot. was (Author: JIRAUSER287471): An initial look at this suggests that the writer is corrupting the data somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) harmless changes from "list" to "array", but there's also two examples of data being corrupted. See the screenshot. !image-2022-04-07-11-49-38-982.png! > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR
[ https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17518968#comment-17518968 ] Timothy Miller commented on PARQUET-2069: - An initial look at this suggests that the writer is corrupting the data somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) harmless changes from "list" to "array", but there's also two examples of data being corrupted. See the screenshot. !image-2022-04-07-11-49-38-982.png! > Parquet file containing arrays, written by Parquet-MR, cannot be read again > by Parquet-MR > - > > Key: PARQUET-2069 > URL: https://issues.apache.org/jira/browse/PARQUET-2069 > Project: Parquet > Issue Type: Bug > Components: parquet-avro >Affects Versions: 1.12.0 > Environment: Windows 10 >Reporter: Devon Kozenieski >Priority: Blocker > Attachments: modified.parquet, original.parquet > > > In the attached files, there is one original file, and one written modified > file that results after reading the original file and writing it back with > Parquet-MR, with a few values modified. The schema should not be modified, > since the schema of the input file is used as the schema to write the output > file. However, the output file has a slightly modified schema that then > cannot be read back the same way again with Parquet-MR, resulting in the > exception message: java.lang.ClassCastException: optional binary element > (STRING) is not a group > My guess is that the issue lies in the Avro schema conversion. > The Parquet files attached have some arrays and some nested fields. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2126) Thread safety bug in CodecFactory
[ https://issues.apache.org/jira/browse/PARQUET-2126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17518883#comment-17518883 ] Timothy Miller commented on PARQUET-2126: - This bug isn't affecting me. My employer has tasked me with improving the performance of Trino on Parquet sources, and I've found that to do that, I have to make improvements to ParquetMR. Well, while I have to learn a lot about ParquetMR anyway, I might as well help fix some bugs in the process. I've somewhat randomly picked PARQUET-2069 to work on, but it would be even better to work on what the maintainers think are the most important problems. Let me know. > Thread safety bug in CodecFactory > - > > Key: PARQUET-2126 > URL: https://issues.apache.org/jira/browse/PARQUET-2126 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: James Turton >Priority: Major > > The code for returning Compressor objects to the caller goes to some lengths > to achieve thread safety, including keeping Codec objects in an Apache > Commons pool that has thread-safe borrow semantics. This is all undone by > the BytesCompressor and BytesDecompressor Maps in > org.apache.parquet.hadoop.CodecFactory which end up caching single compressor > and decompressor instances due to code in CodecFactory@getCompressor and > CodecFactory@getDecompressor. When the caller runs multiple threads, those > threads end up sharing compressor and decompressor instances. > For compressors based on Xerial Snappy this bug has no effect because that > library is itself thread safe. But when BuiltInGzipCompressor from Hadoop is > selected for the CompressionCodecName.GZIP case, serious problems ensue. > That class is not thread safe and sharing one instance of it between threads > produces both silent data corruption and JVM crashes. > To fix this situation, parquet-mr should stop caching single compressor and > decompressor instances. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2126) Thread safety bug in CodecFactory
[ https://issues.apache.org/jira/browse/PARQUET-2126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17518336#comment-17518336 ] Timothy Miller commented on PARQUET-2126: - Does the resolution of DRILL-8139 mean that PARQUET-2126 is also resolved? Or are there more steps that are required? > Thread safety bug in CodecFactory > - > > Key: PARQUET-2126 > URL: https://issues.apache.org/jira/browse/PARQUET-2126 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: James Turton >Priority: Major > > The code for returning Compressor objects to the caller goes to some lengths > to achieve thread safety, including keeping Codec objects in an Apache > Commons pool that has thread-safe borrow semantics. This is all undone by > the BytesCompressor and BytesDecompressor Maps in > org.apache.parquet.hadoop.CodecFactory which end up caching single compressor > and decompressor instances due to code in CodecFactory@getCompressor and > CodecFactory@getDecompressor. When the caller runs multiple threads, those > threads end up sharing compressor and decompressor instances. > For compressors based on Xerial Snappy this bug has no effect because that > library is itself thread safe. But when BuiltInGzipCompressor from Hadoop is > selected for the CompressionCodecName.GZIP case, serious problems ensue. > That class is not thread safe and sharing one instance of it between threads > produces both silent data corruption and JVM crashes. > To fix this situation, parquet-mr should stop caching single compressor and > decompressor instances. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Commented] (PARQUET-2135) Performance optimizations: Merged all LittleEndianDataInputStream functionality into ByteBufferInputStream
[ https://issues.apache.org/jira/browse/PARQUET-2135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17516909#comment-17516909 ] Timothy Miller commented on PARQUET-2135: - Extra note: The reason PlainValuesReader still includes an unused LittleEndianDataInputStream member is because if I don't, the build will fail, indicating an incompatible API change. > Performance optimizations: Merged all LittleEndianDataInputStream > functionality into ByteBufferInputStream > -- > > Key: PARQUET-2135 > URL: https://issues.apache.org/jira/browse/PARQUET-2135 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.12.2 >Reporter: Timothy Miller >Priority: Major > > This PR is all performance optimization. In benchmarking with Trino, we find > query performance to improve from 5% to 15%, depending on the query, and that > includes all the I/O time from S3. > The main modification is to merge all of LittleEndianDataInputStream > functionality into ByteBufferInputStream, which yields the following benefits: > * Elimination of extra layers of abstraction and method call overhead > * Enable the use of intrinsics for readInt, readLong, etc. > * Availability of faster access methods like readFully and skipFully, > without the need for helper functions > * Reduces some object creation in the performance critical path > This also includes and enables performance optimizations to: > * ByteBitPackingValuesReader > * PlainValuesReader > * RunLengthBitPackingHybridDecoder > Context: > I've been working on improving Parquet reading performance in Trino, mostly > by profiling while running performance benchmarks and TPCDS queries. This PR > is a subset of the changes I made that have more than doubled the performance > of a lot of TPCDS queries (wall clock time, including the S3 access time). If > you are kind enough to accept these changes, I have more I would like to > contribute. -- This message was sent by Atlassian Jira (v8.20.1#820001)
[jira] [Created] (PARQUET-2135) Performance optimizations: Merged all LittleEndianDataInputStream functionality into ByteBufferInputStream
Timothy Miller created PARQUET-2135: --- Summary: Performance optimizations: Merged all LittleEndianDataInputStream functionality into ByteBufferInputStream Key: PARQUET-2135 URL: https://issues.apache.org/jira/browse/PARQUET-2135 Project: Parquet Issue Type: Improvement Components: parquet-mr Affects Versions: 1.12.2 Reporter: Timothy Miller This PR is all performance optimization. In benchmarking with Trino, we find query performance to improve from 5% to 15%, depending on the query, and that includes all the I/O time from S3. The main modification is to merge all of LittleEndianDataInputStream functionality into ByteBufferInputStream, which yields the following benefits: * Elimination of extra layers of abstraction and method call overhead * Enable the use of intrinsics for readInt, readLong, etc. * Availability of faster access methods like readFully and skipFully, without the need for helper functions * Reduces some object creation in the performance critical path This also includes and enables performance optimizations to: * ByteBitPackingValuesReader * PlainValuesReader * RunLengthBitPackingHybridDecoder Context: I've been working on improving Parquet reading performance in Trino, mostly by profiling while running performance benchmarks and TPCDS queries. This PR is a subset of the changes I made that have more than doubled the performance of a lot of TPCDS queries (wall clock time, including the S3 access time). If you are kind enough to accept these changes, I have more I would like to contribute. -- This message was sent by Atlassian Jira (v8.20.1#820001)