[jira] [Commented] (PARQUET-2219) ParquetFileReader throws a runtime exception when a file contains only headers and now row data
[ https://issues.apache.org/jira/browse/PARQUET-2219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676962#comment-17676962 ] ASF GitHub Bot commented on PARQUET-2219: - shangxinli merged PR #1018: URL: https://github.com/apache/parquet-mr/pull/1018 > 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-2223) Parquet Data Masking for Column Encryption
[ https://issues.apache.org/jira/browse/PARQUET-2223?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676963#comment-17676963 ] ASF GitHub Bot commented on PARQUET-2223: - shangxinli commented on PR #1016: URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1383006808 @ggershinsky Do you have time to have a look? > Parquet Data Masking for Column Encryption > -- > > Key: PARQUET-2223 > URL: https://issues.apache.org/jira/browse/PARQUET-2223 > Project: Parquet > Issue Type: Task >Reporter: Jiashen Zhang >Priority: Minor > > h1. Background > h2. What is Data Masking? > Data masking is the process of obfuscating sensitive data. Instead of > revealing PII data, masking allows us to return NULLs, hashes or redacted > data in its place. With data masking, users who are in the correct permission > groups can retrieve the original data and users without permissions will > receive masked data. > h2. Why do we need it? > * Fined-Grained Access Control > h2. Why do we want to enhance data masking? > > Users might not have all permissions for all columns, existing code doesn’t > have support for us to skip columns that users don’t have permissions to > access. This enhancement will add this support so that users can decide to > skip some columns to avoid decryption error. > h1. Design Requirements > # Users can skip some columns with a configuration > h1. Proposed solution > Key idea is to modify the request schema by removing skipped columns from the > schema. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] shangxinli commented on pull request #1016: PARQUET-2223: Parquet Data Masking Enhancement for Column Encryption
shangxinli commented on PR #1016: URL: https://github.com/apache/parquet-mr/pull/1016#issuecomment-1383006808 @ggershinsky Do you have time to have a look? -- 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
[GitHub] [parquet-mr] shangxinli merged pull request #1018: PARQUET-2219: ParquetFileReader skips empty row group
shangxinli merged PR #1018: URL: https://github.com/apache/parquet-mr/pull/1018 -- 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-2226) Support merge Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676961#comment-17676961 ] ASF GitHub Bot commented on PARQUET-2226: - shangxinli commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1383006013 @chenjunjiedada Do you still have time to review this change? > Support merge Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] shangxinli commented on pull request #1020: PARQUET-2226 Support merge bloom filters
shangxinli commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1383006013 @chenjunjiedada Do you still have time to review this change? -- 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-2227) Refactor different file rewriters to use single implementation
[ https://issues.apache.org/jira/browse/PARQUET-2227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676927#comment-17676927 ] ASF GitHub Bot commented on PARQUET-2227: - gszadovszky commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382840526 > I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation. I do not have a strong opinion for `'\n'` only that we need a character that probably won't be used by any systems writing parquet files. > Refactor different file rewriters to use single implementation > -- > > Key: PARQUET-2227 > URL: https://issues.apache.org/jira/browse/PARQUET-2227 > Project: Parquet > Issue Type: Sub-task > Components: parquet-mr >Reporter: Gang Wu >Assignee: Gang Wu >Priority: Major > > A new ParquetRewriter is implemented to support all logics in the > ColumnPruner, CompressionConverter, ColumnMasker, and ColumnEncrypter. And > refactor all the old rewriters to use ParquetRewriter under the hood. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2227: Refactor several file rewriters to use a new unified ParquetRewriter implementation
gszadovszky commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382840526 > I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation. I do not have a strong opinion for `'\n'` only that we need a character that probably won't be used by any systems writing parquet files. -- 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-2230) Add a new rewrite command powered by ParquetRewriter
Gang Wu created PARQUET-2230: Summary: Add a new rewrite command powered by ParquetRewriter Key: PARQUET-2230 URL: https://issues.apache.org/jira/browse/PARQUET-2230 Project: Parquet Issue Type: Sub-task Components: parquet-cli Reporter: Gang Wu Assignee: Gang Wu parquet-cli has several commands for rewriting files but missing a consolidated one to provide the full features of ParquetRewriter. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (PARQUET-2229) ParquetRewriter supports masking and encrypting the same column
Gang Wu created PARQUET-2229: Summary: ParquetRewriter supports masking and encrypting the same column Key: PARQUET-2229 URL: https://issues.apache.org/jira/browse/PARQUET-2229 Project: Parquet Issue Type: Sub-task Components: parquet-mr Reporter: Gang Wu Assignee: Gang Wu ParquetRewriter does not yet support masking and encrypting the same column. The scope of this task is to enable it. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (PARQUET-2228) ParquetRewriter supports more than one input file
Gang Wu created PARQUET-2228: Summary: ParquetRewriter supports more than one input file Key: PARQUET-2228 URL: https://issues.apache.org/jira/browse/PARQUET-2228 Project: Parquet Issue Type: Sub-task Components: parquet-mr Reporter: Gang Wu Assignee: Gang Wu ParquetRewriter currently supports only one input file. The scope of this task is to support multiple input files and the rewriter merges them into a single one w/o some rewrite options specified. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (PARQUET-2227) Refactor different file rewriters to use single implementation
Gang Wu created PARQUET-2227: Summary: Refactor different file rewriters to use single implementation Key: PARQUET-2227 URL: https://issues.apache.org/jira/browse/PARQUET-2227 Project: Parquet Issue Type: Sub-task Components: parquet-mr Reporter: Gang Wu Assignee: Gang Wu A new ParquetRewriter is implemented to support all logics in the ColumnPruner, CompressionConverter, ColumnMasker, and ColumnEncrypter. And refactor all the old rewriters to use ParquetRewriter under the hood. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676916#comment-17676916 ] ASF GitHub Bot commented on PARQUET-2075: - wgtmac commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382815489 > > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. > > Perfect! :) > > > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it? > > It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`? I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation. > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter
wgtmac commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382815489 > > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. > > Perfect! :) > > > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it? > > It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`? I am afraid some implementations may drop characters after `'\n'` when displaying the string content. Let me do some investigation. -- 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-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676912#comment-17676912 ] ASF GitHub Bot commented on PARQUET-2075: - gszadovszky commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382754916 > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. Perfect! :) > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it? It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] gszadovszky commented on pull request #1014: PARQUET-2075: Implement unified file rewriter
gszadovszky commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382754916 > * I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. Perfect! :) > * Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it? It sounds good to me. Maybe have the latest one at the beginning and use the separator `'\n'`? -- 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-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676909#comment-17676909 ] ASF GitHub Bot commented on PARQUET-2075: - wgtmac commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382752637 > I think it is a great refactor. Thanks a lot for working on it, @wgtmac! In the other hand I've thought about PARQUET-2075 as a request for a new feature in `parquet-cli` that can be used to convert from one parquet file to another with specific configurations. (Later on we might extend it to allow multiple parquet files to be merged/rewritten to one specified and the tool would decide which level of deserialization/serialization is required.) I am fine with handling it in a separate jira but let's make it clear. Either create another jira for this refactor as a prerequisite of PARQUET-2075 or rephrase PARQUET-2075 and create a new for `parquet-cli`. @shangxinli, what do you think? Thanks for your review @gszadovszky - I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. - Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to it? > Unified Rewriter Tool > --- > > Key: PARQUET-2075 > URL: https://issues.apache.org/jira/browse/PARQUET-2075 > Project: Parquet > Issue Type: New Feature >Reporter: Xinli Shang >Assignee: Gang Wu >Priority: Major > > During the discussion of PARQUET-2071, we came up with the idea of a > universal tool to translate the existing file to a different state while > skipping some level steps like encoding/decoding, to gain speed. For example, > only decompress pages and then compress directly. For PARQUET-2071, we only > decrypt and then encrypt directly. This will be useful for the existing data > to onboard Parquet features like column encryption, zstd etc. > We already have tools like trans-compression, column pruning etc. We will > consolidate all these tools with this universal tool. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] wgtmac commented on pull request #1014: PARQUET-2075: Implement unified file rewriter
wgtmac commented on PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#issuecomment-1382752637 > I think it is a great refactor. Thanks a lot for working on it, @wgtmac! In the other hand I've thought about PARQUET-2075 as a request for a new feature in `parquet-cli` that can be used to convert from one parquet file to another with specific configurations. (Later on we might extend it to allow multiple parquet files to be merged/rewritten to one specified and the tool would decide which level of deserialization/serialization is required.) I am fine with handling it in a separate jira but let's make it clear. Either create another jira for this refactor as a prerequisite of PARQUET-2075 or rephrase PARQUET-2075 and create a new for `parquet-cli`. @shangxinli, what do you think? Thanks for your review @gszadovszky - I'd prefer creating a new JIRA for this refactor to be a prerequisite. Merging multiple files to a single one with customized pruning, encryption, and codec is also in my mind and will be supported later. I will create separate JIRAs as sub-tasks of PARQUET-2075 and work on them progressively. - Putting the original `created_by` into `key_value_metadata` is a good idea. However, it is tricky if a file has been rewritten for several times. What about adding a key named `original_created_by` to `key_value_metadata` and concatenating all old `created_by`s to 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-2226) Support merge Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676907#comment-17676907 ] ASF GitHub Bot commented on PARQUET-2226: - yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267028 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: > IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? Thanks for your review. I had added some check before merging bloomFilter. I see that guava does not seem to have additional utility function.If necessary, I can add an utility function. > Support merge Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on a diff in pull request #1020: PARQUET-2226 Support merge bloom filters
yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267028 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: > IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? Thanks for your review. I had added some check before merging bloomFilter. I see that guava does not seem to have additional utility function.If necessary, I can add an utility function. -- 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-2075) Unified Rewriter Tool
[ https://issues.apache.org/jira/browse/PARQUET-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676905#comment-17676905 ] ASF GitHub Bot commented on PARQUET-2075: - gszadovszky commented on code in PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1070274495 ## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java: ## @@ -0,0 +1,733 @@ +/* + * 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.HadoopReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ColumnReader; +import org.apache.parquet.column.ColumnWriteStore; +import org.apache.parquet.column.ColumnWriter; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.impl.ColumnReadStoreImpl; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnEncryptionSetup; +import org.apache.parquet.crypto.InternalFileEncryptor; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DataPageHeader; +import org.apache.parquet.format.DataPageHeaderV2; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.CodecFactory; +import org.apache.parquet.hadoop.ColumnChunkPageWriteStore; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; +import org.apache.parquet.hadoop.util.HadoopCodecs; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.ParquetEncodingException; +import org.apache.parquet.io.api.Converter; +import org.apache.parquet.io.api.GroupConverter; +import org.apache.parquet.io.api.PrimitiveConverter; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH; +import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH; +import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; +import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE; +import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT; + +public class ParquetRewriter implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class); + private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2; + private final byte[] pageBuffer
[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1014: PARQUET-2075: Implement unified file rewriter
gszadovszky commented on code in PR #1014: URL: https://github.com/apache/parquet-mr/pull/1014#discussion_r1070274495 ## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/rewrite/ParquetRewriter.java: ## @@ -0,0 +1,733 @@ +/* + * 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.HadoopReadOptions; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ColumnReader; +import org.apache.parquet.column.ColumnWriteStore; +import org.apache.parquet.column.ColumnWriter; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.impl.ColumnReadStoreImpl; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.compression.CompressionCodecFactory; +import org.apache.parquet.crypto.AesCipher; +import org.apache.parquet.crypto.InternalColumnEncryptionSetup; +import org.apache.parquet.crypto.InternalFileEncryptor; +import org.apache.parquet.format.BlockCipher; +import org.apache.parquet.format.DataPageHeader; +import org.apache.parquet.format.DataPageHeaderV2; +import org.apache.parquet.format.DictionaryPageHeader; +import org.apache.parquet.format.PageHeader; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.CodecFactory; +import org.apache.parquet.hadoop.ColumnChunkPageWriteStore; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.CompressionConverter.TransParquetFileReader; +import org.apache.parquet.hadoop.util.HadoopCodecs; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.internal.column.columnindex.ColumnIndex; +import org.apache.parquet.internal.column.columnindex.OffsetIndex; +import org.apache.parquet.io.ParquetEncodingException; +import org.apache.parquet.io.api.Converter; +import org.apache.parquet.io.api.GroupConverter; +import org.apache.parquet.io.api.PrimitiveConverter; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.apache.parquet.column.ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH; +import static org.apache.parquet.column.ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH; +import static org.apache.parquet.crypto.ModuleCipherFactory.ModuleType; +import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE; +import static org.apache.parquet.hadoop.ParquetWriter.MAX_PADDING_SIZE_DEFAULT; + +public class ParquetRewriter implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(ParquetRewriter.class); + private final int pageBufferSize = ParquetProperties.DEFAULT_PAGE_SIZE * 2; + private final byte[] pageBuffer = new byte[pageBufferSize]; + private TransParquetFileReader reader; + private ParquetFileWriter writer; + private ParquetMetadata meta; + private MessageType schema; + private String createdBy; + private CompressionCodecName codecName =
[jira] [Commented] (PARQUET-2226) Support merge Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676901#comment-17676901 ] ASF GitHub Bot commented on PARQUET-2226: - gszadovszky commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1382737603 One more thing, @yabola. The compatibility tests fail because you have added a new method to a public interface. Even though this interface is not supposed to be implemented by our clients the current project structure does not allow filtering these classes out from the compat checks. The easiest way would be to add a default implementation to the interface which throws an exception (not implemented). > Support merge Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] gszadovszky commented on pull request #1020: PARQUET-2226 Support merge bloom filters
gszadovszky commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1382737603 One more thing, @yabola. The compatibility tests fail because you have added a new method to a public interface. Even though this interface is not supposed to be implemented by our clients the current project structure does not allow filtering these classes out from the compat checks. The easiest way would be to add a default implementation to the interface which throws an exception (not implemented). -- 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-2226) Support merge Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676900#comment-17676900 ] ASF GitHub Bot commented on PARQUET-2226: - gszadovszky commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1382736990 Thanks, @yabola for working on this and also to @wgtmac for reviewing. I do not have much experience with bloom filters so I will rely on your review. Ping me if you have a +1. @chenjunjiedada, any thoughts? > Support merge Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] gszadovszky commented on pull request #1020: PARQUET-2226 Support merge bloom filters
gszadovszky commented on PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#issuecomment-1382736990 Thanks, @yabola for working on this and also to @wgtmac for reviewing. I do not have much experience with bloom filters so I will rely on your review. Ping me if you have a +1. @chenjunjiedada, any thoughts? -- 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] [Updated] (PARQUET-2226) Support merge Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mars updated PARQUET-2226: -- Summary: Support merge Bloom Filter (was: Support union Bloom Filter) > Support merge Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2226) Support union Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676897#comment-17676897 ] ASF GitHub Bot commented on PARQUET-2226: - yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267028 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: > IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? I had added some check before merging bloomFilter. I see that guava does not seem to have additional utility function.If necessary, I can add an utility function. > Support union Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on a diff in pull request #1020: PARQUET-2226 Support merge bloom filters
yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267080 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java: ## @@ -394,4 +395,21 @@ public long hash(float value) { public long hash(Binary value) { return hashFunction.hashBytes(value.getBytes()); } + + @Override + public void putAll(BloomFilter otherBloomFilter) throws IOException { Review Comment: > Could you add some tests to verify if the merged BF is as expected? Sure, I will add some UT later -- 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-2226) Support union Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676898#comment-17676898 ] ASF GitHub Bot commented on PARQUET-2226: - yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267080 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java: ## @@ -394,4 +395,21 @@ public long hash(float value) { public long hash(Binary value) { return hashFunction.hashBytes(value.getBytes()); } + + @Override + public void putAll(BloomFilter otherBloomFilter) throws IOException { Review Comment: > Could you add some tests to verify if the merged BF is as expected? Sure, I will add some UT later > Support union Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on a diff in pull request #1020: PARQUET-2226 Support merge bloom filters
yabola commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070267028 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: > IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? I had added some check before merging bloomFilter. I see that guava does not seem to have additional utility function.If necessary, I can add an utility function. -- 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-2226) Support union Bloom Filter
[ https://issues.apache.org/jira/browse/PARQUET-2226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17676864#comment-17676864 ] ASF GitHub Bot commented on PARQUET-2226: - wgtmac commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070233210 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java: ## @@ -394,4 +395,21 @@ public long hash(float value) { public long hash(Binary value) { return hashFunction.hashBytes(value.getBytes()); } + + @Override + public void putAll(BloomFilter otherBloomFilter) throws IOException { Review Comment: Could you add some tests to verify if the merged BF is as expected? ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: Rename to `merge`? > Support union Bloom Filter > -- > > Key: PARQUET-2226 > URL: https://issues.apache.org/jira/browse/PARQUET-2226 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > We need to collect Parquet's bloom filter of multiple files, and then > synthesize a more comprehensive bloom filter for common use. > Guava supports similar api operations > https://guava.dev/releases/31.0.1-jre/api/docs/src-html/com/google/common/hash/BloomFilter.html#line.252 -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1020: PARQUET-2226 Support merge bloom filters
wgtmac commented on code in PR #1020: URL: https://github.com/apache/parquet-mr/pull/1020#discussion_r1070233210 ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java: ## @@ -394,4 +395,21 @@ public long hash(float value) { public long hash(Binary value) { return hashFunction.hashBytes(value.getBytes()); } + + @Override + public void putAll(BloomFilter otherBloomFilter) throws IOException { Review Comment: Could you add some tests to verify if the merged BF is as expected? ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: IMHO, user needs to know if two BFs are compatible to merge before calling this function. So an utility function to test compatibility of two BFs is also required. WDYT? ## parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java: ## @@ -176,4 +176,10 @@ public String toString() { * @return compress algorithm that the bloom filter apply */ Compression getCompression(); + + /** + * Combines this Bloom filter with another Bloom filter by performing a bitwise OR of the underlying data + * @param otherBloomFilter The Bloom filter to combine this Bloom filter with. + */ + void putAll(BloomFilter otherBloomFilter) throws IOException; Review Comment: Rename to `merge`? -- 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