[jira] [Commented] (PARQUET-2149) Implement async IO for Parquet file reader

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556480#comment-17556480
 ] 

ASF GitHub Bot commented on PARQUET-2149:
-

parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1160693399

   @shangxinli Thank you for the review! I'll address these comments asap.
   I am reviewing the thread pool and its initialization. IMO, it is better if 
there is no default initialization of the pool and the calling 
application/framework does so explicitly. One side effect of the default 
initialization is that the pool is created unnecessarily even if async is off. 
Also, if an application, shades and includes another copy of the library (or 
transitively, many more), then one more thread pool gets created for every 
version of the library included. 
   It is probably a better idea to allow the thread pool to be assigned as a 
per instance variable. The calling application can then decide to use a single 
pool for all instances or a new one per instance whichever use case is better 
for their performance.
   Finally, some large scale testing has revealed a possible resource leak. I'm 
looking into addressing it. 




> Implement async IO for Parquet file reader
> --
>
> Key: PARQUET-2149
> URL: https://issues.apache.org/jira/browse/PARQUET-2149
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Parth Chandra
>Priority: Major
>
> ParquetFileReader's implementation has the following flow (simplified) - 
>       - For every column -> Read from storage in 8MB blocks -> Read all 
> uncompressed pages into output queue 
>       - From output queues -> (downstream ) decompression + decoding
> This flow is serialized, which means that downstream threads are blocked 
> until the data has been read. Because a large part of the time spent is 
> waiting for data from storage, threads are idle and CPU utilization is really 
> low.
> There is no reason why this cannot be made asynchronous _and_ parallel. So 
> For Column _i_ -> reading one chunk until end, from storage -> intermediate 
> output queue -> read one uncompressed page until end -> output queue -> 
> (downstream ) decompression + decoding
> Note that this can be made completely self contained in ParquetFileReader and 
> downstream implementations like Iceberg and Spark will automatically be able 
> to take advantage without code change as long as the ParquetFileReader apis 
> are not changed. 
> In past work with async io  [Drill - async page reader 
> |https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java]
>  , I have seen 2x-3x improvement in reading speed for Parquet files.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[GitHub] [parquet-mr] parthchandra commented on pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader

2022-06-20 Thread GitBox


parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1160693399

   @shangxinli Thank you for the review! I'll address these comments asap.
   I am reviewing the thread pool and its initialization. IMO, it is better if 
there is no default initialization of the pool and the calling 
application/framework does so explicitly. One side effect of the default 
initialization is that the pool is created unnecessarily even if async is off. 
Also, if an application, shades and includes another copy of the library (or 
transitively, many more), then one more thread pool gets created for every 
version of the library included. 
   It is probably a better idea to allow the thread pool to be assigned as a 
per instance variable. The calling application can then decide to use a single 
pool for all instances or a new one per instance whichever use case is better 
for their performance.
   Finally, some large scale testing has revealed a possible resource leak. I'm 
looking into addressing it. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2149) Implement async IO for Parquet file reader

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556467#comment-17556467
 ] 

ASF GitHub Bot commented on PARQUET-2149:
-

steveloughran commented on code in PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#discussion_r901859862


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -126,6 +127,42 @@ public class ParquetFileReader implements Closeable {
 
   public static String PARQUET_READ_PARALLELISM = 
"parquet.metadata.read.parallelism";
 
+  public static int numProcessors = Runtime.getRuntime().availableProcessors();

Review Comment:
   dynamically changing the number of threads/buffer sizes/cache sizes is a 
recurrent source of pain in past work, as once you get to 128 core systems they 
often end up asking for too much of a limited resource





> Implement async IO for Parquet file reader
> --
>
> Key: PARQUET-2149
> URL: https://issues.apache.org/jira/browse/PARQUET-2149
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Reporter: Parth Chandra
>Priority: Major
>
> ParquetFileReader's implementation has the following flow (simplified) - 
>       - For every column -> Read from storage in 8MB blocks -> Read all 
> uncompressed pages into output queue 
>       - From output queues -> (downstream ) decompression + decoding
> This flow is serialized, which means that downstream threads are blocked 
> until the data has been read. Because a large part of the time spent is 
> waiting for data from storage, threads are idle and CPU utilization is really 
> low.
> There is no reason why this cannot be made asynchronous _and_ parallel. So 
> For Column _i_ -> reading one chunk until end, from storage -> intermediate 
> output queue -> read one uncompressed page until end -> output queue -> 
> (downstream ) decompression + decoding
> Note that this can be made completely self contained in ParquetFileReader and 
> downstream implementations like Iceberg and Spark will automatically be able 
> to take advantage without code change as long as the ParquetFileReader apis 
> are not changed. 
> In past work with async io  [Drill - async page reader 
> |https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java]
>  , I have seen 2x-3x improvement in reading speed for Parquet files.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader

2022-06-20 Thread GitBox


steveloughran commented on code in PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#discussion_r901859862


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -126,6 +127,42 @@ public class ParquetFileReader implements Closeable {
 
   public static String PARQUET_READ_PARALLELISM = 
"parquet.metadata.read.parallelism";
 
+  public static int numProcessors = Runtime.getRuntime().availableProcessors();

Review Comment:
   dynamically changing the number of threads/buffer sizes/cache sizes is a 
recurrent source of pain in past work, as once you get to 128 core systems they 
often end up asking for too much of a limited resource



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2134) Incorrect type checking in HadoopStreams.wrap

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556465#comment-17556465
 ] 

ASF GitHub Bot commented on PARQUET-2134:
-

steveloughran commented on code in PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r901856428


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -50,51 +46,45 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (byteBufferReadableClass != null && h2SeekableConstructor != null &&
-byteBufferReadableClass.isInstance(stream.getWrappedStream())) {
-  try {
-return h2SeekableConstructor.newInstance(stream);
-  } catch (InstantiationException | IllegalAccessException e) {
-LOG.warn("Could not instantiate H2SeekableInputStream, falling back to 
byte array reads", e);
-return new H1SeekableInputStream(stream);
-  } catch (InvocationTargetException e) {
-throw new ParquetDecodingException(
-"Could not instantiate H2SeekableInputStream", 
e.getTargetException());
-  }
+if (isWrappedStreamByteBufferReadable(stream)) {
+  return new H2SeekableInputStream(stream);
 } else {
   return new H1SeekableInputStream(stream);
 }
   }
 
-  private static Class getReadableClass() {
-try {
-  return Class.forName("org.apache.hadoop.fs.ByteBufferReadable");
-} catch (ClassNotFoundException | NoClassDefFoundError e) {
-  return null;
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is "the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * That is: all streams which implement ByteBufferReadable
+   * other than FSDataInputStream successfuly support read(ByteBuffer).
+   * This is true for all filesytem clients the hadoop codebase.
+   *
+   * In hadoop 3.3.0+, the StreamCapabilities probe can be used to
+   * check this: only those streams which provide the read(ByteBuffer)
+   * semantics MAY return true for the probe "in:readbytebuffer";
+   * FSDataInputStream will pass the probe down to the underlying stream.
+   *
+   * @param stream stream to probe
+   * @return true if it is safe to a H2SeekableInputStream to access the data
+   */
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+if (stream.hasCapability("in:readbytebuffer")) {

Review Comment:
   no, the StreamCapabilities probe has been around since hadoop 2. it is just 
in 3.3.0 all streams which implement the api return true for this probe...a 
probe which gets passed down the wrapped streams. It avoids looking at the 
wrapped streams as you should be able to trust the response (put differently: 
if something lied it is in trouble)





> Incorrect type checking in HadoopStreams.wrap
> -
>
> Key: PARQUET-2134
> URL: https://issues.apache.org/jira/browse/PARQUET-2134
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.8.3, 1.10.1, 1.11.2, 1.12.2
>Reporter: Todd Gao
>Priority: Minor
>
> The method 
> [HadoopStreams.wrap|https://github.com/apache/parquet-mr/blob/4d062dc37577e719dcecc666f8e837843e44a9be/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L51]
>  wraps an FSDataInputStream to a SeekableInputStream. 
> It checks whether the underlying stream of the passed  FSDataInputStream 
> implements ByteBufferReadable: if true, wraps the FSDataInputStream to 
> H2SeekableInputStream; otherwise, wraps to H1SeekableInputStream.
> In some cases, we may add another wrapper over FSDataInputStream. For 
> example, 
> {code:java}
> class CustomDataInputStream extends FSDataInputStream {
> public CustomDataInputStream(FSDataInputStream original) {
> super(original);
> }
> }
> {code}
> When we create an FSDataInputStream, whose underlying stream does not 
> implements ByteBufferReadable, and then creates a CustomDataInputStream with 
> it. If we use HadoopStreams.wrap to create a SeekableInputStream, we may get 
> an error like 
> {quote}java.lang.UnsupportedOperationException: Byte-buffer read unsupported 
> by input stream{quote}
> We can fix this by taking recursive checks over the underlying stream of 
> FSDataInputStream.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #951: PARQUET-2134: Fix type checking in HadoopStreams.wrap

2022-06-20 Thread GitBox


steveloughran commented on code in PR #951:
URL: https://github.com/apache/parquet-mr/pull/951#discussion_r901856428


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -50,51 +46,45 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (byteBufferReadableClass != null && h2SeekableConstructor != null &&
-byteBufferReadableClass.isInstance(stream.getWrappedStream())) {
-  try {
-return h2SeekableConstructor.newInstance(stream);
-  } catch (InstantiationException | IllegalAccessException e) {
-LOG.warn("Could not instantiate H2SeekableInputStream, falling back to 
byte array reads", e);
-return new H1SeekableInputStream(stream);
-  } catch (InvocationTargetException e) {
-throw new ParquetDecodingException(
-"Could not instantiate H2SeekableInputStream", 
e.getTargetException());
-  }
+if (isWrappedStreamByteBufferReadable(stream)) {
+  return new H2SeekableInputStream(stream);
 } else {
   return new H1SeekableInputStream(stream);
 }
   }
 
-  private static Class getReadableClass() {
-try {
-  return Class.forName("org.apache.hadoop.fs.ByteBufferReadable");
-} catch (ClassNotFoundException | NoClassDefFoundError e) {
-  return null;
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is "the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * That is: all streams which implement ByteBufferReadable
+   * other than FSDataInputStream successfuly support read(ByteBuffer).
+   * This is true for all filesytem clients the hadoop codebase.
+   *
+   * In hadoop 3.3.0+, the StreamCapabilities probe can be used to
+   * check this: only those streams which provide the read(ByteBuffer)
+   * semantics MAY return true for the probe "in:readbytebuffer";
+   * FSDataInputStream will pass the probe down to the underlying stream.
+   *
+   * @param stream stream to probe
+   * @return true if it is safe to a H2SeekableInputStream to access the data
+   */
+  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
+if (stream.hasCapability("in:readbytebuffer")) {

Review Comment:
   no, the StreamCapabilities probe has been around since hadoop 2. it is just 
in 3.3.0 all streams which implement the api return true for this probe...a 
probe which gets passed down the wrapped streams. It avoids looking at the 
wrapped streams as you should be able to trust the response (put differently: 
if something lied it is in trouble)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2150) parquet-protobuf to compile on mac M1

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556450#comment-17556450
 ] 

ASF GitHub Bot commented on PARQUET-2150:
-

steveloughran commented on PR #970:
URL: https://github.com/apache/parquet-mr/pull/970#issuecomment-1160638077

   this patch is based on Dongjoon;s one for hadoop, tells maven to use the x86 
artifact on macbook m1 builds.
   
   the sunchao one switches to a version of protobuf with a genuine mac m1 
artifacts, a version which should also include some CVE fixes.




> parquet-protobuf to compile on mac M1
> -
>
> Key: PARQUET-2150
> URL: https://issues.apache.org/jira/browse/PARQUET-2150
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-protobuf
>Affects Versions: 1.13.0
>Reporter: Steve Loughran
>Priority: Major
>
> parquet-protobuf module fails to compile on Mac M1 because the maven protoc 
> plugin cannot find the native osx-aarch_64:3.16.1  binary.
> the build needs to be tweaked to pick up the x86 binaries



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[GitHub] [parquet-mr] steveloughran commented on pull request #970: PARQUET-2150: parquet-protobuf to compile on Mac M1

2022-06-20 Thread GitBox


steveloughran commented on PR #970:
URL: https://github.com/apache/parquet-mr/pull/970#issuecomment-1160638077

   this patch is based on Dongjoon;s one for hadoop, tells maven to use the x86 
artifact on macbook m1 builds.
   
   the sunchao one switches to a version of protobuf with a genuine mac m1 
artifacts, a version which should also include some CVE fixes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556408#comment-17556408
 ] 

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901748898


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {
+  System.err.println("Warning, Avro schema doesn't match Parquet schema, 
falling back to conversion: " + e.toString());

Review Comment:
   Oversight on my part.





> 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)


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

2022-06-20 Thread GitBox


theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901748898


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {
+  System.err.println("Warning, Avro schema doesn't match Parquet schema, 
falling back to conversion: " + e.toString());

Review Comment:
   Oversight on my part.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556406#comment-17556406
 ] 

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901740673


##
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayListCompatibility.java:
##
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.avro;
+
+import com.google.common.io.Resources;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.junit.Test;
+import java.io.IOException;
+
+public class TestArrayListCompatibility {
+
+  @Test
+  public void testListArrayCompatibility() throws IOException {
+Path testPath = new 
Path(Resources.getResource("list-array-compat.parquet").getFile());
+
+Configuration conf = new Configuration();
+ParquetReader parquetReader =
+  AvroParquetReader.builder(testPath).withConf(conf).build();
+GenericData.Record firstRecord;
+try {
+  firstRecord = (GenericData.Record) parquetReader.read();
+} catch (Exception x) {
+  x.printStackTrace();

Review Comment:
   Ok, I got rid of the extra catch. I'm not sure what kind of exceptions 
parquetReader.read() can throw, though, so we'll see if we get a compile error 
from not specifying it in the function signature. :)





> 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)


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

2022-06-20 Thread GitBox


theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901740673


##
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayListCompatibility.java:
##
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.avro;
+
+import com.google.common.io.Resources;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.junit.Test;
+import java.io.IOException;
+
+public class TestArrayListCompatibility {
+
+  @Test
+  public void testListArrayCompatibility() throws IOException {
+Path testPath = new 
Path(Resources.getResource("list-array-compat.parquet").getFile());
+
+Configuration conf = new Configuration();
+ParquetReader parquetReader =
+  AvroParquetReader.builder(testPath).withConf(conf).build();
+GenericData.Record firstRecord;
+try {
+  firstRecord = (GenericData.Record) parquetReader.read();
+} catch (Exception x) {
+  x.printStackTrace();

Review Comment:
   Ok, I got rid of the extra catch. I'm not sure what kind of exceptions 
parquetReader.read() can throw, though, so we'll see if we get a compile error 
from not specifying it in the function signature. :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556405#comment-17556405
 ] 

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901733632


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I think the underlying problem is that some versions of ParquetMR produce 
*bad schemas*, so when we try to load those same files, parsing fails, since 
the Parquet schema implicit in the file metadata doesn't match up with the 
stored Avro schema. I'm not sure what to do about bad schemas other than to 
throw them away and try a fallback.





> 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)


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

2022-06-20 Thread GitBox


theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901733632


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I think the underlying problem is that some versions of ParquetMR produce 
*bad schemas*, so when we try to load those same files, parsing fails, since 
the Parquet schema implicit in the file metadata doesn't match up with the 
stored Avro schema. I'm not sure what to do about bad schemas other than to 
throw them away and try a fallback.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2161) Row positions are computed incorrectly when range or offset metadata filter is used

2022-06-20 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17556329#comment-17556329
 ] 

ASF GitHub Bot commented on PARQUET-2161:
-

ala opened a new pull request, #978:
URL: https://github.com/apache/parquet-mr/pull/978

   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [x] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-2161
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [x] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
 - Extends `TestParquetReader` suite. 
   
   ### Commits
   
   - [x] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [x] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   




> Row positions are computed incorrectly when range or offset metadata filter 
> is used
> ---
>
> Key: PARQUET-2161
> URL: https://issues.apache.org/jira/browse/PARQUET-2161
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Ala Luszczak
>Priority: Major
>
> The row indexes introduced in PARQUET-2117 are not computed correctly when
> (1) range or offset metadata filter is applied, and
> (2) the first row group was eliminated by the filter
> For example, if a file has two row groups with 10 rows each, and we attempt 
> to only read the 2nd row group, we are going to produce row indexes 0, 1, 2, 
> ..., 9 instead of expected 10, 11, ..., 19.
> This happens because functions `filterFileMetaDataByStart` (used here: 
> https://github.com/apache/parquet-mr/blob/e06384455567c56d5906fc3a152ab00fd8dfdf33/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1453)
>  and `filterFileMetaDataByMidpoint` (used here: 
> https://github.com/apache/parquet-mr/blob/e06384455567c56d5906fc3a152ab00fd8dfdf33/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1460)
>  modify their input `FileMetaData`. To address the issue we need to 
> `generateRowGroupOffsets` before these filters are applied.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)


[GitHub] [parquet-mr] ala opened a new pull request, #978: PARQUET-2161: Fix row index generation in combination with range filtering

2022-06-20 Thread GitBox


ala opened a new pull request, #978:
URL: https://github.com/apache/parquet-mr/pull/978

   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [x] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-2161
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [x] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
 - Extends `TestParquetReader` suite. 
   
   ### Commits
   
   - [x] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [x] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Created] (PARQUET-2161) Row positions are computed incorrectly when range or offset metadata filter is used

2022-06-20 Thread Ala Luszczak (Jira)
Ala Luszczak created PARQUET-2161:
-

 Summary: Row positions are computed incorrectly when range or 
offset metadata filter is used
 Key: PARQUET-2161
 URL: https://issues.apache.org/jira/browse/PARQUET-2161
 Project: Parquet
  Issue Type: Bug
  Components: parquet-mr
Affects Versions: 1.12.3
Reporter: Ala Luszczak


The row indexes introduced in PARQUET-2117 are not computed correctly when
(1) range or offset metadata filter is applied, and
(2) the first row group was eliminated by the filter

For example, if a file has two row groups with 10 rows each, and we attempt to 
only read the 2nd row group, we are going to produce row indexes 0, 1, 2, ..., 
9 instead of expected 10, 11, ..., 19.

This happens because functions `filterFileMetaDataByStart` (used here: 
https://github.com/apache/parquet-mr/blob/e06384455567c56d5906fc3a152ab00fd8dfdf33/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1453)
 and `filterFileMetaDataByMidpoint` (used here: 
https://github.com/apache/parquet-mr/blob/e06384455567c56d5906fc3a152ab00fd8dfdf33/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1460)
 modify their input `FileMetaData`. To address the issue we need to 
`generateRowGroupOffsets` before these filters are applied.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)