[ https://issues.apache.org/jira/browse/PARQUET-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17695546#comment-17695546 ]
ASF GitHub Bot commented on PARQUET-2159: ----------------------------------------- gszadovszky commented on code in PR #1011: URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122744256 ########## parquet-plugins/parquet-encoding-vector/src/test/java/org/apache/parquet/column/values/bitpacking/TestByteBitPacking512VectorLE.java: ########## @@ -0,0 +1,172 @@ +/* + * 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.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertArrayEquals; + +public class TestByteBitPacking512VectorLE { + private static final Logger LOG = LoggerFactory.getLogger(TestByteBitPacking512VectorLE.class); + @Test + public void unpackValuesUsingVector() { + if (ParquetReadRouter.getSupportVectorFromCPUFlags() != VectorSupport.VECTOR_512) { + LOG.info("avx512vbmi and avx512_vbmi2 are not supported, skip this test."); + return; + } Review Comment: This way this is not a quality gate. I think we should not allow bypassing this test here but somehow ensure that the environment is capable of executing it. What do you think? ########## .github/workflows/vector-plugins.yml: ########## @@ -0,0 +1,56 @@ +# 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. + +name: Vector-plugins + +on: [push, pull_request] + +jobs: + build: + + runs-on: ubuntu-latest + strategy: + fail-fast: false + matrix: + java: [ '17' ] + codes: [ 'uncompressed' ] + name: Build Parquet with JDK ${{ matrix.java }} and ${{ matrix.codes }} + + steps: + - uses: actions/checkout@master + - name: Set up JDK ${{ matrix.java }} + uses: actions/setup-java@v1 + with: + java-version: ${{ matrix.java }} + - name: before_install + env: + CI_TARGET_BRANCH: $GITHUB_HEAD_REF + run: | + bash dev/ci-before_install.sh + - name: install + run: | + EXTRA_JAVA_TEST_ARGS=$(mvn help:evaluate -Dexpression=extraJavaTestArgs -q -DforceStdout) + export MAVEN_OPTS="$MAVEN_OPTS $EXTRA_JAVA_TEST_ARGS" + mvn install --batch-mode -Pvector-plugins -DskipTests=true -Dmaven.javadoc.skip=true -Dsource.skip=true -Djava.version=${{ matrix.java }} -pl parquet-encoding,parquet-plugins/parquet-encoding-vector,parquet-plugins/parquet-plugins-benchmarks + - name: verify + env: + TEST_CODECS: ${{ matrix.codes }} + JAVA_VERSION: ${{ matrix.java }} + run: | + EXTRA_JAVA_TEST_ARGS=$(mvn help:evaluate -Dexpression=extraJavaTestArgs -q -DforceStdout) + export MAVEN_OPTS="$MAVEN_OPTS $EXTRA_JAVA_TEST_ARGS" + mvn verify --batch-mode javadoc:javadoc -Pci-test Review Comment: Shouldn't we use `-Pvector-plugins` here as well? Without it we won't actually run the related unit tests only compile the code in the `install` phase. BTW, do we want to execute every tests in this workflow? I've thought the intent is to compile only the necessary modules to execute the tests of `vector-plugins` only. ########## parquet-plugins/parquet-encoding-vector/src/test/java/org/apache/parquet/column/values/bitpacking/TestParquetReadRouter.java: ########## @@ -0,0 +1,61 @@ +/* + * 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.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.junit.Assert.assertArrayEquals; + +public class TestParquetReadRouter { + private static final Logger LOG = LoggerFactory.getLogger(TestParquetReadRouter.class); + + private static final int minBitWidth = 1; + private static final int maxBitWidth = 32; + private static final int outputValues = 1024; + private final int[] output = new int[outputValues]; + private final int[] outputBatch = new int[outputValues]; + private final int[] outputBatchVector = new int[outputValues]; + + @Test + public void testRead() throws IOException { + for (int bitWidth=minBitWidth; bitWidth <= maxBitWidth; bitWidth++) { + byte[] input = new byte[outputValues * bitWidth / 8]; + for (int i = 0; i < input.length; i++) { + input[i] = (byte) i; + } + ByteBufferInputStream inputStream = ByteBufferInputStream.wrap(ByteBuffer.wrap(input)); + + ParquetReadRouter.read(bitWidth, inputStream, 0, output); + ParquetReadRouter.readBatch(bitWidth, inputStream, 0, outputBatch); + assertArrayEquals(output, outputBatch); + if (ParquetReadRouter.getSupportVectorFromCPUFlags() != VectorSupport.VECTOR_512) { + LOG.info("avx512vbmi and avx512_vbmi2 are not supported, skip this test."); + return; + } Review Comment: See my previous comment in the other test class. > 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)