You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by uw...@apache.org on 2017/09/25 12:26:55 UTC

arrow git commit: ARROW-1601: [C++] Do not read extra byte from validity bitmap, add internal::BitmapReader in lieu of macros

Repository: arrow
Updated Branches:
  refs/heads/master b41a4ee23 -> 096b8770a


ARROW-1601: [C++] Do not read extra byte from validity bitmap, add internal::BitmapReader in lieu of macros

@xhochy since this is causing the crash reported in ARROW-1601 we may want to do a patch release 0.7.1 and parquet-cpp 1.3.1

Author: Wes McKinney <we...@twosigma.com>

Closes #1126 from wesm/ARROW-1601 and squashes the following commits:

6cec81c [Wes McKinney] Fix RleDecoder logic with BitmapReader
ba58b8a [Wes McKinney] Fix test name
fa47865 [Wes McKinney] Add BitmapReader class to replace the bitset macros


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/096b8770
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/096b8770
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/096b8770

Branch: refs/heads/master
Commit: 096b8770a67d1911bee2d4d70653b8539bbd74c7
Parents: b41a4ee
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Sep 25 14:24:50 2017 +0200
Committer: Uwe L. Korn <uw...@xhochy.com>
Committed: Mon Sep 25 14:24:50 2017 +0200

----------------------------------------------------------------------
 cpp/build-support/run_clang_format.py |  7 +++--
 cpp/src/arrow/compute/cast.cc         | 34 ++++++++++-----------
 cpp/src/arrow/util/bit-util-test.cc   | 21 +++++++++++++
 cpp/src/arrow/util/bit-util.h         | 49 ++++++++++++++++++++++++++++++
 cpp/src/arrow/util/rle-encoding.h     | 15 ++++-----
 5 files changed, 98 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/096b8770/cpp/build-support/run_clang_format.py
----------------------------------------------------------------------
diff --git a/cpp/build-support/run_clang_format.py b/cpp/build-support/run_clang_format.py
index ac4954c..f1a448f 100755
--- a/cpp/build-support/run_clang_format.py
+++ b/cpp/build-support/run_clang_format.py
@@ -1,4 +1,4 @@
-#!/usr/bin/python
+#!/usr/bin/env python
 # 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
@@ -58,8 +58,9 @@ for directory, subdirs, files in os.walk(SOURCE_DIR):
 # fi
 
 try:
-    subprocess.check_output([CLANG_FORMAT, '-i'] + files_to_format,
-                            stderr=subprocess.STDOUT)
+    cmd = [CLANG_FORMAT, '-i'] + files_to_format
+    subprocess.check_output(cmd, stderr=subprocess.STDOUT)
 except Exception as e:
     print(e)
+    print(' '.join(cmd))
     raise

http://git-wip-us.apache.org/repos/asf/arrow/blob/096b8770/cpp/src/arrow/compute/cast.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compute/cast.cc b/cpp/src/arrow/compute/cast.cc
index 5283bf0..ee838fa 100644
--- a/cpp/src/arrow/compute/cast.cc
+++ b/cpp/src/arrow/compute/cast.cc
@@ -261,8 +261,9 @@ void UnpackFixedSizeBinaryDictionary(FunctionContext* ctx, const Array& indices,
                                      const FixedSizeBinaryArray& dictionary,
                                      ArrayData* output) {
   using index_c_type = typename IndexType::c_type;
-  const uint8_t* valid_bits = indices.null_bitmap_data();
-  INIT_BITSET(valid_bits, indices.offset());
+
+  internal::BitmapReader valid_bits_reader(indices.null_bitmap_data(), indices.offset(),
+                                           indices.length());
 
   const index_c_type* in =
       reinterpret_cast<const index_c_type*>(indices.data()->buffers[1]->data()) +
@@ -271,11 +272,11 @@ void UnpackFixedSizeBinaryDictionary(FunctionContext* ctx, const Array& indices,
   int32_t byte_width =
       static_cast<const FixedSizeBinaryType&>(*output->type).byte_width();
   for (int64_t i = 0; i < indices.length(); ++i) {
-    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+    if (valid_bits_reader.IsSet()) {
       const uint8_t* value = dictionary.Value(in[i]);
       memcpy(out + i * byte_width, value, byte_width);
     }
-    READ_NEXT_BITSET(valid_bits);
+    valid_bits_reader.Next();
   }
 }
 
@@ -293,8 +294,7 @@ struct CastFunctor<
 
     // Check if values and output type match
     DCHECK(values_type.Equals(*output->type))
-      << "Dictionary type: " << values_type
-      << " target type: " << (*output->type);
+        << "Dictionary type: " << values_type << " target type: " << (*output->type);
 
     const Array& indices = *dict_array.indices();
     switch (indices.type()->id()) {
@@ -327,21 +327,21 @@ Status UnpackBinaryDictionary(FunctionContext* ctx, const Array& indices,
   RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), output->type, &builder));
   BinaryBuilder* binary_builder = static_cast<BinaryBuilder*>(builder.get());
 
-  const uint8_t* valid_bits = indices.null_bitmap_data();
-  INIT_BITSET(valid_bits, indices.offset());
+  internal::BitmapReader valid_bits_reader(indices.null_bitmap_data(), indices.offset(),
+                                           indices.length());
 
   const index_c_type* in =
       reinterpret_cast<const index_c_type*>(indices.data()->buffers[1]->data()) +
       indices.offset();
   for (int64_t i = 0; i < indices.length(); ++i) {
-    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+    if (valid_bits_reader.IsSet()) {
       int32_t length;
       const uint8_t* value = dictionary.GetValue(in[i], &length);
       RETURN_NOT_OK(binary_builder->Append(value, length));
     } else {
       RETURN_NOT_OK(binary_builder->AppendNull());
     }
-    READ_NEXT_BITSET(valid_bits);
+    valid_bits_reader.Next();
   }
 
   std::shared_ptr<Array> plain_array;
@@ -366,8 +366,7 @@ struct CastFunctor<T, DictionaryType,
 
     // Check if values and output type match
     DCHECK(values_type.Equals(*output->type))
-      << "Dictionary type: " << values_type
-      << " target type: " << (*output->type);
+        << "Dictionary type: " << values_type << " target type: " << (*output->type);
 
     const Array& indices = *dict_array.indices();
     switch (indices.type()->id()) {
@@ -401,17 +400,17 @@ void UnpackPrimitiveDictionary(const Array& indices, const c_type* dictionary,
                                c_type* out) {
   using index_c_type = typename IndexType::c_type;
 
-  const uint8_t* valid_bits = indices.null_bitmap_data();
-  INIT_BITSET(valid_bits, indices.offset());
+  internal::BitmapReader valid_bits_reader(indices.null_bitmap_data(), indices.offset(),
+                                           indices.length());
 
   const index_c_type* in =
       reinterpret_cast<const index_c_type*>(indices.data()->buffers[1]->data()) +
       indices.offset();
   for (int64_t i = 0; i < indices.length(); ++i) {
-    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+    if (valid_bits_reader.IsSet()) {
       out[i] = dictionary[in[i]];
     }
-    READ_NEXT_BITSET(valid_bits);
+    valid_bits_reader.Next();
   }
 }
 
@@ -429,8 +428,7 @@ struct CastFunctor<T, DictionaryType,
 
     // Check if values and output type match
     DCHECK(values_type.Equals(*output->type))
-      << "Dictionary type: " << values_type
-      << " target type: " << (*output->type);
+        << "Dictionary type: " << values_type << " target type: " << (*output->type);
 
     auto dictionary =
         reinterpret_cast<const c_type*>(type.dictionary()->data()->buffers[1]->data()) +

http://git-wip-us.apache.org/repos/asf/arrow/blob/096b8770/cpp/src/arrow/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/bit-util-test.cc b/cpp/src/arrow/util/bit-util-test.cc
index d838ab9..a5c6cec 100644
--- a/cpp/src/arrow/util/bit-util-test.cc
+++ b/cpp/src/arrow/util/bit-util-test.cc
@@ -72,6 +72,27 @@ TEST(BitUtilTests, TestNextPower2) {
   ASSERT_EQ(1LL << 62, NextPower2((1LL << 62) - 1));
 }
 
+TEST(BitmapReader, DoesNotReadOutOfBounds) {
+  uint8_t bitmap[16] = {0};
+
+  const int length = 128;
+
+  internal::BitmapReader r1(bitmap, 0, length);
+
+  // If this were to read out of bounds, valgrind would tell us
+  for (int i = 0; i < length; ++i) {
+    ASSERT_TRUE(r1.IsNotSet());
+    r1.Next();
+  }
+
+  internal::BitmapReader r2(bitmap, 5, length - 5);
+
+  for (int i = 0; i < (length - 5); ++i) {
+    ASSERT_TRUE(r2.IsNotSet());
+    r2.Next();
+  }
+}
+
 static inline int64_t SlowCountBits(const uint8_t* data, int64_t bit_offset,
                                     int64_t length) {
   int64_t count = 0;

http://git-wip-us.apache.org/repos/asf/arrow/blob/096b8770/cpp/src/arrow/util/bit-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/bit-util.h b/cpp/src/arrow/util/bit-util.h
index b8a8efa..fa0d7a4 100644
--- a/cpp/src/arrow/util/bit-util.h
+++ b/cpp/src/arrow/util/bit-util.h
@@ -48,6 +48,53 @@
 #endif
 
 namespace arrow {
+namespace internal {
+
+class BitmapReader {
+ public:
+  BitmapReader(const uint8_t* bitmap, int64_t start_offset, int64_t length)
+      : bitmap_(bitmap), position_(0), length_(length) {
+    byte_offset_ = start_offset / 8;
+    bit_offset_ = start_offset % 8;
+    current_byte_ = bitmap[byte_offset_];
+  }
+
+#if defined(_MSC_VER)
+  // MSVC is finicky about this cast
+  bool IsSet() const { return (current_byte_ & (1 << bit_offset_)) != 0; }
+#else
+  bool IsSet() const { return current_byte_ & (1 << bit_offset_); }
+#endif
+
+  bool IsNotSet() const { return (current_byte_ & (1 << bit_offset_)) == 0; }
+
+  void Next() {
+    ++bit_offset_;
+    ++position_;
+    if (bit_offset_ == 8) {
+      bit_offset_ = 0;
+      ++byte_offset_;
+      if (ARROW_PREDICT_TRUE(position_ < length_)) {
+        current_byte_ = bitmap_[byte_offset_];
+      }
+    }
+  }
+
+ private:
+  const uint8_t* bitmap_;
+  int64_t position_;
+  int64_t length_;
+
+  uint8_t current_byte_;
+  int64_t byte_offset_;
+  int64_t bit_offset_;
+};
+
+}  // namespace internal
+
+#ifndef ARROW_NO_DEPRECATED_API
+
+// \deprecated Since > 0.7.0
 
 #define INIT_BITSET(valid_bits_vector, valid_bits_index)            \
   int64_t byte_offset_##valid_bits_vector = (valid_bits_index) / 8; \
@@ -62,6 +109,8 @@ namespace arrow {
     bitset_##valid_bits_vector = valid_bits_vector[byte_offset_##valid_bits_vector]; \
   }
 
+#endif
+
 // TODO(wesm): The source from Impala was depending on boost::make_unsigned
 //
 // We add a partial stub implementation here

http://git-wip-us.apache.org/repos/asf/arrow/blob/096b8770/cpp/src/arrow/util/rle-encoding.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/rle-encoding.h b/cpp/src/arrow/util/rle-encoding.h
index f4c8a77..f343b74 100644
--- a/cpp/src/arrow/util/rle-encoding.h
+++ b/cpp/src/arrow/util/rle-encoding.h
@@ -352,11 +352,12 @@ inline int RleDecoder::GetBatchWithDictSpaced(const T* dictionary, T* values,
   DCHECK_GE(bit_width_, 0);
   int values_read = 0;
   int remaining_nulls = null_count;
-  INIT_BITSET(valid_bits, static_cast<int>(valid_bits_offset));
+
+  internal::BitmapReader bit_reader(valid_bits, valid_bits_offset, batch_size);
 
   while (values_read < batch_size) {
-    bool is_valid = (bitset_valid_bits & (1 << bit_offset_valid_bits)) != 0;
-    READ_NEXT_BITSET(valid_bits);
+    bool is_valid = bit_reader.IsSet();
+    bit_reader.Next();
 
     if (is_valid) {
       if ((repeat_count_ == 0) && (literal_count_ == 0)) {
@@ -369,14 +370,14 @@ inline int RleDecoder::GetBatchWithDictSpaced(const T* dictionary, T* values,
         repeat_count_--;
 
         while (repeat_count_ > 0 && (values_read + repeat_batch) < batch_size) {
-          if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+          if (bit_reader.IsSet()) {
             repeat_count_--;
           } else {
             remaining_nulls--;
           }
           repeat_batch++;
 
-          READ_NEXT_BITSET(valid_bits);
+          bit_reader.Next();
         }
         std::fill(values + values_read, values + values_read + repeat_batch, value);
         values_read += repeat_batch;
@@ -397,7 +398,7 @@ inline int RleDecoder::GetBatchWithDictSpaced(const T* dictionary, T* values,
 
         // Read the first bitset to the end
         while (literals_read < literal_batch) {
-          if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+          if (bit_reader.IsSet()) {
             values[values_read + literals_read + skipped] =
                 dictionary[indices[literals_read]];
             literals_read++;
@@ -405,7 +406,7 @@ inline int RleDecoder::GetBatchWithDictSpaced(const T* dictionary, T* values,
             skipped++;
           }
 
-          READ_NEXT_BITSET(valid_bits);
+          bit_reader.Next();
         }
         literal_count_ -= literal_batch;
         values_read += literal_batch + skipped;