You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by "wgtmac (via GitHub)" <gi...@apache.org> on 2023/02/01 07:18:07 UTC

[GitHub] [orc] wgtmac commented on a diff in pull request #1375: ORC-1356: [C++] Use Intel AVX-512 instructions to accelerate the Rle-bit-packing decode

wgtmac commented on code in PR #1375:
URL: https://github.com/apache/orc/pull/1375#discussion_r1092780103


##########
CMakeLists.txt:
##########
@@ -67,6 +67,10 @@ option(BUILD_CPP_ENABLE_METRICS
     "Enable the metrics collection at compile phase"
     OFF)
 
+option(BUILD_ENABLE_AVX512

Review Comment:
   ```suggestion
   option(BUILD_CPP_AVX512
   ```



##########
CMakeLists.txt:
##########
@@ -67,6 +67,10 @@ option(BUILD_CPP_ENABLE_METRICS
     "Enable the metrics collection at compile phase"
     OFF)
 
+option(BUILD_ENABLE_AVX512
+    "Enable AVX512 vector decode of bit-packing"

Review Comment:
   ```suggestion
       "Enable build with AVX512 at compile time"
   ```



##########
CMakeLists.txt:
##########
@@ -157,6 +172,139 @@ elseif (MSVC)
   set (WARN_FLAGS "${WARN_FLAGS} -wd4146") # unary minus operator applied to unsigned type, result still unsigned
 endif ()
 
+include(CheckCXXCompilerFlag)
+include(CheckCXXSourceCompiles)
+message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}")
+
+if(NOT DEFINED ORC_CPU_FLAG)
+  if(CMAKE_SYSTEM_PROCESSOR MATCHES "AMD64|X86|x86|i[3456]86|x64")
+    set(ORC_CPU_FLAG "x86")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|ARM64|arm64")
+    set(ORC_CPU_FLAG "aarch64")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm$|armv[4-7]")
+    set(ORC_CPU_FLAG "aarch32")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "powerpc|ppc")
+    set(ORC_CPU_FLAG "ppc")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "s390x")
+    set(ORC_CPU_FLAG "s390x")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64")
+    set(ORC_CPU_FLAG "riscv64")
+  else()
+    message(FATAL_ERROR "Unknown system processor")
+  endif()
+endif()
+
+# Check architecture specific compiler flags
+if(ORC_CPU_FLAG STREQUAL "x86")
+  # x86/amd64 compiler flags, msvc/gcc/clang
+  if(MSVC)
+    set(ORC_SSE4_2_FLAG "")
+    set(ORC_AVX2_FLAG "/arch:AVX2")
+    set(ORC_AVX512_FLAG "/arch:AVX512")
+    set(CXX_SUPPORTS_SSE4_2 TRUE)
+  else()
+    set(ORC_SSE4_2_FLAG "-msse4.2")
+    set(ORC_AVX2_FLAG "-march=haswell")
+    # skylake-avx512 consists of AVX512F,AVX512BW,AVX512VL,AVX512CD,AVX512DQ
+    set(ORC_AVX512_FLAG "-march=native -mbmi2")
+    # Append the avx2/avx512 subset option also, fix issue ORC-9877 for homebrew-cpp

Review Comment:
   What does `fix issue ORC-9877 for homebrew-cpp` mean?



##########
CMakeLists.txt:
##########
@@ -157,6 +172,139 @@ elseif (MSVC)
   set (WARN_FLAGS "${WARN_FLAGS} -wd4146") # unary minus operator applied to unsigned type, result still unsigned
 endif ()
 
+include(CheckCXXCompilerFlag)
+include(CheckCXXSourceCompiles)
+message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}")
+
+if(NOT DEFINED ORC_CPU_FLAG)

Review Comment:
   Are we supposed to support ppc, s390x and riscv64? The CI checks do not cover these architectures so we are unable to verify and maintain them. 
   
   cc @dongjoon-hyun 



##########
CMakeLists.txt:
##########
@@ -157,6 +172,139 @@ elseif (MSVC)
   set (WARN_FLAGS "${WARN_FLAGS} -wd4146") # unary minus operator applied to unsigned type, result still unsigned
 endif ()
 
+include(CheckCXXCompilerFlag)

Review Comment:
   The architecture detecting logic below worth a separate file under `cmake_modules` directory and be included here.



##########
CMakeLists.txt:
##########
@@ -157,6 +172,139 @@ elseif (MSVC)
   set (WARN_FLAGS "${WARN_FLAGS} -wd4146") # unary minus operator applied to unsigned type, result still unsigned
 endif ()
 
+include(CheckCXXCompilerFlag)
+include(CheckCXXSourceCompiles)
+message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}")
+
+if(NOT DEFINED ORC_CPU_FLAG)
+  if(CMAKE_SYSTEM_PROCESSOR MATCHES "AMD64|X86|x86|i[3456]86|x64")
+    set(ORC_CPU_FLAG "x86")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|ARM64|arm64")
+    set(ORC_CPU_FLAG "aarch64")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm$|armv[4-7]")
+    set(ORC_CPU_FLAG "aarch32")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "powerpc|ppc")
+    set(ORC_CPU_FLAG "ppc")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "s390x")
+    set(ORC_CPU_FLAG "s390x")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64")
+    set(ORC_CPU_FLAG "riscv64")
+  else()
+    message(FATAL_ERROR "Unknown system processor")
+  endif()
+endif()
+
+# Check architecture specific compiler flags
+if(ORC_CPU_FLAG STREQUAL "x86")
+  # x86/amd64 compiler flags, msvc/gcc/clang
+  if(MSVC)
+    set(ORC_SSE4_2_FLAG "")

Review Comment:
   If this patch aims for AVX512 only, we can remove SSE4 and AVX2 for now. So flags like `ORC_AVX2_FLAG`, `CXX_SUPPORTS_SSE4_2`, and `CXX_SUPPORTS_AVX2` can be removed for now.



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit
+
+#define CPUID_AVX512_MASK \
+  (CPUID_AVX512F | CPUID_AVX512CD | CPUID_AVX512VL | CPUID_AVX512BW | CPUID_AVX512DQ)
+
+  enum class Arch { PX_ARCH = 0, AVX2_ARCH = 1, AVX512_ARCH = 2 };
+
+  Arch detectPlatform() {
+    Arch detectedPlatform = Arch::PX_ARCH;
+    int cpuInfo[4];
+    cpuid(cpuInfo, 1);
+
+    bool avx512SupportCpu = cpuInfo[1] & CPUID_AVX512_MASK;

Review Comment:
   ```suggestion
       bool avx512Supported = cpuInfo[1] & CPUID_AVX512_MASK;
   ```



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit
+
+#define CPUID_AVX512_MASK \
+  (CPUID_AVX512F | CPUID_AVX512CD | CPUID_AVX512VL | CPUID_AVX512BW | CPUID_AVX512DQ)
+
+  enum class Arch { PX_ARCH = 0, AVX2_ARCH = 1, AVX512_ARCH = 2 };
+
+  Arch detectPlatform() {
+    Arch detectedPlatform = Arch::PX_ARCH;
+    int cpuInfo[4];
+    cpuid(cpuInfo, 1);
+
+    bool avx512SupportCpu = cpuInfo[1] & CPUID_AVX512_MASK;
+    bool osUsesXSaveXStore = cpuInfo[2] & EXC_OSXSAVE;

Review Comment:
   ```suggestion
       bool xsaveSupported = cpuInfo[2] & EXC_OSXSAVE;
   ```



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit
+
+#define CPUID_AVX512_MASK \
+  (CPUID_AVX512F | CPUID_AVX512CD | CPUID_AVX512VL | CPUID_AVX512BW | CPUID_AVX512DQ)
+
+  enum class Arch { PX_ARCH = 0, AVX2_ARCH = 1, AVX512_ARCH = 2 };
+
+  Arch detectPlatform() {
+    Arch detectedPlatform = Arch::PX_ARCH;
+    int cpuInfo[4];
+    cpuid(cpuInfo, 1);
+
+    bool avx512SupportCpu = cpuInfo[1] & CPUID_AVX512_MASK;
+    bool osUsesXSaveXStore = cpuInfo[2] & EXC_OSXSAVE;
+
+    if (avx512SupportCpu && osUsesXSaveXStore) {
+      // Check if XMM state and YMM state are saved
+#ifdef _WIN32
+      unsigned long long xcrFeatureMask = _xgetbv(0); /* min VS2010 SP1 compiler is required */

Review Comment:
   What does `xcr` mean?



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit
+
+#define CPUID_AVX512_MASK \
+  (CPUID_AVX512F | CPUID_AVX512CD | CPUID_AVX512VL | CPUID_AVX512BW | CPUID_AVX512DQ)
+
+  enum class Arch { PX_ARCH = 0, AVX2_ARCH = 1, AVX512_ARCH = 2 };
+
+  Arch detectPlatform() {

Review Comment:
   Should we rename the function and the file name to `detect architecture`?



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit

Review Comment:
   Could you give a more meaningful name or add a line of comment?



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32
+
+#include "intrin.h"
+//  Windows CPUID
+#define cpuid(info, x) __cpuidex(info, x, 0)
+#else
+//  GCC Intrinsics
+#include <cpuid.h>
+#include <dlfcn.h>
+
+  void cpuid(int info[4], int InfoType) {
+    __cpuid_count(InfoType, 0, info[0], info[1], info[2], info[3]);
+  }
+
+  unsigned long long xgetbv(unsigned int index) {
+    unsigned int eax, edx;
+    __asm__ __volatile__("xgetbv;" : "=a"(eax), "=d"(edx) : "c"(index));
+    return ((unsigned long long)edx << 32) | eax;
+  }
+
+#endif
+
+#define CPUID_AVX512F 0x00100000
+#define CPUID_AVX512CD 0x00200000
+#define CPUID_AVX512VL 0x04000000
+#define CPUID_AVX512BW 0x01000000
+#define CPUID_AVX512DQ 0x02000000
+#define EXC_OSXSAVE 0x08000000  // 27th  bit
+
+#define CPUID_AVX512_MASK \
+  (CPUID_AVX512F | CPUID_AVX512CD | CPUID_AVX512VL | CPUID_AVX512BW | CPUID_AVX512DQ)
+
+  enum class Arch { PX_ARCH = 0, AVX2_ARCH = 1, AVX512_ARCH = 2 };
+
+  Arch detectPlatform() {
+    Arch detectedPlatform = Arch::PX_ARCH;

Review Comment:
   ```suggestion
       Arch arch = Arch::PX_ARCH;
   ```



##########
c++/src/RLEv2.hh:
##########
@@ -230,6 +265,14 @@ namespace orc {
     uint32_t curByte;                   // Used by anything that uses readLongs
     DataBuffer<int64_t> unpackedPatch;  // Used by PATCHED_BASE
     DataBuffer<int64_t> literals;       // Values of the current run
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    uint8_t

Review Comment:
   How about move the comment above each variable definition? This will be more readable.



##########
c++/src/DetectPlatform.hh:
##########
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_DETECTPLATFORM_HH
+#define ORC_DETECTPLATFORM_HH
+
+#if defined(__GNUC__) || defined(__clang__)
+DIAGNOSTIC_IGNORE("-Wold-style-cast")
+#endif
+
+namespace orc {
+#ifdef _WIN32

Review Comment:
   Platform dependent function like `cpuid` can be defined in the file `Adaptor.hh.in`



##########
c++/src/RLEv2.hh:
##########
@@ -189,13 +192,45 @@ namespace orc {
       resetReadLongs();
     }
 
+    void resetBufferStart(uint64_t len, bool resetBuf, uint32_t backupLen);
     unsigned char readByte();
 
     int64_t readLongBE(uint64_t bsz);
     int64_t readVslong();
     uint64_t readVulong();
     void readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs);
-    void plainUnpackLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs);
+    void plainUnpackLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs,
+                          uint64_t& startBit);
+
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    void unrolledUnpackVector1(int64_t* data, uint64_t offset, uint64_t len);

Review Comment:
   Rename to `vectorUnpackX` ?



##########
c++/src/RleDecoderV2.cc:
##########
@@ -18,11 +18,35 @@
 
 #include "Adaptor.hh"
 #include "Compression.hh"
+#include "DetectPlatform.hh"
 #include "RLEV2Util.hh"
 #include "RLEv2.hh"
 #include "Utils.hh"
+#include "VectorDecoder.hh"
 
 namespace orc {
+  void RleDecoderV2::resetBufferStart(uint64_t len, bool resetBuf, uint32_t backupByteLen) {
+    uint64_t restLen = bufferEnd - bufferStart;

Review Comment:
   ```suggestion
       uint64_t remainingLen = bufferEnd - bufferStart;
   ```



##########
CMakeLists.txt:
##########
@@ -157,6 +172,139 @@ elseif (MSVC)
   set (WARN_FLAGS "${WARN_FLAGS} -wd4146") # unary minus operator applied to unsigned type, result still unsigned
 endif ()
 
+include(CheckCXXCompilerFlag)
+include(CheckCXXSourceCompiles)
+message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}")
+
+if(NOT DEFINED ORC_CPU_FLAG)
+  if(CMAKE_SYSTEM_PROCESSOR MATCHES "AMD64|X86|x86|i[3456]86|x64")
+    set(ORC_CPU_FLAG "x86")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|ARM64|arm64")
+    set(ORC_CPU_FLAG "aarch64")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm$|armv[4-7]")
+    set(ORC_CPU_FLAG "aarch32")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "powerpc|ppc")
+    set(ORC_CPU_FLAG "ppc")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "s390x")
+    set(ORC_CPU_FLAG "s390x")
+  elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "riscv64")
+    set(ORC_CPU_FLAG "riscv64")
+  else()
+    message(FATAL_ERROR "Unknown system processor")
+  endif()
+endif()
+
+# Check architecture specific compiler flags
+if(ORC_CPU_FLAG STREQUAL "x86")
+  # x86/amd64 compiler flags, msvc/gcc/clang
+  if(MSVC)
+    set(ORC_SSE4_2_FLAG "")
+    set(ORC_AVX2_FLAG "/arch:AVX2")
+    set(ORC_AVX512_FLAG "/arch:AVX512")
+    set(CXX_SUPPORTS_SSE4_2 TRUE)
+  else()
+    set(ORC_SSE4_2_FLAG "-msse4.2")
+    set(ORC_AVX2_FLAG "-march=haswell")
+    # skylake-avx512 consists of AVX512F,AVX512BW,AVX512VL,AVX512CD,AVX512DQ
+    set(ORC_AVX512_FLAG "-march=native -mbmi2")
+    # Append the avx2/avx512 subset option also, fix issue ORC-9877 for homebrew-cpp
+    set(ORC_AVX2_FLAG "${ORC_AVX2_FLAG} -mavx2")
+    set(ORC_AVX512_FLAG
+        "${ORC_AVX512_FLAG} -mavx512f -mavx512cd -mavx512vl -mavx512dq -mavx512bw -mavx512vbmi")
+  endif()
+  check_cxx_compiler_flag(${ORC_AVX512_FLAG} CXX_SUPPORTS_AVX512)
+  if(MINGW)
+    # https://gcc.gnu.org/bugzilla/show_bug.cgi?id=65782
+    message(STATUS "Disable AVX512 support on MINGW for now")
+  else()
+    # Check for AVX512 support in the compiler.
+    set(OLD_CMAKE_REQURED_FLAGS ${CMAKE_REQUIRED_FLAGS})
+    set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} ${ORC_AVX512_FLAG}")
+    check_cxx_source_compiles("
+      #ifdef _MSC_VER
+      #include <intrin.h>
+      #else
+      #include <immintrin.h>
+      #endif
+
+      int main() {
+        __m512i mask = _mm512_set1_epi32(0x1);
+        char out[32];
+        _mm512_storeu_si512(out, mask);
+        return 0;
+      }"
+      CXX_SUPPORTS_AVX512)
+    set(CMAKE_REQUIRED_FLAGS ${OLD_CMAKE_REQURED_FLAGS})
+  endif()
+  # Runtime SIMD level it can get from compiler and ORC_RUNTIME_SIMD_LEVEL
+  if(CXX_SUPPORTS_SSE4_2 AND ORC_RUNTIME_SIMD_LEVEL MATCHES
+                             "^(SSE4_2|AVX2|AVX512|MAX)$")
+    set(ORC_HAVE_RUNTIME_SSE4_2 ON)
+    set(ORC_SIMD_LEVEL "SSE4_2")
+    add_definitions(-DORC_HAVE_RUNTIME_SSE4_2)
+  endif()
+  if(CXX_SUPPORTS_AVX2 AND ORC_RUNTIME_SIMD_LEVEL MATCHES "^(AVX2|AVX512|MAX)$")
+    set(ORC_HAVE_RUNTIME_AVX2 ON)
+    set(ORC_SIMD_LEVEL "AVX2")
+    add_definitions(-DORC_HAVE_RUNTIME_AVX2 -DORC_HAVE_RUNTIME_BMI2)
+  endif()
+  if(BUILD_ENABLE_AVX512 AND CXX_SUPPORTS_AVX512 AND ORC_RUNTIME_SIMD_LEVEL MATCHES "^(AVX512|MAX)$")
+    message(STATUS "Enable the AVX512 vector decode of bit-packing")
+    set(ORC_HAVE_RUNTIME_AVX512 ON)
+    set(ORC_SIMD_LEVEL "AVX512")
+    add_definitions(-DORC_HAVE_RUNTIME_AVX512 -DORC_HAVE_RUNTIME_BMI2)
+  else ()
+    message(STATUS "Disable the AVX512 vector decode of bit-packing")
+  endif()
+  if(ORC_SIMD_LEVEL STREQUAL "DEFAULT")
+    set(ORC_SIMD_LEVEL "NONE")
+  endif()
+elseif(ORC_CPU_FLAG STREQUAL "ppc")
+  # power compiler flags, gcc/clang only
+  set(ORC_ALTIVEC_FLAG "-maltivec")
+  check_cxx_compiler_flag(${ORC_ALTIVEC_FLAG} CXX_SUPPORTS_ALTIVEC)
+  if(ORC_SIMD_LEVEL STREQUAL "DEFAULT")
+    set(ORC_SIMD_LEVEL "NONE")
+  endif()
+elseif(ORC_CPU_FLAG STREQUAL "aarch64")
+  # Arm64 compiler flags, gcc/clang only
+  set(ORC_ARMV8_MARCH "armv8-a")
+  check_cxx_compiler_flag("-march=${ORC_ARMV8_MARCH}+sve" CXX_SUPPORTS_SVE)
+  if(ORC_SIMD_LEVEL STREQUAL "DEFAULT")
+    set(ORC_SIMD_LEVEL "NEON")
+  endif()
+endif()
+
+# Only enable additional instruction sets if they are supported
+if(ORC_CPU_FLAG STREQUAL "x86")
+  if(MINGW)
+    # Enable _xgetbv() intrinsic to query OS support for ZMM register saves
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mxsave")
+  endif()
+  if(ORC_SIMD_LEVEL STREQUAL "AVX512")
+    if(NOT CXX_SUPPORTS_AVX512)
+      message(FATAL_ERROR "AVX512 required but compiler doesn't support it.")
+    endif()
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${ORC_AVX512_FLAG}")
+    add_definitions(-DORC_HAVE_AVX512 -DORC_HAVE_AVX2 -DORC_HAVE_BMI2
+                    -DORC_HAVE_SSE4_2)
+  elseif(ORC_SIMD_LEVEL STREQUAL "AVX2")

Review Comment:
   We can remove levels other than `AVX512` for now to make it simpler.



##########
c++/src/RleDecoderV2.cc:
##########
@@ -67,6 +91,149 @@ namespace orc {
   }
 
   void RleDecoderV2::readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs) {
+    uint64_t startBit = 0;
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    const auto runtimeEnable = getenv("ENABLE_RUNTIME_AVX512");
+    std::string avxRuntimeEnable = runtimeEnable == nullptr ? "OFF" : std::string(runtimeEnable);
+    if (detectPlatform() == Arch::AVX512_ARCH && strcasecmp(avxRuntimeEnable.c_str(), "on") == 0) {
+      switch (fbs) {

Review Comment:
   We can wrap lines between 99 and 234 to a separate function named something like `readLongsAvx512`



##########
c++/src/VectorDecoder.hh:
##########
@@ -0,0 +1,488 @@
+/**
+ * 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.
+ */
+
+#ifndef VECTOR_DECODER_HH
+#define VECTOR_DECODER_HH
+
+#include <string.h>

Review Comment:
   Move it below line 24?



##########
c++/src/RleDecoderV2.cc:
##########
@@ -67,6 +91,149 @@ namespace orc {
   }
 
   void RleDecoderV2::readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs) {
+    uint64_t startBit = 0;
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    const auto runtimeEnable = getenv("ENABLE_RUNTIME_AVX512");

Review Comment:
   Can we add a flag or enum value to class `RleDecoderV2` so it can decide how to dispatch functions at runtime? In this way, we can simply make the decision at the creation time of `RleDecoderV2`. Otherwise the decision is made on every call to `readLongs`.



##########
c++/test/TestRleVectorDecoder.cc:
##########
@@ -0,0 +1,608 @@
+/**
+ * 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.
+ */
+
+#include <inttypes.h>

Review Comment:
   I assume this include file will fail on some platforms (MSVC?)



##########
c++/src/RleDecoderV2.cc:
##########
@@ -97,10 +264,4151 @@ namespace orc {
         return;
       default:
         // Fallback to the default implementation for deprecated bit size.
-        plainUnpackLongs(data, offset, len, fbs);
+        plainUnpackLongs(data, offset, len, fbs, startBit);
+        return;
+    }
+#endif
+  }
+
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+  void RleDecoderV2::unrolledUnpackVector1(int64_t* data, uint64_t offset, uint64_t len) {

Review Comment:
   Do you have the script that generates the vectorized code? It would be great if it is committed alongside for future maintenance.



##########
c++/src/RleDecoderV2.cc:
##########
@@ -67,6 +91,149 @@ namespace orc {
   }
 
   void RleDecoderV2::readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs) {
+    uint64_t startBit = 0;
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    const auto runtimeEnable = getenv("ENABLE_RUNTIME_AVX512");
+    std::string avxRuntimeEnable = runtimeEnable == nullptr ? "OFF" : std::string(runtimeEnable);
+    if (detectPlatform() == Arch::AVX512_ARCH && strcasecmp(avxRuntimeEnable.c_str(), "on") == 0) {
+      switch (fbs) {
+        case 1:
+          unrolledUnpackVector1(data, offset, len);
+          return;
+        case 2:
+          unrolledUnpackVector2(data, offset, len);
+          return;
+        case 3:
+          unrolledUnpackVector3(data, offset, len);
+          return;
+        case 4:
+          unrolledUnpackVector4(data, offset, len);
+          return;
+        case 5:
+          unrolledUnpackVector5(data, offset, len);
+          return;
+        case 6:
+          unrolledUnpackVector6(data, offset, len);
+          return;
+        case 7:
+          unrolledUnpackVector7(data, offset, len);
+          return;
+        case 8:
+          unrolledUnpack8(data, offset, len);
+          return;
+        case 9:
+          unrolledUnpackVector9(data, offset, len);
+          return;
+        case 10:
+          unrolledUnpackVector10(data, offset, len);
+          return;
+        case 11:
+          unrolledUnpackVector11(data, offset, len);
+          return;
+        case 12:
+          unrolledUnpackVector12(data, offset, len);
+          return;
+        case 13:
+          unrolledUnpackVector13(data, offset, len);
+          return;
+        case 14:
+          unrolledUnpackVector14(data, offset, len);
+          return;
+        case 15:
+          unrolledUnpackVector15(data, offset, len);
+          return;
+        case 16:
+          unrolledUnpackVector16(data, offset, len);
+          return;
+        case 17:
+          unrolledUnpackVector17(data, offset, len);
+          return;
+        case 18:
+          unrolledUnpackVector18(data, offset, len);
+          return;
+        case 19:
+          unrolledUnpackVector19(data, offset, len);
+          return;
+        case 20:
+          unrolledUnpackVector20(data, offset, len);
+          return;
+        case 21:
+          unrolledUnpackVector21(data, offset, len);
+          return;
+        case 22:
+          unrolledUnpackVector22(data, offset, len);
+          return;
+        case 23:
+          unrolledUnpackVector23(data, offset, len);
+          return;
+        case 24:
+          unrolledUnpackVector24(data, offset, len);
+          return;
+        case 26:
+          unrolledUnpackVector26(data, offset, len);
+          return;
+        case 28:
+          unrolledUnpackVector28(data, offset, len);
+          return;
+        case 30:
+          unrolledUnpackVector30(data, offset, len);
+          return;
+        case 32:
+          unrolledUnpackVector32(data, offset, len);
+          return;
+        case 40:

Review Comment:
   It seems cases here and below are handled by `plainUnpackLongs` already.



##########
CMakeLists.txt:
##########
@@ -67,6 +67,10 @@ option(BUILD_CPP_ENABLE_METRICS
     "Enable the metrics collection at compile phase"
     OFF)
 
+option(BUILD_ENABLE_AVX512
+    "Enable AVX512 vector decode of bit-packing"

Review Comment:
   IMO, here we do not need to say what AVX512 is used for.



##########
c++/test/TestRleVectorDecoder.cc:
##########
@@ -0,0 +1,608 @@
+/**
+ * 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.
+ */
+
+#include <inttypes.h>
+
+#include <cstdlib>
+
+#include "MemoryOutputStream.hh"
+#include "RLEv2.hh"
+#include "wrap/gtest-wrapper.h"
+#include "wrap/orc-proto-wrapper.hh"
+
+#ifdef __clang__
+DIAGNOSTIC_IGNORE("-Wmissing-variable-declarations")
+#endif
+
+namespace orc {
+
+  using ::testing::TestWithParam;
+  using ::testing::Values;
+
+  const int DEFAULT_MEM_STREAM_SIZE = 1024 * 1024;  // 1M
+
+  class RleVectorTest : public TestWithParam<bool> {

Review Comment:
   ```suggestion
     class RleV2BitUnpackAvx512Test : public TestWithParam<bool> {
   ```



##########
c++/src/VectorDecoder.hh:
##########
@@ -0,0 +1,488 @@
+/**
+ * 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.
+ */
+
+#ifndef VECTOR_DECODER_HH

Review Comment:
   Is this file specific to AVX512? If true, we had better rename the header file and variable names to explicitly speak AVX512. For example, rename `VectorDecoder.hh` to `BitUnpackerAvx512.hh`



##########
c++/src/RLEv2.hh:
##########
@@ -230,6 +265,14 @@ namespace orc {
     uint32_t curByte;                   // Used by anything that uses readLongs
     DataBuffer<int64_t> unpackedPatch;  // Used by PATCHED_BASE
     DataBuffer<int64_t> literals;       // Values of the current run
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    uint8_t

Review Comment:
   In addition, my concern here is that if we support different instruction sets, here will add more buffers.



##########
c++/src/RLEv2.hh:
##########
@@ -189,13 +192,45 @@ namespace orc {
       resetReadLongs();
     }
 
+    void resetBufferStart(uint64_t len, bool resetBuf, uint32_t backupLen);
     unsigned char readByte();
 
     int64_t readLongBE(uint64_t bsz);
     int64_t readVslong();
     uint64_t readVulong();
     void readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs);
-    void plainUnpackLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs);
+    void plainUnpackLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs,
+                          uint64_t& startBit);
+
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    void unrolledUnpackVector1(int64_t* data, uint64_t offset, uint64_t len);

Review Comment:
   Why not define them in `RleDecoderV2.cc` and delete the declarations here? They are not supposed to be used elsewhere.



##########
c++/src/RleDecoderV2.cc:
##########
@@ -67,6 +91,149 @@ namespace orc {
   }
 
   void RleDecoderV2::readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs) {
+    uint64_t startBit = 0;
+#if defined(ORC_HAVE_RUNTIME_AVX512)
+    const auto runtimeEnable = getenv("ENABLE_RUNTIME_AVX512");
+    std::string avxRuntimeEnable = runtimeEnable == nullptr ? "OFF" : std::string(runtimeEnable);
+    if (detectPlatform() == Arch::AVX512_ARCH && strcasecmp(avxRuntimeEnable.c_str(), "on") == 0) {
+      switch (fbs) {

Review Comment:
   It also would be better to put these AVX512 functions in a separate header like `VectorDecoder.hh` (I have suggested renaming it in another comment). In this way, we can add future implementation much easier. Probably the functions should not rely on the internal class variables but use input parameters instead.



-- 
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: issues-unsubscribe@orc.apache.org

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