[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2075:
-

ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1064348254


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune 

[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter

2023-01-08 Thread GitBox


ggershinsky commented on code in PR #1014:
URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1064348254


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/RewriteOptions.java:
##
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.hadoop.rewrite;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.crypto.FileEncryptionProperties;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.util.List;
+import java.util.Map;
+
+// A set of options to create a ParquetRewriter.
+public class RewriteOptions {
+
+  final Configuration conf;
+  final Path inputFile;
+  final Path outputFile;
+  final List pruneColumns;
+  final CompressionCodecName newCodecName;
+  final Map maskColumns;
+  final List encryptColumns;
+  final FileEncryptionProperties fileEncryptionProperties;
+
+  private RewriteOptions(Configuration conf,
+ Path inputFile,
+ Path outputFile,
+ List pruneColumns,
+ CompressionCodecName newCodecName,
+ Map maskColumns,
+ List encryptColumns,
+ FileEncryptionProperties fileEncryptionProperties) {
+this.conf = conf;
+this.inputFile = inputFile;
+this.outputFile = outputFile;
+this.pruneColumns = pruneColumns;
+this.newCodecName = newCodecName;
+this.maskColumns = maskColumns;
+this.encryptColumns = encryptColumns;
+this.fileEncryptionProperties = fileEncryptionProperties;
+  }
+
+  public Configuration getConf() {
+return conf;
+  }
+
+  public Path getInputFile() {
+return inputFile;
+  }
+
+  public Path getOutputFile() {
+return outputFile;
+  }
+
+  public List getPruneColumns() {
+return pruneColumns;
+  }
+
+  public CompressionCodecName getNewCodecName() {
+return newCodecName;
+  }
+
+  public Map getMaskColumns() {
+return maskColumns;
+  }
+
+  public List getEncryptColumns() {
+return encryptColumns;
+  }
+
+  public FileEncryptionProperties getFileEncryptionProperties() {
+return fileEncryptionProperties;
+  }
+
+  // Builder to create a RewriterOptions.
+  public static class Builder {
+private Configuration conf;
+private Path inputFile;
+private Path outputFile;
+private List pruneColumns;
+private CompressionCodecName newCodecName;
+private Map maskColumns;
+private List encryptColumns;
+private FileEncryptionProperties fileEncryptionProperties;
+
+public Builder(Configuration conf, Path inputFile, Path outputFile) {
+  this.conf = conf;
+  this.inputFile = inputFile;
+  this.outputFile = outputFile;
+}
+
+public Builder prune(List columns) {
+  this.pruneColumns = columns;
+  return this;
+}
+
+public Builder transform(CompressionCodecName newCodecName) {
+  this.newCodecName = newCodecName;
+  return this;
+}
+
+public Builder mask(Map maskColumns) {
+  this.maskColumns = maskColumns;
+  return this;
+}
+
+public Builder encrypt(List encryptColumns) {
+  this.encryptColumns = encryptColumns;
+  return this;
+}
+
+public Builder encryptionProperties(FileEncryptionProperties 
fileEncryptionProperties) {
+  this.fileEncryptionProperties = fileEncryptionProperties;
+  return this;
+}
+
+public RewriteOptions build() {
+  Preconditions.checkArgument(inputFile != null, "Input file is required");
+  Preconditions.checkArgument(outputFile != null, "Output file is 
required");
+
+  if (pruneColumns != null) {
+if (maskColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!maskColumns.containsKey(pruneColumn),
+"Cannot prune and mask same column");
+  }
+}
+
+if (encryptColumns != null) {
+  for (String pruneColumn : pruneColumns) {
+Preconditions.checkArgument(!encryptColumns.contains(pruneColumn),
+

[jira] [Reopened] (PARQUET-1980) Build and test Apache Parquet on ARM64 CPU architecture

2023-01-08 Thread Gabor Szadovszky (Jira)


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

Gabor Szadovszky reopened PARQUET-1980:
---

[~mgrigorov],

PMC just got a note from Apache IT that they are about to "move away from 
Travis at the beginning of 2023". I don't know if Github actions are now 
suitable for ARM64 or there any other solutions for this. If you have time, 
could you please take a look?

> Build and test Apache Parquet on ARM64 CPU architecture
> ---
>
> Key: PARQUET-1980
> URL: https://issues.apache.org/jira/browse/PARQUET-1980
> Project: Parquet
>  Issue Type: Test
>  Components: parquet-format
>Reporter: Martin Tzvetanov Grigorov
>Assignee: Martin Tzvetanov Grigorov
>Priority: Minor
>  Labels: pull-request-available
> Fix For: 1.12.0
>
>
> More and more deployments are being done on ARM64 machines.
> It would be good to make sure Parquet MR project builds fine on it.
> The project moved from TravisCI to GitHub Actions recently (PARQUET-1969) but 
> .travis.yml could be re-intorduced for ARM64 until GitHub Actions provide 
> aarch64 nodes!



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2224) Publish SBOM artifacts

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2224:
-

dongjoon-hyun commented on PR #1017:
URL: https://github.com/apache/parquet-mr/pull/1017#issuecomment-1375203719

   Thank you, @ggershinsky !




> Publish SBOM artifacts
> --
>
> Key: PARQUET-2224
> URL: https://issues.apache.org/jira/browse/PARQUET-2224
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Dongjoon Hyun
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] dongjoon-hyun commented on pull request #1017: PARQUET-2224: Publish SBOM artifacts

2023-01-08 Thread GitBox


dongjoon-hyun commented on PR #1017:
URL: https://github.com/apache/parquet-mr/pull/1017#issuecomment-1375203719

   Thank you, @ggershinsky !


-- 
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-2224) Publish SBOM artifacts

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2224:
-

dongjoon-hyun commented on PR #1017:
URL: https://github.com/apache/parquet-mr/pull/1017#issuecomment-1374985673

   Thank you, @wgtmac and @sunchao .




> Publish SBOM artifacts
> --
>
> Key: PARQUET-2224
> URL: https://issues.apache.org/jira/browse/PARQUET-2224
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Dongjoon Hyun
>Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] dongjoon-hyun commented on pull request #1017: PARQUET-2224: Publish SBOM artifacts

2023-01-08 Thread GitBox


dongjoon-hyun commented on PR #1017:
URL: https://github.com/apache/parquet-mr/pull/1017#issuecomment-1374985673

   Thank you, @wgtmac and @sunchao .


-- 
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-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1374806195

   @gszadovszky @ggershinsky @shangxinli @sunchao Could you please take a look 
when you have time?
   
   cc @emkornfield 




> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] wgtmac commented on pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-08 Thread GitBox


wgtmac commented on PR #1018:
URL: https://github.com/apache/parquet-mr/pull/1018#issuecomment-1374806195

   @gszadovszky @ggershinsky @shangxinli @sunchao Could you please take a look 
when you have time?
   
   cc @emkornfield 


-- 
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] [Assigned] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-08 Thread Gang Wu (Jira)


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

Gang Wu reassigned PARQUET-2219:


Assignee: Gang Wu

> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Assignee: Gang Wu
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data

2023-01-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2219:
-

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

   ### Jira
   
   My PR addresses the 
[PARQUET-2219](https://issues.apache.org/jira/browse/PARQUET/PARQUET-2219).
   
   ### Tests
   
   My PR adds the following unit test to read parquet file with empty row group:
   - 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderEmptyBlock.java
   
   ### Commits
   
   The parquet specs does not forbid empty row group and some implementations 
are able to generate files with empty row group. The commit aims to make 
ParquetFileReader robust by skipping empty row group while reading.
   




> ParquetFileReader throws a runtime exception when a file contains only 
> headers and now row data
> ---
>
> Key: PARQUET-2219
> URL: https://issues.apache.org/jira/browse/PARQUET-2219
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.1
>Reporter: chris stockton
>Priority: Minor
>
> Google BigQuery has an option to export table data to Parquet-formatted 
> files, but some of these files are written with header data only.  When this 
> happens and these files are opened with the ParquetFileReader, an exception 
> is thrown:
> {{RuntimeException("Illegal row group of 0 rows");}}
> It seems like the ParquetFileReader should not throw an exception when it 
> encounters such a file.
> https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L949



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] wgtmac opened a new pull request, #1018: PARQUET-2219: ParquetFileReader skips empty row group

2023-01-08 Thread GitBox


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

   ### Jira
   
   My PR addresses the 
[PARQUET-2219](https://issues.apache.org/jira/browse/PARQUET/PARQUET-2219).
   
   ### Tests
   
   My PR adds the following unit test to read parquet file with empty row group:
   - 
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetReaderEmptyBlock.java
   
   ### Commits
   
   The parquet specs does not forbid empty row group and some implementations 
are able to generate files with empty row group. The commit aims to make 
ParquetFileReader robust by skipping empty row group while reading.
   


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