[jira] [Commented] (PARQUET-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17685077#comment-17685077 ] ASF GitHub Bot commented on PARQUET-2159: - jiangjiguang commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1098239530 ## parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java: ## @@ -0,0 +1,133 @@ +/* + * 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.column.values.bitpacking; + +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This is a utils class which is used for big data applications(such as Spark Flink) + * + * - For Intel CPU, Flags avx512vbmi && avx512_vbmi2 can have better performance gains + */ +public class ParquetReadRouter { + private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class); + + private static volatile Boolean vector; + + public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException { +if (supportVector()) { + readBatchVector(bitWidth, in, currentCount, currentBuffer); +} else { + readBatchVector(bitWidth, in, currentCount, currentBuffer); Review Comment: @jatin-bhateja please task the updated code > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
jiangjiguang commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1098239530 ## parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java: ## @@ -0,0 +1,133 @@ +/* + * 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.column.values.bitpacking; + +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This is a utils class which is used for big data applications(such as Spark Flink) + * + * - For Intel CPU, Flags avx512vbmi && avx512_vbmi2 can have better performance gains + */ +public class ParquetReadRouter { + private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class); + + private static volatile Boolean vector; + + public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException { +if (supportVector()) { + readBatchVector(bitWidth, in, currentCount, currentBuffer); +} else { + readBatchVector(bitWidth, in, currentCount, currentBuffer); Review Comment: @jatin-bhateja please task the updated code -- 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-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17685075#comment-17685075 ] ASF GitHub Bot commented on PARQUET-2159: - jiangjiguang commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1098236516 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: @wgtmac @sunchao @dongjoon-hyun Yes, I have integrated it with Apache Spark, and finished TPC-H testing. Because of Parquet ByteBitPacking512VectorLE class only will be loaded when used, we can add configuration such as "spark.sql.parquet.vector.enable", if false, any java vector api will not load, so it is backward compatible with older JDK versions like JDK8. > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
jiangjiguang commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1098236516 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: @wgtmac @sunchao @dongjoon-hyun Yes, I have integrated it with Apache Spark, and finished TPC-H testing. Because of Parquet ByteBitPacking512VectorLE class only will be loaded when used, we can add configuration such as "spark.sql.parquet.vector.enable", if false, any java vector api will not load, so it is backward compatible with older JDK versions like JDK8. -- 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-2237) Improve performance when filters in RowGroupFilter can match exactly
[ https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17685021#comment-17685021 ] ASF GitHub Bot commented on PARQUET-2237: - yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1420123553 @wgtmac @shangxinli I thought of a way to avoid interface modification and distinguish by Boolean objects. Please take a look > Improve performance when filters in RowGroupFilter can match exactly > > > Key: PARQUET-2237 > URL: https://issues.apache.org/jira/browse/PARQUET-2237 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > If we can accurately judge by the minMax status, we don’t need to load the > dictionary from filesystem and compare one by one anymore. > Similarly , Bloomfilter needs to load from filesystem, it may costs time and > memory. If we can exactly determine the existence/nonexistence of the value > from minMax or dictionary filters , then we can avoid using Bloomfilter to > Improve performance. > For example, > # read data greater than {{x1}} in the block, if minMax in status is all > greater than {{{}x1{}}}, then we don't need to read dictionary and compare > one by one. > # If we already have page dictionaries and have compared one by one, we > don't need to read BloomFilter and compare. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly
yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1420123553 @wgtmac @shangxinli I thought of a way to avoid interface modification and distinguish by Boolean objects. Please take 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
[jira] [Commented] (PARQUET-831) Corrupt Parquet Files
[ https://issues.apache.org/jira/browse/PARQUET-831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684988#comment-17684988 ] ASF GitHub Bot commented on PARQUET-831: wgtmac commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1098084801 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: Thanks for the explanation! IMHO, we may not have any plan to make a release based on the legacy version. Please advise if I have missed anything. @shangxinli > Corrupt Parquet Files > - > > Key: PARQUET-831 > URL: https://issues.apache.org/jira/browse/PARQUET-831 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.7.0 > Environment: HDP-2.5.3.0 Spark-2.0.2 >Reporter: Steve Severance >Priority: Major > > I am getting corrupt parquet files as the result of a spark job. The write > job completes with no errors but when I read the data again I get the > following error: > org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in > block -1 in file > hdfs://MYPATH/part-r-4-b5c93a19-2f75-4c04-b798-de9cb463f02f.gz.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:228) > at > org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201) > at > org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:128) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithoutKey$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.NegativeArraySizeException > at > org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:755) > at >
[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1022: PARQUET-831: fix estimate page size check overflow corrupting parquet
wgtmac commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1098084801 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: Thanks for the explanation! IMHO, we may not have any plan to make a release based on the legacy version. Please advise if I have missed anything. @shangxinli -- 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-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684935#comment-17684935 ] ASF GitHub Bot commented on PARQUET-2159: - dongjoon-hyun commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097900504 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: Ya, +1 for @sunchao 's comment. Actually, that's the big hurdle in the Apache Spark codebase. It was tricky even ML-part change. Given that Parquet is in `SQL` part, I'm not sure how much we are able to isolate this, @wgtmac . > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] dongjoon-hyun commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
dongjoon-hyun commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097900504 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: Ya, +1 for @sunchao 's comment. Actually, that's the big hurdle in the Apache Spark codebase. It was tricky even ML-part change. Given that Parquet is in `SQL` part, I'm not sure how much we are able to isolate this, @wgtmac . -- 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] sunchao commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
sunchao commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097875149 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: It should be possible but we need to make sure it is backward compatible since Spark also compiles with older JDK versions like JDK8. You can take a look at https://github.com/apache/spark/pull/30810 which does something similar. -- 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-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684922#comment-17684922 ] ASF GitHub Bot commented on PARQUET-2159: - sunchao commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097875149 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: It should be possible but we need to make sure it is backward compatible since Spark also compiles with older JDK versions like JDK8. You can take a look at https://github.com/apache/spark/pull/30810 which does something similar. > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-831) Corrupt Parquet Files
[ https://issues.apache.org/jira/browse/PARQUET-831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684829#comment-17684829 ] ASF GitHub Bot commented on PARQUET-831: jianchun commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097699497 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: The new test case is not applicable in master branch as later code has significantly refactored. After refactor ColumnWriterV1 is no longer responsible to slice and flush pages itself. Other components will call its "writePage" method instead. (Moreover, the new components slicing pages don't have this floating point div by 0 bug. Refactored code has 0 check.) In 0.10 and prior, ColumnWriterV1 is responsible for slicing pages where this bug resides and this test targets. > Corrupt Parquet Files > - > > Key: PARQUET-831 > URL: https://issues.apache.org/jira/browse/PARQUET-831 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.7.0 > Environment: HDP-2.5.3.0 Spark-2.0.2 >Reporter: Steve Severance >Priority: Major > > I am getting corrupt parquet files as the result of a spark job. The write > job completes with no errors but when I read the data again I get the > following error: > org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in > block -1 in file > hdfs://MYPATH/part-r-4-b5c93a19-2f75-4c04-b798-de9cb463f02f.gz.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:228) > at > org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201) > at > org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:128) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithoutKey$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at >
[GitHub] [parquet-mr] jianchun commented on a diff in pull request #1022: PARQUET-831: fix estimate page size check overflow corrupting parquet
jianchun commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097699497 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: The new test case is not applicable in master branch as later code has significantly refactored. After refactor ColumnWriterV1 is no longer responsible to slice and flush pages itself. Other components will call its "writePage" method instead. (Moreover, the new components slicing pages don't have this floating point div by 0 bug. Refactored code has 0 check.) In 0.10 and prior, ColumnWriterV1 is responsible for slicing pages where this bug resides and this test targets. -- 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-831) Corrupt Parquet Files
[ https://issues.apache.org/jira/browse/PARQUET-831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684828#comment-17684828 ] ASF GitHub Bot commented on PARQUET-831: jianchun commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097699497 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: The new test case is not applicable in master branch as later code has significantly refactored. After refactor ColumnWriterV1 is no longer responsible to slice and flush pages itself. Other components will call its "writePage" method instead. In 0.10 and prior, ColumnWriterV1 is responsible for slicing pages where this bug resides and this test targets. > Corrupt Parquet Files > - > > Key: PARQUET-831 > URL: https://issues.apache.org/jira/browse/PARQUET-831 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.7.0 > Environment: HDP-2.5.3.0 Spark-2.0.2 >Reporter: Steve Severance >Priority: Major > > I am getting corrupt parquet files as the result of a spark job. The write > job completes with no errors but when I read the data again I get the > following error: > org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in > block -1 in file > hdfs://MYPATH/part-r-4-b5c93a19-2f75-4c04-b798-de9cb463f02f.gz.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:228) > at > org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201) > at > org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:128) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithoutKey$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at
[GitHub] [parquet-mr] jianchun commented on a diff in pull request #1022: PARQUET-831: fix estimate page size check overflow corrupting parquet
jianchun commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097699497 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: The new test case is not applicable in master branch as later code has significantly refactored. After refactor ColumnWriterV1 is no longer responsible to slice and flush pages itself. Other components will call its "writePage" method instead. In 0.10 and prior, ColumnWriterV1 is responsible for slicing pages where this bug resides and this test targets. -- 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-2237) Improve performance when filters in RowGroupFilter can match exactly
[ https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684800#comment-17684800 ] ASF GitHub Bot commented on PARQUET-2237: - yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419363481 retest this please > Improve performance when filters in RowGroupFilter can match exactly > > > Key: PARQUET-2237 > URL: https://issues.apache.org/jira/browse/PARQUET-2237 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > If we can accurately judge by the minMax status, we don’t need to load the > dictionary from filesystem and compare one by one anymore. > Similarly , Bloomfilter needs to load from filesystem, it may costs time and > memory. If we can exactly determine the existence/nonexistence of the value > from minMax or dictionary filters , then we can avoid using Bloomfilter to > Improve performance. > For example, > # read data greater than {{x1}} in the block, if minMax in status is all > greater than {{{}x1{}}}, then we don't need to read dictionary and compare > one by one. > # If we already have page dictionaries and have compared one by one, we > don't need to read BloomFilter and compare. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly
yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419363481 retest this please -- 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-2237) Improve performance when filters in RowGroupFilter can match exactly
[ https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684798#comment-17684798 ] ASF GitHub Bot commented on PARQUET-2237: - yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419361716 I thought of a way to use Boolean objects to distinguish the different types, It doesn't need to modify the visitor return type now. But it seems Jenkins machine was broken?... > Improve performance when filters in RowGroupFilter can match exactly > > > Key: PARQUET-2237 > URL: https://issues.apache.org/jira/browse/PARQUET-2237 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > If we can accurately judge by the minMax status, we don’t need to load the > dictionary from filesystem and compare one by one anymore. > Similarly , Bloomfilter needs to load from filesystem, it may costs time and > memory. If we can exactly determine the existence/nonexistence of the value > from minMax or dictionary filters , then we can avoid using Bloomfilter to > Improve performance. > For example, > # read data greater than {{x1}} in the block, if minMax in status is all > greater than {{{}x1{}}}, then we don't need to read dictionary and compare > one by one. > # If we already have page dictionaries and have compared one by one, we > don't need to read BloomFilter and compare. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] yabola commented on pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly
yabola commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419361716 I thought of a way to use Boolean objects to distinguish the different types, It doesn't need to modify the visitor return type now. But it seems Jenkins machine was broken?... -- 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-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684756#comment-17684756 ] ASF GitHub Bot commented on PARQUET-2159: - wgtmac commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097538545 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: @dongjoon-hyun @sunchao Could you please take a look and advise if it is possible to integrate it with Apache Spark? Thanks! > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
wgtmac commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1097538545 ## README.md: ## @@ -83,6 +83,16 @@ Parquet is a very active project, and new features are being added quickly. Here * Column stats * Delta encoding * Index pages +* Java Vector API support + +## Java Vector API support Review Comment: @dongjoon-hyun @sunchao Could you please take a look and advise if it is possible to integrate it with Apache Spark? Thanks! -- 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-2237) Improve performance when filters in RowGroupFilter can match exactly
[ https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684747#comment-17684747 ] ASF GitHub Bot commented on PARQUET-2237: - shangxinli commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419254754 +1, let's not modify the signature. > Improve performance when filters in RowGroupFilter can match exactly > > > Key: PARQUET-2237 > URL: https://issues.apache.org/jira/browse/PARQUET-2237 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > If we can accurately judge by the minMax status, we don’t need to load the > dictionary from filesystem and compare one by one anymore. > Similarly , Bloomfilter needs to load from filesystem, it may costs time and > memory. If we can exactly determine the existence/nonexistence of the value > from minMax or dictionary filters , then we can avoid using Bloomfilter to > Improve performance. > For example, > # read data greater than {{x1}} in the block, if minMax in status is all > greater than {{{}x1{}}}, then we don't need to read dictionary and compare > one by one. > # If we already have page dictionaries and have compared one by one, we > don't need to read BloomFilter and compare. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] shangxinli commented on pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly
shangxinli commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1419254754 +1, let's not modify the signature. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: dev-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
[jira] [Commented] (PARQUET-2239) Replace log4j1 with reload4j
[ https://issues.apache.org/jira/browse/PARQUET-2239?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684607#comment-17684607 ] Steve Loughran commented on PARQUET-2239: - good, but trickier than you think as you have to do lots of excludes of other imports of log4j, slf4j look at HADOOP-18088 as an example of what to do > Replace log4j1 with reload4j > > > Key: PARQUET-2239 > URL: https://issues.apache.org/jira/browse/PARQUET-2239 > Project: Parquet > Issue Type: Improvement >Reporter: Akshat Mathur >Priority: Major > Labels: pick-me-up > > Due to multiple CVE in log4j1, replace log4j dependency with reload4j. > More about reload4j: https://reload4j.qos.ch/ -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Created] (PARQUET-2239) Replace log4j1 with reload4j
Akshat Mathur created PARQUET-2239: -- Summary: Replace log4j1 with reload4j Key: PARQUET-2239 URL: https://issues.apache.org/jira/browse/PARQUET-2239 Project: Parquet Issue Type: Improvement Reporter: Akshat Mathur Due to multiple CVE in log4j1, replace log4j dependency with reload4j. More about reload4j: https://reload4j.qos.ch/ -- This message was sent by Atlassian Jira (v8.20.10#820010)
[jira] [Commented] (PARQUET-2159) Parquet bit-packing de/encode optimization
[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684539#comment-17684539 ] ASF GitHub Bot commented on PARQUET-2159: - jiangjiguang commented on PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1418781509 @wgtmac I added doc about how big data applications use Java Vector API > Parquet bit-packing de/encode optimization > -- > > Key: PARQUET-2159 > URL: https://issues.apache.org/jira/browse/PARQUET-2159 > Project: Parquet > Issue Type: Improvement > Components: parquet-mr >Affects Versions: 1.13.0 >Reporter: Fang-Xie >Assignee: Fang-Xie >Priority: Major > Fix For: 1.13.0 > > Attachments: image-2022-06-15-22-56-08-396.png, > image-2022-06-15-22-57-15-964.png, image-2022-06-15-22-58-01-442.png, > image-2022-06-15-22-58-40-704.png > > > Current Spark use Parquet-mr as parquet reader/writer library, but the > built-in bit-packing en/decode is not efficient enough. > Our optimization for Parquet bit-packing en/decode with jdk.incubator.vector > in Open JDK18 brings prominent performance improvement. > Due to Vector API is added to OpenJDK since 16, So this optimization request > JDK16 or higher. > *Below are our test results* > Functional test is based on open-source parquet-mr Bit-pack decoding > function: *_public final void unpack8Values(final byte[] in, final int inPos, > final int[] out, final int outPos)_* __ > compared with our implementation with vector API *_public final void > unpack8Values_vec(final byte[] in, final int inPos, final int[] out, final > int outPos)_* > We tested 10 pairs (open source parquet bit unpacking vs ours optimized > vectorized SIMD implementation) decode function with bit > width=\{1,2,3,4,5,6,7,8,9,10}, below are test results: > !image-2022-06-15-22-56-08-396.png|width=437,height=223! > We integrated our bit-packing decode implementation into parquet-mr, tested > the parquet batch reader ability from Spark VectorizedParquetRecordReader > which get parquet column data by the batch way. We construct parquet file > with different row count and column count, the column data type is Int32, the > maximum int value is 127 which satisfies bit pack encode with bit width=7, > the count of the row is from 10k to 100 million and the count of the column > is from 1 to 4. > !image-2022-06-15-22-57-15-964.png|width=453,height=229! > !image-2022-06-15-22-58-01-442.png|width=439,height=217! > !image-2022-06-15-22-58-40-704.png|width=415,height=208! -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
jiangjiguang commented on PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1418781509 @wgtmac I added doc about how big data applications use Java Vector API -- 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-831) Corrupt Parquet Files
[ https://issues.apache.org/jira/browse/PARQUET-831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684524#comment-17684524 ] ASF GitHub Bot commented on PARQUET-831: wgtmac commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097088924 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: Does the new test case fail in the master branch without your fix? > Corrupt Parquet Files > - > > Key: PARQUET-831 > URL: https://issues.apache.org/jira/browse/PARQUET-831 > Project: Parquet > Issue Type: Bug > Components: parquet-mr >Affects Versions: 1.7.0 > Environment: HDP-2.5.3.0 Spark-2.0.2 >Reporter: Steve Severance >Priority: Major > > I am getting corrupt parquet files as the result of a spark job. The write > job completes with no errors but when I read the data again I get the > following error: > org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in > block -1 in file > hdfs://MYPATH/part-r-4-b5c93a19-2f75-4c04-b798-de9cb463f02f.gz.parquet > at > org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:228) > at > org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:201) > at > org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:128) > at > org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:91) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.agg_doAggregateWithoutKey$(Unknown > Source) > at > org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown > Source) > at > org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) > at > org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) > at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408) > at > org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.NegativeArraySizeException > at > org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:755) > at > org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:494) > at >
[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1022: PARQUET-831: fix estimate page size check overflow corrupting parquet
wgtmac commented on code in PR #1022: URL: https://github.com/apache/parquet-mr/pull/1022#discussion_r1097088924 ## parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnWriterV1.java: ## @@ -0,0 +1,66 @@ +/* + * 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.column.impl; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.page.PageWriter; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.Test; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TestColumnWriterV1 { + + @Test + public void testEstimateNextSizeCheckOnZeroUsedMem() throws Exception { Review Comment: Does the new test case fail in the master branch without your fix? -- 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-2237) Improve performance when filters in RowGroupFilter can match exactly
[ https://issues.apache.org/jira/browse/PARQUET-2237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17684522#comment-17684522 ] ASF GitHub Bot commented on PARQUET-2237: - wgtmac commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418711576 Unfortunately we cannot modify the signature of any public methods. My suggestion was to make the new enum serves as an internal state of the visitor (and probably use it to terminate evaluation early). Then add a new method to return the final state. Does it work? > Improve performance when filters in RowGroupFilter can match exactly > > > Key: PARQUET-2237 > URL: https://issues.apache.org/jira/browse/PARQUET-2237 > Project: Parquet > Issue Type: Improvement >Reporter: Mars >Priority: Major > > If we can accurately judge by the minMax status, we don’t need to load the > dictionary from filesystem and compare one by one anymore. > Similarly , Bloomfilter needs to load from filesystem, it may costs time and > memory. If we can exactly determine the existence/nonexistence of the value > from minMax or dictionary filters , then we can avoid using Bloomfilter to > Improve performance. > For example, > # read data greater than {{x1}} in the block, if minMax in status is all > greater than {{{}x1{}}}, then we don't need to read dictionary and compare > one by one. > # If we already have page dictionaries and have compared one by one, we > don't need to read BloomFilter and compare. -- This message was sent by Atlassian Jira (v8.20.10#820010)
[GitHub] [parquet-mr] wgtmac commented on pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly
wgtmac commented on PR #1023: URL: https://github.com/apache/parquet-mr/pull/1023#issuecomment-1418711576 Unfortunately we cannot modify the signature of any public methods. My suggestion was to make the new enum serves as an internal state of the visitor (and probably use it to terminate evaluation early). Then add a new method to return the final state. Does it work? -- 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