You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by we...@apache.org on 2017/06/27 21:07:37 UTC

[3/3] parquet-cpp git commit: PARQUET-1045: Remove code that's being moved to Apache Arrow in ARROW-1154

PARQUET-1045: Remove code that's being moved to Apache Arrow in ARROW-1154

I verified locally that this works on Linux with the current iteration of https://github.com/apache/arrow/pull/785, will update the commit in ThirdpartyToolchain.cmake after that is merged to get a passing build

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

Closes #364 from wesm/PARQUET-1045 and squashes the following commits:

fede8d9 [Wes McKinney] Update Arrow version to include ARROW-1159
b44b4ab [Wes McKinney] Fix benchmark. Only build Zlib on Windows when building Thrift
83d2289 [Wes McKinney] Update Arrow version
8914c16 [Wes McKinney] Remove code that was moved to Apache Arrow, refactoring


Project: http://git-wip-us.apache.org/repos/asf/parquet-cpp/repo
Commit: http://git-wip-us.apache.org/repos/asf/parquet-cpp/commit/2f5ef895
Tree: http://git-wip-us.apache.org/repos/asf/parquet-cpp/tree/2f5ef895
Diff: http://git-wip-us.apache.org/repos/asf/parquet-cpp/diff/2f5ef895

Branch: refs/heads/master
Commit: 2f5ef8957851fe13dfb1b8c67f7a6786730a404e
Parents: 84db929
Author: Wes McKinney <we...@twosigma.com>
Authored: Tue Jun 27 17:07:30 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Tue Jun 27 17:07:30 2017 -0400

----------------------------------------------------------------------
 CMakeLists.txt                             |    2 -
 benchmarks/decode_benchmark.cc             |    6 +-
 cmake_modules/ThirdpartyToolchain.cmake    |   53 +-
 src/parquet/arrow/reader.cc                |    6 +-
 src/parquet/arrow/writer.cc                |    9 +-
 src/parquet/column_reader.cc               |    7 +-
 src/parquet/column_reader.h                |   10 +-
 src/parquet/column_writer.cc               |    7 +-
 src/parquet/column_writer.h                |   10 +-
 src/parquet/encoding-internal.h            |   45 +-
 src/parquet/encoding-test.cc               |    5 +-
 src/parquet/encoding.h                     |    3 +-
 src/parquet/types.h                        |    3 +-
 src/parquet/util/CMakeLists.txt            |   10 -
 src/parquet/util/bit-stream-utils.h        |  170 --
 src/parquet/util/bit-stream-utils.inline.h |  258 --
 src/parquet/util/bit-util-test.cc          |  190 --
 src/parquet/util/bit-util.h                |  340 ---
 src/parquet/util/bpacking.h                | 3342 -----------------------
 src/parquet/util/compiler-util.h           |   63 -
 src/parquet/util/cpu-info.cc               |  208 --
 src/parquet/util/cpu-info.h                |   92 -
 src/parquet/util/hash-util.h               |  258 --
 src/parquet/util/logging.h                 |  105 +-
 src/parquet/util/memory.cc                 |    4 +-
 src/parquet/util/memory.h                  |    2 +
 src/parquet/util/rle-encoding.h            |  599 ----
 src/parquet/util/rle-test.cc               |  460 ----
 src/parquet/util/sse-util.h                |  237 --
 29 files changed, 93 insertions(+), 6411 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 47984e6..0627b00 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -618,8 +618,6 @@ set(LIBPARQUET_SRCS
 
   src/parquet/parquet_constants.cpp
   src/parquet/parquet_types.cpp
-
-  src/parquet/util/cpu-info.cc
   src/parquet/util/memory.cc
 )
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/benchmarks/decode_benchmark.cc
----------------------------------------------------------------------
diff --git a/benchmarks/decode_benchmark.cc b/benchmarks/decode_benchmark.cc
index 44776a8..5514d8b 100644
--- a/benchmarks/decode_benchmark.cc
+++ b/benchmarks/decode_benchmark.cc
@@ -40,10 +40,10 @@ class DeltaBitPackEncoder {
 
   uint8_t* Encode(int* encoded_len) {
     uint8_t* result = new uint8_t[10 * 1024 * 1024];
-    int num_mini_blocks = parquet::BitUtil::Ceil(num_values() - 1, mini_block_size_);
+    int num_mini_blocks = arrow::BitUtil::Ceil(num_values() - 1, mini_block_size_);
     uint8_t* mini_block_widths = NULL;
 
-    parquet::BitWriter writer(result, 10 * 1024 * 1024);
+    arrow::BitWriter writer(result, 10 * 1024 * 1024);
 
     // Writer the size of each block. We only use 1 block currently.
     writer.PutVlqInt(num_mini_blocks * mini_block_size_);
@@ -83,7 +83,7 @@ class DeltaBitPackEncoder {
 
       // The bit width for this block is the number of bits needed to store
       // (max_delta - min_delta).
-      int bit_width = parquet::BitUtil::NumRequiredBits(max_delta - min_delta);
+      int bit_width = arrow::BitUtil::NumRequiredBits(max_delta - min_delta);
       mini_block_widths[i] = bit_width;
 
       // Encode this mini blocking using min_delta and bit_width

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/cmake_modules/ThirdpartyToolchain.cmake
----------------------------------------------------------------------
diff --git a/cmake_modules/ThirdpartyToolchain.cmake b/cmake_modules/ThirdpartyToolchain.cmake
index 5c4e565..716debc 100644
--- a/cmake_modules/ThirdpartyToolchain.cmake
+++ b/cmake_modules/ThirdpartyToolchain.cmake
@@ -109,34 +109,6 @@ set(LIBS ${LIBS} ${Boost_LIBRARIES})
 # ----------------------------------------------------------------------
 # ZLIB
 
-set(ZLIB_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/zlib_ep/src/zlib_ep-install")
-set(ZLIB_HOME "${ZLIB_PREFIX}")
-set(ZLIB_INCLUDE_DIR "${ZLIB_PREFIX}/include")
-if (MSVC)
-  if (${UPPERCASE_BUILD_TYPE} STREQUAL "DEBUG")
-    set(ZLIB_STATIC_LIB_NAME zlibstaticd.lib)
-  else()
-    set(ZLIB_STATIC_LIB_NAME zlibstatic.lib)
-  endif()
-else()
-  set(ZLIB_STATIC_LIB_NAME libz.a)
-endif()
-set(ZLIB_STATIC_LIB "${ZLIB_PREFIX}/lib/${ZLIB_STATIC_LIB_NAME}")
-set(ZLIB_CMAKE_ARGS -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}
-  -DCMAKE_INSTALL_PREFIX=${ZLIB_PREFIX}
-  -DCMAKE_C_FLAGS=${EP_C_FLAGS}
-  -DBUILD_SHARED_LIBS=OFF)
-ExternalProject_Add(zlib_ep
-  URL "http://zlib.net/fossils/zlib-1.2.8.tar.gz"
-  BUILD_BYPRODUCTS "${ZLIB_STATIC_LIB}"
-  ${ZLIB_BUILD_BYPRODUCTS}
-  CMAKE_ARGS ${ZLIB_CMAKE_ARGS})
-
-include_directories(SYSTEM ${ZLIB_INCLUDE_DIR})
-add_library(zlibstatic STATIC IMPORTED)
-set_target_properties(zlibstatic PROPERTIES IMPORTED_LOCATION ${ZLIB_STATIC_LIB})
-add_dependencies(zlibstatic zlib_ep)
-
 # ----------------------------------------------------------------------
 # Thrift
 
@@ -144,6 +116,29 @@ add_dependencies(zlibstatic zlib_ep)
 find_package(Thrift)
 
 if (NOT THRIFT_FOUND)
+  set(ZLIB_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/zlib_ep/src/zlib_ep-install")
+  set(ZLIB_HOME "${ZLIB_PREFIX}")
+  set(ZLIB_INCLUDE_DIR "${ZLIB_PREFIX}/include")
+  if (MSVC)
+    if (${UPPERCASE_BUILD_TYPE} STREQUAL "DEBUG")
+      set(ZLIB_STATIC_LIB_NAME zlibstaticd.lib)
+    else()
+      set(ZLIB_STATIC_LIB_NAME zlibstatic.lib)
+    endif()
+  else()
+    set(ZLIB_STATIC_LIB_NAME libz.a)
+  endif()
+  set(ZLIB_STATIC_LIB "${ZLIB_PREFIX}/lib/${ZLIB_STATIC_LIB_NAME}")
+  set(ZLIB_CMAKE_ARGS -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}
+    -DCMAKE_INSTALL_PREFIX=${ZLIB_PREFIX}
+    -DCMAKE_C_FLAGS=${EP_C_FLAGS}
+    -DBUILD_SHARED_LIBS=OFF)
+  ExternalProject_Add(zlib_ep
+    URL "http://zlib.net/fossils/zlib-1.2.8.tar.gz"
+    BUILD_BYPRODUCTS "${ZLIB_STATIC_LIB}"
+    ${ZLIB_BUILD_BYPRODUCTS}
+    CMAKE_ARGS ${ZLIB_CMAKE_ARGS})
+
   set(THRIFT_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/thrift_ep/src/thrift_ep-install")
   set(THRIFT_HOME "${THRIFT_PREFIX}")
   set(THRIFT_INCLUDE_DIR "${THRIFT_PREFIX}/include")
@@ -341,7 +336,7 @@ if (NOT ARROW_FOUND)
     -DARROW_BUILD_TESTS=OFF)
 
   if ("$ENV{PARQUET_ARROW_VERSION}" STREQUAL "")
-    set(ARROW_VERSION "98f7cac6e162d9775d615d07b9867c1ec0030f82")
+    set(ARROW_VERSION "a58893882ac8acd1ac4a5036685cbf09a9a09673")
   else()
     set(ARROW_VERSION "$ENV{PARQUET_ARROW_VERSION}")
   endif()

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/arrow/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/reader.cc b/src/parquet/arrow/reader.cc
index a3a26c9..dd1c9d2 100644
--- a/src/parquet/arrow/reader.cc
+++ b/src/parquet/arrow/reader.cc
@@ -26,12 +26,12 @@
 #include <thread>
 #include <vector>
 
+#include "arrow/api.h"
+#include "arrow/util/bit-util.h"
+
 #include "parquet/arrow/schema.h"
-#include "parquet/util/bit-util.h"
 #include "parquet/util/schema-util.h"
 
-#include "arrow/api.h"
-
 using arrow::Array;
 using arrow::BooleanArray;
 using arrow::Column;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/arrow/writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/writer.cc b/src/parquet/arrow/writer.cc
index b8cb45c..1e3f6de 100644
--- a/src/parquet/arrow/writer.cc
+++ b/src/parquet/arrow/writer.cc
@@ -21,14 +21,13 @@
 #include <string>
 #include <vector>
 
-#include "parquet/util/bit-util.h"
-#include "parquet/util/logging.h"
-
-#include "parquet/arrow/schema.h"
-
 #include "arrow/api.h"
+#include "arrow/util/bit-util.h"
 #include "arrow/visitor_inline.h"
 
+#include "parquet/arrow/schema.h"
+#include "parquet/util/logging.h"
+
 using arrow::Array;
 using arrow::BinaryArray;
 using arrow::FixedSizeBinaryArray;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/column_reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_reader.cc b/src/parquet/column_reader.cc
index f63f6f1..ce6936d 100644
--- a/src/parquet/column_reader.cc
+++ b/src/parquet/column_reader.cc
@@ -21,10 +21,11 @@
 #include <cstdint>
 #include <memory>
 
+#include "arrow/util/rle-encoding.h"
+
 #include "parquet/column_page.h"
 #include "parquet/encoding-internal.h"
 #include "parquet/properties.h"
-#include "parquet/util/rle-encoding.h"
 
 using arrow::MemoryPool;
 
@@ -45,7 +46,7 @@ int LevelDecoder::SetData(Encoding::type encoding, int16_t max_level,
       num_bytes = *reinterpret_cast<const int32_t*>(data);
       const uint8_t* decoder_data = data + sizeof(int32_t);
       if (!rle_decoder_) {
-        rle_decoder_.reset(new RleDecoder(decoder_data, num_bytes, bit_width_));
+        rle_decoder_.reset(new ::arrow::RleDecoder(decoder_data, num_bytes, bit_width_));
       } else {
         rle_decoder_->Reset(decoder_data, num_bytes, bit_width_);
       }
@@ -55,7 +56,7 @@ int LevelDecoder::SetData(Encoding::type encoding, int16_t max_level,
       num_bytes =
           static_cast<int32_t>(BitUtil::Ceil(num_buffered_values * bit_width_, 8));
       if (!bit_packed_decoder_) {
-        bit_packed_decoder_.reset(new BitReader(data, num_bytes));
+        bit_packed_decoder_.reset(new ::arrow::BitReader(data, num_bytes));
       } else {
         bit_packed_decoder_->Reset(data, num_bytes);
       }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/column_reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_reader.h b/src/parquet/column_reader.h
index f4b8b02..e733d67 100644
--- a/src/parquet/column_reader.h
+++ b/src/parquet/column_reader.h
@@ -36,11 +36,15 @@
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"
 
-namespace parquet {
+namespace arrow {
 
 class BitReader;
 class RleDecoder;
 
+}  // namespace arrow
+
+namespace parquet {
+
 class PARQUET_EXPORT LevelDecoder {
  public:
   LevelDecoder();
@@ -58,8 +62,8 @@ class PARQUET_EXPORT LevelDecoder {
   int bit_width_;
   int num_values_remaining_;
   Encoding::type encoding_;
-  std::unique_ptr<RleDecoder> rle_decoder_;
-  std::unique_ptr<BitReader> bit_packed_decoder_;
+  std::unique_ptr<::arrow::RleDecoder> rle_decoder_;
+  std::unique_ptr<::arrow::BitReader> bit_packed_decoder_;
 };
 
 class PARQUET_EXPORT ColumnReader {

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/column_writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_writer.cc b/src/parquet/column_writer.cc
index c13d4a0..21550da 100644
--- a/src/parquet/column_writer.cc
+++ b/src/parquet/column_writer.cc
@@ -17,15 +17,20 @@
 
 #include "parquet/column_writer.h"
 
+#include "arrow/util/bit-util.h"
+#include "arrow/util/rle-encoding.h"
+
 #include "parquet/encoding-internal.h"
 #include "parquet/properties.h"
 #include "parquet/statistics.h"
 #include "parquet/util/logging.h"
 #include "parquet/util/memory.h"
-#include "parquet/util/rle-encoding.h"
 
 namespace parquet {
 
+using BitWriter = ::arrow::BitWriter;
+using RleEncoder = ::arrow::RleEncoder;
+
 LevelEncoder::LevelEncoder() {}
 LevelEncoder::~LevelEncoder() {}
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/column_writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_writer.h b/src/parquet/column_writer.h
index 4e113de..1637780 100644
--- a/src/parquet/column_writer.h
+++ b/src/parquet/column_writer.h
@@ -30,11 +30,15 @@
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"
 
-namespace parquet {
+namespace arrow {
 
 class BitWriter;
 class RleEncoder;
 
+}  // namespace arrow
+
+namespace parquet {
+
 class PARQUET_EXPORT LevelEncoder {
  public:
   LevelEncoder();
@@ -61,8 +65,8 @@ class PARQUET_EXPORT LevelEncoder {
   int bit_width_;
   int rle_length_;
   Encoding::type encoding_;
-  std::unique_ptr<RleEncoder> rle_encoder_;
-  std::unique_ptr<BitWriter> bit_packed_encoder_;
+  std::unique_ptr<::arrow::RleEncoder> rle_encoder_;
+  std::unique_ptr<::arrow::BitWriter> bit_packed_encoder_;
 };
 
 static constexpr int WRITE_BATCH_SIZE = 1000;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/encoding-internal.h
----------------------------------------------------------------------
diff --git a/src/parquet/encoding-internal.h b/src/parquet/encoding-internal.h
index 61b8e24..88d781f 100644
--- a/src/parquet/encoding-internal.h
+++ b/src/parquet/encoding-internal.h
@@ -24,21 +24,23 @@
 #include <memory>
 #include <vector>
 
-#include <arrow/util/bit-util.h>
+#include "arrow/util/bit-stream-utils.h"
+#include "arrow/util/bit-util.h"
+#include "arrow/util/cpu-info.h"
+#include "arrow/util/hash-util.h"
+#include "arrow/util/rle-encoding.h"
 
 #include "parquet/encoding.h"
 #include "parquet/exception.h"
 #include "parquet/schema.h"
 #include "parquet/types.h"
-#include "parquet/util/bit-stream-utils.inline.h"
-#include "parquet/util/bit-util.h"
-#include "parquet/util/cpu-info.h"
-#include "parquet/util/hash-util.h"
 #include "parquet/util/memory.h"
-#include "parquet/util/rle-encoding.h"
 
 namespace parquet {
 
+namespace BitUtil = ::arrow::BitUtil;
+using HashUtil = ::arrow::HashUtil;
+
 class ColumnDescriptor;
 
 // ----------------------------------------------------------------------
@@ -136,7 +138,7 @@ class PlainDecoder<BooleanType> : public Decoder<BooleanType> {
 
   virtual void SetData(int num_values, const uint8_t* data, int len) {
     num_values_ = num_values;
-    bit_reader_ = BitReader(data, len);
+    bit_reader_ = ::arrow::BitReader(data, len);
   }
 
   // Two flavors of bool decoding
@@ -161,7 +163,7 @@ class PlainDecoder<BooleanType> : public Decoder<BooleanType> {
   }
 
  private:
-  BitReader bit_reader_;
+  ::arrow::BitReader bit_reader_;
 };
 
 // ----------------------------------------------------------------------
@@ -196,7 +198,7 @@ class PlainEncoder<BooleanType> : public Encoder<BooleanType> {
         bits_available_(kInMemoryDefaultCapacity * 8),
         bits_buffer_(AllocateBuffer(pool, kInMemoryDefaultCapacity)),
         values_sink_(new InMemoryOutputStream(pool)) {
-    bit_writer_.reset(new BitWriter(
+    bit_writer_.reset(new ::arrow::BitWriter(
         bits_buffer_->mutable_data(), static_cast<int>(bits_buffer_->size())));
   }
 
@@ -260,7 +262,7 @@ class PlainEncoder<BooleanType> : public Encoder<BooleanType> {
 
  protected:
   int bits_available_;
-  std::unique_ptr<BitWriter> bit_writer_;
+  std::unique_ptr<::arrow::BitWriter> bit_writer_;
   std::shared_ptr<PoolBuffer> bits_buffer_;
   std::unique_ptr<InMemoryOutputStream> values_sink_;
 };
@@ -325,12 +327,13 @@ class DictionaryDecoder : public Decoder<Type> {
     uint8_t bit_width = *data;
     ++data;
     --len;
-    idx_decoder_ = RleDecoder(data, len, bit_width);
+    idx_decoder_ = ::arrow::RleDecoder(data, len, bit_width);
   }
 
   int Decode(T* buffer, int max_values) override {
     max_values = std::min(max_values, num_values_);
-    int decoded_values = idx_decoder_.GetBatchWithDict(dictionary_, buffer, max_values);
+    int decoded_values =
+        idx_decoder_.GetBatchWithDict(dictionary_.data(), buffer, max_values);
     if (decoded_values != max_values) { ParquetException::EofException(); }
     num_values_ -= max_values;
     return max_values;
@@ -338,8 +341,8 @@ class DictionaryDecoder : public Decoder<Type> {
 
   int DecodeSpaced(T* buffer, int num_values, int null_count, const uint8_t* valid_bits,
       int64_t valid_bits_offset) override {
-    int decoded_values = idx_decoder_.GetBatchWithDictSpaced(
-        dictionary_, buffer, num_values, null_count, valid_bits, valid_bits_offset);
+    int decoded_values = idx_decoder_.GetBatchWithDictSpaced(dictionary_.data(), buffer,
+        num_values, null_count, valid_bits, valid_bits_offset);
     if (decoded_values != num_values) { ParquetException::EofException(); }
     return decoded_values;
   }
@@ -354,7 +357,7 @@ class DictionaryDecoder : public Decoder<Type> {
   // pointers).
   std::shared_ptr<PoolBuffer> byte_array_data_;
 
-  RleDecoder idx_decoder_;
+  ::arrow::RleDecoder idx_decoder_;
 };
 
 template <typename Type>
@@ -446,7 +449,7 @@ class DictEncoder : public Encoder<DType> {
         dict_encoded_size_(0),
         type_length_(desc->type_length()) {
     hash_slots_.Assign(hash_table_size_, HASH_SLOT_EMPTY);
-    if (!CpuInfo::initialized()) { CpuInfo::Init(); }
+    if (!::arrow::CpuInfo::initialized()) { ::arrow::CpuInfo::Init(); }
   }
 
   virtual ~DictEncoder() { DCHECK(buffered_indices_.empty()); }
@@ -464,9 +467,9 @@ class DictEncoder : public Encoder<DType> {
     // reserve
     // an extra "RleEncoder::MinBufferSize" bytes. These extra bytes won't be used
     // but not reserving them would cause the encoder to fail.
-    return 1 + RleEncoder::MaxBufferSize(
+    return 1 + ::arrow::RleEncoder::MaxBufferSize(
                    bit_width(), static_cast<int>(buffered_indices_.size())) +
-           RleEncoder::MinBufferSize(bit_width());
+           ::arrow::RleEncoder::MinBufferSize(bit_width());
   }
 
   /// The minimum bit width required to encode the currently buffered indices.
@@ -727,7 +730,7 @@ inline int DictEncoder<DType>::WriteIndices(uint8_t* buffer, int buffer_len) {
   ++buffer;
   --buffer_len;
 
-  RleEncoder encoder(buffer, buffer_len, bit_width());
+  ::arrow::RleEncoder encoder(buffer, buffer_len, bit_width());
   for (int index : buffered_indices_) {
     if (!encoder.Put(index)) return -1;
   }
@@ -756,7 +759,7 @@ class DeltaBitPackDecoder : public Decoder<DType> {
 
   virtual void SetData(int num_values, const uint8_t* data, int len) {
     num_values_ = num_values;
-    decoder_ = BitReader(data, len);
+    decoder_ = ::arrow::BitReader(data, len);
     values_current_block_ = 0;
     values_current_mini_block_ = 0;
   }
@@ -819,7 +822,7 @@ class DeltaBitPackDecoder : public Decoder<DType> {
     return max_values;
   }
 
-  BitReader decoder_;
+  ::arrow::BitReader decoder_;
   int32_t values_current_block_;
   int32_t num_mini_blocks_;
   uint64_t values_per_mini_block_;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/encoding-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/encoding-test.cc b/src/parquet/encoding-test.cc
index 2e78036..dcd813d 100644
--- a/src/parquet/encoding-test.cc
+++ b/src/parquet/encoding-test.cc
@@ -22,10 +22,11 @@
 #include <string>
 #include <vector>
 
+#include "arrow/util/bit-util.h"
+
 #include "parquet/encoding-internal.h"
 #include "parquet/schema.h"
 #include "parquet/types.h"
-#include "parquet/util/bit-util.h"
 #include "parquet/util/memory.h"
 #include "parquet/util/test-common.h"
 
@@ -64,7 +65,7 @@ TEST(VectorBooleanTest, TestEncodeDecode) {
   ASSERT_EQ(nvalues, values_decoded);
 
   for (int i = 0; i < nvalues; ++i) {
-    ASSERT_EQ(draws[i], BitUtil::GetArrayBit(decode_data, i)) << i;
+    ASSERT_EQ(draws[i], BitUtil::GetBit(decode_data, i)) << i;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encoding.h b/src/parquet/encoding.h
index 7c51cfd..1417e98 100644
--- a/src/parquet/encoding.h
+++ b/src/parquet/encoding.h
@@ -21,12 +21,11 @@
 #include <cstdint>
 #include <memory>
 
-#include <arrow/util/bit-util.h>
+#include "arrow/util/bit-util.h"
 
 #include "parquet/exception.h"
 #include "parquet/schema.h"
 #include "parquet/types.h"
-#include "parquet/util/bit-util.h"
 #include "parquet/util/memory.h"
 
 namespace parquet {

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/types.h
----------------------------------------------------------------------
diff --git a/src/parquet/types.h b/src/parquet/types.h
index 7ec3825..38015c4 100644
--- a/src/parquet/types.h
+++ b/src/parquet/types.h
@@ -24,7 +24,8 @@
 #include <sstream>
 #include <string>
 
-#include "parquet/util/compiler-util.h"
+#include "arrow/util/compiler-util.h"
+
 #include "parquet/util/visibility.h"
 
 namespace parquet {

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/parquet/util/CMakeLists.txt b/src/parquet/util/CMakeLists.txt
index e8fbdc7..4717266 100644
--- a/src/parquet/util/CMakeLists.txt
+++ b/src/parquet/util/CMakeLists.txt
@@ -17,19 +17,11 @@
 
 # Headers: util
 install(FILES
-  bit-stream-utils.h
-  bit-stream-utils.inline.h
-  bit-util.h
   buffer-builder.h
-  compiler-util.h
-  cpu-info.h
-  hash-util.h
   logging.h
   macros.h
   memory.h
-  rle-encoding.h
   stopwatch.h
-  sse-util.h
   visibility.h
   DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/parquet/util")
 
@@ -52,7 +44,5 @@ if (PARQUET_BUILD_BENCHMARKS)
   endif()
 endif()
 
-ADD_PARQUET_TEST(bit-util-test)
 ADD_PARQUET_TEST(comparison-test)
 ADD_PARQUET_TEST(memory-test)
-ADD_PARQUET_TEST(rle-test)

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/util/bit-stream-utils.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/bit-stream-utils.h b/src/parquet/util/bit-stream-utils.h
deleted file mode 100644
index 497a960..0000000
--- a/src/parquet/util/bit-stream-utils.h
+++ /dev/null
@@ -1,170 +0,0 @@
-// 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.
-
-// From Apache Impala as of 2016-01-29
-
-#ifndef PARQUET_UTIL_BIT_STREAM_UTILS_H
-#define PARQUET_UTIL_BIT_STREAM_UTILS_H
-
-#include <algorithm>
-#include <cstdint>
-#include <string.h>
-
-#include "parquet/util/bit-util.h"
-#include "parquet/util/compiler-util.h"
-#include "parquet/util/logging.h"
-
-namespace parquet {
-
-/// Utility class to write bit/byte streams.  This class can write data to either be
-/// bit packed or byte aligned (and a single stream that has a mix of both).
-/// This class does not allocate memory.
-class BitWriter {
- public:
-  /// buffer: buffer to write bits to.  Buffer should be preallocated with
-  /// 'buffer_len' bytes.
-  BitWriter(uint8_t* buffer, int buffer_len) : buffer_(buffer), max_bytes_(buffer_len) {
-    Clear();
-  }
-
-  void Clear() {
-    buffered_values_ = 0;
-    byte_offset_ = 0;
-    bit_offset_ = 0;
-  }
-
-  /// The number of current bytes written, including the current byte (i.e. may include a
-  /// fraction of a byte). Includes buffered values.
-  int bytes_written() const {
-    return byte_offset_ + static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
-  }
-  uint8_t* buffer() const { return buffer_; }
-  int buffer_len() const { return max_bytes_; }
-
-  /// Writes a value to buffered_values_, flushing to buffer_ if necessary.  This is bit
-  /// packed.  Returns false if there was not enough space. num_bits must be <= 32.
-  bool PutValue(uint64_t v, int num_bits);
-
-  /// Writes v to the next aligned byte using num_bytes. If T is larger than
-  /// num_bytes, the extra high-order bytes will be ignored. Returns false if
-  /// there was not enough space.
-  template <typename T>
-  bool PutAligned(T v, int num_bytes);
-
-  /// Write a Vlq encoded int to the buffer.  Returns false if there was not enough
-  /// room.  The value is written byte aligned.
-  /// For more details on vlq:
-  /// en.wikipedia.org/wiki/Variable-length_quantity
-  bool PutVlqInt(uint32_t v);
-
-  // Writes an int zigzag encoded.
-  bool PutZigZagVlqInt(int32_t v);
-
-  /// Get a pointer to the next aligned byte and advance the underlying buffer
-  /// by num_bytes.
-  /// Returns NULL if there was not enough space.
-  uint8_t* GetNextBytePtr(int num_bytes = 1);
-
-  /// Flushes all buffered values to the buffer. Call this when done writing to
-  /// the buffer.  If 'align' is true, buffered_values_ is reset and any future
-  /// writes will be written to the next byte boundary.
-  void Flush(bool align = false);
-
- private:
-  uint8_t* buffer_;
-  int max_bytes_;
-
-  /// Bit-packed values are initially written to this variable before being memcpy'd to
-  /// buffer_. This is faster than writing values byte by byte directly to buffer_.
-  uint64_t buffered_values_;
-
-  int byte_offset_;  // Offset in buffer_
-  int bit_offset_;   // Offset in buffered_values_
-};
-
-/// Utility class to read bit/byte stream.  This class can read bits or bytes
-/// that are either byte aligned or not.  It also has utilities to read multiple
-/// bytes in one read (e.g. encoded int).
-class BitReader {
- public:
-  /// 'buffer' is the buffer to read from.  The buffer's length is 'buffer_len'.
-  BitReader(const uint8_t* buffer, int buffer_len)
-      : buffer_(buffer), max_bytes_(buffer_len), byte_offset_(0), bit_offset_(0) {
-    int num_bytes = std::min(8, max_bytes_ - byte_offset_);
-    memcpy(&buffered_values_, buffer_ + byte_offset_, num_bytes);
-  }
-
-  BitReader() : buffer_(NULL), max_bytes_(0) {}
-
-  void Reset(const uint8_t* buffer, int buffer_len) {
-    buffer_ = buffer;
-    max_bytes_ = buffer_len;
-    byte_offset_ = 0;
-    bit_offset_ = 0;
-    int num_bytes = std::min(8, max_bytes_ - byte_offset_);
-    memcpy(&buffered_values_, buffer_ + byte_offset_, num_bytes);
-  }
-
-  /// Gets the next value from the buffer.  Returns true if 'v' could be read or false if
-  /// there are not enough bytes left. num_bits must be <= 32.
-  template <typename T>
-  bool GetValue(int num_bits, T* v);
-
-  /// Get a number of values from the buffer. Return the number of values actually read.
-  template <typename T>
-  int GetBatch(int num_bits, T* v, int batch_size);
-
-  /// Reads a 'num_bytes'-sized value from the buffer and stores it in 'v'. T
-  /// needs to be a little-endian native type and big enough to store
-  /// 'num_bytes'. The value is assumed to be byte-aligned so the stream will
-  /// be advanced to the start of the next byte before 'v' is read. Returns
-  /// false if there are not enough bytes left.
-  template <typename T>
-  bool GetAligned(int num_bytes, T* v);
-
-  /// Reads a vlq encoded int from the stream.  The encoded int must start at
-  /// the beginning of a byte. Return false if there were not enough bytes in
-  /// the buffer.
-  bool GetVlqInt(int32_t* v);
-
-  // Reads a zigzag encoded int `into` v.
-  bool GetZigZagVlqInt(int32_t* v);
-
-  /// Returns the number of bytes left in the stream, not including the current
-  /// byte (i.e., there may be an additional fraction of a byte).
-  int bytes_left() {
-    return max_bytes_ - (byte_offset_ + static_cast<int>(BitUtil::Ceil(bit_offset_, 8)));
-  }
-
-  /// Maximum byte length of a vlq encoded int
-  static const int MAX_VLQ_BYTE_LEN = 5;
-
- private:
-  const uint8_t* buffer_;
-  int max_bytes_;
-
-  /// Bytes are memcpy'd from buffer_ and values are read from this variable. This is
-  /// faster than reading values byte by byte directly from buffer_.
-  uint64_t buffered_values_;
-
-  int byte_offset_;  // Offset in buffer_
-  int bit_offset_;   // Offset in buffered_values_
-};
-
-}  // namespace parquet
-
-#endif  // PARQUET_UTIL_BIT_STREAM_UTILS_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/util/bit-stream-utils.inline.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/bit-stream-utils.inline.h b/src/parquet/util/bit-stream-utils.inline.h
deleted file mode 100644
index 5db1639..0000000
--- a/src/parquet/util/bit-stream-utils.inline.h
+++ /dev/null
@@ -1,258 +0,0 @@
-// 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.
-
-// From Apache Impala as of 2016-01-29
-
-#ifndef PARQUET_UTIL_BIT_STREAM_UTILS_INLINE_H
-#define PARQUET_UTIL_BIT_STREAM_UTILS_INLINE_H
-
-#include <algorithm>
-
-#include "parquet/util/bit-stream-utils.h"
-#include "parquet/util/bpacking.h"
-
-namespace parquet {
-
-inline bool BitWriter::PutValue(uint64_t v, int num_bits) {
-  // TODO: revisit this limit if necessary (can be raised to 64 by fixing some edge cases)
-  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;
-
-  buffered_values_ |= v << bit_offset_;
-  bit_offset_ += num_bits;
-
-  if (UNLIKELY(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;
-    byte_offset_ += 8;
-    bit_offset_ -= 64;
-    buffered_values_ = v >> (num_bits - bit_offset_);
-  }
-  DCHECK_LT(bit_offset_, 64);
-  return true;
-}
-
-inline void BitWriter::Flush(bool align) {
-  int num_bytes = static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
-  DCHECK_LE(byte_offset_ + num_bytes, max_bytes_);
-  memcpy(buffer_ + byte_offset_, &buffered_values_, num_bytes);
-
-  if (align) {
-    buffered_values_ = 0;
-    byte_offset_ += num_bytes;
-    bit_offset_ = 0;
-  }
-}
-
-inline uint8_t* BitWriter::GetNextBytePtr(int num_bytes) {
-  Flush(/* align */ true);
-  DCHECK_LE(byte_offset_, max_bytes_);
-  if (byte_offset_ + num_bytes > max_bytes_) return NULL;
-  uint8_t* ptr = buffer_ + byte_offset_;
-  byte_offset_ += num_bytes;
-  return ptr;
-}
-
-template <typename T>
-inline bool BitWriter::PutAligned(T val, int num_bytes) {
-  uint8_t* ptr = GetNextBytePtr(num_bytes);
-  if (ptr == NULL) return false;
-  memcpy(ptr, &val, num_bytes);
-  return true;
-}
-
-inline bool BitWriter::PutVlqInt(uint32_t v) {
-  bool result = true;
-  while ((v & 0xFFFFFF80) != 0L) {
-    result &= PutAligned<uint8_t>((v & 0x7F) | 0x80, 1);
-    v >>= 7;
-  }
-  result &= PutAligned<uint8_t>(v & 0x7F, 1);
-  return result;
-}
-
-template <typename T>
-inline void GetValue_(int num_bits, T* v, int max_bytes, const uint8_t* buffer,
-    int* bit_offset, int* byte_offset, uint64_t* buffered_values) {
-#ifdef _MSC_VER
-#pragma warning(push)
-#pragma warning(disable : 4800)
-#endif
-  *v = static_cast<T>(
-      BitUtil::TrailingBits(*buffered_values, *bit_offset + num_bits) >> *bit_offset);
-#ifdef _MSC_VER
-#pragma warning(pop)
-#endif
-  *bit_offset += num_bits;
-  if (*bit_offset >= 64) {
-    *byte_offset += 8;
-    *bit_offset -= 64;
-
-    int bytes_remaining = max_bytes - *byte_offset;
-    if (LIKELY(bytes_remaining >= 8)) {
-      memcpy(buffered_values, buffer + *byte_offset, 8);
-    } else {
-      memcpy(buffered_values, buffer + *byte_offset, bytes_remaining);
-    }
-#ifdef _MSC_VER
-#pragma warning(push)
-#pragma warning(disable : 4800 4805)
-#endif
-    // Read bits of v that crossed into new buffered_values_
-    *v |= BitUtil::TrailingBits(*buffered_values, *bit_offset)
-          << (num_bits - *bit_offset);
-#ifdef _MSC_VER
-#pragma warning(pop)
-#endif
-    DCHECK_LE(*bit_offset, 64);
-  }
-}
-
-template <typename T>
-inline bool BitReader::GetValue(int num_bits, T* v) {
-  return GetBatch(num_bits, v, 1) == 1;
-}
-
-template <typename T>
-inline int BitReader::GetBatch(int num_bits, T* v, int batch_size) {
-  DCHECK(buffer_ != NULL);
-  // TODO: revisit this limit if necessary
-  DCHECK_LE(num_bits, 32);
-  DCHECK_LE(num_bits, static_cast<int>(sizeof(T) * 8));
-
-  int bit_offset = bit_offset_;
-  int byte_offset = byte_offset_;
-  uint64_t buffered_values = buffered_values_;
-  int max_bytes = max_bytes_;
-  const uint8_t* buffer = buffer_;
-
-  uint64_t needed_bits = num_bits * batch_size;
-  uint64_t remaining_bits = (max_bytes - byte_offset) * 8 - bit_offset;
-  if (remaining_bits < needed_bits) {
-    batch_size = static_cast<int>(remaining_bits) / num_bits;
-  }
-
-  int i = 0;
-  if (UNLIKELY(bit_offset != 0)) {
-    for (; i < batch_size && bit_offset != 0; ++i) {
-      GetValue_(num_bits, &v[i], max_bytes, buffer, &bit_offset, &byte_offset,
-          &buffered_values);
-    }
-  }
-
-  if (sizeof(T) == 4) {
-    int num_unpacked = unpack32(reinterpret_cast<const uint32_t*>(buffer + byte_offset),
-        reinterpret_cast<uint32_t*>(v + i), batch_size - i, num_bits);
-    i += num_unpacked;
-    byte_offset += num_unpacked * num_bits / 8;
-  } else {
-    const int buffer_size = 1024;
-    uint32_t unpack_buffer[buffer_size];
-    while (i < batch_size) {
-      int unpack_size = std::min(buffer_size, batch_size - i);
-      int num_unpacked = unpack32(reinterpret_cast<const uint32_t*>(buffer + byte_offset),
-          unpack_buffer, unpack_size, num_bits);
-      if (num_unpacked == 0) { break; }
-      for (int k = 0; k < num_unpacked; ++k) {
-#ifdef _MSC_VER
-#pragma warning(push)
-#pragma warning(disable : 4800)
-#endif
-        v[i + k] = unpack_buffer[k];
-#ifdef _MSC_VER
-#pragma warning(pop)
-#endif
-      }
-      i += num_unpacked;
-      byte_offset += num_unpacked * num_bits / 8;
-    }
-  }
-
-  int bytes_remaining = max_bytes - byte_offset;
-  if (bytes_remaining >= 8) {
-    memcpy(&buffered_values, buffer + byte_offset, 8);
-  } else {
-    memcpy(&buffered_values, buffer + byte_offset, bytes_remaining);
-  }
-
-  for (; i < batch_size; ++i) {
-    GetValue_(
-        num_bits, &v[i], max_bytes, buffer, &bit_offset, &byte_offset, &buffered_values);
-  }
-
-  bit_offset_ = bit_offset;
-  byte_offset_ = byte_offset;
-  buffered_values_ = buffered_values;
-
-  return batch_size;
-}
-
-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;
-
-  // Advance byte_offset to next unread byte and read num_bytes
-  byte_offset_ += bytes_read;
-  memcpy(v, buffer_ + byte_offset_, num_bytes);
-  byte_offset_ += num_bytes;
-
-  // Reset buffered_values_
-  bit_offset_ = 0;
-  int bytes_remaining = max_bytes_ - byte_offset_;
-  if (LIKELY(bytes_remaining >= 8)) {
-    memcpy(&buffered_values_, buffer_ + byte_offset_, 8);
-  } else {
-    memcpy(&buffered_values_, buffer_ + byte_offset_, bytes_remaining);
-  }
-  return true;
-}
-
-inline bool BitReader::GetVlqInt(int32_t* v) {
-  *v = 0;
-  int shift = 0;
-  int num_bytes = 0;
-  uint8_t byte = 0;
-  do {
-    if (!GetAligned<uint8_t>(1, &byte)) return false;
-    *v |= (byte & 0x7F) << shift;
-    shift += 7;
-    DCHECK_LE(++num_bytes, MAX_VLQ_BYTE_LEN);
-  } while ((byte & 0x80) != 0);
-  return true;
-}
-
-inline bool BitWriter::PutZigZagVlqInt(int32_t v) {
-  uint32_t u = (v << 1) ^ (v >> 31);
-  return PutVlqInt(u);
-}
-
-inline bool BitReader::GetZigZagVlqInt(int32_t* v) {
-  int32_t u_signed;
-  if (!GetVlqInt(&u_signed)) return false;
-  uint32_t u = static_cast<uint32_t>(u_signed);
-  *reinterpret_cast<uint32_t*>(v) = (u >> 1) ^ -(static_cast<int32_t>(u & 1));
-  return true;
-}
-
-}  // namespace parquet
-
-#endif  // PARQUET_UTIL_BIT_STREAM_UTILS_INLINE_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/util/bit-util-test.cc b/src/parquet/util/bit-util-test.cc
deleted file mode 100644
index bc3e182..0000000
--- a/src/parquet/util/bit-util-test.cc
+++ /dev/null
@@ -1,190 +0,0 @@
-// 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.
-
-// From Apache Impala as of 2016-01-29
-
-#include <gtest/gtest.h>
-#include <limits.h>
-#include <stdio.h>
-#include <stdlib.h>
-
-#include <boost/utility.hpp>
-
-#include <iostream>
-
-#include "parquet/util/bit-stream-utils.inline.h"
-#include "parquet/util/bit-util.h"
-#include "parquet/util/cpu-info.h"
-
-namespace parquet {
-
-static void ensure_cpu_info_initialized() {
-  if (!CpuInfo::initialized()) { CpuInfo::Init(); }
-}
-
-TEST(BitUtil, Ceil) {
-  EXPECT_EQ(BitUtil::Ceil(0, 1), 0);
-  EXPECT_EQ(BitUtil::Ceil(1, 1), 1);
-  EXPECT_EQ(BitUtil::Ceil(1, 2), 1);
-  EXPECT_EQ(BitUtil::Ceil(1, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(7, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(8, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(9, 8), 2);
-  EXPECT_EQ(BitUtil::Ceil(9, 9), 1);
-  EXPECT_EQ(BitUtil::Ceil(10000000000, 10), 1000000000);
-  EXPECT_EQ(BitUtil::Ceil(10, 10000000000), 1);
-  EXPECT_EQ(BitUtil::Ceil(100000000000, 10000000000), 10);
-}
-
-TEST(BitUtil, RoundUp) {
-  EXPECT_EQ(BitUtil::RoundUp(0, 1), 0);
-  EXPECT_EQ(BitUtil::RoundUp(1, 1), 1);
-  EXPECT_EQ(BitUtil::RoundUp(1, 2), 2);
-  EXPECT_EQ(BitUtil::RoundUp(6, 2), 6);
-  EXPECT_EQ(BitUtil::RoundUp(7, 3), 9);
-  EXPECT_EQ(BitUtil::RoundUp(9, 9), 9);
-  EXPECT_EQ(BitUtil::RoundUp(10000000001, 10), 10000000010);
-  EXPECT_EQ(BitUtil::RoundUp(10, 10000000000), 10000000000);
-  EXPECT_EQ(BitUtil::RoundUp(100000000000, 10000000000), 100000000000);
-}
-
-TEST(BitUtil, RoundDown) {
-  EXPECT_EQ(BitUtil::RoundDown(0, 1), 0);
-  EXPECT_EQ(BitUtil::RoundDown(1, 1), 1);
-  EXPECT_EQ(BitUtil::RoundDown(1, 2), 0);
-  EXPECT_EQ(BitUtil::RoundDown(6, 2), 6);
-  EXPECT_EQ(BitUtil::RoundDown(7, 3), 6);
-  EXPECT_EQ(BitUtil::RoundDown(9, 9), 9);
-  EXPECT_EQ(BitUtil::RoundDown(10000000001, 10), 10000000000);
-  EXPECT_EQ(BitUtil::RoundDown(10, 10000000000), 0);
-  EXPECT_EQ(BitUtil::RoundDown(100000000000, 10000000000), 100000000000);
-}
-
-TEST(BitUtil, Popcount) {
-  ensure_cpu_info_initialized();
-
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(0 1 0 1 0 1 0 1)), 4);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(0 1 0 1 0 1 0 1)), 4);
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(1 1 1 1 0 1 0 1)), 6);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(1 1 1 1 0 1 0 1)), 6);
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(1 1 1 1 1 1 1 1)), 8);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(1 1 1 1 1 1 1 1)), 8);
-  EXPECT_EQ(BitUtil::Popcount(0), 0);
-  EXPECT_EQ(BitUtil::PopcountNoHw(0), 0);
-}
-
-TEST(BitUtil, TrailingBits) {
-  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 0), 0);
-  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 1), 1);
-  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 64),
-      BOOST_BINARY(1 1 1 1 1 1 1 1));
-  EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 100),
-      BOOST_BINARY(1 1 1 1 1 1 1 1));
-  EXPECT_EQ(BitUtil::TrailingBits(0, 1), 0);
-  EXPECT_EQ(BitUtil::TrailingBits(0, 64), 0);
-  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 0), 0);
-  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 63), 0);
-  EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 64), 1LL << 63);
-}
-
-TEST(BitUtil, ByteSwap) {
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint32_t>(0)), 0);
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint32_t>(0x11223344)), 0x44332211);
-
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<int32_t>(0)), 0);
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<int32_t>(0x11223344)), 0x44332211);
-
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint64_t>(0)), 0);
-  EXPECT_EQ(
-      BitUtil::ByteSwap(static_cast<uint64_t>(0x1122334455667788)), 0x8877665544332211);
-
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<int64_t>(0)), 0);
-  EXPECT_EQ(
-      BitUtil::ByteSwap(static_cast<int64_t>(0x1122334455667788)), 0x8877665544332211);
-
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<int16_t>(0)), 0);
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<int16_t>(0x1122)), 0x2211);
-
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint16_t>(0)), 0);
-  EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint16_t>(0x1122)), 0x2211);
-}
-
-TEST(BitUtil, Log2) {
-  EXPECT_EQ(BitUtil::Log2(1), 0);
-  EXPECT_EQ(BitUtil::Log2(2), 1);
-  EXPECT_EQ(BitUtil::Log2(3), 2);
-  EXPECT_EQ(BitUtil::Log2(4), 2);
-  EXPECT_EQ(BitUtil::Log2(5), 3);
-  EXPECT_EQ(BitUtil::Log2(INT_MAX), 31);
-  EXPECT_EQ(BitUtil::Log2(UINT_MAX), 32);
-  EXPECT_EQ(BitUtil::Log2(ULLONG_MAX), 64);
-}
-
-TEST(BitUtil, RoundUpToPowerOf2) {
-  EXPECT_EQ(BitUtil::RoundUpToPowerOf2(7, 8), 8);
-  EXPECT_EQ(BitUtil::RoundUpToPowerOf2(8, 8), 8);
-  EXPECT_EQ(BitUtil::RoundUpToPowerOf2(9, 8), 16);
-}
-
-TEST(BitUtil, RoundDownToPowerOf2) {
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(7, 8), 0);
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(8, 8), 8);
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(9, 8), 8);
-}
-
-TEST(BitUtil, RoundUpDown) {
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(7), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(8), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(9), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(7), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(8), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(9), 1);
-
-  EXPECT_EQ(BitUtil::RoundUpNumi32(31), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi32(32), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi32(33), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(31), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(32), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(33), 1);
-
-  EXPECT_EQ(BitUtil::RoundUpNumi64(63), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi64(64), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi64(65), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(63), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(64), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(65), 1);
-}
-
-void TestZigZag(int32_t v) {
-  uint8_t buffer[BitReader::MAX_VLQ_BYTE_LEN];
-  BitWriter writer(buffer, sizeof(buffer));
-  BitReader reader(buffer, sizeof(buffer));
-  writer.PutZigZagVlqInt(v);
-  int32_t result;
-  EXPECT_TRUE(reader.GetZigZagVlqInt(&result));
-  EXPECT_EQ(v, result);
-}
-
-TEST(BitStreamUtil, ZigZag) {
-  TestZigZag(0);
-  TestZigZag(1);
-  TestZigZag(-1);
-  TestZigZag(std::numeric_limits<int32_t>::max());
-  TestZigZag(-std::numeric_limits<int32_t>::max());
-}
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/2f5ef895/src/parquet/util/bit-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/bit-util.h b/src/parquet/util/bit-util.h
deleted file mode 100644
index e315b5f..0000000
--- a/src/parquet/util/bit-util.h
+++ /dev/null
@@ -1,340 +0,0 @@
-// 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.
-
-// From Apache Impala as of 2016-01-29
-
-#ifndef PARQUET_UTIL_BIT_UTIL_H
-#define PARQUET_UTIL_BIT_UTIL_H
-
-#if defined(__APPLE__)
-#include <machine/endian.h>
-#elif defined(_WIN32)
-#define __LITTLE_ENDIAN 1
-#else
-#include <endian.h>
-#endif
-
-#if defined(_MSC_VER)
-#define PARQUET_BYTE_SWAP64 _byteswap_uint64
-#define PARQUET_BYTE_SWAP32 _byteswap_ulong
-#else
-#define PARQUET_BYTE_SWAP64 __builtin_bswap64
-#define PARQUET_BYTE_SWAP32 __builtin_bswap32
-#endif
-
-#include <cstdint>
-
-#include "parquet/util/compiler-util.h"
-
-#ifdef PARQUET_USE_SSE
-#include "parquet/util/cpu-info.h"
-#include "parquet/util/sse-util.h"
-#endif
-
-namespace parquet {
-
-#define INIT_BITSET(valid_bits_vector, valid_bits_index)        \
-  int byte_offset_##valid_bits_vector = (valid_bits_index) / 8; \
-  int bit_offset_##valid_bits_vector = (valid_bits_index) % 8;  \
-  uint8_t bitset_##valid_bits_vector = valid_bits_vector[byte_offset_##valid_bits_vector];
-
-#define READ_NEXT_BITSET(valid_bits_vector)                                          \
-  bit_offset_##valid_bits_vector++;                                                  \
-  if (bit_offset_##valid_bits_vector == 8) {                                         \
-    bit_offset_##valid_bits_vector = 0;                                              \
-    byte_offset_##valid_bits_vector++;                                               \
-    bitset_##valid_bits_vector = valid_bits_vector[byte_offset_##valid_bits_vector]; \
-  }
-
-// TODO(wesm): The source from Impala was depending on boost::make_unsigned
-//
-// We add a partial stub implementation here
-
-template <typename T>
-struct make_unsigned {};
-
-template <>
-struct make_unsigned<int8_t> {
-  typedef uint8_t type;
-};
-
-template <>
-struct make_unsigned<int16_t> {
-  typedef uint16_t type;
-};
-
-template <>
-struct make_unsigned<int32_t> {
-  typedef uint32_t type;
-};
-
-template <>
-struct make_unsigned<int64_t> {
-  typedef uint64_t type;
-};
-
-/// Utility class to do standard bit tricks
-class BitUtil {
- public:
-  /// Returns the ceil of value/divisor
-  static inline int64_t Ceil(int64_t value, int64_t divisor) {
-    return value / divisor + (value % divisor != 0);
-  }
-
-  /// Returns 'value' rounded up to the nearest multiple of 'factor'
-  static inline int64_t RoundUp(int64_t value, int64_t factor) {
-    return (value + (factor - 1)) / factor * factor;
-  }
-
-  /// Returns 'value' rounded down to the nearest multiple of 'factor'
-  static inline int64_t RoundDown(int64_t value, int64_t factor) {
-    return (value / factor) * factor;
-  }
-
-  /// Returns the smallest power of two that contains v. Taken from
-  /// http://graphics.stanford.edu/~seander/bithacks.html#RoundUpPowerOf2
-  /// TODO: Pick a better name, as it is not clear what happens when the input is
-  /// already a power of two.
-  static inline int64_t NextPowerOfTwo(int64_t v) {
-    --v;
-    v |= v >> 1;
-    v |= v >> 2;
-    v |= v >> 4;
-    v |= v >> 8;
-    v |= v >> 16;
-    v |= v >> 32;
-    ++v;
-    return v;
-  }
-
-  /// Returns 'value' rounded up to the nearest multiple of 'factor' when factor is
-  /// a power of two
-  static inline int RoundUpToPowerOf2(int value, int factor) {
-    // DCHECK((factor > 0) && ((factor & (factor - 1)) == 0));
-    return (value + (factor - 1)) & ~(factor - 1);
-  }
-
-  static inline int RoundDownToPowerOf2(int value, int factor) {
-    // DCHECK((factor > 0) && ((factor & (factor - 1)) == 0));
-    return value & ~(factor - 1);
-  }
-
-  /// Specialized round up and down functions for frequently used factors,
-  /// like 8 (bits->bytes), 32 (bits->i32), and 64 (bits->i64).
-  /// Returns the rounded up number of bytes that fit the number of bits.
-  static inline uint32_t RoundUpNumBytes(uint32_t bits) { return (bits + 7) >> 3; }
-
-  /// Returns the rounded down number of bytes that fit the number of bits.
-  static inline uint32_t RoundDownNumBytes(uint32_t bits) { return bits >> 3; }
-
-  /// Returns the rounded up to 32 multiple. Used for conversions of bits to i32.
-  static inline uint32_t RoundUpNumi32(uint32_t bits) { return (bits + 31) >> 5; }
-
-  /// Returns the rounded up 32 multiple.
-  static inline uint32_t RoundDownNumi32(uint32_t bits) { return bits >> 5; }
-
-  /// Returns the rounded up to 64 multiple. Used for conversions of bits to i64.
-  static inline uint32_t RoundUpNumi64(uint32_t bits) { return (bits + 63) >> 6; }
-
-  /// Returns the rounded down to 64 multiple.
-  static inline uint32_t RoundDownNumi64(uint32_t bits) { return bits >> 6; }
-
-  /// Non hw accelerated pop count.
-  /// TODO: we don't use this in any perf sensitive code paths currently.  There
-  /// might be a much faster way to implement this.
-  static inline int PopcountNoHw(uint64_t x) {
-    int count = 0;
-    for (; x != 0; ++count)
-      x &= x - 1;
-    return count;
-  }
-
-  /// Returns the number of set bits in x
-  static inline int Popcount(uint64_t x) {
-#ifdef PARQUET_USE_SSE
-    if (LIKELY(CpuInfo::IsSupported(CpuInfo::POPCNT))) {
-      return POPCNT_popcnt_u64(x);
-    } else {
-      return PopcountNoHw(x);
-    }
-#else
-    return PopcountNoHw(x);
-#endif
-  }
-
-  // Compute correct population count for various-width signed integers
-  template <typename T>
-  static inline int PopcountSigned(T v) {
-    // Converting to same-width unsigned then extending preserves the bit pattern.
-    return BitUtil::Popcount(static_cast<typename make_unsigned<T>::type>(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;
-    int n = 64 - num_bits;
-    return (v << n) >> n;
-  }
-
-  /// Returns ceil(log2(x)).
-  /// TODO: this could be faster if we use __builtin_clz.  Fix this if this ever shows up
-  /// in a hot path.
-  static inline int Log2(uint64_t x) {
-    // DCHECK_GT(x, 0);
-    if (x == 1) return 0;
-    // Compute result = ceil(log2(x))
-    //                = floor(log2(x - 1)) + 1, for x > 1
-    // by finding the position of the most significant bit (1-indexed) of x - 1
-    // (floor(log2(n)) = MSB(n) (0-indexed))
-    --x;
-    int result = 1;
-    while (x >>= 1)
-      ++result;
-    return result;
-  }
-
-  /// Swaps the byte order (i.e. endianess)
-  static inline int64_t ByteSwap(int64_t value) { return PARQUET_BYTE_SWAP64(value); }
-  static inline uint64_t ByteSwap(uint64_t value) {
-    return static_cast<uint64_t>(PARQUET_BYTE_SWAP64(value));
-  }
-  static inline int32_t ByteSwap(int32_t value) { return PARQUET_BYTE_SWAP32(value); }
-  static inline uint32_t ByteSwap(uint32_t value) {
-    return static_cast<uint32_t>(PARQUET_BYTE_SWAP32(value));
-  }
-  static inline int16_t ByteSwap(int16_t value) {
-    return (((value >> 8) & 0xff) | ((value & 0xff) << 8));
-  }
-  static inline uint16_t ByteSwap(uint16_t value) {
-    return static_cast<uint16_t>(ByteSwap(static_cast<int16_t>(value)));
-  }
-
-  /// Write the swapped bytes into dst. Src and st cannot overlap.
-  static inline void ByteSwap(void* dst, const void* src, int len) {
-    switch (len) {
-      case 1:
-        *reinterpret_cast<int8_t*>(dst) = *reinterpret_cast<const int8_t*>(src);
-        return;
-      case 2:
-        *reinterpret_cast<int16_t*>(dst) =
-            ByteSwap(*reinterpret_cast<const int16_t*>(src));
-        return;
-      case 4:
-        *reinterpret_cast<int32_t*>(dst) =
-            ByteSwap(*reinterpret_cast<const int32_t*>(src));
-        return;
-      case 8:
-        *reinterpret_cast<int64_t*>(dst) =
-            ByteSwap(*reinterpret_cast<const int64_t*>(src));
-        return;
-      default:
-        break;
-    }
-
-    uint8_t* d = reinterpret_cast<uint8_t*>(dst);
-    const uint8_t* s = reinterpret_cast<const uint8_t*>(src);
-    for (int i = 0; i < len; ++i) {
-      d[i] = s[len - i - 1];
-    }
-  }
-
-/// Converts to big endian format (if not already in big endian) from the
-/// machine's native endian format.
-#if __BYTE_ORDER == __LITTLE_ENDIAN
-  static inline int64_t ToBigEndian(int64_t value) { return ByteSwap(value); }
-  static inline uint64_t ToBigEndian(uint64_t value) { return ByteSwap(value); }
-  static inline int32_t ToBigEndian(int32_t value) { return ByteSwap(value); }
-  static inline uint32_t ToBigEndian(uint32_t value) { return ByteSwap(value); }
-  static inline int16_t ToBigEndian(int16_t value) { return ByteSwap(value); }
-  static inline uint16_t ToBigEndian(uint16_t value) { return ByteSwap(value); }
-#else
-  static inline int64_t ToBigEndian(int64_t val) { return val; }
-  static inline uint64_t ToBigEndian(uint64_t val) { return val; }
-  static inline int32_t ToBigEndian(int32_t val) { return val; }
-  static inline uint32_t ToBigEndian(uint32_t val) { return val; }
-  static inline int16_t ToBigEndian(int16_t val) { return val; }
-  static inline uint16_t ToBigEndian(uint16_t val) { return val; }
-#endif
-
-/// Converts from big endian format to the machine's native endian format.
-#if __BYTE_ORDER == __LITTLE_ENDIAN
-  static inline int64_t FromBigEndian(int64_t value) { return ByteSwap(value); }
-  static inline uint64_t FromBigEndian(uint64_t value) { return ByteSwap(value); }
-  static inline int32_t FromBigEndian(int32_t value) { return ByteSwap(value); }
-  static inline uint32_t FromBigEndian(uint32_t value) { return ByteSwap(value); }
-  static inline int16_t FromBigEndian(int16_t value) { return ByteSwap(value); }
-  static inline uint16_t FromBigEndian(uint16_t value) { return ByteSwap(value); }
-#else
-  static inline int64_t FromBigEndian(int64_t val) { return val; }
-  static inline uint64_t FromBigEndian(uint64_t val) { return val; }
-  static inline int32_t FromBigEndian(int32_t val) { return val; }
-  static inline uint32_t FromBigEndian(uint32_t val) { return val; }
-  static inline int16_t FromBigEndian(int16_t val) { return val; }
-  static inline uint16_t FromBigEndian(uint16_t val) { return val; }
-#endif
-
-  // Logical right shift for signed integer types
-  // This is needed because the C >> operator does arithmetic right shift
-  // Negative shift amounts lead to undefined behavior
-  template <typename T>
-  static T ShiftRightLogical(T v, int shift) {
-    // Conversion to unsigned ensures most significant bits always filled with 0's
-    return static_cast<typename make_unsigned<T>::type>(v) >> shift;
-  }
-
-  // Get an specific bit of a numeric type
-  template <typename T>
-  static inline int8_t GetBit(T v, int bitpos) {
-    T masked = v & (static_cast<T>(0x1) << bitpos);
-    return static_cast<int8_t>(ShiftRightLogical(masked, bitpos));
-  }
-
-  // Set a specific bit to 1
-  // Behavior when bitpos is negative is undefined
-  template <typename T>
-  static T SetBit(T v, int bitpos) {
-    return v | (static_cast<T>(0x1) << bitpos);
-  }
-
-  static inline bool GetArrayBit(const uint8_t* bits, int i) {
-    return (bits[i / 8] & (1 << (i % 8))) != 0;
-  }
-
-  static inline void SetArrayBit(uint8_t* bits, int i, bool is_set) {
-    bits[i / 8] |= (1 << (i % 8)) * is_set;
-  }
-
-  // Set a specific bit to 0
-  // Behavior when bitpos is negative is undefined
-  template <typename T>
-  static T UnsetBit(T v, int bitpos) {
-    return v & ~(static_cast<T>(0x1) << bitpos);
-  }
-
-  // Returns the minimum number of bits needed to represent the value of 'x'
-  static inline int NumRequiredBits(uint64_t x) {
-    for (int i = 63; i >= 0; --i) {
-      if (x & (UINT64_C(1) << i)) return i + 1;
-    }
-    return 0;
-  }
-};
-
-}  // namespace parquet
-
-#endif  // PARQUET_UTIL_BIT_UTIL_H