You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/05/12 16:16:27 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #9424: ARROW-11502: [C++] Optimize Arrow ByteStreamSplitDecode with Neon

pitrou commented on a change in pull request #9424:
URL: https://github.com/apache/arrow/pull/9424#discussion_r631192243



##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -23,25 +23,33 @@
 #include <stdint.h>
 #include <algorithm>
 
+#if defined(ARROW_HAVE_NEON) || defined(ARROW_HAVE_SSE4_2)
+#include <xsimd/xsimd.hpp>
+// Enable the SIMD for ByteStreamSplit Decoder
+#define ARROW_HAVE_SIMD_DECODE_SPLIT
+#endif
+
 #ifdef ARROW_HAVE_SSE4_2
-// Enable the SIMD for ByteStreamSplit Encoder/Decoder
-#define ARROW_HAVE_SIMD_SPLIT
+// Enable the SIMD for ByteStreamSplit Encoder
+#define ARROW_HAVE_SIMD_ENCODE_SPLIT
 #endif  // ARROW_HAVE_SSE4_2
 
 namespace arrow {
 namespace util {
 namespace internal {
 
-#if defined(ARROW_HAVE_SSE4_2)
+#ifdef ARROW_HAVE_SIMD_DECODE_SPLIT
 template <typename T>
-void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t stride,
-                               T* out) {
+void ByteStreamSplitDecode128bit(const uint8_t* data, int64_t num_values, int64_t stride,

Review comment:
       Nit: call this `ByteStreamSplitDecodeSIMD128`?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -23,25 +23,33 @@
 #include <stdint.h>
 #include <algorithm>
 
+#if defined(ARROW_HAVE_NEON) || defined(ARROW_HAVE_SSE4_2)
+#include <xsimd/xsimd.hpp>
+// Enable the SIMD for ByteStreamSplit Decoder
+#define ARROW_HAVE_SIMD_DECODE_SPLIT
+#endif
+
 #ifdef ARROW_HAVE_SSE4_2

Review comment:
       Why isn't NEON enabled here?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -63,30 +71,34 @@ void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t
   // Stage 1: AAAA BBBB CCCC DDDD
   // Stage 2: ACAC ACAC BDBD BDBD
   // Stage 3: ABCD ABCD ABCD ABCD
-  __m128i stage[kNumStreamsLog2 + 1U][kNumStreams];
+  simd_batch stage[kNumStreamsLog2 + 1U][kNumStreams];
   constexpr size_t kNumStreamsHalf = kNumStreams / 2U;
 
   for (int64_t i = 0; i < num_blocks; ++i) {
     for (size_t j = 0; j < kNumStreams; ++j) {
-      stage[0][j] = _mm_loadu_si128(
-          reinterpret_cast<const __m128i*>(&data[i * sizeof(__m128i) + j * stride]));
+      stage[0][j] = simd_batch(
+          reinterpret_cast<const int8_t*>(&data[i * dTypeSize128bits + j * stride]),
+          xsimd::unaligned_mode());
     }
     for (size_t step = 0; step < kNumStreamsLog2; ++step) {
       for (size_t j = 0; j < kNumStreamsHalf; ++j) {
         stage[step + 1U][j * 2] =
-            _mm_unpacklo_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_lo(stage[step][j], stage[step][kNumStreamsHalf + j]);
         stage[step + 1U][j * 2 + 1U] =
-            _mm_unpackhi_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_hi(stage[step][j], stage[step][kNumStreamsHalf + j]);
       }
     }
     for (size_t j = 0; j < kNumStreams; ++j) {
-      _mm_storeu_si128(reinterpret_cast<__m128i*>(
-                           &output_data[(i * kNumStreams + j) * sizeof(__m128i)]),
-                       stage[kNumStreamsLog2][j]);
+      xsimd::store_simd<int8_t, int8_t>(

Review comment:
       I don't think you need to pass the template parameters explicitly?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -23,25 +23,33 @@
 #include <stdint.h>
 #include <algorithm>
 
+#if defined(ARROW_HAVE_NEON) || defined(ARROW_HAVE_SSE4_2)
+#include <xsimd/xsimd.hpp>
+// Enable the SIMD for ByteStreamSplit Decoder
+#define ARROW_HAVE_SIMD_DECODE_SPLIT
+#endif
+
 #ifdef ARROW_HAVE_SSE4_2
-// Enable the SIMD for ByteStreamSplit Encoder/Decoder
-#define ARROW_HAVE_SIMD_SPLIT
+// Enable the SIMD for ByteStreamSplit Encoder
+#define ARROW_HAVE_SIMD_ENCODE_SPLIT
 #endif  // ARROW_HAVE_SSE4_2
 
 namespace arrow {
 namespace util {
 namespace internal {
 
-#if defined(ARROW_HAVE_SSE4_2)
+#ifdef ARROW_HAVE_SIMD_DECODE_SPLIT
 template <typename T>
-void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t stride,
-                               T* out) {
+void ByteStreamSplitDecode128bit(const uint8_t* data, int64_t num_values, int64_t stride,
+                                 T* out) {
+  using simd_batch = xsimd::batch<int8_t, 16>;
+  constexpr size_t dTypeSize128bits = sizeof(int8_t) * 16;

Review comment:
       Can you call this `kBatchSize`?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -63,30 +71,34 @@ void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t
   // Stage 1: AAAA BBBB CCCC DDDD
   // Stage 2: ACAC ACAC BDBD BDBD
   // Stage 3: ABCD ABCD ABCD ABCD
-  __m128i stage[kNumStreamsLog2 + 1U][kNumStreams];
+  simd_batch stage[kNumStreamsLog2 + 1U][kNumStreams];
   constexpr size_t kNumStreamsHalf = kNumStreams / 2U;
 
   for (int64_t i = 0; i < num_blocks; ++i) {
     for (size_t j = 0; j < kNumStreams; ++j) {
-      stage[0][j] = _mm_loadu_si128(
-          reinterpret_cast<const __m128i*>(&data[i * sizeof(__m128i) + j * stride]));
+      stage[0][j] = simd_batch(
+          reinterpret_cast<const int8_t*>(&data[i * dTypeSize128bits + j * stride]),
+          xsimd::unaligned_mode());
     }
     for (size_t step = 0; step < kNumStreamsLog2; ++step) {
       for (size_t j = 0; j < kNumStreamsHalf; ++j) {
         stage[step + 1U][j * 2] =
-            _mm_unpacklo_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_lo(stage[step][j], stage[step][kNumStreamsHalf + j]);
         stage[step + 1U][j * 2 + 1U] =
-            _mm_unpackhi_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_hi(stage[step][j], stage[step][kNumStreamsHalf + j]);
       }
     }
     for (size_t j = 0; j < kNumStreams; ++j) {
-      _mm_storeu_si128(reinterpret_cast<__m128i*>(
-                           &output_data[(i * kNumStreams + j) * sizeof(__m128i)]),
-                       stage[kNumStreamsLog2][j]);
+      xsimd::store_simd<int8_t, int8_t>(
+          reinterpret_cast<int8_t*>(
+              &output_data[(i * kNumStreams + j) * dTypeSize128bits]),
+          stage[kNumStreamsLog2][j], xsimd::unaligned_mode());
     }
   }
 }
+#endif  // ARROW_HAVE_SIMD_DECODE_SPLIT
 
+#if defined(ARROW_HAVE_SSE4_2)
 template <typename T>
 void ByteStreamSplitEncodeSse2(const uint8_t* raw_values, const size_t num_values,

Review comment:
       Can you please convert this to use xsimd as well?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -63,30 +71,34 @@ void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t
   // Stage 1: AAAA BBBB CCCC DDDD
   // Stage 2: ACAC ACAC BDBD BDBD
   // Stage 3: ABCD ABCD ABCD ABCD
-  __m128i stage[kNumStreamsLog2 + 1U][kNumStreams];
+  simd_batch stage[kNumStreamsLog2 + 1U][kNumStreams];
   constexpr size_t kNumStreamsHalf = kNumStreams / 2U;
 
   for (int64_t i = 0; i < num_blocks; ++i) {
     for (size_t j = 0; j < kNumStreams; ++j) {
-      stage[0][j] = _mm_loadu_si128(
-          reinterpret_cast<const __m128i*>(&data[i * sizeof(__m128i) + j * stride]));
+      stage[0][j] = simd_batch(
+          reinterpret_cast<const int8_t*>(&data[i * dTypeSize128bits + j * stride]),
+          xsimd::unaligned_mode());
     }
     for (size_t step = 0; step < kNumStreamsLog2; ++step) {
       for (size_t j = 0; j < kNumStreamsHalf; ++j) {
         stage[step + 1U][j * 2] =
-            _mm_unpacklo_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_lo(stage[step][j], stage[step][kNumStreamsHalf + j]);
         stage[step + 1U][j * 2 + 1U] =
-            _mm_unpackhi_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_hi(stage[step][j], stage[step][kNumStreamsHalf + j]);
       }
     }
     for (size_t j = 0; j < kNumStreams; ++j) {
-      _mm_storeu_si128(reinterpret_cast<__m128i*>(
-                           &output_data[(i * kNumStreams + j) * sizeof(__m128i)]),
-                       stage[kNumStreamsLog2][j]);
+      xsimd::store_simd<int8_t, int8_t>(
+          reinterpret_cast<int8_t*>(
+              &output_data[(i * kNumStreams + j) * dTypeSize128bits]),
+          stage[kNumStreamsLog2][j], xsimd::unaligned_mode());
     }
   }
 }
+#endif  // ARROW_HAVE_SIMD_DECODE_SPLIT
 
+#if defined(ARROW_HAVE_SSE4_2)

Review comment:
       Why not `ARROW_HAVE_SIMD_ENCODE_SPLIT`?

##########
File path: cpp/src/parquet/encoding_benchmark.cc
##########
@@ -371,17 +371,22 @@ BENCHMARK(BM_ByteStreamSplitDecode_Double_Scalar)->Range(MIN_RANGE, MAX_RANGE);
 BENCHMARK(BM_ByteStreamSplitEncode_Float_Scalar)->Range(MIN_RANGE, MAX_RANGE);
 BENCHMARK(BM_ByteStreamSplitEncode_Double_Scalar)->Range(MIN_RANGE, MAX_RANGE);
 
-#if defined(ARROW_HAVE_SSE4_2)
-static void BM_ByteStreamSplitDecode_Float_Sse2(benchmark::State& state) {
+#if defined(ARROW_HAVE_SSE4_2) || defined(ARROW_HAVE_NEON)
+static void BM_ByteStreamSplitDecode_Float_128bit(benchmark::State& state) {

Review comment:
       Perhaps "SIMD128" rather than "128bit".

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -545,21 +557,23 @@ void ByteStreamSplitEncodeAvx512(const uint8_t* raw_values, const size_t num_val
 }
 #endif  // ARROW_HAVE_AVX512
 
-#if defined(ARROW_HAVE_SIMD_SPLIT)
+#if defined(ARROW_HAVE_SIMD_DECODE_SPLIT)
 template <typename T>
 void inline ByteStreamSplitDecodeSimd(const uint8_t* data, int64_t num_values,
                                       int64_t stride, T* out) {
 #if defined(ARROW_HAVE_AVX512)
   return ByteStreamSplitDecodeAvx512(data, num_values, stride, out);
 #elif defined(ARROW_HAVE_AVX2)
   return ByteStreamSplitDecodeAvx2(data, num_values, stride, out);
-#elif defined(ARROW_HAVE_SSE4_2)
-  return ByteStreamSplitDecodeSse2(data, num_values, stride, out);
+#elif defined(ARROW_HAVE_SSE4_2) || defined(ARROW_HAVE_NEON)

Review comment:
       Why not `ARROW_HAVE_SIMD_ENCODE_SPLIT`?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -545,21 +557,23 @@ void ByteStreamSplitEncodeAvx512(const uint8_t* raw_values, const size_t num_val
 }
 #endif  // ARROW_HAVE_AVX512
 
-#if defined(ARROW_HAVE_SIMD_SPLIT)
+#if defined(ARROW_HAVE_SIMD_DECODE_SPLIT)
 template <typename T>
 void inline ByteStreamSplitDecodeSimd(const uint8_t* data, int64_t num_values,
                                       int64_t stride, T* out) {
 #if defined(ARROW_HAVE_AVX512)
   return ByteStreamSplitDecodeAvx512(data, num_values, stride, out);
 #elif defined(ARROW_HAVE_AVX2)
   return ByteStreamSplitDecodeAvx2(data, num_values, stride, out);

Review comment:
       Can the AVX versions also be converted to use xsimd?
   

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -63,30 +71,34 @@ void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t
   // Stage 1: AAAA BBBB CCCC DDDD
   // Stage 2: ACAC ACAC BDBD BDBD
   // Stage 3: ABCD ABCD ABCD ABCD
-  __m128i stage[kNumStreamsLog2 + 1U][kNumStreams];
+  simd_batch stage[kNumStreamsLog2 + 1U][kNumStreams];
   constexpr size_t kNumStreamsHalf = kNumStreams / 2U;
 
   for (int64_t i = 0; i < num_blocks; ++i) {
     for (size_t j = 0; j < kNumStreams; ++j) {
-      stage[0][j] = _mm_loadu_si128(
-          reinterpret_cast<const __m128i*>(&data[i * sizeof(__m128i) + j * stride]));
+      stage[0][j] = simd_batch(
+          reinterpret_cast<const int8_t*>(&data[i * dTypeSize128bits + j * stride]),
+          xsimd::unaligned_mode());
     }
     for (size_t step = 0; step < kNumStreamsLog2; ++step) {
       for (size_t j = 0; j < kNumStreamsHalf; ++j) {
         stage[step + 1U][j * 2] =
-            _mm_unpacklo_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_lo(stage[step][j], stage[step][kNumStreamsHalf + j]);
         stage[step + 1U][j * 2 + 1U] =
-            _mm_unpackhi_epi8(stage[step][j], stage[step][kNumStreamsHalf + j]);
+            xsimd::zip_hi(stage[step][j], stage[step][kNumStreamsHalf + j]);
       }
     }
     for (size_t j = 0; j < kNumStreams; ++j) {
-      _mm_storeu_si128(reinterpret_cast<__m128i*>(
-                           &output_data[(i * kNumStreams + j) * sizeof(__m128i)]),
-                       stage[kNumStreamsLog2][j]);
+      xsimd::store_simd<int8_t, int8_t>(
+          reinterpret_cast<int8_t*>(

Review comment:
       Is the `reinterpret_cast` still needed here?

##########
File path: cpp/src/arrow/util/byte_stream_split.h
##########
@@ -63,30 +71,34 @@ void ByteStreamSplitDecodeSse2(const uint8_t* data, int64_t num_values, int64_t
   // Stage 1: AAAA BBBB CCCC DDDD
   // Stage 2: ACAC ACAC BDBD BDBD
   // Stage 3: ABCD ABCD ABCD ABCD
-  __m128i stage[kNumStreamsLog2 + 1U][kNumStreams];
+  simd_batch stage[kNumStreamsLog2 + 1U][kNumStreams];
   constexpr size_t kNumStreamsHalf = kNumStreams / 2U;
 
   for (int64_t i = 0; i < num_blocks; ++i) {
     for (size_t j = 0; j < kNumStreams; ++j) {
-      stage[0][j] = _mm_loadu_si128(
-          reinterpret_cast<const __m128i*>(&data[i * sizeof(__m128i) + j * stride]));
+      stage[0][j] = simd_batch(
+          reinterpret_cast<const int8_t*>(&data[i * dTypeSize128bits + j * stride]),

Review comment:
       Is the `reinterpret_cast` still needed 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.

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