You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/09/03 19:08:08 UTC

arrow git commit: ARROW-1452: [C++] Make macros in arrow/util/macros.h more unique

Repository: arrow
Updated Branches:
  refs/heads/master 1a8dafb2b -> ebca1af3b


ARROW-1452: [C++] Make macros in arrow/util/macros.h more unique

This fixes a bug encounted in third party use: https://github.com/mapd/mapd-core/blob/master/QueryEngine/ResultSet.h#L37

cc @asuhan

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

Closes #1030 from wesm/ARROW-1452 and squashes the following commits:

ab7611cf [Wes McKinney] Disallow copy and assign in ArrayBuilder
2a4db73f [Wes McKinney] Remove DISALLOW_COPY_AND_ASSIGN from some abstract classes. Revert MANUALLY_ALIGNED_STRUCT
39f10d64 [Wes McKinney] Remove UNLIKELY, LIKELY macros as they may conflict with thirdparty projects, consolidate compiler-util.h with macros.h
994f5bfe [Wes McKinney] Add unique prefix to DISALLOW_COPY_AND_ASSIGN
c3121df0 [Wes McKinney] Rename UNUSED to ARROW_UNUSED


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

Branch: refs/heads/master
Commit: ebca1af3b40f16d8f00cfbfbbe28881e2d267bf4
Parents: 1a8dafb
Author: Wes McKinney <we...@twosigma.com>
Authored: Sun Sep 3 15:07:59 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Sun Sep 3 15:07:59 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/array.h                    |  3 --
 cpp/src/arrow/buffer.h                   |  2 +-
 cpp/src/arrow/builder.cc                 |  4 +--
 cpp/src/arrow/builder.h                  |  2 +-
 cpp/src/arrow/compare.cc                 |  2 +-
 cpp/src/arrow/io/hdfs.h                  |  6 ++--
 cpp/src/arrow/io/interfaces.h            |  3 --
 cpp/src/arrow/ipc/dictionary.h           |  2 +-
 cpp/src/arrow/ipc/message.h              |  2 +-
 cpp/src/arrow/python/arrow_to_pandas.cc  |  2 +-
 cpp/src/arrow/python/common.h            |  2 +-
 cpp/src/arrow/tensor.h                   |  2 +-
 cpp/src/arrow/type.h                     |  3 --
 cpp/src/arrow/util/bit-stream-utils.h    | 16 ++++++-----
 cpp/src/arrow/util/bit-util.h            |  8 +++---
 cpp/src/arrow/util/compiler-util.h       | 40 ++-------------------------
 cpp/src/arrow/util/compression_snappy.cc |  5 ----
 cpp/src/arrow/util/hash-util.h           |  2 +-
 cpp/src/arrow/util/key_value_metadata.h  |  2 +-
 cpp/src/arrow/util/macros.h              | 32 ++++++++++++++++++---
 cpp/src/arrow/util/rle-encoding.h        |  6 ++--
 21 files changed, 62 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 57c1f27..8052071 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -254,9 +254,6 @@ class ARROW_EXPORT Array {
     }
     data_ = data;
   }
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(Array);
 };
 
 static inline std::ostream& operator<<(std::ostream& os, const Array& x) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/buffer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index ed128fd..d5b2a26 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -97,7 +97,7 @@ class ARROW_EXPORT Buffer {
   std::shared_ptr<Buffer> parent_;
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(Buffer);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Buffer);
 };
 
 /// \brief Create Buffer referencing std::string memory

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/builder.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index daaf49c..71f0bab 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -793,8 +793,8 @@ Status DictionaryBuilder<T>::Append(const Scalar& value) {
     hash_slots_[j] = index;
     RETURN_NOT_OK(AppendDictionary(value));
 
-    if (UNLIKELY(static_cast<int32_t>(dict_builder_.length()) >
-                 hash_table_size_ * kMaxHashTableLoad)) {
+    if (ARROW_PREDICT_FALSE(static_cast<int32_t>(dict_builder_.length()) >
+                            hash_table_size_ * kMaxHashTableLoad)) {
       RETURN_NOT_OK(DoubleTableSize());
     }
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/builder.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index 8e41266..cb1e4ba 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -156,7 +156,7 @@ class ARROW_EXPORT ArrayBuilder {
   void UnsafeSetNotNull(int64_t length);
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(ArrayBuilder);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(ArrayBuilder);
 };
 
 class ARROW_EXPORT NullBuilder : public ArrayBuilder {

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/compare.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index beb22e7..1b84603 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -261,7 +261,7 @@ class RangeEqualsVisitor {
   }
 
   Status Visit(const NullArray& left) {
-    UNUSED(left);
+    ARROW_UNUSED(left);
     result_ = true;
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/io/hdfs.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/io/hdfs.h b/cpp/src/arrow/io/hdfs.h
index f5ed082..431ff5e 100644
--- a/cpp/src/arrow/io/hdfs.h
+++ b/cpp/src/arrow/io/hdfs.h
@@ -169,7 +169,7 @@ class ARROW_EXPORT HadoopFileSystem : public FileSystem {
   std::unique_ptr<HadoopFileSystemImpl> impl_;
 
   HadoopFileSystem();
-  DISALLOW_COPY_AND_ASSIGN(HadoopFileSystem);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(HadoopFileSystem);
 };
 
 #ifndef ARROW_NO_DEPRECATED_API
@@ -211,7 +211,7 @@ class ARROW_EXPORT HdfsReadableFile : public RandomAccessFile {
 
   friend class HadoopFileSystem::HadoopFileSystemImpl;
 
-  DISALLOW_COPY_AND_ASSIGN(HdfsReadableFile);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(HdfsReadableFile);
 };
 
 // Naming this file OutputStream because it does not support seeking (like the
@@ -238,7 +238,7 @@ class ARROW_EXPORT HdfsOutputStream : public OutputStream {
 
   HdfsOutputStream();
 
-  DISALLOW_COPY_AND_ASSIGN(HdfsOutputStream);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(HdfsOutputStream);
 };
 
 Status ARROW_EXPORT HaveLibHdfs();

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/io/interfaces.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/io/interfaces.h b/cpp/src/arrow/io/interfaces.h
index dd3f32d..59aed75 100644
--- a/cpp/src/arrow/io/interfaces.h
+++ b/cpp/src/arrow/io/interfaces.h
@@ -79,9 +79,6 @@ class ARROW_EXPORT FileInterface {
   FileInterface() {}
   FileMode::type mode_;
   void set_mode(FileMode::type mode) { mode_ = mode; }
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(FileInterface);
 };
 
 class ARROW_EXPORT Seekable {

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/ipc/dictionary.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/dictionary.h b/cpp/src/arrow/ipc/dictionary.h
index 13cf7a0..4130e9b 100644
--- a/cpp/src/arrow/ipc/dictionary.h
+++ b/cpp/src/arrow/ipc/dictionary.h
@@ -68,7 +68,7 @@ class ARROW_EXPORT DictionaryMemo {
   // Map of dictionary id to dictionary array
   DictionaryMap id_to_dictionary_;
 
-  DISALLOW_COPY_AND_ASSIGN(DictionaryMemo);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(DictionaryMemo);
 };
 
 }  // namespace ipc

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/ipc/message.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/message.h b/cpp/src/arrow/ipc/message.h
index 6fb8de7..4bc4384 100644
--- a/cpp/src/arrow/ipc/message.h
+++ b/cpp/src/arrow/ipc/message.h
@@ -117,7 +117,7 @@ class ARROW_EXPORT Message {
   class MessageImpl;
   std::unique_ptr<MessageImpl> impl_;
 
-  DISALLOW_COPY_AND_ASSIGN(Message);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Message);
 };
 
 ARROW_EXPORT std::string FormatMessageType(Message::Type type);

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/python/arrow_to_pandas.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index fe4d63b..8ba3f46 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -260,7 +260,7 @@ class PandasBlock {
   int64_t* placement_data_;
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(PandasBlock);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(PandasBlock);
 };
 
 template <typename T>

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/python/common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index 7f94f95..e315771 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -56,7 +56,7 @@ class ARROW_EXPORT PyAcquireGIL {
  private:
   bool acquired_gil_;
   PyGILState_STATE state_;
-  DISALLOW_COPY_AND_ASSIGN(PyAcquireGIL);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(PyAcquireGIL);
 };
 
 #define PYARROW_IS_PY2 PY_MAJOR_VERSION <= 2

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/tensor.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/tensor.h b/cpp/src/arrow/tensor.h
index d1cd3cf..4e4c6b8 100644
--- a/cpp/src/arrow/tensor.h
+++ b/cpp/src/arrow/tensor.h
@@ -111,7 +111,7 @@ class ARROW_EXPORT Tensor {
   std::vector<std::string> dim_names_;
 
  private:
-  DISALLOW_COPY_AND_ASSIGN(Tensor);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(Tensor);
 };
 
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 7026aee..aeb6130 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -157,9 +157,6 @@ class ARROW_EXPORT DataType {
  protected:
   Type::type id_;
   std::vector<std::shared_ptr<Field>> children_;
-
- private:
-  DISALLOW_COPY_AND_ASSIGN(DataType);
 };
 
 // TODO(wesm): Remove this from parquet-cpp

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/bit-stream-utils.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/bit-stream-utils.h b/cpp/src/arrow/util/bit-stream-utils.h
index d312fef..8c1f067 100644
--- a/cpp/src/arrow/util/bit-stream-utils.h
+++ b/cpp/src/arrow/util/bit-stream-utils.h
@@ -26,8 +26,8 @@
 
 #include "arrow/util/bit-util.h"
 #include "arrow/util/bpacking.h"
-#include "arrow/util/compiler-util.h"
 #include "arrow/util/logging.h"
+#include "arrow/util/macros.h"
 
 namespace arrow {
 
@@ -171,12 +171,13 @@ inline bool BitWriter::PutValue(uint64_t v, int num_bits) {
   DCHECK_LE(num_bits, 32);
   DCHECK_EQ(v >> num_bits, 0) << "v = " << v << ", num_bits = " << num_bits;
 
-  if (UNLIKELY(byte_offset_ * 8 + bit_offset_ + num_bits > max_bytes_ * 8)) return false;
+  if (ARROW_PREDICT_FALSE(byte_offset_ * 8 + bit_offset_ + num_bits > max_bytes_ * 8))
+    return false;
 
   buffered_values_ |= v << bit_offset_;
   bit_offset_ += num_bits;
 
-  if (UNLIKELY(bit_offset_ >= 64)) {
+  if (ARROW_PREDICT_FALSE(bit_offset_ >= 64)) {
     // Flush buffered_values_ and write out bits of v that did not fit
     memcpy(buffer_ + byte_offset_, &buffered_values_, 8);
     buffered_values_ = 0;
@@ -247,7 +248,7 @@ inline void GetValue_(int num_bits, T* v, int max_bytes, const uint8_t* buffer,
     *bit_offset -= 64;
 
     int bytes_remaining = max_bytes - *byte_offset;
-    if (LIKELY(bytes_remaining >= 8)) {
+    if (ARROW_PREDICT_TRUE(bytes_remaining >= 8)) {
       memcpy(buffered_values, buffer + *byte_offset, 8);
     } else {
       memcpy(buffered_values, buffer + *byte_offset, bytes_remaining);
@@ -293,7 +294,7 @@ inline int BitReader::GetBatch(int num_bits, T* v, int batch_size) {
   }
 
   int i = 0;
-  if (UNLIKELY(bit_offset != 0)) {
+  if (ARROW_PREDICT_FALSE(bit_offset != 0)) {
     for (; i < batch_size && bit_offset != 0; ++i) {
       detail::GetValue_(num_bits, &v[i], max_bytes, buffer, &bit_offset, &byte_offset,
                         &buffered_values);
@@ -355,7 +356,8 @@ template <typename T>
 inline bool BitReader::GetAligned(int num_bytes, T* v) {
   DCHECK_LE(num_bytes, static_cast<int>(sizeof(T)));
   int bytes_read = static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
-  if (UNLIKELY(byte_offset_ + bytes_read + num_bytes > max_bytes_)) return false;
+  if (ARROW_PREDICT_FALSE(byte_offset_ + bytes_read + num_bytes > max_bytes_))
+    return false;
 
   // Advance byte_offset to next unread byte and read num_bytes
   byte_offset_ += bytes_read;
@@ -365,7 +367,7 @@ inline bool BitReader::GetAligned(int num_bytes, T* v) {
   // Reset buffered_values_
   bit_offset_ = 0;
   int bytes_remaining = max_bytes_ - byte_offset_;
-  if (LIKELY(bytes_remaining >= 8)) {
+  if (ARROW_PREDICT_TRUE(bytes_remaining >= 8)) {
     memcpy(&buffered_values_, buffer_ + byte_offset_, 8);
   } else {
     memcpy(&buffered_values_, buffer_ + byte_offset_, bytes_remaining);

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/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 52e78bf..da2826f 100644
--- a/cpp/src/arrow/util/bit-util.h
+++ b/cpp/src/arrow/util/bit-util.h
@@ -39,7 +39,7 @@
 #include <memory>
 #include <vector>
 
-#include "arrow/util/compiler-util.h"
+#include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
 
 #ifdef ARROW_USE_SSE
@@ -251,7 +251,7 @@ static inline int PopcountNoHw(uint64_t x) {
 /// Returns the number of set bits in x
 static inline int Popcount(uint64_t x) {
 #ifdef ARROW_USE_SSE
-  if (LIKELY(CpuInfo::IsSupported(CpuInfo::POPCNT))) {
+  if (ARROW_PREDICT_TRUE(CpuInfo::IsSupported(CpuInfo::POPCNT))) {
     return POPCNT_popcnt_u64(x);
   } else {
     return PopcountNoHw(x);
@@ -270,8 +270,8 @@ static inline int PopcountSigned(T v) {
 
 /// Returns the 'num_bits' least-significant bits of 'v'.
 static inline uint64_t TrailingBits(uint64_t v, int num_bits) {
-  if (UNLIKELY(num_bits == 0)) return 0;
-  if (UNLIKELY(num_bits >= 64)) return v;
+  if (ARROW_PREDICT_FALSE(num_bits == 0)) return 0;
+  if (ARROW_PREDICT_FALSE(num_bits >= 64)) return v;
   int n = 64 - num_bits;
   return (v << n) >> n;
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/compiler-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/compiler-util.h b/cpp/src/arrow/util/compiler-util.h
index ccbe545..820a9b0 100644
--- a/cpp/src/arrow/util/compiler-util.h
+++ b/cpp/src/arrow/util/compiler-util.h
@@ -15,45 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
+// Deprecated header, here for backwards compatibility in parquet-cpp
+
 #ifndef ARROW_UTIL_COMPILER_UTIL_H
 #define ARROW_UTIL_COMPILER_UTIL_H
 
-// Branch prediction macro hints for GCC
-#ifdef LIKELY
-#undef LIKELY
-#endif
-
-#ifdef UNLIKELY
-#undef UNLIKELY
-#endif
-
-#ifdef _MSC_VER
-#define LIKELY(expr) expr
-#define UNLIKELY(expr) expr
-#else
-#define LIKELY(expr) __builtin_expect(!!(expr), 1)
-#define UNLIKELY(expr) __builtin_expect(!!(expr), 0)
-#endif
-
-#define PREFETCH(addr) __builtin_prefetch(addr)
-
-// macros to disable padding
-// these macros are portable across different compilers and platforms
-//[https://github.com/google/flatbuffers/blob/master/include/flatbuffers/flatbuffers.h#L1355]
-#if defined(_MSC_VER)
-#define MANUALLY_ALIGNED_STRUCT(alignment) \
-  __pragma(pack(1));                       \
-  struct __declspec(align(alignment))
-#define STRUCT_END(name, size) \
-  __pragma(pack());            \
-  static_assert(sizeof(name) == size, "compiler breaks packing rules")
-#elif defined(__GNUC__) || defined(__clang__)
-#define MANUALLY_ALIGNED_STRUCT(alignment) \
-  _Pragma("pack(1)") struct __attribute__((aligned(alignment)))
-#define STRUCT_END(name, size) \
-  _Pragma("pack()") static_assert(sizeof(name) == size, "compiler breaks packing rules")
-#else
-#error Unknown compiler, please define structure alignment macros
-#endif
+#include "arrow/util/macros.h"
 
 #endif  // ARROW_UTIL_COMPILER_UTIL_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/compression_snappy.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/compression_snappy.cc b/cpp/src/arrow/util/compression_snappy.cc
index 542831f..e284bd4 100644
--- a/cpp/src/arrow/util/compression_snappy.cc
+++ b/cpp/src/arrow/util/compression_snappy.cc
@@ -17,11 +17,6 @@
 
 #include "arrow/util/compression_snappy.h"
 
-// Work around warning caused by Snappy include
-#ifdef DISALLOW_COPY_AND_ASSIGN
-#undef DISALLOW_COPY_AND_ASSIGN
-#endif
-
 #include <cstddef>
 #include <cstdint>
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/hash-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/hash-util.h b/cpp/src/arrow/util/hash-util.h
index 4c049c1..d5fb212 100644
--- a/cpp/src/arrow/util/hash-util.h
+++ b/cpp/src/arrow/util/hash-util.h
@@ -22,9 +22,9 @@
 
 #include <cstdint>
 
-#include "arrow/util/compiler-util.h"
 #include "arrow/util/cpu-info.h"
 #include "arrow/util/logging.h"
+#include "arrow/util/macros.h"
 #include "arrow/util/sse-util.h"
 
 namespace arrow {

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/key_value_metadata.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/key_value_metadata.h b/cpp/src/arrow/util/key_value_metadata.h
index 3d60213..adea2bc 100644
--- a/cpp/src/arrow/util/key_value_metadata.h
+++ b/cpp/src/arrow/util/key_value_metadata.h
@@ -55,7 +55,7 @@ class ARROW_EXPORT KeyValueMetadata {
   std::vector<std::string> keys_;
   std::vector<std::string> values_;
 
-  DISALLOW_COPY_AND_ASSIGN(KeyValueMetadata);
+  ARROW_DISALLOW_COPY_AND_ASSIGN(KeyValueMetadata);
 };
 
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/cpp/src/arrow/util/macros.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/macros.h b/cpp/src/arrow/util/macros.h
index fe2d768..9352149 100644
--- a/cpp/src/arrow/util/macros.h
+++ b/cpp/src/arrow/util/macros.h
@@ -19,13 +19,13 @@
 #define ARROW_UTIL_MACROS_H
 
 // From Google gutil
-#ifndef DISALLOW_COPY_AND_ASSIGN
-#define DISALLOW_COPY_AND_ASSIGN(TypeName) \
-  TypeName(const TypeName&) = delete;      \
+#ifndef ARROW_DISALLOW_COPY_AND_ASSIGN
+#define ARROW_DISALLOW_COPY_AND_ASSIGN(TypeName) \
+  TypeName(const TypeName&) = delete;            \
   TypeName& operator=(const TypeName&) = delete
 #endif
 
-#define UNUSED(x) (void)x
+#define ARROW_UNUSED(x) (void)x
 
 //
 // GCC can be told that a certain branch is not likely to be taken (for
@@ -37,14 +37,17 @@
 #define ARROW_PREDICT_FALSE(x) (__builtin_expect(x, 0))
 #define ARROW_PREDICT_TRUE(x) (__builtin_expect(!!(x), 1))
 #define ARROW_NORETURN __attribute__((noreturn))
+#define ARROW_PREFETCH(addr) __builtin_prefetch(addr)
 #elif defined(_MSC_VER)
 #define ARROW_NORETURN __declspec(noreturn)
 #define ARROW_PREDICT_FALSE(x) x
 #define ARROW_PREDICT_TRUE(x) x
+#define ARROW_PREFETCH(addr)
 #else
 #define ARROW_NORETURN
 #define ARROW_PREDICT_FALSE(x) x
 #define ARROW_PREDICT_TRUE(x) x
+#define ARROW_PREFETCH(addr)
 #endif
 
 #if (defined(__GNUC__) || defined(__APPLE__))
@@ -55,4 +58,25 @@
 #define ARROW_MUST_USE_RESULT
 #endif
 
+// macros to disable padding
+// these macros are portable across different compilers and platforms
+//[https://github.com/google/flatbuffers/blob/master/include/flatbuffers/flatbuffers.h#L1355]
+#if !defined(MANUALLY_ALIGNED_STRUCT)
+#if defined(_MSC_VER)
+#define MANUALLY_ALIGNED_STRUCT(alignment) \
+  __pragma(pack(1));                             \
+  struct __declspec(align(alignment))
+#define STRUCT_END(name, size) \
+  __pragma(pack());            \
+  static_assert(sizeof(name) == size, "compiler breaks packing rules")
+#elif defined(__GNUC__) || defined(__clang__)
+#define MANUALLY_ALIGNED_STRUCT(alignment) \
+  _Pragma("pack(1)") struct __attribute__((aligned(alignment)))
+#define STRUCT_END(name, size) \
+  _Pragma("pack()") static_assert(sizeof(name) == size, "compiler breaks packing rules")
+#else
+#error Unknown compiler, please define structure alignment macros
+#endif
+#endif // !defined(MANUALLY_ALIGNED_STRUCT)
+
 #endif  // ARROW_UTIL_MACROS_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/ebca1af3/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 e690778..f4c8a77 100644
--- a/cpp/src/arrow/util/rle-encoding.h
+++ b/cpp/src/arrow/util/rle-encoding.h
@@ -26,7 +26,7 @@
 
 #include "arrow/util/bit-stream-utils.h"
 #include "arrow/util/bit-util.h"
-#include "arrow/util/compiler-util.h"
+#include "arrow/util/macros.h"
 
 namespace arrow {
 
@@ -446,9 +446,9 @@ bool RleDecoder::NextCounts() {
 /// it decides whether they should be encoded as a literal or repeated run.
 inline bool RleEncoder::Put(uint64_t value) {
   DCHECK(bit_width_ == 64 || value < (1ULL << bit_width_));
-  if (UNLIKELY(buffer_full_)) return false;
+  if (ARROW_PREDICT_FALSE(buffer_full_)) return false;
 
-  if (LIKELY(current_value_ == value)) {
+  if (ARROW_PREDICT_TRUE(current_value_ == value)) {
     ++repeat_count_;
     if (repeat_count_ > 8) {
       // This is just a continuation of the current run, no need to buffer the