You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by GitBox <gi...@apache.org> on 2023/01/12 06:29:42 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_r1067721200


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

Review Comment:
   We need at least two levels of control of this setting:
   - An option like this to enable compiling the library with AVX512 enabled.
   - A setting to disable runtime dispatch to AVX512.



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

Review Comment:
   I think `BUILD_ENABLE_AVX512` is enough.



##########
c++/src/DetectPlatform.hh:
##########
@@ -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.
+ */
+
+#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 arch_t {
+    px_arch     = 0,
+    avx2_arch   = 1,
+    avx512_arch = 2
+  };
+
+  arch_t detect_platform() {

Review Comment:
   ```suggestion
     arch_t detectPlatform() {
   ```



##########
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 ENABLE_AVX512
+    void unrolledUnpackVector1(int64_t *data, uint64_t offset, uint64_t len);

Review Comment:
   Could you do a little bit refactoring to use the same function signatures but dispatch to different implementations? This can make it easy to add support for other SIMD implementations. You may want to check this for reference: https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/bpacking.h



##########
c++/src/DetectPlatform.hh:
##########
@@ -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.
+ */
+
+#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 arch_t {
+    px_arch     = 0,
+    avx2_arch   = 1,
+    avx512_arch = 2
+  };
+
+  arch_t detect_platform() {
+    arch_t detected_platform = arch_t::px_arch;
+    int    cpu_info[4];

Review Comment:
   Please fix similar naming of variables.



##########
c++/src/DetectPlatform.hh:
##########
@@ -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.
+ */
+
+#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 arch_t {

Review Comment:
   Please use `enum class` and match the naming style.



##########
c++/src/DetectPlatform.hh:
##########
@@ -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.
+ */
+
+#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 arch_t {
+    px_arch     = 0,
+    avx2_arch   = 1,
+    avx512_arch = 2
+  };
+
+  arch_t detect_platform() {
+    arch_t detected_platform = arch_t::px_arch;
+    int    cpu_info[4];

Review Comment:
   ```suggestion
       int    cpuInfo[4];
   ```



##########
c++/src/VectorDecoder.hh:
##########
@@ -0,0 +1,506 @@
+/**
+ * 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 <immintrin.h>

Review Comment:
   This should also be protected by macro as it is not always available.



##########
c++/src/RleDecoderV2.cc:
##########
@@ -67,6 +91,147 @@ namespace orc {
   }
 
   void RleDecoderV2::readLongs(int64_t* data, uint64_t offset, uint64_t len, uint64_t fbs) {
+    uint64_t startBit = 0;
+#if ENABLE_AVX512

Review Comment:
   It would be better to be able to disable it at the runtime.



##########
c++/src/DetectPlatform.hh:
##########
@@ -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.
+ */
+
+#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 arch_t {
+    px_arch     = 0,
+    avx2_arch   = 1,
+    avx512_arch = 2
+  };
+
+  arch_t detect_platform() {

Review Comment:
   Can you add a test?



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

Review Comment:
   Can we do something like below to check if the CPU supports AVX512 at compile time?
   
   https://github.com/apache/arrow/blob/master/cpp/cmake_modules/SetupCxxFlags.cmake#L45
   
   Then we can enable it by default and disable it by options.



-- 
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