[jira] [Commented] (PARQUET-1978) Provide a tool to show the complete footer

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1978:
-

ggershinsky commented on a change in pull request #867:
URL: https://github.com/apache/parquet-mr/pull/867#discussion_r573915631



##
File path: 
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java
##
@@ -0,0 +1,150 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.parquet.bytes.BytesUtils.readIntLittleEndian;
+import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC;
+import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.format.CliUtils;
+import org.apache.parquet.format.Util;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.SeekableInputStream;
+import org.slf4j.Logger;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.PropertyAccessor;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+
+@Parameters(commandDescription = "Print the Parquet file footer in json 
format")
+public class ShowFooterCommand extends BaseCommand {
+
+  public ShowFooterCommand(Logger console) {
+super(console);
+  }
+
+  @Parameter(description = "", required = true)
+  String target;
+
+  @Parameter(names = { "-r", "--raw" }, description = "Print the raw thrift 
object of the footer")
+  boolean raw = false;
+
+  @Override
+  public int run() throws IOException {
+InputFile inputFile = HadoopInputFile.fromPath(qualifiedPath(target), 
getConf());
+
+String json;
+if (raw) {
+  json = readRawFooter(inputFile);
+} else {
+  json = readFooter(inputFile);
+}
+console.info(json);
+
+return 0;
+  }
+
+  private String readFooter(InputFile inputFile) throws 
JsonProcessingException, IOException {
+String json;
+try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) {
+  ParquetMetadata footer = reader.getFooter();
+  ObjectMapper mapper = createObjectMapper();
+  mapper.setVisibility(PropertyAccessor.ALL, Visibility.NONE);
+  mapper.setVisibility(PropertyAccessor.FIELD, Visibility.ANY);
+  json = 
mapper.writerWithDefaultPrettyPrinter().writeValueAsString(footer);
+}
+return json;
+  }
+
+  private ObjectMapper createObjectMapper() {
+ObjectMapper mapper = new ObjectMapper();
+mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+return mapper;
+  }
+
+  private String readRawFooter(InputFile file) throws IOException {
+long fileLen = file.getLength();
+
+int FOOTER_LENGTH_SIZE = 4;
+if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC 
+ data + footer + footerIndex + MAGIC
+  throw new RuntimeException("Not a Parquet file (length is too low: " + 
fileLen + ")");
+}
+
+try (SeekableInputStream f = file.newStream()) {
+  // Read footer length and magic string - with a single seek
+  byte[] magic = new byte[MAGIC.length];
+  long fileMetadataLengthIndex = fileLen - magic.length - 
FOOTER_LENGTH_SIZE;
+  f.seek(fileMetadataLengthIndex);
+  int fileMetadataLength = readIntLittleEndian(f);
+  f.readFully(magic);
+
+  if (Arrays.equals(EFMAGIC, magic)) {
+throw new RuntimeException("Encrypted Parquet files a

[jira] [Commented] (PARQUET-1979) Optional bloom_filter_offset is filled if no bloom filter is present

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1979:
-

gszadovszky opened a new pull request #869:
URL: https://github.com/apache/parquet-mr/pull/869


   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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


> Optional bloom_filter_offset is filled if no bloom filter is present
> 
>
> Key: PARQUET-1979
> URL: https://issues.apache.org/jira/browse/PARQUET-1979
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Currently the field 
> [bloom_filter_offset|https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L750]
>  is filled (with the value {{0}}) even if no bloom filter is saved for the 
> related column chunk. This is not correct as this field is optional so we 
> should not fill it in this case.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky opened a new pull request #869: PARQUET-1979: bloom_filter_offset is filled if there are no bloom filters

2021-02-10 Thread GitBox


gszadovszky opened a new pull request #869:
URL: https://github.com/apache/parquet-mr/pull/869


   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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




[GitHub] [parquet-mr] ggershinsky commented on a change in pull request #867: PARQUET-1978: Provide a tool to show the complete footer

2021-02-10 Thread GitBox


ggershinsky commented on a change in pull request #867:
URL: https://github.com/apache/parquet-mr/pull/867#discussion_r573915631



##
File path: 
parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowFooterCommand.java
##
@@ -0,0 +1,150 @@
+/*
+ * 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.cli.commands;
+
+import static org.apache.parquet.bytes.BytesUtils.readIntLittleEndian;
+import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC;
+import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.format.CliUtils;
+import org.apache.parquet.format.Util;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.SeekableInputStream;
+import org.slf4j.Logger;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.PropertyAccessor;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+
+@Parameters(commandDescription = "Print the Parquet file footer in json 
format")
+public class ShowFooterCommand extends BaseCommand {
+
+  public ShowFooterCommand(Logger console) {
+super(console);
+  }
+
+  @Parameter(description = "", required = true)
+  String target;
+
+  @Parameter(names = { "-r", "--raw" }, description = "Print the raw thrift 
object of the footer")
+  boolean raw = false;
+
+  @Override
+  public int run() throws IOException {
+InputFile inputFile = HadoopInputFile.fromPath(qualifiedPath(target), 
getConf());
+
+String json;
+if (raw) {
+  json = readRawFooter(inputFile);
+} else {
+  json = readFooter(inputFile);
+}
+console.info(json);
+
+return 0;
+  }
+
+  private String readFooter(InputFile inputFile) throws 
JsonProcessingException, IOException {
+String json;
+try (ParquetFileReader reader = ParquetFileReader.open(inputFile)) {
+  ParquetMetadata footer = reader.getFooter();
+  ObjectMapper mapper = createObjectMapper();
+  mapper.setVisibility(PropertyAccessor.ALL, Visibility.NONE);
+  mapper.setVisibility(PropertyAccessor.FIELD, Visibility.ANY);
+  json = 
mapper.writerWithDefaultPrettyPrinter().writeValueAsString(footer);
+}
+return json;
+  }
+
+  private ObjectMapper createObjectMapper() {
+ObjectMapper mapper = new ObjectMapper();
+mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false);
+mapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+return mapper;
+  }
+
+  private String readRawFooter(InputFile file) throws IOException {
+long fileLen = file.getLength();
+
+int FOOTER_LENGTH_SIZE = 4;
+if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { // MAGIC 
+ data + footer + footerIndex + MAGIC
+  throw new RuntimeException("Not a Parquet file (length is too low: " + 
fileLen + ")");
+}
+
+try (SeekableInputStream f = file.newStream()) {
+  // Read footer length and magic string - with a single seek
+  byte[] magic = new byte[MAGIC.length];
+  long fileMetadataLengthIndex = fileLen - magic.length - 
FOOTER_LENGTH_SIZE;
+  f.seek(fileMetadataLengthIndex);
+  int fileMetadataLength = readIntLittleEndian(f);
+  f.readFully(magic);
+
+  if (Arrays.equals(EFMAGIC, magic)) {
+throw new RuntimeException("Encrypted Parquet files are not 
supported.");

Review comment:
   technically, EFMAGIC is for encrypted files with encrypted footer. 
Encrypted files can also have the regular MAGIC, if created in plaintext footer 
mode.





[jira] [Commented] (PARQUET-1977) Invalid data_page_offset

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1977:
-

gszadovszky opened a new pull request #868:
URL: https://github.com/apache/parquet-mr/pull/868


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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


> Invalid data_page_offset
> 
>
> Key: PARQUET-1977
> URL: https://issues.apache.org/jira/browse/PARQUET-1977
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> The thrift field 
> [{{data_page_offset}}|https://github.com/gszadovszky/parquet-format/blob/master/src/main/thrift/parquet.thrift#L733]
>  is filled with incorrect value. Currently, it always points to the beginning 
> of the column chunk which is not correct according to the spec in case there 
> is a dictionary page. This is not a regression as it was written incorrectly 
> since the beginning of parquet-mr.
> Meanwhile PARQUET-1850 fixed that we never wrote the field 
> [{{dictionary_page_offset}}|https://github.com/gszadovszky/parquet-format/blob/master/src/main/thrift/parquet.thrift#L739].
>  After the fix we correctly write this field if there is a dictionary filter. 
> The problem is we are using the same value to fill both fields. So there are 
> two possibilities:
> * There is no dictionary page in the column chunk so {{data_page_offset}} is 
> filled with the correct value while {{dictionary_page_offset}} is not filled 
> which is still correct. We are good.
> * There is a dictionary page at the beginning of the column chunk so 
> {{data_page_offset}} and {{dictionary_page_offset}} are both contains the 
> same value. This is not only a regression but it causes issues in other 
> implementations (e.g. Impala) where footer validation is more strict than in 
> parquet-mr because {{dictionary_page_offset}} shall be less than 
> {{data_page_offset}} at all time if it is filled.
> So, we need to fill {{data_page_offset}} correctly.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky opened a new pull request #868: PARQUET-1977: Invalid data_page_offset

2021-02-10 Thread GitBox


gszadovszky opened a new pull request #868:
URL: https://github.com/apache/parquet-mr/pull/868


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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




[jira] [Updated] (PARQUET-1979) Optional bloom_filter_offset is filled if no bloom filter is present

2021-02-10 Thread Gabor Szadovszky (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-1979?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Gabor Szadovszky updated PARQUET-1979:
--
Description: Currently the field 
[bloom_filter_offset|https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L750]
 is filled (with the value {{0}}) even if no bloom filter is saved for the 
related column chunk. This is not correct as this field is optional so we 
should not fill it in this case.  (was: Currently the field 
[bloom_filter_offset|https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L750]
 is filled (with the value `0`) even if there is no bloom filter is saved for 
the related column chunk. This is not correct as this field is optional so we 
should not fill it in this case.)

> Optional bloom_filter_offset is filled if no bloom filter is present
> 
>
> Key: PARQUET-1979
> URL: https://issues.apache.org/jira/browse/PARQUET-1979
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Currently the field 
> [bloom_filter_offset|https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L750]
>  is filled (with the value {{0}}) even if no bloom filter is saved for the 
> related column chunk. This is not correct as this field is optional so we 
> should not fill it in this case.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (PARQUET-1979) Optional bloom_filter_offset is filled if no bloom filter is present

2021-02-10 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-1979:
-

 Summary: Optional bloom_filter_offset is filled if no bloom filter 
is present
 Key: PARQUET-1979
 URL: https://issues.apache.org/jira/browse/PARQUET-1979
 Project: Parquet
  Issue Type: Bug
Affects Versions: 1.12.0
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


Currently the field 
[bloom_filter_offset|https://github.com/apache/parquet-format/blob/master/src/main/thrift/parquet.thrift#L750]
 is filled (with the value `0`) even if there is no bloom filter is saved for 
the related column chunk. This is not correct as this field is optional so we 
should not fill it in this case.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1966) Fix build with JDK11 for JDK8

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1966:
-

shangxinli commented on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776722038


   > > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > > "should be within 10% of the expected value (expected = 453745044 actual 
= 505046624)"
   > 
   > Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis.
   > 
   > But this change is not related to the flaky test. It is related to the 
issue caused by the first RC built with jdk11 and failed in a jre8 runtime.
   
   Sounds good



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.

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


> Fix build with JDK11 for JDK8
> -
>
> Key: PARQUET-1966
> URL: https://issues.apache.org/jira/browse/PARQUET-1966
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Blocker
>
> However the target is set to 1.8 it seems to be not enough as of building 
> with JDK11 it fails at runtime with the following exception: 
> {code:java}
> ava.lang.NoSuchMethodError: 
> java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
> at 
> org.apache.parquet.bytes.CapacityByteArrayOutputStream.write(CapacityByteArrayOutputStream.java:197)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeOrAppendBitPackedRun(RunLengthBitPackingHybridEncoder.java:193)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeInt(RunLengthBitPackingHybridEncoder.java:179)
> at 
> org.apache.parquet.column.values.dictionary.DictionaryValuesWriter.getBytes(DictionaryValuesWriter.java:167)
> at 
> org.apache.parquet.column.values.fallback.FallbackValuesWriter.getBytes(FallbackValuesWriter.java:74)
> at 
> org.apache.parquet.column.impl.ColumnWriterV1.writePage(ColumnWriterV1.java:60)
> at 
> org.apache.parquet.column.impl.ColumnWriterBase.writePage(ColumnWriterBase.java:387)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.sizeCheck(ColumnWriteStoreBase.java:235)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.endRecord(ColumnWriteStoreBase.java:222)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreV1.endRecord(ColumnWriteStoreV1.java:29)
> at 
> org.apache.parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endMessage(MessageColumnIO.java:307)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.consumeMessage(ParquetWriteSupport.scala:465)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:148)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:54)
> at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:138)
> {code}
> To reproduce execute the following.
> {code}
> export JAVA_HOME={the path to the JDK11 home}
> mvn clean install -Djvm={the path to the JRE8 java executable}
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] shangxinli commented on pull request #858: PARQUET-1966: Fix build with JDK11 for JDK8

2021-02-10 Thread GitBox


shangxinli commented on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776722038


   > > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > > "should be within 10% of the expected value (expected = 453745044 actual 
= 505046624)"
   > 
   > Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis.
   > 
   > But this change is not related to the flaky test. It is related to the 
issue caused by the first RC built with jdk11 and failed in a jre8 runtime.
   
   Sounds good



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.

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




[jira] [Commented] (PARQUET-1978) Provide a tool to show the complete footer

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1978:
-

gszadovszky opened a new pull request #867:
URL: https://github.com/apache/parquet-mr/pull/867


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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


> Provide a tool to show the complete footer
> --
>
> Key: PARQUET-1978
> URL: https://issues.apache.org/jira/browse/PARQUET-1978
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cli
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> In many cases of debugging it would be nice to have a tool to check the whole 
> footer of a parquet file. It would be also nice to see both the content of 
> the parquet-mr internal footer object structure and the original thrift 
> objects.
>  
> (As of parquet-tools is deprecated we should develop this in parquet-cli.)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky opened a new pull request #867: PARQUET-1978: Provide a tool to show the complete footer

2021-02-10 Thread GitBox


gszadovszky opened a new pull request #867:
URL: https://github.com/apache/parquet-mr/pull/867


   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] 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-XXX
 - 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
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] 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
   
   - [ ] 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.

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




[jira] [Created] (PARQUET-1978) Provide a tool to show the complete footer

2021-02-10 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-1978:
-

 Summary: Provide a tool to show the complete footer
 Key: PARQUET-1978
 URL: https://issues.apache.org/jira/browse/PARQUET-1978
 Project: Parquet
  Issue Type: Improvement
  Components: parquet-cli
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


In many cases of debugging it would be nice to have a tool to check the whole 
footer of a parquet file. It would be also nice to see both the content of the 
parquet-mr internal footer object structure and the original thrift objects.

 

(As of parquet-tools is deprecated we should develop this in parquet-cli.)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1966) Fix build with JDK11 for JDK8

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1966:
-

gszadovszky edited a comment on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776598248


   > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > 
   > "should be within 10% of the expected value (expected = 453745044 actual = 
505046624)"
   
   Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis. 
   
   But this change is not related to the flaky test. It is related to the issue 
caused by the first RC built with jdk11 and failed in a jre8 runtime. 



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.

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


> Fix build with JDK11 for JDK8
> -
>
> Key: PARQUET-1966
> URL: https://issues.apache.org/jira/browse/PARQUET-1966
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Blocker
>
> However the target is set to 1.8 it seems to be not enough as of building 
> with JDK11 it fails at runtime with the following exception: 
> {code:java}
> ava.lang.NoSuchMethodError: 
> java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
> at 
> org.apache.parquet.bytes.CapacityByteArrayOutputStream.write(CapacityByteArrayOutputStream.java:197)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeOrAppendBitPackedRun(RunLengthBitPackingHybridEncoder.java:193)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeInt(RunLengthBitPackingHybridEncoder.java:179)
> at 
> org.apache.parquet.column.values.dictionary.DictionaryValuesWriter.getBytes(DictionaryValuesWriter.java:167)
> at 
> org.apache.parquet.column.values.fallback.FallbackValuesWriter.getBytes(FallbackValuesWriter.java:74)
> at 
> org.apache.parquet.column.impl.ColumnWriterV1.writePage(ColumnWriterV1.java:60)
> at 
> org.apache.parquet.column.impl.ColumnWriterBase.writePage(ColumnWriterBase.java:387)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.sizeCheck(ColumnWriteStoreBase.java:235)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.endRecord(ColumnWriteStoreBase.java:222)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreV1.endRecord(ColumnWriteStoreV1.java:29)
> at 
> org.apache.parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endMessage(MessageColumnIO.java:307)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.consumeMessage(ParquetWriteSupport.scala:465)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:148)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:54)
> at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:138)
> {code}
> To reproduce execute the following.
> {code}
> export JAVA_HOME={the path to the JDK11 home}
> mvn clean install -Djvm={the path to the JRE8 java executable}
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky edited a comment on pull request #858: PARQUET-1966: Fix build with JDK11 for JDK8

2021-02-10 Thread GitBox


gszadovszky edited a comment on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776598248


   > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > 
   > "should be within 10% of the expected value (expected = 453745044 actual = 
505046624)"
   
   Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis. 
   
   But this change is not related to the flaky test. It is related to the issue 
caused by the first RC built with jdk11 and failed in a jre8 runtime. 



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.

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




[jira] [Commented] (PARQUET-1966) Fix build with JDK11 for JDK8

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1966:
-

gszadovszky commented on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776598248


   > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > 
   > "should be within 10% of the expected value (expected = 453745044 actual = 
505046624)"
   
   Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis. 



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.

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


> Fix build with JDK11 for JDK8
> -
>
> Key: PARQUET-1966
> URL: https://issues.apache.org/jira/browse/PARQUET-1966
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Blocker
>
> However the target is set to 1.8 it seems to be not enough as of building 
> with JDK11 it fails at runtime with the following exception: 
> {code:java}
> ava.lang.NoSuchMethodError: 
> java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
> at 
> org.apache.parquet.bytes.CapacityByteArrayOutputStream.write(CapacityByteArrayOutputStream.java:197)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeOrAppendBitPackedRun(RunLengthBitPackingHybridEncoder.java:193)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeInt(RunLengthBitPackingHybridEncoder.java:179)
> at 
> org.apache.parquet.column.values.dictionary.DictionaryValuesWriter.getBytes(DictionaryValuesWriter.java:167)
> at 
> org.apache.parquet.column.values.fallback.FallbackValuesWriter.getBytes(FallbackValuesWriter.java:74)
> at 
> org.apache.parquet.column.impl.ColumnWriterV1.writePage(ColumnWriterV1.java:60)
> at 
> org.apache.parquet.column.impl.ColumnWriterBase.writePage(ColumnWriterBase.java:387)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.sizeCheck(ColumnWriteStoreBase.java:235)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.endRecord(ColumnWriteStoreBase.java:222)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreV1.endRecord(ColumnWriteStoreV1.java:29)
> at 
> org.apache.parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endMessage(MessageColumnIO.java:307)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.consumeMessage(ParquetWriteSupport.scala:465)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:148)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:54)
> at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:138)
> {code}
> To reproduce execute the following.
> {code}
> export JAVA_HOME={the path to the JDK11 home}
> mvn clean install -Djvm={the path to the JRE8 java executable}
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky commented on pull request #858: PARQUET-1966: Fix build with JDK11 for JDK8

2021-02-10 Thread GitBox


gszadovszky commented on pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#issuecomment-776598248


   > For the failed test "testMemoryManagerUpperLimit", I am not sure it is 
caused by this change, or the test itself is unstable. It seems the failure is 
because the pool size is larger than expected.
   > 
   > "should be within 10% of the expected value (expected = 453745044 actual = 
505046624)"
   
   Yes, the test is flaky. Based on git history we've had to increase the 
percentage 3 times already. (I am not sure if this test make sense this way but 
I did not want to drop it for now.) Meanwhile, I have not seen this test fail 
until the transition to the github actions. I guess there are some 
environmental differences between the new actions and Travis. 



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.

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




[jira] [Commented] (PARQUET-1966) Fix build with JDK11 for JDK8

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1966:
-

gszadovszky commented on a change in pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#discussion_r573598674



##
File path: pom.xml
##
@@ -393,6 +393,7 @@
 
 org.apache.maven.plugins
 maven-compiler-plugin
+3.8.1

Review comment:
   I would not update it for now (just before the release). I think, it is 
better to bump such versions for the next release. The bump of the 
`maven-compiler-plugin` was required to support the `release` config.





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.

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


> Fix build with JDK11 for JDK8
> -
>
> Key: PARQUET-1966
> URL: https://issues.apache.org/jira/browse/PARQUET-1966
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Blocker
>
> However the target is set to 1.8 it seems to be not enough as of building 
> with JDK11 it fails at runtime with the following exception: 
> {code:java}
> ava.lang.NoSuchMethodError: 
> java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
> at 
> org.apache.parquet.bytes.CapacityByteArrayOutputStream.write(CapacityByteArrayOutputStream.java:197)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeOrAppendBitPackedRun(RunLengthBitPackingHybridEncoder.java:193)
> at 
> org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder.writeInt(RunLengthBitPackingHybridEncoder.java:179)
> at 
> org.apache.parquet.column.values.dictionary.DictionaryValuesWriter.getBytes(DictionaryValuesWriter.java:167)
> at 
> org.apache.parquet.column.values.fallback.FallbackValuesWriter.getBytes(FallbackValuesWriter.java:74)
> at 
> org.apache.parquet.column.impl.ColumnWriterV1.writePage(ColumnWriterV1.java:60)
> at 
> org.apache.parquet.column.impl.ColumnWriterBase.writePage(ColumnWriterBase.java:387)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.sizeCheck(ColumnWriteStoreBase.java:235)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreBase.endRecord(ColumnWriteStoreBase.java:222)
> at 
> org.apache.parquet.column.impl.ColumnWriteStoreV1.endRecord(ColumnWriteStoreV1.java:29)
> at 
> org.apache.parquet.io.MessageColumnIO$MessageColumnIORecordConsumer.endMessage(MessageColumnIO.java:307)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.consumeMessage(ParquetWriteSupport.scala:465)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:148)
> at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.write(ParquetWriteSupport.scala:54)
> at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:138)
> {code}
> To reproduce execute the following.
> {code}
> export JAVA_HOME={the path to the JDK11 home}
> mvn clean install -Djvm={the path to the JRE8 java executable}
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky commented on a change in pull request #858: PARQUET-1966: Fix build with JDK11 for JDK8

2021-02-10 Thread GitBox


gszadovszky commented on a change in pull request #858:
URL: https://github.com/apache/parquet-mr/pull/858#discussion_r573598674



##
File path: pom.xml
##
@@ -393,6 +393,7 @@
 
 org.apache.maven.plugins
 maven-compiler-plugin
+3.8.1

Review comment:
   I would not update it for now (just before the release). I think, it is 
better to bump such versions for the next release. The bump of the 
`maven-compiler-plugin` was required to support the `release` config.





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.

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




[jira] [Resolved] (PARQUET-1970) Make minor releases source compatible

2021-02-10 Thread Gabor Szadovszky (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-1970?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Gabor Szadovszky resolved PARQUET-1970.
---
Fix Version/s: 1.12.0
   Resolution: Fixed

> Make minor releases source compatible
> -
>
> Key: PARQUET-1970
> URL: https://issues.apache.org/jira/browse/PARQUET-1970
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
> Fix For: 1.12.0
>
>
> Currently the compatibility checker ({{japicmp-maven-plugin}}) is configured 
> to allow source incompatible changes for minor releases. It does not seem to 
> be a great burden not allow such changes.
> Meanwhile a potential [bug|https://github.com/siom79/japicmp/issues/281] is 
> discovered in the tool. Let's wait for this issue to be clarified because if 
> it is not a bug then changes in parquet-mr will be required.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (PARQUET-1970) Make minor releases source compatible

2021-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-1970:
-

gszadovszky merged pull request #861:
URL: https://github.com/apache/parquet-mr/pull/861


   



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.

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


> Make minor releases source compatible
> -
>
> Key: PARQUET-1970
> URL: https://issues.apache.org/jira/browse/PARQUET-1970
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.12.0
>Reporter: Gabor Szadovszky
>Assignee: Gabor Szadovszky
>Priority: Major
>
> Currently the compatibility checker ({{japicmp-maven-plugin}}) is configured 
> to allow source incompatible changes for minor releases. It does not seem to 
> be a great burden not allow such changes.
> Meanwhile a potential [bug|https://github.com/siom79/japicmp/issues/281] is 
> discovered in the tool. Let's wait for this issue to be clarified because if 
> it is not a bug then changes in parquet-mr will be required.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [parquet-mr] gszadovszky merged pull request #861: PARQUET-1970: Make minor releases source compatible

2021-02-10 Thread GitBox


gszadovszky merged pull request #861:
URL: https://github.com/apache/parquet-mr/pull/861


   



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.

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




[jira] [Created] (PARQUET-1977) Invalid data_page_offset

2021-02-10 Thread Gabor Szadovszky (Jira)
Gabor Szadovszky created PARQUET-1977:
-

 Summary: Invalid data_page_offset
 Key: PARQUET-1977
 URL: https://issues.apache.org/jira/browse/PARQUET-1977
 Project: Parquet
  Issue Type: Bug
  Components: parquet-mr
Affects Versions: 1.12.0
Reporter: Gabor Szadovszky
Assignee: Gabor Szadovszky


The thrift field 
[{{data_page_offset}}|https://github.com/gszadovszky/parquet-format/blob/master/src/main/thrift/parquet.thrift#L733]
 is filled with incorrect value. Currently, it always points to the beginning 
of the column chunk which is not correct according to the spec in case there is 
a dictionary page. This is not a regression as it was written incorrectly since 
the beginning of parquet-mr.
Meanwhile PARQUET-1850 fixed that we never wrote the field 
[{{dictionary_page_offset}}|https://github.com/gszadovszky/parquet-format/blob/master/src/main/thrift/parquet.thrift#L739].
 After the fix we correctly write this field if there is a dictionary filter. 
The problem is we are using the same value to fill both fields. So there are 
two possibilities:
* There is no dictionary page in the column chunk so {{data_page_offset}} is 
filled with the correct value while {{dictionary_page_offset}} is not filled 
which is still correct. We are good.
* There is a dictionary page at the beginning of the column chunk so 
{{data_page_offset}} and {{dictionary_page_offset}} are both contains the same 
value. This is not only a regression but it causes issues in other 
implementations (e.g. Impala) where footer validation is more strict than in 
parquet-mr because {{dictionary_page_offset}} shall be less than 
{{data_page_offset}} at all time if it is filled.

So, we need to fill {{data_page_offset}} correctly.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)