You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/11/15 07:32:17 UTC

[GitHub] [parquet-mr] jiangjiguang opened a new pull request, #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

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

   The PR includes 3 aspects:
   1. Use java17 vector api to decode bit-packing ,  the performance gain is 4x ~ 8x according to the microbenchmark 
   2. Upgrade the project to java17 to support java vector api
   3. Add ParquetReadRouter to compatible with different platform when computing engines(such as spark) read parquet.
   
   ### Jira
   - https://issues.apache.org/jira/projects/PARQUET/issues/PARQUET-2159?filter=allopenissues
   
   ### Tests
   
   - Add unit tests org.apache.parquet.column.values.bitpacking.TestByteBitPackingVectorLE
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Documentation
   - The PR adds maven profile vector to enable generate java17 vector bit-packing decode opt. code, and run junit tests: mvn clean install -P vector
   - The PR needs Intel Ice Lake CPU to run junit tests
   - The contributors are jiangjiguang  jiyu1021 guangzegu  Fang-Xie  , and co-contributor is  jatin-bhateja  
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122860932


##########
.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:
   fix 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122538089


##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   @wgtmac I updated the vector-plugins workflow, it only specifies modules needed to execute



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1119607355


##########
README.md:
##########
@@ -83,6 +83,20 @@ 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
+Parquet-MR has supported Java Vector API to speed up reading, to enable this feature:

Review Comment:
   It might worth mentioning that it is an experimental feature/ongoing development. Also that it is currently not part of the parquet distribution (release).



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] sunchao commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1099630871


##########
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:
   Ah thanks! this looks promising and looking forward to the Spark PR!



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089859464


##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.*;

Review Comment:
   Please fix the import.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_512VECTOR = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int VALUES_PER_PACKED = 8;

Review Comment:
   ```suggestion
     private static final int NUM_VALUES_TO_PACK = 8;
   ```



##########
parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestParquetReadRouter.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestParquetReadRouter {
+
+  /**
+   * The range of bitWidth is 1 ~ 32, change it directly if test other bitWidth.
+   */
+  private static final int bitWidth = 7;

Review Comment:
   It would be good to add cases of different bit widths.



##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   Should we remove this generated file from git? I assume it will be automatically created at maven build time, like other generators do.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_512VECTOR = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int VALUES_PER_PACKED = 8;
+
+  private static volatile Boolean vector;

Review Comment:
   Does it make sense to use an enum instead of boolean to represent longest available vector support? Something like this:
   ```
   enum VectorSupport {
     NONE,
     VECTOR_128,
     VECTOR_256,
     VECTOR_512
   }
   ```
   This makes it more extensible.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_512VECTOR = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int VALUES_PER_PACKED = 8;
+
+  private static volatile Boolean vector;
+
+  // The is just a logical method, most call readBatchUsing512Vector or readBatch directly given that you should know your computer system

Review Comment:
   ```suggestion
     // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
   ```



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,26 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+
+  }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {
+

Review Comment:
   I mean why is it an empty function? Shouldn't it throw an unimplemented exception?



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_512VECTOR = 64;

Review Comment:
   ```suggestion
     private static final int BYTES_PER_VECTOR_512 = 64;
   ```



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1116420531


##########
pom.xml:
##########
@@ -151,6 +151,9 @@
     <module>parquet-scala</module>
     <module>parquet-thrift</module>
     <module>parquet-hadoop-bundle</module>
+    <!--
+    <module>plugins/parquet-encoding-vector</module>

Review Comment:
   This means user has to remove the comment to enable it which seems not very friendly. We can use `<profile>` to enable module optionally.



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/Packer.java:
##########
@@ -86,6 +105,7 @@ private static Object getStaticField(String className, String fieldName) {
   static IntPackerFactory leIntPackerFactory = getIntPackerFactory("LemireBitPackingLE");
   static BytePackerFactory beBytePackerFactory = getBytePackerFactory("ByteBitPackingBE");
   static BytePackerFactory leBytePackerFactory = getBytePackerFactory("ByteBitPackingLE");
+  static BytePackerFactory leBytePacker512VectorFactory = null;

Review Comment:
   Please add a comment to explain leBytePacker512VectorFactory will be initialized lazily.



##########
plugins/parquet-encoding-vector/pom.xml:
##########
@@ -0,0 +1,131 @@
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>org.apache.parquet</groupId>
+    <artifactId>parquet</artifactId>
+    <version>1.13.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-encoding-vector</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Apache Parquet Encodings Vector</name>
+  <url>https://parquet.apache.org</url>
+
+  <properties>
+    <maven.compiler.source>17</maven.compiler.source>

Review Comment:
   If we add a new profile in the root pom file, then we can set compiler version there and avoid hardcode here.



##########
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
+Parquet-MR has supported Java Vector API to speed up reading, to enable the function:
+* Java 17+, 64-bit
+* For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains(ICE Lake or newer processor).
+* mvn clean package -P java17-target -P vector

Review Comment:
   Please update the document to reflect the latest change.



##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class uses the java17 vector API, add VM options --add-modules=jdk.incubator.vector
+ */
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@Warmup(iterations = 1, batchSize = 100000)
+@Measurement(iterations = 1, batchSize = 100000)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ByteBitPackingVectorBenchmarks {

Review Comment:
   If the parquet-encoding-vector does not build, what will happen to this file? Does it fail?



##########
plugins/parquet-encoding-vector/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPacking512VectorLE.java:
##########
@@ -0,0 +1,3010 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ *

Review Comment:
   Remove it or add something?



##########
plugins/parquet-encoding-vector/src/main/java/org/apache/parquet/column/values/bitpacking/VectorSupport.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+/**
+ * Vector bits which has be supported

Review Comment:
   ```suggestion
    * Supported bit widths to use 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1116427502


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   +1 for the option 2. But why not enable it by a new profile?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1114179342


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   I would suggest to have a separate source directory for java17 that gets "activated" only in case of the related profile is activated. This generator solution is misleading to me.
   
   To keep this part of code clean I would also suggest to include java17 and the related class compile/unit test execution in the github actions so they would be executed on the PRs. (The only difference remains is we won't ship these in our releases.)
   
   What do you think, @wgtmac, @jiangjiguang?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115471862


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @gszadovszky Even though the java vector API can check the CPU Flags(low level check), but if users use java vector api on computer which dose not contain avx512vbmi && avx512_vbmi2, the java vector API how to do ?  throw exception?  fall back ?  
   
   Besides,  the CPU Flags are becoming more and more,  and  Java vector api is also increasing. maybe Java vector api dose not know which CPU Flags  the users need. 
   I think  users should have already known their computer(including CPU Flags). They can decide whether to  enable parquet java17 vector optimization. We only need show what CPU Flags needed in README.md
   
   For example, velox(https://github.com/facebookincubator/velox) shows only what CPU Flags needed.
   ![image](https://user-images.githubusercontent.com/12368495/220878861-a760cea7-1c67-46a3-8700-1e4ce9ee8224.png)
   
    



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1327495379

   @jiangjiguang, 
   
   What do you mean by this change is compatible with java8? The vectorized encoding/decoding requires java17 runtime, right? If yes, we will need to have a way release artifacts for java17 and keep java8 compatibility with others. We need to have an agreement (at community level) how we want to achieve this.
   
   By running through the change related to the compile it seems that if you have a jdk17 in the compile environment it will be compiled for a java17 runtime automatically. Since we do not have a specific environment for making a release (shame on us) it highly depends on the environment of the dev how creates the release. So, we shall either add a `release` configuration for `8` to the `jdk7` profile (if it makes sense) or do not have the `jdk17` profile activated automatically. Otherwise we might release artifacts that are not compatible < java17.
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1092845969


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   Do you have any script to generate the code here? If true, it would be great to commit it as well.



##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.*;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class uses the java17 vector API, add VM options --add-modules=jdk.incubator.vector
+ */
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@Warmup(iterations = 1, batchSize = 100000)
+@Measurement(iterations = 1, batchSize = 100000)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ByteBitPackingVectorBenchmarks {
+
+  /**
+   * The range of bitWidth is 1 ~ 32, change it directly if test other bitWidth.
+   */
+  private static final int bitWidth = 7;
+  private static final int outputValues = 1024;
+  private final byte[] input = new byte[outputValues * bitWidth / 8];
+  private final int[] output = new int[outputValues];
+  private final int[] outputVector = new int[outputValues];
+
+  @Setup(Level.Trial)
+  public void getInputBytes() {
+    for (int i = 0; i < input.length; i++) {
+      input[i] = (byte) i;
+    }
+  }
+
+  @Benchmark
+  public void testUnpack() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    for (int i = 0, j = 0; i < input.length; i += bitWidth, j += 8) {
+      bytePacker.unpack8Values(input, i, output, j);
+    }
+  }
+
+  @Benchmark
+  public void testUnpackVector() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker bytePackerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);

Review Comment:
   Could you elaborate more? @jatin-bhateja 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089852900


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGeneratorVector.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.*;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGeneratorVector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPackingVector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPackingVectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian

Review Comment:
   I have created 3 sub-tasks(including Int for Big Endian、Long for Big Endian、Long for Little Endian) with https://issues.apache.org/jira/browse/PARQUET-2159



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1449362200

   > > 
   > 
   > @gszadovszky @wgtmac This feature need avx512vbmi and avx512_vbmi2 instruction set, so it needs github action runners with intel ice lake. I do not know how to select runners with Intel Ice Lake ? So I have submitted the help ([actions/runner#2467](https://github.com/actions/runner/issues/2467)).
   
   @gszadovszky  @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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1449363247

   @gszadovszky @wgtmac  I have added a new workflow named Vector-plugins, can you run 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1449379971

   > > @gszadovszky @wgtmac I have added a new workflow named Vector-plugins, can you run it ?
   > 
   > It seems that this PR is closed. Could you please reopen it and see if it can run automatically?
   
   @wgtmac sorry, may be my wrong click, I have reopened 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1113019945


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */

Review Comment:
   But if this code is written by hand (which is completely find) why don't we check it in instead of having the generator thing?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1114063818


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   > So by generating this class simply copies the file form `resources` to a source dir under the name of `.java`. Why do we need code generation then?
   
   I guess the reason is that it does not compile without java17.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115471862


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @gszadovszky Even though the java vector API can check the CPU Flags(low level check), but if users use java vector api on computer which dose not contain avx512vbmi && avx512_vbmi2, the java vector API how to do ?  throw exception?  fall back ?  
   
   Besides,  the CPU Flags are becoming more and more,  and  Java vector api is also increasing. maybe Java vector api dose not know which CPU Flags  the users need. 
   I think  users should have already known their computer(including CPU Flags). They can decide whether to  enable parquet java17 vector optimization. 
   
   For example, velox(https://github.com/facebookincubator/velox) shows only what CPU Flags needed.
   <img width="1085" alt="image" src="https://user-images.githubusercontent.com/12368495/220878686-53b6eec5-89e9-4958-8802-ff5fd97076bd.png">
    



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118070050


##########
plugins/parquet-encoding-vector/src/main/java/org/apache/parquet/column/values/bitpacking/ByteBitPacking512VectorLE.java:
##########
@@ -0,0 +1,3010 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ *

Review Comment:
   I have added comments



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118070147


##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/Packer.java:
##########
@@ -86,6 +105,7 @@ private static Object getStaticField(String className, String fieldName) {
   static IntPackerFactory leIntPackerFactory = getIntPackerFactory("LemireBitPackingLE");
   static BytePackerFactory beBytePackerFactory = getBytePackerFactory("ByteBitPackingBE");
   static BytePackerFactory leBytePackerFactory = getBytePackerFactory("ByteBitPackingLE");
+  static BytePackerFactory leBytePacker512VectorFactory = null;

Review Comment:
   I have added some comments



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] CascadingCloud commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "CascadingCloud (via GitHub)" <gi...@apache.org>.
CascadingCloud commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1646860071

   > @jiangjiguang Sorry to bother you, I am not sure if the use of the AVX 512 will still make other CPU cores to downshift frequency. If so, is there a way to manually turn off this feature now? Otherwise, the new version may be difficult to promote and use in the enterprise.
   
   spark.sql.parquet.vector512.read.enabled 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1109362203


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */

Review Comment:
   @gszadovszky  I have changed the comments.
   Due to use Java Vector API, it needs rearrange/shuffle/lanewise operations, so it is very difficult to create automatically ByteBitPacking512VectorLE like other generators do. On the contrary,  It is relatively simple to code directly instead of generating.
   I think more important is the finished code than how that code was generated. 
   Besides, the class is a plugin based on java17 to make it compatible with java8
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1109345316


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @gszadovszky avx512vbmi and avx512_vbmi2 are CPU Flags which contains some instruction set. The code of this PR  has used the instruction set. Without them, we cannot get benefit from 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121226362


##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   because they have been run in the Test workflow. I think vector-plugins should run only the module associated with 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122821855


##########
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:
   @gszadovszky I agree with you. 
   1、I have verified with `lscpu` command that there are not avx512vbmi and avx512_vbmi2 instruction set on actions runner.
   2、I have checked the docs(https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners), github actions do not support to select runner with specific instruction set.
   3、I have resubmitted the help on how to select specific runner(https://github.com/orgs/community/discussions/48955)
   * I think there are two ways to fix it:
     * a. skip vector related tests until github actions support to select specific runner.
     * b. I will add a **self-hosted runner** for parquet-mr repo with cloud instance  to run vector related tests. But the runner is not long time since I have to pay money for it.
   
   What do you think ?  @gszadovszky 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1451385421

   I'd request sign off from @gszadovszky @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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1407602919

   @wgtmac PTAK again


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1407344417

   > Sorry for the delay. I have left some comments and the implementation is overall looking good. Thanks @jiangjiguang for your effort!
   > 
   > My main concern is the extensibility to support other instruction sets. In addition, it seems to me that the java vector api is still incubating. As I am not a java expert, do we have the risk of unstable API?
   
   @wgtmac Jatin is a java expert, @jatin-bhateja Can you help give an answer? 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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118072844


##########
plugins/parquet-encoding-vector/pom.xml:
##########
@@ -0,0 +1,131 @@
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <groupId>org.apache.parquet</groupId>
+    <artifactId>parquet</artifactId>
+    <version>1.13.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>parquet-encoding-vector</artifactId>
+  <packaging>jar</packaging>
+
+  <name>Apache Parquet Encodings Vector</name>
+  <url>https://parquet.apache.org</url>
+
+  <properties>
+    <maven.compiler.source>17</maven.compiler.source>

Review Comment:
   I have removed the property  and use <release>17</release> like parent pom.xml
   plugins/parquet-encoding-vector is independent and it needs some configurations(such as extraJavaVectorArgs), but other modules will not need, so I think it is best placed in its own module



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1116689274


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   I agree with @wgtmac. Option 2 with maven profile support seems the best way to go.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115229390


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   > @jiangjiguang, I understand how it works. What I would have expected from the Vector API is to implement some help to check on the CPU flags.
   
   Hi @gszadovszky , vector APIs are just in time compiled into efficient instruction sequence, this instruction sequence can vary based on CPU features set of the target platform,  by introducing additional feature checks for vectorized unpacking we are trying to prevent any perf degradations over non-X86 platforms, over the time these features checks may be relaxed once we quantify the performance improvements on other targets.
   
   One can use JVM option -XX:+PrintIntrinsics to determine if  VectorAPIs used in the source is intrinsified for a particular target. 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118075418


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @gszadovszky I got it. As far as I know, there is no such library available for java. I think we can try to achieve implement a library like that with JNI in the future. But there is no time line. If I have finished it, I will update here.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1114220877


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   +1 for @gszadovszky 
   
   The core idea is to isolate the new feature as an individual unit and minimize the burden of future maintenance.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1094047290


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   In this case, the script is not necessary. Manual bit-unpacking code is error-prone, we really rely on the quality and coverage of test cases.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] dongjoon-hyun commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1350805103

   @wgtmac  I have resubmitted the PR, can you review it again ?


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1048036480


##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,16 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   Please add appropriate comments over newly added definitions, like the ones over top of scalar routines.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115408615


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   >  I was wondering, though if there is an existing 3rd party implementation to check things like that
   
   Hi @gszadovszky , Any implementation using VectorAPI will need some fine tuning since algorithm may not be performant if any API does not get intrinsified for a specified target.
   
   >  burden of maintaining this code.
   
   I am not aware of any 3rd party java based library which vectorizes parquet-mr bit packing.  Intel has also contributed a VectorAPI base version of [PForR Java library](https://github.com/lemire/JavaFastPFOR/pull/51) which does integral bit packing which is other encoding scheme being used in parquet-mr but its a different format.
   
   ![PFOR_VS_PARQUET_FORMAT](https://user-images.githubusercontent.com/59989778/220858640-26c07f03-9415-4152-834e-1bc90f02c554.png)
   
    



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118070969


##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class uses the java17 vector API, add VM options --add-modules=jdk.incubator.vector
+ */
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@Warmup(iterations = 1, batchSize = 100000)
+@Measurement(iterations = 1, batchSize = 100000)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ByteBitPackingVectorBenchmarks {

Review Comment:
   Yes, it will fail, so I new a new module plugins/parquet-plugins-benchmarks  and put the class into the module.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118074325


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */

Review Comment:
   I have checked it in instead of having the generator thing



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122863674


##########
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:
   @gszadovszky thanks, I have used assume from junit and added vector-plugins in vector-plugins.yml.
   
   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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1109367368


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @gszadovszky I have added the maven profile named vector, when execute mvn clean package -P vector, the class will be run and reads ByteBitPacking512VectorLE file to generate ByteBitPacking512VectorLE.java



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089666125


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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).

Review Comment:
   ```suggestion
    * Utility class for big data applications (such as Apache Spark and Apache Flink).
   ```



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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:
   ```suggestion
         readBatch(bitWidth, in, currentCount, currentBuffer);
   ```



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes
+    int totalByteCountVector = totalByteCount - 64;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= 64, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      // values are bit packed 8 at a time, so reading bitWidth will always work
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += 8;
+    }
+  }
+
+  public static Boolean supportVector() {
+    if (vector != null) {
+      return vector;
+    }
+    synchronized (ParquetReadRouter.class) {
+      if (vector == null) {
+        synchronized (ParquetReadRouter.class) {
+          vector = avx512Flag();

Review Comment:
   I assume the java vector api is agnostic to specific instruction set. So this function name is a little bit misleading to me since it binds the vector to AVX-512. Better to organize the code to be more extensible for other instruction sets.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes
+    int totalByteCountVector = totalByteCount - 64;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= 64, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      // values are bit packed 8 at a time, so reading bitWidth will always work
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += 8;
+    }
+  }
+
+  public static Boolean supportVector() {
+    if (vector != null) {
+      return vector;
+    }
+    synchronized (ParquetReadRouter.class) {
+      if (vector == null) {
+        synchronized (ParquetReadRouter.class) {
+          vector = avx512Flag();
+        }
+      }
+    }
+    return vector;
+  }
+
+  private static boolean avx512Flag() {

Review Comment:
   Looks like these routines will be executed every time once `supportVector()` is called.



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,26 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+
+  }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {
+

Review Comment:
   It is better to throw here and above?



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,26 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   Do we have a standard norm to define a function when it leverages the Vector API? Does it sound good to rename it to `unpackValuesUsingVector` or `vectorUnpackValues`? The current name is misleading.



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGeneratorVector.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.*;

Review Comment:
   Please do not use `*` in the import.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes

Review Comment:
   Can we make 64 bytes to be an input parameter? My intention is not to fix it so we can support other instruction sets as well. In my experience, sometimes AVX2 outperforms AVX512 in certain workloads, especially for bitpacking.



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes
+    int totalByteCountVector = totalByteCount - 64;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= 64, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      // values are bit packed 8 at a time, so reading bitWidth will always work
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += 8;
+    }
+  }
+
+  public static Boolean supportVector() {
+    if (vector != null) {
+      return vector;
+    }
+    synchronized (ParquetReadRouter.class) {
+      if (vector == null) {
+        synchronized (ParquetReadRouter.class) {
+          vector = avx512Flag();
+        }
+      }
+    }
+    return vector;
+  }
+
+  private static boolean avx512Flag() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return false;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return true;
+          }
+        }
+      }
+    } catch (Exception ex) {
+      LOG.warn("Not getting CPU info error");

Review Comment:
   ```suggestion
         LOG.warn("Failed to gett CPU info");
   ```



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;

Review Comment:
   Avoid magic numbers like 8 and 64 and declare them as constants.



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -31,6 +31,13 @@ public abstract class BytePacker {
 
   private final int bitWidth;
 
+  /**
+   * unpack the output int values at a time.

Review Comment:
   ```suggestion
      * Number of integer values to be unpacked at a time.
   ```



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/Packer.java:
##########
@@ -101,6 +121,10 @@ private static Object getStaticField(String className, String fieldName) {
    */
   public abstract BytePacker newBytePacker(int width);
 
+  public BytePacker newBytePackerVector(int width) {
+    throw new RuntimeException("This function must be implemented by subclasses!");

Review Comment:
   ```suggestion
       throw new RuntimeException("newBytePackerVector must be implemented by subclasses!");
   ```



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGeneratorVector.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.*;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGeneratorVector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPackingVector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPackingVectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian

Review Comment:
   Would you mind opening the JIRAs to break down the work for better tracking?



##########
pom.xml:
##########
@@ -659,5 +666,74 @@
         </plugins>
       </build>
     </profile>
+
+    <profile>
+      <id>java17-target</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <release>17</release>
+              <compilerArgs combine.children="append">
+                <compilerArg>${extraJavaVectorTestArgs}</compilerArg>
+              </compilerArgs>
+            </configuration>
+          </plugin>
+
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <argLine>${surefire.argLine} ${extraJavaTestArgs} ${extraJavaVectorTestArgs}</argLine>
+              <systemPropertyVariables>

Review Comment:
   Why are these needed?



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1044195380


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class ParquetReadRouter {

Review Comment:
   Yes, this class will be used for big data applications (spark flink ...) as they use now,  I will add some unit test.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] sunchao commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1404125417

   Sorry for the bad experience, @jiangjiguang. Unfortunately, I don't have too much time for the parquet community.
   @wgtmac, @jatin-bhateja, since you were the ones who actively reviewed this PR I would like to ask for your approvals. I won't have time to properly review this but I am happy to push it if you agree.


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121226362


##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   because these modules have been execute in the Test workflow. I think vector-plugins should execute only the modules associated with vector. vector-plugins should not execute repeated part with Test workflow.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1454747482

   I have merged it. Thanks @jiangjiguang @jatin-bhateja for the contribution and @gszadovszky for the review!


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1454364112

   Hi @gszadovszky , @wgtmac Thanks for your reviews and guidance during this long review process. May we request to you please merge this PR by adding following people as  co-authors ( jiangjiguang jiyu1021 guangzegu Fang-Xie jatin-bhateja) alternatively make @jiangjiguang a committer so that he can merge on our behalf.


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1114064922


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   +1 for the request for official support to reduce maintenance burden.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115435366


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @jiangjiguang , can you please respond to @gszadovszky  query.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118319141


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   Not sure whether this answer works in this case: https://stackoverflow.com/a/13383092



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1448194127

   > 
   @gszadovszky @wgtmac  This feature need avx512vbmi and avx512_vbmi2 instruction set, so it needs github action runners with intel ice lake. I do not know how to select runners with Intel Ice Lake ? So I have submitted the help (https://github.com/actions/runner/issues/2467).
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1449381064

   > @wgtmac sorry, may be my wrong click, I have reopened it
   
   NP. Seems it is running. https://github.com/apache/parquet-mr/actions/runs/4300429406/jobs/7496638049


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121218282


##########
.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,brotli', 'gzip,snappy' ]

Review Comment:
   To reduce some resource consumption, `uncompressed` seems enough.



##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   Why is `-pl -parquet-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift` required here?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121226677


##########
.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,brotli', 'gzip,snappy' ]

Review Comment:
   it makes sense



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1328723973

   @jiangjiguang,
   
   With you current implementation if parquet-mr is compiled with jdk17 than it will not be compatible with java8 while any jdk under 17 will generate a code compatible with java8 since `release` is set to 8. My problem is this exact behaviour. I would suggest keep generating java8 compatible artifacts with using any versions of jdks (including 17) by default and let it compile for java17 (and the vectorized stuff) when specific profile is activated manually.
   
   About starting the discussion about java17 compatibility. I am currently not really active in the community and not even following the dev list. Maybe, @shangxinli or @ggershinsky would like to drive this topic?


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089903619


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
       This is a good question! 
       Why I put the generated file in the resources directory? Due to use Java Vector, it needs rearrange/shuffle/lanewise operations, so it is difficult to create automatically ByteBitPacking512VectorLE like other generators do. On the contrary,  It is relatively simple to code directly instead of generating.
       I think more important is the finished code than how that code was generated. 
       In fact, I've done part of the work to generate ByteBitPacking512VectorLE like other generators do. But is the generating valuable if it increases the workload ?
   @wgtmac Do you think it is necessary to generate ByteBitPacking512VectorLE like other generators do ?  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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089903619


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
       This is a good question! 
       Why I put the generated file in the resources directory? Due to use Java Vector, it needs rearrange/shuffle/lanewise operations, so it is difficult to create automatically ByteBitPacking512VectorLE like other generators do. On the contrary,  It is relatively simple to code directly instead of generating.
       I think more important is the finished code than how that code was generated. 
       In fact, I've done part of the work to generate ByteBitPacking512VectorLE like other generators do. But is the generating valuable if it if it increases the workload ?
   @wgtmac Do you think it is necessary to generate ByteBitPacking512VectorLE like other generators do ?  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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089861017


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes

Review Comment:
   @wgtmac It is very difficult to make 64 bytes to be an input parameter.  The PR using only AVX512 Vector to optimize parquet decode. As you said, sometimes AVX2 outperforms better, but we should know the specific workloads for AVX2 so that we can make a choice to use AVX2 or AVX512. 



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089861113


##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,26 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+
+  }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {
+

Review Comment:
   @wgtmac I don't understand it, not throw exception here. Can you you describe in detail ? 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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] sunchao commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1099446478


##########
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:
   @jiangjiguang sounds good, could you share the TPC-H benchmark results too?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] WangYuxing0924 commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "WangYuxing0924 (via GitHub)" <gi...@apache.org>.
WangYuxing0924 commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1099601047


##########
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:
   @sunchao , [here](https://issues.apache.org/jira/projects/PARQUET/issues/PARQUET-2159) shows the micro-benchmark of bitpack function and test report from Spark vectorizedparquetRecordReader (scan operatories). Most tpch queries are join-related operatories so the hotspot lies in the join/shuffle stage. bitpack optimization would be beneficial for SQL filter query



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1119492456


##########
pom.xml:
##########
@@ -659,5 +662,13 @@
         </plugins>
       </build>
     </profile>
+
+    <profile>
+      <id>plugins</id>

Review Comment:
   @wgtmac  @gszadovszky  vector-plugins +1, and it can show the feature of this PR. 
    I have renamed plugins to parquet-plugins.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122821855


##########
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:
   @gszadovszky I agree with you. 
   1、I have verified with `lscpu` command that there are not avx512vbmi and avx512_vbmi2 instruction set on actions runner.
   2、I have checked the docs(https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners), github actions do not support to select runner with specific instruction set.
   3、I have resubmitted the help on how to select specific runner(https://github.com/orgs/community/discussions/48955)
   * I think there are two ways to fix it:
     * a. skip vector related tests until github actions support to select specific runner.
     * b. I will add a **self-hosted runner** for parquet-mr repo with cloud instance  to run vector related tests. But the runner will be not running for a long time since I have to pay money for it.
   
   What do you think ?  @gszadovszky 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122821855


##########
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:
   @gszadovszky I agree with you. 
   1、I have verified with `lscpu` command that there are not avx512vbmi and avx512_vbmi2 instruction set on actions runner.
   2、I have checked the docs(https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners), github actions do not support to select runner with specific instruction set.
   3、I have resubmitted the help on how to select specific runner(https://github.com/orgs/community/discussions/48955)
   * I think there are two ways to fix it:
     * a. skip vector related tests until github actions support to select specific runner.
     * b. I will add a self-hosted runner for parquet-mr repo with cloud instance  to run vector related tests. But the runner is not long time since I have to pay money for it.
   
   What do you think ?  @gszadovszky 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122821855


##########
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:
   @gszadovszky I agree with you. 
   1、I have verified with `lscpu command` that there are not avx512vbmi and avx512_vbmi2 instruction set on actions runner.
   2、I have checked the docs(https://docs.github.com/en/actions/using-github-hosted-runners/about-github-hosted-runners), github actions do not support to select runner with specific instruction set.
   3、I have resubmitted the help on how to select specific runner(https://github.com/orgs/community/discussions/48955)
   I think there are two ways to fix it:
   a. skip vector related tests until github actions support to select specific runner.
   b. I will add a self-hosted runner for parquet-mr repo with cloud instance  to run vector related tests. But the runner is not long time since I have to pay money for it.
   
   What do you think ?  @gszadovszky 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121226362


##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   because these module have been run in the Test workflow. I think vector-plugins should run only the modules associated with vector



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1413064662

   > 
   
   @wgtmac I know your concern: 
    1. I will keep the content of the PR updated if needed when java changed.
    2. I have coded a test to verify generated code, org.apache.parquet.column.values.bitpacking.TestByteBitPacking512VectorLE
    3. I have finished the TPC-H integrated Testing with spark, maybe I can write a document to give best practice to test them


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1327398085

   @gszadovszky I agree on it. As far as I know some products(such as hadoop presto flink ) are working towards upgrading to java17.  Spark already supports java17, trino requires a minimum java version is java17.0.3,  So I think parquet  should support java17 as soon as possible and be compatible with java8, because of java17 vector can bring 4x ~ 8x performance gain for parquet encode/decode. The PR uses "maven profile -P vector"  and  "code gen" to support java17 and is compatible with java8 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1328405659

   > @gszadovszky That what I said supports java17 and is compatible with java8 depends on compile environment. Can you start a discussion on how to support java17 ?
   
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1321101130

   @wangyum @gszadovszky This PR always fails to build, I do not know why.   Is the reason of failure "1 workflow awaiting approval" ? please help me, this is my first PR to parquet-mr community, 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1113017123


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   So by generating this class simply copies the file form `resources` to a source dir under the name of `.java`. Why do we need code generation then?



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115317241


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @gszadovszky @wgtmac  I add a new module named parquet-encoding-vector on directory plugins, the new module implements all code about Vector. I think it can keep code clean about java17 and  related class compile/unit test execution
   
   Let name "the new module" option2, and  "the code I realized before" option1:
   option2's features are:
   1、the vector code is isolate, an individual unit and minimize the burden of future maintenance.
   2、users have to add manully <module>plugins/parquet-encoding-vector</module> in parent pom.xml to enable java17 vector optimization
   
   option1's features are:
   1、the vector code is not isolate
   2、users have to maven parameters -P java17-target -P vector on command Line to enable java17 vector optimization
    
   I think option2 is better 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089700107


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGeneratorVector.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.*;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGeneratorVector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPackingVector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPackingVectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian

Review Comment:
   OK, I will create JIRAs to  break down the 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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089700482


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGeneratorVector.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.*;

Review Comment:
   OK, I will remote it next update



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089903619


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
       This is a good question! 
       Why I put the generated file in the resources directory? Due to use Java Vector, it needs rearrange/shuffle/lanewise operations, so it is difficult to create automatically ByteBitPacking512VectorLE like other generators do. On the contrary,  It is relatively simple to code directly instead of generating.
       I think more important is the finished code than how that code was generated. 
       In fact, I've done part of the work to generate ByteBitPacking512VectorLE like other generators do. But is the generating valuable it if it increases the workload ?
   @wgtmac Do you think it is necessary to generate ByteBitPacking512VectorLE like other generators do ?  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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1407605213

   > @wgtmac PTAK again
   
   Generally this patch looks good to me now. Thanks @jiangjiguang for working on it!
   
   Could you approve the workflow and take another pass? @gszadovszky @shangxinli @ggershinsky 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089859296


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes
+    int totalByteCountVector = totalByteCount - 64;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= 64, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      // values are bit packed 8 at a time, so reading bitWidth will always work
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += 8;
+    }
+  }
+
+  public static Boolean supportVector() {
+    if (vector != null) {
+      return vector;
+    }
+    synchronized (ParquetReadRouter.class) {
+      if (vector == null) {
+        synchronized (ParquetReadRouter.class) {
+          vector = avx512Flag();
+        }
+      }
+    }
+    return vector;
+  }
+
+  private static boolean avx512Flag() {

Review Comment:
   @wgtmac yes, I have added some comments to guide users how to use this class.



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089893184


##########
parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestParquetReadRouter.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertArrayEquals;
+
+public class TestParquetReadRouter {
+
+  /**
+   * The range of bitWidth is 1 ~ 32, change it directly if test other bitWidth.
+   */
+  private static final int bitWidth = 7;

Review Comment:
   Good idea!



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1326087771

   @jiangjiguang, Apache introduced a process to require approvals for test executions to keep resources under control. I've approved it so the tests are running. Not sure though when I will have time to properly review 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1326291164

   @gszadovszky I resubmitted the code, can you approve the workflow again ? 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1120068917


##########
README.md:
##########
@@ -83,6 +83,20 @@ 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
+Parquet-MR has supported Java Vector API to speed up reading, to enable this feature:

Review Comment:
   I have added the experimental flag



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118074070


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @gszadovszky @wgtmac I agree with you, but I can't solve how to exclude a module(parquet-encoding-vector) in the profile by default.
   In fact, The current solution is also simply and user only go to the plugins/parquet-encoding-vector and build with maven



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1427240074

   > > @wgtmac PTAK again
   > 
   > Generally this patch looks good to me now. Thanks @jiangjiguang for working on it!
   > 
   > Could you approve the workflow and take another pass? @gszadovszky @shangxinli @ggershinsky
   
   @gszadovszky @shangxinli @ggershinsky Can you take a look the pr ? 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1112946589


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @jiangjiguang, I understand how it works. What I would have expected from the Vector API is to implement some help to check on the CPU flags.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1122834543


##########
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:
   @jiangjiguang, I wouldn't expect you to spend money on it. I accept there is no proper solution for now. Since this is more for the developers who work on this part and it does not part of the official parquet-mr distribution I am fine with not enforcing these tests to be executed.
   I would suggest to use `Assume` from `junit`. This way the tests would be marked as _skipped_ instead of having them _passed_.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
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.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1454733852

   > Hi @gszadovszky , @wgtmac Thanks for your reviews and guidance during this long review process. May we request to you please merge this PR by adding following people as co-authors ( @jiangjiguang @jiyu1021 @guangzegu @Fang-Xie @jatin-bhateja) alternatively make @jiangjiguang a committer so that he can merge on our behalf.
   
   @jatin-bhateja I have added them as co-authors on the first commit.
   ![image](https://user-images.githubusercontent.com/12368495/222903521-648745d9-e389-48d0-b223-9abfb308d125.png)
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1326934392

   @gszadovszky Could you review the PR ? 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089699240


##########
pom.xml:
##########
@@ -659,5 +666,74 @@
         </plugins>
       </build>
     </profile>
+
+    <profile>
+      <id>java17-target</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+              <release>17</release>
+              <compilerArgs combine.children="append">
+                <compilerArg>${extraJavaVectorTestArgs}</compilerArg>
+              </compilerArgs>
+            </configuration>
+          </plugin>
+
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-surefire-plugin</artifactId>
+            <configuration>
+              <argLine>${surefire.argLine} ${extraJavaTestArgs} ${extraJavaVectorTestArgs}</argLine>
+              <systemPropertyVariables>

Review Comment:
   extraJavaVectorTestArgs is only for Java17 Vector module, I have rename it to extraJavaVectorArgs, and move it to profile vector



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089905196


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   OK, I got your point now. 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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1408198456

   > > Sorry for the delay. I have left some comments and the implementation is overall looking good. Thanks @jiangjiguang for your effort!
   > > My main concern is the extensibility to support other instruction sets. In addition, it seems to me that the java vector api is still incubating. As I am not a java expert, do we have the risk of unstable API?
   > 
   > @wgtmac Jatin is a java expert, @jatin-bhateja Can you help give an answer? thanks.
   
   Hi @wgtmac , our patch vectorizes unpacking algorithm for various decode bit sizes, entire new functionality is exposed through a plugin interface **ParquetReadRouter**, in order to prevent any performance regressions over other targets we have enabled the new functionality only for X86 targets with valid features, this limitation can be removed over time.
   
   VectorAPI made its appearance in JDK16 and has been maturing since then with each successive release. I do not have a firm timeline for you at this point on its incubation exit and being exposed as a preview feature.  Intent here is to enable parquet-mr community developers to make use of the plugin in parquet reader and provide us with early feedback, we are also in process of vectorizing packer algorithm.
   
   Being a large project we plan to do this incrementally, we seek your guidance here in pushing this patch through. 


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1407314603

   > @jiangjiguang, please check test failures.
   
   @gszadovszky 
   
   > @jiangjiguang, please check test failures.
   
   @gszadovszky Thank you for your attention about the PR, I have fixed it and the test successes.


-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1093985661


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   @wgtmac I have the script, but it generate only the code partly. 
   It needs hard work and lots of time to complete the script(I don't think it is necessary). 
   In fact, the code is completed mostly by manually.
   Should I commit the script which is partly completed ? 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] shangxinli commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1336210632

   Thank @gszadovszky a lot for helping with this PR! 
   
   +1 for what @gszadovszky said. The mainstream runtime JDK is still 1.8. Parquet is one of the underlying building blocks for many big data applications.  The bare minimum, for now, is to keep java8 compatible. Otherwise forcing applications to upgrade to jdk17 because of Parquet is disruptive and impacts adoptions. 
   
   @jiangjiguang, I am very happy to see you have this PR to help the Parquet community.  Would you mind starting an email discussion to dev@parquet.apache.org for this topic? 


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1040642048


##########
parquet-generator/src/main/resources/ByteBitPackingVectorLE:
##########
@@ -0,0 +1,3215 @@
+/*
+ * 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;

Review Comment:
   Should we add a comment here to let reader know this is an auto-generated source file and should not edit it directly? Just in case.



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingVectorGenerator.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.bitpacking;
+
+import java.io.*;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.

Review Comment:
   Why is this intentional?



##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class ParquetReadRouter {

Review Comment:
   It seems that this class is not used anywhere.



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/GeneratorVector.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.encoding;
+
+import org.apache.parquet.encoding.bitpacking.ByteBasedBitPackingVectorGenerator;
+
+/**
+ * main class for code generation hook in build for encodings vector generation
+ */
+public class GeneratorVector {

Review Comment:
   Should we add a separate package like `vectorbitpacking` and rename this class to `Generator` so that it is consistent with other Generators in the parquet-generator?



##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +114,15 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+
+  }

Review Comment:
   Please add an empty line between method bodies.



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/GeneratorVector.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.encoding;
+
+import org.apache.parquet.encoding.bitpacking.ByteBasedBitPackingVectorGenerator;
+
+/**
+ * main class for code generation hook in build for encodings vector generation
+ */
+public class GeneratorVector {
+  public static void main(String[] args) throws Exception {
+    Generator.main(args);

Review Comment:
   Why mixing these two generators together? IMHO, making generators to do their jobs separately is less error-prone to users.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1350287549

   > This work looks promising! It would be great if you can add some micro-benchmark to parquet-benchmarks.
   
   @wgtmac I have add the micro-benchmark to parquet-benchmarks, this is the result:
   
   ![image](https://user-images.githubusercontent.com/12368495/207491959-a1a22134-98fd-45f6-aa08-1934584e0fbb.png)
   


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1400441832

   > Thank @gszadovszky a lot for helping with this PR!
   > 
   > +1 for what @gszadovszky said. The mainstream runtime JDK is still 1.8. Parquet is one of the underlying building blocks for many big data applications. The bare minimum, for now, is to keep java8 compatible. Otherwise forcing applications to upgrade to jdk17 because of Parquet is disruptive and impacts adoptions.
   > 
   > @jiangjiguang, I am very happy to see you have this PR to help the Parquet community. Would you mind starting an email discussion to [dev@parquet.apache.org](mailto:dev@parquet.apache.org) for this topic?
   > 
   > cc @ggershinsky @wgtmac
   
   @shangxinli 
   
   > 
   
   @gszadovszky @shangxinli @wgtmac I have started the discussion about how to upgrade java17 over a month, but nobody involved!  So I have updated the PR, it does not  involve how to upgrade java17.
   The default compilation is java8
   Just add maven build parameters  -P java17-target -P vector and get the expected jars  when people want to use java17 vector to speed up parquet decode


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1048037258


##########
parquet-generator/src/main/resources/ByteBitPackingVectorLE:
##########
@@ -0,0 +1,3218 @@
+/*
+ * 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 jdk.incubator.vector.*;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPackingVectorLE {
+  private static final BytePacker[] packers = new BytePacker[33];
+
+  static {
+    packers[0] = new Packer0();
+    packers[1] = new Packer1();
+    packers[2] = new Packer2();
+    packers[3] = new Packer3();
+    packers[4] = new Packer4();
+    packers[5] = new Packer5();
+    packers[6] = new Packer6();
+    packers[7] = new Packer7();
+    packers[8] = new Packer8();
+    packers[9] = new Packer9();
+    packers[10] = new Packer10();
+    packers[11] = new Packer11();
+    packers[12] = new Packer12();
+    packers[13] = new Packer13();
+    packers[14] = new Packer14();
+    packers[15] = new Packer15();
+    packers[16] = new Packer16();
+    packers[17] = new Packer17();
+    packers[18] = new Packer18();
+    packers[19] = new Packer19();
+    packers[20] = new Packer20();
+    packers[21] = new Packer21();
+    packers[22] = new Packer22();
+    packers[23] = new Packer23();
+    packers[24] = new Packer24();
+    packers[25] = new Packer25();
+    packers[26] = new Packer26();
+    packers[27] = new Packer27();
+    packers[28] = new Packer28();
+    packers[29] = new Packer29();
+    packers[30] = new Packer30();
+    packers[31] = new Packer31();
+    packers[32] = new Packer32();
+  }
+
+  public static final BytePackerFactory factory = new BytePackerFactory() {
+    public BytePacker newBytePacker(int bitWidth) {
+      return packers[bitWidth];
+    }
+  };
+
+  private static final class Packer0 extends BytePacker {
+    private int unpackCount = 0;
+
+    private Packer0() {
+      super(0);
+    }
+
+    public int getUnpackCount() {
+      return unpackCount;
+    }
+
+    public final void pack8Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void pack32Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   All these empty definitions can be removed if we introduce a new class ByteVectorPacker which inherit from existing BytePacker.
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1411584288

   > > > Sorry for the delay. I have left some comments and the implementation is overall looking good. Thanks @jiangjiguang for your effort!
   > > > My main concern is the extensibility to support other instruction sets. In addition, it seems to me that the java vector api is still incubating. As I am not a java expert, do we have the risk of unstable API?
   > > 
   > > 
   > > @wgtmac Jatin is a java expert, @jatin-bhateja Can you help give an answer? thanks.
   > 
   > Hi @wgtmac , our patch vectorizes unpacking algorithm for various decode bit sizes, entire new functionality is exposed through a plugin interface **ParquetReadRouter**, in order to prevent any performance regressions over other targets we have enabled the new functionality only for X86 targets with valid features, this limitation can be removed over time.
   > 
   > VectorAPI made its appearance in JDK16 and has been maturing since then with each successive release. I do not have a firm timeline for you at this point on its incubation exit and being exposed as a preview feature. Intent here is to enable parquet-mr community developers to make use of the plugin in parquet reader and provide us with early feedback, we are also in process of vectorizing packer algorithm.
   > 
   > Being a large project we plan to do this incrementally, we seek your guidance in pushing this patch through either on master or a separate development branch.
   
   Thanks for your explanation @jatin-bhateja! 
   
   So when vector API is finalized in the future java release, we may need to change the VM options to enable it accordingly.
   
   BTW, I may not be able to verify the generated code line by line. Please advice the best practice to test them according to your experience. Thanks @jatin-bhateja  


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1094064182


##########
parquet-generator/src/main/resources/ByteBitPacking512VectorLE:
##########
@@ -0,0 +1,3095 @@
+/*
+ * 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 jdk.incubator.vector.ByteVector;
+import jdk.incubator.vector.IntVector;
+import jdk.incubator.vector.LongVector;
+import jdk.incubator.vector.ShortVector;
+import jdk.incubator.vector.Vector;
+import jdk.incubator.vector.VectorMask;
+import jdk.incubator.vector.VectorOperators;
+import jdk.incubator.vector.VectorShuffle;
+import jdk.incubator.vector.VectorSpecies;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPacking512VectorLE {

Review Comment:
   > In this case, the script is not necessary. Manual bit-unpacking code is error-prone, we really rely on the quality and coverage of test cases.
   
   @wgtmac  I strongly agree with you, so I try my best to cover all aspects(different aspects from bitWidth 1 to 32) in class TestByteBitPacking512VectorLE. Besides, I have done the TPC-H testing, and compared the query result with before optimization. In short, I have done more work to ensure the code quality.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1044192235


##########
parquet-generator/src/main/resources/ByteBitPackingVectorLE:
##########
@@ -0,0 +1,3215 @@
+/*
+ * 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;

Review Comment:
   Good idea!



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang closed pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang closed pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization
URL: https://github.com/apache/parquet-mr/pull/1011


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089859139


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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 containing avx512vbmi and 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);
+    }
+  }
+
+  public static void readBatchVector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / 8 * bitWidth;
+    int totalByteCount = currentCount * bitWidth / 8;
+
+    // register of avx512 are 512 bits, and can load up to 64 bytes
+    int totalByteCountVector = totalByteCount - 64;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= 64, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += 8) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      // values are bit packed 8 at a time, so reading bitWidth will always work
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += 8;
+    }
+  }
+
+  public static Boolean supportVector() {
+    if (vector != null) {
+      return vector;
+    }
+    synchronized (ParquetReadRouter.class) {
+      if (vector == null) {
+        synchronized (ParquetReadRouter.class) {
+          vector = avx512Flag();

Review Comment:
   About "the java vector api is agnostic to specific instruction set." @jatin-bhateja Can you help with details ? thanks.
   @wgtmac The PR using only avx512 vector to optimize parquet decode. so I have renamed related classes and methods to to be more extensible for other instruction sets(such as AVX2)



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089859452


##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,26 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  /**
+   * unpack bitWidth bytes from input at inPos into {unpackCount} values in output at outPos using Java Vector API.
+   * @param input the input bytes
+   * @param inPos where to read from in input
+   * @param output the output values
+   * @param outPos where to write to in output
+   */
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   @wgtmac I have reviewed the Java Vector Module name norm, and renamed related classed and methods, unpackValuesUsingVector looks good, so I used 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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1089885328


##########
parquet-column/src/main/java/org/apache/parquet/column/values/bitpacking/ParquetReadRouter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_512VECTOR = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int VALUES_PER_PACKED = 8;
+
+  private static volatile Boolean vector;

Review Comment:
   Good idea!



-- 
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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1107005376


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);

Review Comment:
   When do we want to implement these TODOs? If it is not for an already upcoming change to address, I would suggest having more detailed descriptions here. Why do we need these? In what circumstances would it couse issues if we do not implement them?



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */

Review Comment:
   Is this comment still stands? I cannot find the generated file checked in. BTW, I don't think it is a good practice to check in generated files. It is better to automatically generate them in `generate-sources` phase and then compile them with the other source files.



##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   It looks like quite hacky. Doesn't the vector API support this somehow?



##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   I do not really get how this generation works. Some description would be great.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac merged pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac merged PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] LuciferYang commented on pull request #1011: PARQUET-2159: Vectorized BytePacker decoder using Java VectorAPI

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1495563620

   @jiangjiguang Sorry to bother you, I am not sure if the use of the AVX 512 will still make other CPU cores to downshift frequency. If so, is there a way to manually turn off this feature now? Otherwise, the new version will be difficult to promote and use in the enterprise.


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115317241


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @gszadovszky @wgtmac  I add a new module named parquet-encoding-vector on directory plugins, the new module implements all code about Vector. I think it can keep code clean about java17 and  related class compile/unit test execution
   
   Let name currently code "the new module" option2, and  "the code I realized before" option1:
   option2's features are:
   1、the vector code is isolate, an individual unit and minimize the burden of future maintenance.
   2、users have to add manully <module>plugins/parquet-encoding-vector</module> in parent pom.xml to enable java17 vector optimization
   
   option1's features are:
   1、the vector code is not isolate
   2、users have to maven parameters -P java17-target -P vector on command Line to enable java17 vector optimization
    
   I think option2 is better 
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115429762


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   Sorry, @jatin-bhateja, I was misleading. I was talking about this very code about checking the CPU flags not the whole vectorization. I accept we need to check the CPU flags. I only think that the way we are doing is quite low level. It might be implemented already by a 3rd party that might support other OS etc.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115144554


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @jatin-bhateja Can you help give an answer ? 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115229390


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   > @jiangjiguang, I understand how it works. What I would have expected from the Vector API is to implement some help to check on the CPU flags.
   
   Hi @gszadovszky , vector APIs are just in time compiled into efficient instruction sequence, this instruction sequence can vary based on CPU features set of the target platform,  by introducing additional feature checks for vectorized unpacking we are trying to prevent any perf degradations over non-X86 platforms, over the time these features checks may be relaxed once we quantify the performance improvements on other targets.
   
   One can use JVM option -XX:+PrintIntrinsics to determine if  VectorAPI used in the source is intrinsified for a particular target. 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115317241


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @gszadovszky @wgtmac  I add a new module named parquet-encoding-vector on directory plugins, the new module implements all code about Vector. I think it can keep code clean about java17 and  related class compile/unit test execution.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115408615


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   >  I was wondering, though if there is an existing 3rd party implementation to check things like that
   
   Hi @gszadovszky , Any implementation using VectorAPI will need some fine tuning since algorithm may not be performant if any API does not get intrinsified for a specified target.
   
   >  burden of maintaining this code.
   
   I am not aware of any 3rd party java based library which vectorizes parquet-mr bit packing.  Intel has also contributed a VectorAPI based version of [PForR Java library](https://github.com/lemire/JavaFastPFOR/pull/51) which does integral bit packing which is other encoding scheme being used in parquet-mr but its a different format.
   
   ![PFOR_VS_PARQUET_FORMAT](https://user-images.githubusercontent.com/59989778/220858640-26c07f03-9415-4152-834e-1bc90f02c554.png)
   
    



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118073030


##########
pom.xml:
##########
@@ -151,6 +151,9 @@
     <module>parquet-scala</module>
     <module>parquet-thrift</module>
     <module>parquet-hadoop-bundle</module>
+    <!--
+    <module>plugins/parquet-encoding-vector</module>

Review Comment:
   I have removed it, user need go in plugins/parquet-encoding-vector and build with maven



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1327156483

   @jiangjiguang, however I agree on upgrading to newer java versions (and to have performance benefits like this one) it is not always an easy thing to do. For example our main historical user Hadoop is still on java 1.8 which means they would not be able to upgrade parquet after releasing this one. I think it worth a discussion on the dev list and maybe even a formal vote on 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1404160615

   @jiangjiguang, please check test failures.


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1407554736

   @wgtmac PTAK again  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: devnull-unsubscribe@infra.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1109339286


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);

Review Comment:
   @gszadovszky As far as I know, most JVM depends on Little Endian, I haven't see that JVM depends on Big Endian, So I only finished the code of Little Endian. "Long for Little Endian" affects on parquet page v2 version, I have finished the code, I will submit it to parquet-mr community after this PR. 
   Indeed, it has no side effects(can't benefit from Java Vector API) if we do not implement them, 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1109339286


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);

Review Comment:
   @gszadovszky As far as I know, most JVM depends on Little Endian, I haven't see that JVM depends on Big Endian, So I only finished the code of Little Endian. "Long for Little Endian" affects on parquet page v2 version, I have finished the code, I will submit it to parquet-mr community after this PR. 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1115327777


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @jatin-bhateja, thanks for the explanation. This is fine to check CPU flags. I was wondering, though if there is an existing 3rd party implementation to check things like that so we don't take the burden of maintaining this 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1113018967


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);

Review Comment:
   Let's clean up the TODOs in the next PR then. TODOs in the code in their own mean nothing. Nobody will pick them up to fix them. We either remove the TODOs or at least create a jira for them.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1044199960


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/GeneratorVector.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.encoding;
+
+import org.apache.parquet.encoding.bitpacking.ByteBasedBitPackingVectorGenerator;
+
+/**
+ * main class for code generation hook in build for encodings vector generation
+ */
+public class GeneratorVector {
+  public static void main(String[] args) throws Exception {
+    Generator.main(args);

Review Comment:
   because bitpacking java17 vector decode depends on the current decode way, so I think it have to run Generator before GeneratorVector



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jatin-bhateja (via GitHub)" <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1096553161


##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.*;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class uses the java17 vector API, add VM options --add-modules=jdk.incubator.vector
+ */
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@Warmup(iterations = 1, batchSize = 100000)
+@Measurement(iterations = 1, batchSize = 100000)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ByteBitPackingVectorBenchmarks {
+
+  /**
+   * The range of bitWidth is 1 ~ 32, change it directly if test other bitWidth.
+   */
+  private static final int bitWidth = 7;
+  private static final int outputValues = 1024;
+  private final byte[] input = new byte[outputValues * bitWidth / 8];
+  private final int[] output = new int[outputValues];
+  private final int[] outputVector = new int[outputValues];
+
+  @Setup(Level.Trial)
+  public void getInputBytes() {
+    for (int i = 0; i < input.length; i++) {
+      input[i] = (byte) i;
+    }
+  }
+
+  @Benchmark
+  public void testUnpack() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    for (int i = 0, j = 0; i < input.length; i += bitWidth, j += 8) {
+      bytePacker.unpack8Values(input, i, output, j);
+    }
+  }
+
+  @Benchmark
+  public void testUnpackVector() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker bytePackerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);

Review Comment:
   > Could you elaborate more? @jatin-bhateja
   
   Idea was to emit scalar routines also in vector packer so that user can access both scalar and vector routines through one vector packer instance. 
   But this can be addressed later since currently scalar packer routines are generated at build time and vector packer routines are hand crafted. Existing scalar packer are nested static final classes which makes extending them difficult.



##########
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:
   Else block should have call to readBatch method.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] Fang-Xie commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "Fang-Xie (via GitHub)" <gi...@apache.org>.
Fang-Xie commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1099602361


##########
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:
   @sunchao , [here](https://issues.apache.org/jira/projects/PARQUET/issues/PARQUET-2159) shows the micro-benchmark of bitpack function and test report from Spark vectorizedparquetRecordReader (scan operatories). Most tpch queries are join-related operatories so the hotspot lies in the join/shuffle stage. bitpack optimization would be beneficial for SQL filter query



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jatin-bhateja commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jatin-bhateja commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1048036480


##########
parquet-encoding/src/main/java/org/apache/parquet/column/values/bitpacking/BytePacker.java:
##########
@@ -105,4 +116,16 @@ public void unpack8Values(final byte[] input, final int inPos, final int[] outpu
   public void unpack32Values(byte[] input, int inPos, int[] output, int outPos) {
     unpack32Values(ByteBuffer.wrap(input), inPos, output, outPos);
   }
+
+  public void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   Please add appropriate comments over newly added definitions, like the ones over top over scalar routines.



##########
parquet-generator/src/main/resources/ByteBitPackingVectorLE:
##########
@@ -0,0 +1,3218 @@
+/*
+ * 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 jdk.incubator.vector.*;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPackingVectorLE {
+  private static final BytePacker[] packers = new BytePacker[33];
+
+  static {
+    packers[0] = new Packer0();
+    packers[1] = new Packer1();
+    packers[2] = new Packer2();
+    packers[3] = new Packer3();
+    packers[4] = new Packer4();
+    packers[5] = new Packer5();
+    packers[6] = new Packer6();
+    packers[7] = new Packer7();
+    packers[8] = new Packer8();
+    packers[9] = new Packer9();
+    packers[10] = new Packer10();
+    packers[11] = new Packer11();
+    packers[12] = new Packer12();
+    packers[13] = new Packer13();
+    packers[14] = new Packer14();
+    packers[15] = new Packer15();
+    packers[16] = new Packer16();
+    packers[17] = new Packer17();
+    packers[18] = new Packer18();
+    packers[19] = new Packer19();
+    packers[20] = new Packer20();
+    packers[21] = new Packer21();
+    packers[22] = new Packer22();
+    packers[23] = new Packer23();
+    packers[24] = new Packer24();
+    packers[25] = new Packer25();
+    packers[26] = new Packer26();
+    packers[27] = new Packer27();
+    packers[28] = new Packer28();
+    packers[29] = new Packer29();
+    packers[30] = new Packer30();
+    packers[31] = new Packer31();
+    packers[32] = new Packer32();
+  }
+
+  public static final BytePackerFactory factory = new BytePackerFactory() {
+    public BytePacker newBytePacker(int bitWidth) {
+      return packers[bitWidth];
+    }
+  };
+
+  private static final class Packer0 extends BytePacker {
+    private int unpackCount = 0;
+
+    private Packer0() {
+      super(0);
+    }
+
+    public int getUnpackCount() {
+      return unpackCount;
+    }
+
+    public final void pack8Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void pack32Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   We also need to ensure that we invoke both scalar generators and vector generators for jdk17 profile. 



##########
parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/ByteBitPackingVectorBenchmarks.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.benchmarks;
+
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.openjdk.jmh.annotations.*;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This class uses the java17 vector API, add VM options --add-modules=jdk.incubator.vector
+ */
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@Warmup(iterations = 1, batchSize = 100000)
+@Measurement(iterations = 1, batchSize = 100000)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ByteBitPackingVectorBenchmarks {
+
+  /**
+   * The range of bitWidth is 1 ~ 32, change it directly if test other bitWidth.
+   */
+  private static final int bitWidth = 7;
+  private static final int outputValues = 1024;
+  private final byte[] input = new byte[outputValues * bitWidth / 8];
+  private final int[] output = new int[outputValues];
+  private final int[] outputVector = new int[outputValues];
+
+  @Setup(Level.Trial)
+  public void getInputBytes() {
+    for (int i = 0; i < input.length; i++) {
+      input[i] = (byte) i;
+    }
+  }
+
+  @Benchmark
+  public void testUnpack() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    for (int i = 0, j = 0; i < input.length; i += bitWidth, j += 8) {
+      bytePacker.unpack8Values(input, i, output, j);
+    }
+  }
+
+  @Benchmark
+  public void testUnpackVector() {
+    BytePacker bytePacker = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker bytePackerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);

Review Comment:
   As suggested in my following comments, it will be useful if we are able to access all the routines through one packer instance. 



##########
parquet-generator/src/main/resources/ByteBitPackingVectorLE:
##########
@@ -0,0 +1,3218 @@
+/*
+ * 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 jdk.incubator.vector.*;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is an auto-generated source file and should not edit it directly.
+ */
+public abstract class ByteBitPackingVectorLE {
+  private static final BytePacker[] packers = new BytePacker[33];
+
+  static {
+    packers[0] = new Packer0();
+    packers[1] = new Packer1();
+    packers[2] = new Packer2();
+    packers[3] = new Packer3();
+    packers[4] = new Packer4();
+    packers[5] = new Packer5();
+    packers[6] = new Packer6();
+    packers[7] = new Packer7();
+    packers[8] = new Packer8();
+    packers[9] = new Packer9();
+    packers[10] = new Packer10();
+    packers[11] = new Packer11();
+    packers[12] = new Packer12();
+    packers[13] = new Packer13();
+    packers[14] = new Packer14();
+    packers[15] = new Packer15();
+    packers[16] = new Packer16();
+    packers[17] = new Packer17();
+    packers[18] = new Packer18();
+    packers[19] = new Packer19();
+    packers[20] = new Packer20();
+    packers[21] = new Packer21();
+    packers[22] = new Packer22();
+    packers[23] = new Packer23();
+    packers[24] = new Packer24();
+    packers[25] = new Packer25();
+    packers[26] = new Packer26();
+    packers[27] = new Packer27();
+    packers[28] = new Packer28();
+    packers[29] = new Packer29();
+    packers[30] = new Packer30();
+    packers[31] = new Packer31();
+    packers[32] = new Packer32();
+  }
+
+  public static final BytePackerFactory factory = new BytePackerFactory() {
+    public BytePacker newBytePacker(int bitWidth) {
+      return packers[bitWidth];
+    }
+  };
+
+  private static final class Packer0 extends BytePacker {
+    private int unpackCount = 0;
+
+    private Packer0() {
+      super(0);
+    }
+
+    public int getUnpackCount() {
+      return unpackCount;
+    }
+
+    public final void pack8Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void pack32Values(final int[] in, final int inPos, final byte[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack8Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final byte[] in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpack32Values(final ByteBuffer in, final int inPos, final int[] out, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final byte[] input, final int inPos, final int[] output, final int outPos) {
+    }
+
+    public final void unpackValuesVector(final ByteBuffer input, final int inPos, final int[] output, final int outPos) {

Review Comment:
   All these empty definitions can be removed if we introduce a new class ByteVectorPacker which inherit from existing BytePacker.
   



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1325814649

   @wangyum Could you review the PR ? 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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by GitBox <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1046764737


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/bitpacking/ByteBasedBitPackingVectorGenerator.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.encoding.bitpacking;
+
+import java.io.*;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.

Review Comment:
   This class is used to generate ByteBitPackingVectorLE class, which unpackValues/packValues with java17 vector api.
   If use  java17 vector api directly, errors will occur when compiling or developing with java8(<java17).



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#issuecomment-1449365344

   > @gszadovszky @wgtmac I have added a new workflow named Vector-plugins, can you run it ?
   
   It seems that this PR is closed. Could you please reopen it and see if it can run automatically?


-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1119478321


##########
pom.xml:
##########
@@ -659,5 +662,13 @@
         </plugins>
       </build>
     </profile>
+
+    <profile>
+      <id>plugins</id>

Review Comment:
   `plugins` is a little bit generic to me. Rename it to `encoding-plugin` or `vector-plugins`? Any suggestion?  @gszadovszky 
   
   In addition, please rename `plugins` folder to `parquet-plugins` to follow naming of other sub-directories.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118663395


##########
parquet-generator/src/main/java/org/apache/parquet/encoding/vectorbitpacking/BitPackingGenerator512Vector.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.encoding.vectorbitpacking;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * This class generates vector bit packers that pack the most significant bit first.
+ * The result of the generation is checked in. To regenerate the code run this class and check in the result.
+ */
+public class BitPackingGenerator512Vector {
+  private static final String CLASS_NAME_PREFIX_FOR_INT = "ByteBitPacking512Vector";
+  private static final String CLASS_NAME_PREFIX_FOR_LONG = "ByteBitPacking512VectorForLong";
+
+  public static void main(String[] args) throws Exception {
+    String basePath = args[0];
+    //TODO: Int for Big Endian
+    //generateScheme(false, true, basePath);
+
+    // Int for Little Endian
+    generateScheme(false, false, basePath);
+
+    //TODO: Long for Big Endian
+    //generateScheme(true, true, basePath);
+
+    //TODO: Long for Little Endian
+    //generateScheme(true, false, basePath);
+  }
+
+  private static void generateScheme(boolean isLong, boolean msbFirst,

Review Comment:
   @wgtmac It makes sense. I have added profile plugins.  mvn clean package -P plugins  to enable this feature. 



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "gszadovszky (via GitHub)" <gi...@apache.org>.
gszadovszky commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1116694655


##########
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;
+
+/**
+ * Utility class for big data applications (such as Apache Spark and Apache Flink).
+ * For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains.
+ */
+public class ParquetReadRouter {
+  private static final Logger LOG = LoggerFactory.getLogger(ParquetReadRouter.class);
+
+  private static final int BITS_PER_BYTE = 8;
+
+  // register of avx512 are 512 bits, and can load up to 64 bytes
+  private static final int BYTES_PER_VECTOR_512 = 64;
+
+  // values are bit packed 8 at a time, so reading bitWidth will always work
+  private static final int NUM_VALUES_TO_PACK = 8;
+
+  private static final VectorSupport vectorSupport;
+
+  static {
+    vectorSupport = getSupportVectorFromCPUFlags();
+  }
+
+  // Dispatches to use vector when available. Directly call readBatchUsing512Vector() if you are sure about it.
+  public static void read(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    switch (vectorSupport) {
+      case VECTOR_512:
+        readBatchUsing512Vector(bitWidth, in, currentCount, currentBuffer);
+        break;
+      default:
+        readBatch(bitWidth, in, currentCount, currentBuffer);
+    }
+  }
+
+  // Call the method directly if your computer system contains avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatchUsing512Vector(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws IOException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    BytePacker packerVector = Packer.LITTLE_ENDIAN.newBytePackerVector(bitWidth);
+    int valueIndex = 0;
+    int byteIndex = 0;
+    int unpackCount = packerVector.getUnpackCount();
+    int inputByteCountPerVector = packerVector.getUnpackCount() / BITS_PER_BYTE * bitWidth;
+    int totalByteCount = currentCount * bitWidth / BITS_PER_BYTE;
+    int totalByteCountVector = totalByteCount - BYTES_PER_VECTOR_512;
+    ByteBuffer buffer = in.slice(totalByteCount);
+    if (buffer.hasArray()) {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      // If the remaining bytes size <= {BYTES_PER_512VECTOR}, the remaining bytes are unpacked by packer
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer.array(), buffer.arrayOffset() + buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    } else {
+      for (; byteIndex < totalByteCountVector; byteIndex += inputByteCountPerVector, valueIndex += unpackCount) {
+        packerVector.unpackValuesUsingVector(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+      for (; byteIndex < totalByteCount; byteIndex += bitWidth, valueIndex += NUM_VALUES_TO_PACK) {
+        packer.unpack8Values(buffer, buffer.position() + byteIndex, currentBuffer, valueIndex);
+      }
+    }
+  }
+
+  // Call the method directly if your computer system doesn't contain avx512vbmi and avx512_vbmi2 CPU Flags
+  public static void readBatch(int bitWidth, ByteBufferInputStream in, int currentCount, int[] currentBuffer) throws EOFException {
+    BytePacker packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth);
+    int valueIndex = 0;
+    while (valueIndex < currentCount) {
+      ByteBuffer buffer = in.slice(bitWidth);
+      packer.unpack8Values(buffer, buffer.position(), currentBuffer, valueIndex);
+      valueIndex += NUM_VALUES_TO_PACK;
+    }
+  }
+
+  private static VectorSupport getSupportVectorFromCPUFlags() {
+    try {
+      String os = System.getProperty("os.name");
+      if (os == null || !os.toLowerCase().startsWith("linux")) {
+        return VectorSupport.NONE;
+      }
+      List<String> allLines = Files.readAllLines(Paths.get("/proc/cpuinfo"), StandardCharsets.UTF_8);
+      for (String line : allLines) {
+        if (line != null && line.startsWith("flags")) {
+          int index = line.indexOf(":");
+          if (index < 0) {
+            continue;
+          }
+          line = line.substring(index + 1);
+          Set<String> flagsSet = Arrays.stream(line.split(" ")).collect(Collectors.toSet());
+          if (flagsSet.contains("avx512vbmi") && flagsSet.contains("avx512_vbmi2")) {
+            return VectorSupport.VECTOR_512;
+          }
+        }
+      }

Review Comment:
   @jiangjiguang, I understand and do accept to check the CPU flags here. My concern is only about the actual implementation that we are reading `/proc/cpuinfo` directly. If there is a 3rd party library that implements the extraction of the CPU flags in a nicer way (platform independent etc.) it would be better to use it. If there is no such library available for java then I am fine going forward with this approach.



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1118069968


##########
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
+Parquet-MR has supported Java Vector API to speed up reading, to enable the function:
+* Java 17+, 64-bit
+* For Intel CPU, Flags containing avx512vbmi and avx512_vbmi2 can have better performance gains(ICE Lake or newer processor).
+* mvn clean package -P java17-target -P vector

Review Comment:
   I have updated the document



-- 
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-unsubscribe@parquet.apache.org

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


[GitHub] [parquet-mr] jiangjiguang commented on a diff in pull request #1011: PARQUET-2159: java17 vector parquet bit-packing decode optimization

Posted by "jiangjiguang (via GitHub)" <gi...@apache.org>.
jiangjiguang commented on code in PR #1011:
URL: https://github.com/apache/parquet-mr/pull/1011#discussion_r1121226362


##########
.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,brotli', 'gzip,snappy' ]
+    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-hadoop,-parquet-arrow,-parquet-avro,-parquet-benchmarks,-parquet-cli,-parquet-column,-parquet-hadoop-bundle,-parquet-jackson,-parquet-pig,-parquet-pig-bundle,-parquet-protobuf,-parquet-thrift

Review Comment:
   because these modules(parquet-hadoop parquet-arrow ...) have been executed in the Test workflow. I think vector-plugins should execute only the modules associated with vector. vector-plugins should not execute repeated part with Test workflow.



-- 
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-unsubscribe@parquet.apache.org

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