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

[1/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Repository: parquet-cpp
Updated Branches:
  refs/heads/master 491182c22 -> 84db929ec


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/schema-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/schema-test.cc b/src/parquet/schema-test.cc
index 58f23df..203a312 100644
--- a/src/parquet/schema-test.cc
+++ b/src/parquet/schema-test.cc
@@ -337,13 +337,13 @@ TEST_F(TestGroupNode, FieldIndex) {
   NodeVector fields = Fields1();
   GroupNode group("group", Repetition::REQUIRED, fields);
   for (size_t i = 0; i < fields.size(); i++) {
-      auto field = group.field(static_cast<int>(i));
-      ASSERT_EQ(i, group.FieldIndex(*field.get()));
+    auto field = group.field(static_cast<int>(i));
+    ASSERT_EQ(i, group.FieldIndex(*field.get()));
   }
 
   // Test a non field node
-  auto non_field_alien = Int32("alien", Repetition::REQUIRED); // other name
-  auto non_field_familiar = Int32("one", Repetition::REPEATED); // other node
+  auto non_field_alien = Int32("alien", Repetition::REQUIRED);   // other name
+  auto non_field_familiar = Int32("one", Repetition::REPEATED);  // other node
   ASSERT_TRUE(group.FieldIndex(*non_field_alien.get()) < 0);
   ASSERT_TRUE(group.FieldIndex(*non_field_familiar.get()) < 0);
 }
@@ -664,13 +664,13 @@ TEST_F(TestSchemaDescriptor, BuildTree) {
   ASSERT_EQ(descr_.Column(5)->path()->ToDotString(), "bag.records.item3");
 
   for (int i = 0; i < nleaves; ++i) {
-      auto col = descr_.Column(i);
-      ASSERT_EQ(i, descr_.ColumnIndex(*col->schema_node().get()));
+    auto col = descr_.Column(i);
+    ASSERT_EQ(i, descr_.ColumnIndex(*col->schema_node().get()));
   }
 
   // Test non-column nodes find
-  NodePtr non_column_alien = Int32("alien", Repetition::REQUIRED); // other path
-  NodePtr non_column_familiar = Int32("a", Repetition::REPEATED); // other node
+  NodePtr non_column_alien = Int32("alien", Repetition::REQUIRED);  // other path
+  NodePtr non_column_familiar = Int32("a", Repetition::REPEATED);   // other node
   ASSERT_TRUE(descr_.ColumnIndex(*non_column_alien.get()) < 0);
   ASSERT_TRUE(descr_.ColumnIndex(*non_column_familiar.get()) < 0);
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/statistics-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/statistics-test.cc b/src/parquet/statistics-test.cc
new file mode 100644
index 0000000..cbc761d
--- /dev/null
+++ b/src/parquet/statistics-test.cc
@@ -0,0 +1,358 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include <algorithm>
+#include <array>
+#include <cstdint>
+#include <cstring>
+#include <memory>
+#include <vector>
+
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/file/reader.h"
+#include "parquet/file/writer.h"
+#include "parquet/schema.h"
+#include "parquet/statistics.h"
+#include "parquet/test-specialization.h"
+#include "parquet/test-util.h"
+#include "parquet/thrift.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+
+using arrow::default_memory_pool;
+using arrow::MemoryPool;
+
+namespace parquet {
+
+using schema::NodePtr;
+using schema::PrimitiveNode;
+using schema::GroupNode;
+
+namespace test {
+
+template <typename TestType>
+class TestRowGroupStatistics : public PrimitiveTypedTest<TestType> {
+ public:
+  using T = typename TestType::c_type;
+  using TypedStats = TypedRowGroupStatistics<TestType>;
+
+  std::vector<T> GetDeepCopy(
+      const std::vector<T>&);  // allocates new memory for FLBA/ByteArray
+
+  T* GetValuesPointer(std::vector<T>&);
+  void DeepFree(std::vector<T>&);
+
+  void TestMinMaxEncode() {
+    this->GenerateData(1000);
+
+    TypedStats statistics1(this->schema_.Column(0));
+    statistics1.Update(this->values_ptr_, this->values_.size(), 0);
+    std::string encoded_min = statistics1.EncodeMin();
+    std::string encoded_max = statistics1.EncodeMax();
+
+    TypedStats statistics2(this->schema_.Column(0), encoded_min, encoded_max,
+        this->values_.size(), 0, 0, true);
+
+    TypedStats statistics3(this->schema_.Column(0));
+    std::vector<uint8_t> valid_bits(
+        BitUtil::RoundUpNumBytes(static_cast<uint32_t>(this->values_.size())) + 1, 255);
+    statistics3.UpdateSpaced(
+        this->values_ptr_, valid_bits.data(), 0, this->values_.size(), 0);
+    std::string encoded_min_spaced = statistics3.EncodeMin();
+    std::string encoded_max_spaced = statistics3.EncodeMax();
+
+    ASSERT_EQ(encoded_min, statistics2.EncodeMin());
+    ASSERT_EQ(encoded_max, statistics2.EncodeMax());
+    ASSERT_EQ(statistics1.min(), statistics2.min());
+    ASSERT_EQ(statistics1.max(), statistics2.max());
+    ASSERT_EQ(encoded_min_spaced, statistics2.EncodeMin());
+    ASSERT_EQ(encoded_max_spaced, statistics2.EncodeMax());
+    ASSERT_EQ(statistics3.min(), statistics2.min());
+    ASSERT_EQ(statistics3.max(), statistics2.max());
+  }
+
+  void TestReset() {
+    this->GenerateData(1000);
+
+    TypedStats statistics(this->schema_.Column(0));
+    statistics.Update(this->values_ptr_, this->values_.size(), 0);
+    ASSERT_EQ(this->values_.size(), statistics.num_values());
+
+    statistics.Reset();
+    ASSERT_EQ(0, statistics.null_count());
+    ASSERT_EQ(0, statistics.num_values());
+    ASSERT_EQ("", statistics.EncodeMin());
+    ASSERT_EQ("", statistics.EncodeMax());
+  }
+
+  void TestMerge() {
+    int num_null[2];
+    random_numbers(2, 42, 0, 100, num_null);
+
+    TypedStats statistics1(this->schema_.Column(0));
+    this->GenerateData(1000);
+    statistics1.Update(
+        this->values_ptr_, this->values_.size() - num_null[0], num_null[0]);
+
+    TypedStats statistics2(this->schema_.Column(0));
+    this->GenerateData(1000);
+    statistics2.Update(
+        this->values_ptr_, this->values_.size() - num_null[1], num_null[1]);
+
+    TypedStats total(this->schema_.Column(0));
+    total.Merge(statistics1);
+    total.Merge(statistics2);
+
+    ASSERT_EQ(num_null[0] + num_null[1], total.null_count());
+    ASSERT_EQ(this->values_.size() * 2 - num_null[0] - num_null[1], total.num_values());
+    ASSERT_EQ(total.min(), std::min(statistics1.min(), statistics2.min()));
+    ASSERT_EQ(total.max(), std::max(statistics1.max(), statistics2.max()));
+  }
+
+  void TestFullRoundtrip(int64_t num_values, int64_t null_count) {
+    this->GenerateData(num_values);
+
+    // compute statistics for the whole batch
+    TypedStats expected_stats(this->schema_.Column(0));
+    expected_stats.Update(this->values_ptr_, num_values - null_count, null_count);
+
+    auto sink = std::make_shared<InMemoryOutputStream>();
+    auto gnode = std::static_pointer_cast<GroupNode>(this->node_);
+    std::shared_ptr<WriterProperties> writer_properties =
+        WriterProperties::Builder().enable_statistics("column")->build();
+    auto file_writer = ParquetFileWriter::Open(sink, gnode, writer_properties);
+    auto row_group_writer = file_writer->AppendRowGroup(num_values);
+    auto column_writer =
+        static_cast<TypedColumnWriter<TestType>*>(row_group_writer->NextColumn());
+
+    // simulate the case when data comes from multiple buffers,
+    // in which case special care is necessary for FLBA/ByteArray types
+    for (int i = 0; i < 2; i++) {
+      int64_t batch_num_values = i ? num_values - num_values / 2 : num_values / 2;
+      int64_t batch_null_count = i ? null_count : 0;
+      DCHECK(null_count <= num_values);  // avoid too much headache
+      std::vector<int16_t> definition_levels(batch_null_count, 0);
+      definition_levels.insert(
+          definition_levels.end(), batch_num_values - batch_null_count, 1);
+      auto beg = this->values_.begin() + i * num_values / 2;
+      auto end = beg + batch_num_values;
+      std::vector<T> batch = GetDeepCopy(std::vector<T>(beg, end));
+      T* batch_values_ptr = GetValuesPointer(batch);
+      column_writer->WriteBatch(
+          batch_num_values, definition_levels.data(), nullptr, batch_values_ptr);
+      DeepFree(batch);
+    }
+    column_writer->Close();
+    row_group_writer->Close();
+    file_writer->Close();
+
+    auto buffer = sink->GetBuffer();
+    auto source = std::make_shared<::arrow::io::BufferReader>(buffer);
+    auto file_reader = ParquetFileReader::Open(source);
+    auto rg_reader = file_reader->RowGroup(0);
+    auto column_chunk = rg_reader->metadata()->ColumnChunk(0);
+    if (!column_chunk->is_stats_set()) return;
+    std::shared_ptr<RowGroupStatistics> stats = column_chunk->statistics();
+    // check values after serialization + deserialization
+    ASSERT_EQ(null_count, stats->null_count());
+    ASSERT_EQ(num_values - null_count, stats->num_values());
+    ASSERT_EQ(expected_stats.EncodeMin(), stats->EncodeMin());
+    ASSERT_EQ(expected_stats.EncodeMax(), stats->EncodeMax());
+  }
+};
+
+template <typename TestType>
+typename TestType::c_type* TestRowGroupStatistics<TestType>::GetValuesPointer(
+    std::vector<typename TestType::c_type>& values) {
+  return values.data();
+}
+
+template <>
+bool* TestRowGroupStatistics<BooleanType>::GetValuesPointer(std::vector<bool>& values) {
+  static std::vector<uint8_t> bool_buffer;
+  bool_buffer.clear();
+  bool_buffer.resize(values.size());
+  std::copy(values.begin(), values.end(), bool_buffer.begin());
+  return reinterpret_cast<bool*>(bool_buffer.data());
+}
+
+template <typename TestType>
+typename std::vector<typename TestType::c_type>
+TestRowGroupStatistics<TestType>::GetDeepCopy(
+    const std::vector<typename TestType::c_type>& values) {
+  return values;
+}
+
+template <>
+std::vector<FLBA> TestRowGroupStatistics<FLBAType>::GetDeepCopy(
+    const std::vector<FLBA>& values) {
+  std::vector<FLBA> copy;
+  MemoryPool* pool = ::arrow::default_memory_pool();
+  for (const FLBA& flba : values) {
+    uint8_t* ptr;
+    PARQUET_THROW_NOT_OK(pool->Allocate(FLBA_LENGTH, &ptr));
+    memcpy(ptr, flba.ptr, FLBA_LENGTH);
+    copy.emplace_back(ptr);
+  }
+  return copy;
+}
+
+template <>
+std::vector<ByteArray> TestRowGroupStatistics<ByteArrayType>::GetDeepCopy(
+    const std::vector<ByteArray>& values) {
+  std::vector<ByteArray> copy;
+  MemoryPool* pool = default_memory_pool();
+  for (const ByteArray& ba : values) {
+    uint8_t* ptr;
+    PARQUET_THROW_NOT_OK(pool->Allocate(ba.len, &ptr));
+    memcpy(ptr, ba.ptr, ba.len);
+    copy.emplace_back(ba.len, ptr);
+  }
+  return copy;
+}
+
+template <typename TestType>
+void TestRowGroupStatistics<TestType>::DeepFree(
+    std::vector<typename TestType::c_type>& values) {}
+
+template <>
+void TestRowGroupStatistics<FLBAType>::DeepFree(std::vector<FLBA>& values) {
+  MemoryPool* pool = default_memory_pool();
+  for (FLBA& flba : values) {
+    auto ptr = const_cast<uint8_t*>(flba.ptr);
+    memset(ptr, 0, FLBA_LENGTH);
+    pool->Free(ptr, FLBA_LENGTH);
+  }
+}
+
+template <>
+void TestRowGroupStatistics<ByteArrayType>::DeepFree(std::vector<ByteArray>& values) {
+  MemoryPool* pool = default_memory_pool();
+  for (ByteArray& ba : values) {
+    auto ptr = const_cast<uint8_t*>(ba.ptr);
+    memset(ptr, 0, ba.len);
+    pool->Free(ptr, ba.len);
+  }
+}
+
+template <>
+void TestRowGroupStatistics<ByteArrayType>::TestMinMaxEncode() {
+  this->GenerateData(1000);
+  // Test that we encode min max strings correctly
+  TypedRowGroupStatistics<ByteArrayType> statistics1(this->schema_.Column(0));
+  statistics1.Update(this->values_ptr_, this->values_.size(), 0);
+  std::string encoded_min = statistics1.EncodeMin();
+  std::string encoded_max = statistics1.EncodeMax();
+
+  // encoded is same as unencoded
+  ASSERT_EQ(encoded_min,
+      std::string((const char*)statistics1.min().ptr, statistics1.min().len));
+  ASSERT_EQ(encoded_max,
+      std::string((const char*)statistics1.max().ptr, statistics1.max().len));
+
+  TypedRowGroupStatistics<ByteArrayType> statistics2(this->schema_.Column(0), encoded_min,
+      encoded_max, this->values_.size(), 0, 0, true);
+
+  ASSERT_EQ(encoded_min, statistics2.EncodeMin());
+  ASSERT_EQ(encoded_max, statistics2.EncodeMax());
+  ASSERT_EQ(statistics1.min(), statistics2.min());
+  ASSERT_EQ(statistics1.max(), statistics2.max());
+}
+
+using TestTypes = ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
+    ByteArrayType, FLBAType, BooleanType>;
+
+TYPED_TEST_CASE(TestRowGroupStatistics, TestTypes);
+
+TYPED_TEST(TestRowGroupStatistics, MinMaxEncode) {
+  this->SetUpSchema(Repetition::REQUIRED);
+  this->TestMinMaxEncode();
+}
+
+TYPED_TEST(TestRowGroupStatistics, Reset) {
+  this->SetUpSchema(Repetition::OPTIONAL);
+  this->TestReset();
+}
+
+TYPED_TEST(TestRowGroupStatistics, FullRoundtrip) {
+  this->SetUpSchema(Repetition::OPTIONAL);
+  this->TestFullRoundtrip(100, 31);
+  this->TestFullRoundtrip(1000, 415);
+  this->TestFullRoundtrip(10000, 926);
+}
+
+template <typename TestType>
+class TestNumericRowGroupStatistics : public TestRowGroupStatistics<TestType> {};
+
+using NumericTypes = ::testing::Types<Int32Type, Int64Type, FloatType, DoubleType>;
+
+TYPED_TEST_CASE(TestNumericRowGroupStatistics, NumericTypes);
+
+TYPED_TEST(TestNumericRowGroupStatistics, Merge) {
+  this->SetUpSchema(Repetition::OPTIONAL);
+  this->TestMerge();
+}
+
+TEST(CorruptStatistics, Basics) {
+  ApplicationVersion version("parquet-mr version 1.8.0");
+  SchemaDescriptor schema;
+  schema::NodePtr node;
+  std::vector<schema::NodePtr> fields;
+  // Test Physical Types
+  fields.push_back(schema::PrimitiveNode::Make(
+      "col1", Repetition::OPTIONAL, Type::INT32, LogicalType::NONE));
+  fields.push_back(schema::PrimitiveNode::Make(
+      "col2", Repetition::OPTIONAL, Type::BYTE_ARRAY, LogicalType::NONE));
+  // Test Logical Types
+  fields.push_back(schema::PrimitiveNode::Make(
+      "col3", Repetition::OPTIONAL, Type::INT32, LogicalType::DATE));
+  fields.push_back(schema::PrimitiveNode::Make(
+      "col4", Repetition::OPTIONAL, Type::INT32, LogicalType::UINT_32));
+  fields.push_back(schema::PrimitiveNode::Make("col5", Repetition::OPTIONAL,
+      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::INTERVAL, 12));
+  fields.push_back(schema::PrimitiveNode::Make(
+      "col6", Repetition::OPTIONAL, Type::BYTE_ARRAY, LogicalType::UTF8));
+  node = schema::GroupNode::Make("schema", Repetition::REQUIRED, fields);
+  schema.Init(node);
+
+  format::ColumnChunk col_chunk;
+  col_chunk.meta_data.__isset.statistics = true;
+  auto column_chunk1 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(0), &version);
+  ASSERT_TRUE(column_chunk1->is_stats_set());
+  auto column_chunk2 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(1), &version);
+  ASSERT_FALSE(column_chunk2->is_stats_set());
+  auto column_chunk3 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(2), &version);
+  ASSERT_TRUE(column_chunk3->is_stats_set());
+  auto column_chunk4 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(3), &version);
+  ASSERT_FALSE(column_chunk4->is_stats_set());
+  auto column_chunk5 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(4), &version);
+  ASSERT_FALSE(column_chunk5->is_stats_set());
+  auto column_chunk6 = ColumnChunkMetaData::Make(
+      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(5), &version);
+  ASSERT_FALSE(column_chunk6->is_stats_set());
+}
+
+}  // namespace test
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/statistics.cc
----------------------------------------------------------------------
diff --git a/src/parquet/statistics.cc b/src/parquet/statistics.cc
new file mode 100644
index 0000000..d99140e
--- /dev/null
+++ b/src/parquet/statistics.cc
@@ -0,0 +1,244 @@
+// 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.
+
+#include <algorithm>
+#include <cstring>
+
+#include "parquet/encoding-internal.h"
+#include "parquet/exception.h"
+#include "parquet/statistics.h"
+#include "parquet/util/comparison.h"
+#include "parquet/util/memory.h"
+
+using arrow::default_memory_pool;
+using arrow::MemoryPool;
+
+namespace parquet {
+
+template <typename DType>
+TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(
+    const ColumnDescriptor* schema, MemoryPool* pool)
+    : pool_(pool),
+      min_buffer_(AllocateBuffer(pool_, 0)),
+      max_buffer_(AllocateBuffer(pool_, 0)) {
+  SetDescr(schema);
+  Reset();
+}
+
+template <typename DType>
+TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(const typename DType::c_type& min,
+    const typename DType::c_type& max, int64_t num_values, int64_t null_count,
+    int64_t distinct_count)
+    : pool_(default_memory_pool()),
+      min_buffer_(AllocateBuffer(pool_, 0)),
+      max_buffer_(AllocateBuffer(pool_, 0)) {
+  IncrementNumValues(num_values);
+  IncrementNullCount(null_count);
+  IncrementDistinctCount(distinct_count);
+
+  Copy(min, &min_, min_buffer_.get());
+  Copy(max, &max_, max_buffer_.get());
+  has_min_max_ = true;
+}
+
+template <typename DType>
+TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(const ColumnDescriptor* schema,
+    const std::string& encoded_min, const std::string& encoded_max, int64_t num_values,
+    int64_t null_count, int64_t distinct_count, bool has_min_max, MemoryPool* pool)
+    : pool_(pool),
+      min_buffer_(AllocateBuffer(pool_, 0)),
+      max_buffer_(AllocateBuffer(pool_, 0)) {
+  IncrementNumValues(num_values);
+  IncrementNullCount(null_count);
+  IncrementDistinctCount(distinct_count);
+
+  SetDescr(schema);
+
+  if (!encoded_min.empty()) { PlainDecode(encoded_min, &min_); }
+  if (!encoded_max.empty()) { PlainDecode(encoded_max, &max_); }
+  has_min_max_ = has_min_max;
+}
+
+template <typename DType>
+bool TypedRowGroupStatistics<DType>::HasMinMax() const {
+  return has_min_max_;
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::Reset() {
+  ResetCounts();
+  has_min_max_ = false;
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::Update(
+    const T* values, int64_t num_not_null, int64_t num_null) {
+  DCHECK(num_not_null >= 0);
+  DCHECK(num_null >= 0);
+
+  IncrementNullCount(num_null);
+  IncrementNumValues(num_not_null);
+  // TODO: support distinct count?
+  if (num_not_null == 0) return;
+
+  Compare<T> compare(descr_);
+  auto batch_minmax = std::minmax_element(values, values + num_not_null, compare);
+  if (!has_min_max_) {
+    has_min_max_ = true;
+    Copy(*batch_minmax.first, &min_, min_buffer_.get());
+    Copy(*batch_minmax.second, &max_, max_buffer_.get());
+  } else {
+    Copy(std::min(min_, *batch_minmax.first, compare), &min_, min_buffer_.get());
+    Copy(std::max(max_, *batch_minmax.second, compare), &max_, max_buffer_.get());
+  }
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::UpdateSpaced(const T* values,
+    const uint8_t* valid_bits, int64_t valid_bits_offset, int64_t num_not_null,
+    int64_t num_null) {
+  DCHECK(num_not_null >= 0);
+  DCHECK(num_null >= 0);
+
+  IncrementNullCount(num_null);
+  IncrementNumValues(num_not_null);
+  // TODO: support distinct count?
+  if (num_not_null == 0) return;
+
+  Compare<T> compare(descr_);
+  INIT_BITSET(valid_bits, static_cast<int>(valid_bits_offset));
+  // Find first valid entry and use that for min/max
+  // As (num_not_null != 0) there must be one
+  int64_t length = num_null + num_not_null;
+  int64_t i = 0;
+  for (; i < length; i++) {
+    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) { break; }
+    READ_NEXT_BITSET(valid_bits);
+  }
+  T min = values[i];
+  T max = values[i];
+  for (; i < length; i++) {
+    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
+      if (compare(values[i], min)) {
+        min = values[i];
+      } else if (compare(max, values[i])) {
+        max = values[i];
+      }
+    }
+    READ_NEXT_BITSET(valid_bits);
+  }
+  if (!has_min_max_) {
+    has_min_max_ = true;
+    Copy(min, &min_, min_buffer_.get());
+    Copy(max, &max_, max_buffer_.get());
+  } else {
+    Copy(std::min(min_, min, compare), &min_, min_buffer_.get());
+    Copy(std::max(max_, max, compare), &max_, max_buffer_.get());
+  }
+}
+
+template <typename DType>
+const typename DType::c_type& TypedRowGroupStatistics<DType>::min() const {
+  return min_;
+}
+
+template <typename DType>
+const typename DType::c_type& TypedRowGroupStatistics<DType>::max() const {
+  return max_;
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::Merge(const TypedRowGroupStatistics<DType>& other) {
+  this->MergeCounts(other);
+
+  if (!other.HasMinMax()) return;
+
+  if (!has_min_max_) {
+    Copy(other.min_, &this->min_, min_buffer_.get());
+    Copy(other.max_, &this->max_, max_buffer_.get());
+    has_min_max_ = true;
+    return;
+  }
+
+  Compare<T> compare(descr_);
+  Copy(std::min(this->min_, other.min_, compare), &this->min_, min_buffer_.get());
+  Copy(std::max(this->max_, other.max_, compare), &this->max_, max_buffer_.get());
+}
+
+template <typename DType>
+std::string TypedRowGroupStatistics<DType>::EncodeMin() {
+  std::string s;
+  if (HasMinMax()) this->PlainEncode(min_, &s);
+  return s;
+}
+
+template <typename DType>
+std::string TypedRowGroupStatistics<DType>::EncodeMax() {
+  std::string s;
+  if (HasMinMax()) this->PlainEncode(max_, &s);
+  return s;
+}
+
+template <typename DType>
+EncodedStatistics TypedRowGroupStatistics<DType>::Encode() {
+  EncodedStatistics s;
+  if (HasMinMax()) {
+    s.set_min(this->EncodeMin());
+    s.set_max(this->EncodeMax());
+  }
+  s.set_null_count(this->null_count());
+  return s;
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::PlainEncode(const T& src, std::string* dst) {
+  PlainEncoder<DType> encoder(descr(), pool_);
+  encoder.Put(&src, 1);
+  auto buffer = encoder.FlushValues();
+  auto ptr = reinterpret_cast<const char*>(buffer->data());
+  dst->assign(ptr, buffer->size());
+}
+
+template <typename DType>
+void TypedRowGroupStatistics<DType>::PlainDecode(const std::string& src, T* dst) {
+  PlainDecoder<DType> decoder(descr());
+  decoder.SetData(
+      1, reinterpret_cast<const uint8_t*>(src.c_str()), static_cast<int>(src.size()));
+  decoder.Decode(dst, 1);
+}
+
+template <>
+void TypedRowGroupStatistics<ByteArrayType>::PlainEncode(const T& src, std::string* dst) {
+  dst->assign(reinterpret_cast<const char*>(src.ptr), src.len);
+}
+
+template <>
+void TypedRowGroupStatistics<ByteArrayType>::PlainDecode(const std::string& src, T* dst) {
+  dst->len = static_cast<uint32_t>(src.size());
+  dst->ptr = reinterpret_cast<const uint8_t*>(src.c_str());
+}
+
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<BooleanType>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int32Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int64Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int96Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<FloatType>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<DoubleType>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<ByteArrayType>;
+template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<FLBAType>;
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/statistics.h
----------------------------------------------------------------------
diff --git a/src/parquet/statistics.h b/src/parquet/statistics.h
new file mode 100644
index 0000000..c6a2487
--- /dev/null
+++ b/src/parquet/statistics.h
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef PARQUET_COLUMN_STATISTICS_H
+#define PARQUET_COLUMN_STATISTICS_H
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "parquet/schema.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/visibility.h"
+
+namespace parquet {
+
+class PARQUET_EXPORT EncodedStatistics {
+  std::shared_ptr<std::string> max_, min_;
+
+ public:
+  EncodedStatistics()
+      : max_(std::make_shared<std::string>()), min_(std::make_shared<std::string>()) {}
+
+  const std::string& max() const { return *max_; }
+  const std::string& min() const { return *min_; }
+
+  int64_t null_count = 0;
+  int64_t distinct_count = 0;
+
+  bool has_min = false;
+  bool has_max = false;
+  bool has_null_count = false;
+  bool has_distinct_count = false;
+
+  inline bool is_set() const {
+    return has_min || has_max || has_null_count || has_distinct_count;
+  }
+
+  inline EncodedStatistics& set_max(const std::string& value) {
+    *max_ = value;
+    has_max = true;
+    return *this;
+  }
+
+  inline EncodedStatistics& set_min(const std::string& value) {
+    *min_ = value;
+    has_min = true;
+    return *this;
+  }
+
+  inline EncodedStatistics& set_null_count(int64_t value) {
+    null_count = value;
+    has_null_count = true;
+    return *this;
+  }
+
+  inline EncodedStatistics& set_distinct_count(int64_t value) {
+    distinct_count = value;
+    has_distinct_count = true;
+    return *this;
+  }
+};
+
+template <typename DType>
+class PARQUET_EXPORT TypedRowGroupStatistics;
+
+class PARQUET_EXPORT RowGroupStatistics
+    : public std::enable_shared_from_this<RowGroupStatistics> {
+ public:
+  int64_t null_count() const { return statistics_.null_count; }
+  int64_t distinct_count() const { return statistics_.distinct_count; }
+  int64_t num_values() const { return num_values_; }
+
+  virtual bool HasMinMax() const = 0;
+  virtual void Reset() = 0;
+
+  // Plain-encoded minimum value
+  virtual std::string EncodeMin() = 0;
+
+  // Plain-encoded maximum value
+  virtual std::string EncodeMax() = 0;
+
+  virtual EncodedStatistics Encode() = 0;
+
+  virtual ~RowGroupStatistics() {}
+
+  Type::type physical_type() const { return descr_->physical_type(); }
+
+ protected:
+  const ColumnDescriptor* descr() const { return descr_; }
+  void SetDescr(const ColumnDescriptor* schema) { descr_ = schema; }
+
+  void IncrementNullCount(int64_t n) { statistics_.null_count += n; }
+
+  void IncrementNumValues(int64_t n) { num_values_ += n; }
+
+  void IncrementDistinctCount(int64_t n) { statistics_.distinct_count += n; }
+
+  void MergeCounts(const RowGroupStatistics& other) {
+    this->statistics_.null_count += other.statistics_.null_count;
+    this->statistics_.distinct_count += other.statistics_.distinct_count;
+    this->num_values_ += other.num_values_;
+  }
+
+  void ResetCounts() {
+    this->statistics_.null_count = 0;
+    this->statistics_.distinct_count = 0;
+    this->num_values_ = 0;
+  }
+
+  const ColumnDescriptor* descr_ = nullptr;
+  int64_t num_values_ = 0;
+  EncodedStatistics statistics_;
+};
+
+template <typename DType>
+class TypedRowGroupStatistics : public RowGroupStatistics {
+ public:
+  using T = typename DType::c_type;
+
+  TypedRowGroupStatistics(const ColumnDescriptor* schema,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
+
+  TypedRowGroupStatistics(const T& min, const T& max, int64_t num_values,
+      int64_t null_count, int64_t distinct_count);
+
+  TypedRowGroupStatistics(const ColumnDescriptor* schema, const std::string& encoded_min,
+      const std::string& encoded_max, int64_t num_values, int64_t null_count,
+      int64_t distinct_count, bool has_min_max,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
+
+  bool HasMinMax() const override;
+  void Reset() override;
+  void Merge(const TypedRowGroupStatistics<DType>& other);
+
+  void Update(const T* values, int64_t num_not_null, int64_t num_null);
+  void UpdateSpaced(const T* values, const uint8_t* valid_bits, int64_t valid_bits_spaced,
+      int64_t num_not_null, int64_t num_null);
+
+  const T& min() const;
+  const T& max() const;
+
+  std::string EncodeMin() override;
+  std::string EncodeMax() override;
+  EncodedStatistics Encode() override;
+
+ private:
+  bool has_min_max_ = false;
+  T min_;
+  T max_;
+  ::arrow::MemoryPool* pool_;
+
+  void PlainEncode(const T& src, std::string* dst);
+  void PlainDecode(const std::string& src, T* dst);
+  void Copy(const T& src, T* dst, PoolBuffer* buffer);
+
+  std::shared_ptr<PoolBuffer> min_buffer_, max_buffer_;
+};
+
+template <typename DType>
+inline void TypedRowGroupStatistics<DType>::Copy(const T& src, T* dst, PoolBuffer*) {
+  *dst = src;
+}
+
+template <>
+inline void TypedRowGroupStatistics<FLBAType>::Copy(
+    const FLBA& src, FLBA* dst, PoolBuffer* buffer) {
+  if (dst->ptr == src.ptr) return;
+  uint32_t len = descr_->type_length();
+  PARQUET_THROW_NOT_OK(buffer->Resize(len, false));
+  std::memcpy(buffer->mutable_data(), src.ptr, len);
+  *dst = FLBA(buffer->data());
+}
+
+template <>
+inline void TypedRowGroupStatistics<ByteArrayType>::Copy(
+    const ByteArray& src, ByteArray* dst, PoolBuffer* buffer) {
+  if (dst->ptr == src.ptr) return;
+  PARQUET_THROW_NOT_OK(buffer->Resize(src.len, false));
+  std::memcpy(buffer->mutable_data(), src.ptr, src.len);
+  *dst = ByteArray(src.len, buffer->data());
+}
+
+template <>
+void TypedRowGroupStatistics<ByteArrayType>::PlainEncode(const T& src, std::string* dst);
+
+template <>
+void TypedRowGroupStatistics<ByteArrayType>::PlainDecode(const std::string& src, T* dst);
+
+typedef TypedRowGroupStatistics<BooleanType> BoolStatistics;
+typedef TypedRowGroupStatistics<Int32Type> Int32Statistics;
+typedef TypedRowGroupStatistics<Int64Type> Int64Statistics;
+typedef TypedRowGroupStatistics<Int96Type> Int96Statistics;
+typedef TypedRowGroupStatistics<FloatType> FloatStatistics;
+typedef TypedRowGroupStatistics<DoubleType> DoubleStatistics;
+typedef TypedRowGroupStatistics<ByteArrayType> ByteArrayStatistics;
+typedef TypedRowGroupStatistics<FLBAType> FLBAStatistics;
+
+#if defined(__GNUC__) && !defined(__clang__)
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wattributes"
+#endif
+
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<BooleanType>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int32Type>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int64Type>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int96Type>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<FloatType>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<DoubleType>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<ByteArrayType>;
+PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<FLBAType>;
+
+#if defined(__GNUC__) && !defined(__clang__)
+#pragma GCC diagnostic pop
+#endif
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_STATISTICS_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/test-specialization.h
----------------------------------------------------------------------
diff --git a/src/parquet/test-specialization.h b/src/parquet/test-specialization.h
new file mode 100644
index 0000000..6bd1dee
--- /dev/null
+++ b/src/parquet/test-specialization.h
@@ -0,0 +1,172 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#ifndef PARQUET_COLUMN_TEST_SPECIALIZATION_H
+#define PARQUET_COLUMN_TEST_SPECIALIZATION_H
+
+#include <algorithm>
+#include <limits>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include "parquet/test-util.h"
+
+namespace parquet {
+
+namespace test {
+
+template <>
+void InitValues<bool>(int num_values, vector<bool>& values, vector<uint8_t>& buffer) {
+  values = flip_coins(num_values, 0);
+}
+
+template <>
+void InitValues<ByteArray>(
+    int num_values, vector<ByteArray>& values, vector<uint8_t>& buffer) {
+  int max_byte_array_len = 12;
+  int num_bytes = max_byte_array_len + sizeof(uint32_t);
+  size_t nbytes = num_values * num_bytes;
+  buffer.resize(nbytes);
+  random_byte_array(num_values, 0, buffer.data(), values.data(), max_byte_array_len);
+}
+
+template <>
+void InitValues<FLBA>(int num_values, vector<FLBA>& values, vector<uint8_t>& buffer) {
+  size_t nbytes = num_values * FLBA_LENGTH;
+  buffer.resize(nbytes);
+  random_fixed_byte_array(num_values, 0, buffer.data(), FLBA_LENGTH, values.data());
+}
+
+template <>
+void InitValues<Int96>(int num_values, vector<Int96>& values, vector<uint8_t>& buffer) {
+  random_Int96_numbers(num_values, 0, std::numeric_limits<int32_t>::min(),
+      std::numeric_limits<int32_t>::max(), values.data());
+}
+
+inline std::string TestColumnName(int i) {
+  std::stringstream col_name;
+  col_name << "column_" << i;
+  return col_name.str();
+}
+
+// This class lives here because of its dependency on the InitValues specializations.
+template <typename TestType>
+class PrimitiveTypedTest : public ::testing::Test {
+ public:
+  typedef typename TestType::c_type T;
+
+  void SetUpSchema(Repetition::type repetition, int num_columns = 1) {
+    std::vector<schema::NodePtr> fields;
+
+    for (int i = 0; i < num_columns; ++i) {
+      std::string name = TestColumnName(i);
+      fields.push_back(schema::PrimitiveNode::Make(
+          name, repetition, TestType::type_num, LogicalType::NONE, FLBA_LENGTH));
+    }
+    node_ = schema::GroupNode::Make("schema", Repetition::REQUIRED, fields);
+    schema_.Init(node_);
+  }
+
+  void GenerateData(int64_t num_values);
+  void SetupValuesOut(int64_t num_values);
+  void SyncValuesOut();
+
+ protected:
+  schema::NodePtr node_;
+  SchemaDescriptor schema_;
+
+  // Input buffers
+  std::vector<T> values_;
+
+  std::vector<int16_t> def_levels_;
+
+  std::vector<uint8_t> buffer_;
+  // Pointer to the values, needed as we cannot use vector<bool>::data()
+  T* values_ptr_;
+  std::vector<uint8_t> bool_buffer_;
+
+  // Output buffers
+  std::vector<T> values_out_;
+  std::vector<uint8_t> bool_buffer_out_;
+  T* values_out_ptr_;
+};
+
+template <typename TestType>
+void PrimitiveTypedTest<TestType>::SyncValuesOut() {}
+
+template <>
+void PrimitiveTypedTest<BooleanType>::SyncValuesOut() {
+  std::vector<uint8_t>::const_iterator source_iterator = bool_buffer_out_.begin();
+  std::vector<T>::iterator destination_iterator = values_out_.begin();
+  while (source_iterator != bool_buffer_out_.end()) {
+    *destination_iterator++ = *source_iterator++ != 0;
+  }
+}
+
+template <typename TestType>
+void PrimitiveTypedTest<TestType>::SetupValuesOut(int64_t num_values) {
+  values_out_.clear();
+  values_out_.resize(num_values);
+  values_out_ptr_ = values_out_.data();
+}
+
+template <>
+void PrimitiveTypedTest<BooleanType>::SetupValuesOut(int64_t num_values) {
+  values_out_.clear();
+  values_out_.resize(num_values);
+
+  bool_buffer_out_.clear();
+  bool_buffer_out_.resize(num_values);
+  // Write once to all values so we can copy it without getting Valgrind errors
+  // about uninitialised values.
+  std::fill(bool_buffer_out_.begin(), bool_buffer_out_.end(), true);
+  values_out_ptr_ = reinterpret_cast<bool*>(bool_buffer_out_.data());
+}
+
+template <typename TestType>
+void PrimitiveTypedTest<TestType>::GenerateData(int64_t num_values) {
+  def_levels_.resize(num_values);
+  values_.resize(num_values);
+
+  InitValues<T>(static_cast<int>(num_values), values_, buffer_);
+  values_ptr_ = values_.data();
+
+  std::fill(def_levels_.begin(), def_levels_.end(), 1);
+}
+
+template <>
+void PrimitiveTypedTest<BooleanType>::GenerateData(int64_t num_values) {
+  def_levels_.resize(num_values);
+  values_.resize(num_values);
+
+  InitValues<T>(static_cast<int>(num_values), values_, buffer_);
+  bool_buffer_.resize(num_values);
+  std::copy(values_.begin(), values_.end(), bool_buffer_.begin());
+  values_ptr_ = reinterpret_cast<bool*>(bool_buffer_.data());
+
+  std::fill(def_levels_.begin(), def_levels_.end(), 1);
+}
+}  // namespace test
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_TEST_SPECIALIZATION_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/test-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/test-util.h b/src/parquet/test-util.h
new file mode 100644
index 0000000..8657a7f
--- /dev/null
+++ b/src/parquet/test-util.h
@@ -0,0 +1,430 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#ifndef PARQUET_COLUMN_TEST_UTIL_H
+#define PARQUET_COLUMN_TEST_UTIL_H
+
+#include <algorithm>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/encoding-internal.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/test-common.h"
+
+using std::vector;
+using std::shared_ptr;
+
+namespace parquet {
+
+static int FLBA_LENGTH = 12;
+
+bool operator==(const FixedLenByteArray& a, const FixedLenByteArray& b) {
+  return 0 == memcmp(a.ptr, b.ptr, FLBA_LENGTH);
+}
+
+namespace test {
+
+template <typename T>
+static void InitValues(int num_values, vector<T>& values, vector<uint8_t>& buffer) {
+  random_numbers(num_values, 0, std::numeric_limits<T>::min(),
+      std::numeric_limits<T>::max(), values.data());
+}
+
+template <typename T>
+static void InitDictValues(
+    int num_values, int num_dicts, vector<T>& values, vector<uint8_t>& buffer) {
+  int repeat_factor = num_values / num_dicts;
+  InitValues<T>(num_dicts, values, buffer);
+  // add some repeated values
+  for (int j = 1; j < repeat_factor; ++j) {
+    for (int i = 0; i < num_dicts; ++i) {
+      std::memcpy(&values[num_dicts * j + i], &values[i], sizeof(T));
+    }
+  }
+  // computed only dict_per_page * repeat_factor - 1 values < num_values
+  // compute remaining
+  for (int i = num_dicts * repeat_factor; i < num_values; ++i) {
+    std::memcpy(&values[i], &values[i - num_dicts * repeat_factor], sizeof(T));
+  }
+}
+
+class MockPageReader : public PageReader {
+ public:
+  explicit MockPageReader(const vector<shared_ptr<Page>>& pages)
+      : pages_(pages), page_index_(0) {}
+
+  // Implement the PageReader interface
+  virtual shared_ptr<Page> NextPage() {
+    if (page_index_ == static_cast<int>(pages_.size())) {
+      // EOS to consumer
+      return shared_ptr<Page>(nullptr);
+    }
+    return pages_[page_index_++];
+  }
+
+ private:
+  vector<shared_ptr<Page>> pages_;
+  int page_index_;
+};
+
+// TODO(wesm): this is only used for testing for now. Refactor to form part of
+// primary file write path
+template <typename Type>
+class DataPageBuilder {
+ public:
+  typedef typename Type::c_type T;
+
+  // This class writes data and metadata to the passed inputs
+  explicit DataPageBuilder(InMemoryOutputStream* sink)
+      : sink_(sink),
+        num_values_(0),
+        encoding_(Encoding::PLAIN),
+        definition_level_encoding_(Encoding::RLE),
+        repetition_level_encoding_(Encoding::RLE),
+        have_def_levels_(false),
+        have_rep_levels_(false),
+        have_values_(false) {}
+
+  void AppendDefLevels(const vector<int16_t>& levels, int16_t max_level,
+      Encoding::type encoding = Encoding::RLE) {
+    AppendLevels(levels, max_level, encoding);
+
+    num_values_ = std::max(static_cast<int32_t>(levels.size()), num_values_);
+    definition_level_encoding_ = encoding;
+    have_def_levels_ = true;
+  }
+
+  void AppendRepLevels(const vector<int16_t>& levels, int16_t max_level,
+      Encoding::type encoding = Encoding::RLE) {
+    AppendLevels(levels, max_level, encoding);
+
+    num_values_ = std::max(static_cast<int32_t>(levels.size()), num_values_);
+    repetition_level_encoding_ = encoding;
+    have_rep_levels_ = true;
+  }
+
+  void AppendValues(const ColumnDescriptor* d, const vector<T>& values,
+      Encoding::type encoding = Encoding::PLAIN) {
+    PlainEncoder<Type> encoder(d);
+    encoder.Put(&values[0], static_cast<int>(values.size()));
+    std::shared_ptr<Buffer> values_sink = encoder.FlushValues();
+    sink_->Write(values_sink->data(), values_sink->size());
+
+    num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);
+    encoding_ = encoding;
+    have_values_ = true;
+  }
+
+  int32_t num_values() const { return num_values_; }
+
+  Encoding::type encoding() const { return encoding_; }
+
+  Encoding::type rep_level_encoding() const { return repetition_level_encoding_; }
+
+  Encoding::type def_level_encoding() const { return definition_level_encoding_; }
+
+ private:
+  InMemoryOutputStream* sink_;
+
+  int32_t num_values_;
+  Encoding::type encoding_;
+  Encoding::type definition_level_encoding_;
+  Encoding::type repetition_level_encoding_;
+
+  bool have_def_levels_;
+  bool have_rep_levels_;
+  bool have_values_;
+
+  // Used internally for both repetition and definition levels
+  void AppendLevels(
+      const vector<int16_t>& levels, int16_t max_level, Encoding::type encoding) {
+    if (encoding != Encoding::RLE) {
+      ParquetException::NYI("only rle encoding currently implemented");
+    }
+
+    // TODO: compute a more precise maximum size for the encoded levels
+    vector<uint8_t> encode_buffer(levels.size() * 2);
+
+    // We encode into separate memory from the output stream because the
+    // RLE-encoded bytes have to be preceded in the stream by their absolute
+    // size.
+    LevelEncoder encoder;
+    encoder.Init(encoding, max_level, static_cast<int>(levels.size()),
+        encode_buffer.data(), static_cast<int>(encode_buffer.size()));
+
+    encoder.Encode(static_cast<int>(levels.size()), levels.data());
+
+    int32_t rle_bytes = encoder.len();
+    sink_->Write(reinterpret_cast<const uint8_t*>(&rle_bytes), sizeof(int32_t));
+    sink_->Write(encode_buffer.data(), rle_bytes);
+  }
+};
+
+template <>
+void DataPageBuilder<BooleanType>::AppendValues(
+    const ColumnDescriptor* d, const vector<bool>& values, Encoding::type encoding) {
+  if (encoding != Encoding::PLAIN) {
+    ParquetException::NYI("only plain encoding currently implemented");
+  }
+  PlainEncoder<BooleanType> encoder(d);
+  encoder.Put(values, static_cast<int>(values.size()));
+  std::shared_ptr<Buffer> buffer = encoder.FlushValues();
+  sink_->Write(buffer->data(), buffer->size());
+
+  num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);
+  encoding_ = encoding;
+  have_values_ = true;
+}
+
+template <typename Type>
+static shared_ptr<DataPage> MakeDataPage(const ColumnDescriptor* d,
+    const vector<typename Type::c_type>& values, int num_vals, Encoding::type encoding,
+    const uint8_t* indices, int indices_size, const vector<int16_t>& def_levels,
+    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level) {
+  int num_values = 0;
+
+  InMemoryOutputStream page_stream;
+  test::DataPageBuilder<Type> page_builder(&page_stream);
+
+  if (!rep_levels.empty()) { page_builder.AppendRepLevels(rep_levels, max_rep_level); }
+  if (!def_levels.empty()) { page_builder.AppendDefLevels(def_levels, max_def_level); }
+
+  if (encoding == Encoding::PLAIN) {
+    page_builder.AppendValues(d, values, encoding);
+    num_values = page_builder.num_values();
+  } else {  // DICTIONARY PAGES
+    page_stream.Write(indices, indices_size);
+    num_values = std::max(page_builder.num_values(), num_vals);
+  }
+
+  auto buffer = page_stream.GetBuffer();
+
+  return std::make_shared<DataPage>(buffer, num_values, encoding,
+      page_builder.def_level_encoding(), page_builder.rep_level_encoding());
+}
+
+template <typename TYPE>
+class DictionaryPageBuilder {
+ public:
+  typedef typename TYPE::c_type TC;
+  static constexpr int TN = TYPE::type_num;
+
+  // This class writes data and metadata to the passed inputs
+  explicit DictionaryPageBuilder(const ColumnDescriptor* d)
+      : num_dict_values_(0), have_values_(false) {
+    encoder_.reset(new DictEncoder<TYPE>(d, &pool_));
+  }
+
+  ~DictionaryPageBuilder() { pool_.FreeAll(); }
+
+  shared_ptr<Buffer> AppendValues(const vector<TC>& values) {
+    int num_values = static_cast<int>(values.size());
+    // Dictionary encoding
+    encoder_->Put(values.data(), num_values);
+    num_dict_values_ = encoder_->num_entries();
+    have_values_ = true;
+    return encoder_->FlushValues();
+  }
+
+  shared_ptr<Buffer> WriteDict() {
+    std::shared_ptr<PoolBuffer> dict_buffer =
+        AllocateBuffer(::arrow::default_memory_pool(), encoder_->dict_encoded_size());
+    encoder_->WriteDict(dict_buffer->mutable_data());
+    return dict_buffer;
+  }
+
+  int32_t num_values() const { return num_dict_values_; }
+
+ private:
+  ChunkedAllocator pool_;
+  shared_ptr<DictEncoder<TYPE>> encoder_;
+  int32_t num_dict_values_;
+  bool have_values_;
+};
+
+template <>
+DictionaryPageBuilder<BooleanType>::DictionaryPageBuilder(const ColumnDescriptor* d) {
+  ParquetException::NYI("only plain encoding currently implemented for boolean");
+}
+
+template <>
+shared_ptr<Buffer> DictionaryPageBuilder<BooleanType>::WriteDict() {
+  ParquetException::NYI("only plain encoding currently implemented for boolean");
+  return nullptr;
+}
+
+template <>
+shared_ptr<Buffer> DictionaryPageBuilder<BooleanType>::AppendValues(
+    const vector<TC>& values) {
+  ParquetException::NYI("only plain encoding currently implemented for boolean");
+  return nullptr;
+}
+
+template <typename Type>
+static shared_ptr<DictionaryPage> MakeDictPage(const ColumnDescriptor* d,
+    const vector<typename Type::c_type>& values, const vector<int>& values_per_page,
+    Encoding::type encoding, vector<shared_ptr<Buffer>>& rle_indices) {
+  InMemoryOutputStream page_stream;
+  test::DictionaryPageBuilder<Type> page_builder(d);
+  int num_pages = static_cast<int>(values_per_page.size());
+  int value_start = 0;
+
+  for (int i = 0; i < num_pages; i++) {
+    rle_indices.push_back(page_builder.AppendValues(
+        slice(values, value_start, value_start + values_per_page[i])));
+    value_start += values_per_page[i];
+  }
+
+  auto buffer = page_builder.WriteDict();
+
+  return std::make_shared<DictionaryPage>(
+      buffer, page_builder.num_values(), Encoding::PLAIN);
+}
+
+// Given def/rep levels and values create multiple dict pages
+template <typename Type>
+static void PaginateDict(const ColumnDescriptor* d,
+    const vector<typename Type::c_type>& values, const vector<int16_t>& def_levels,
+    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level,
+    int num_levels_per_page, const vector<int>& values_per_page,
+    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::RLE_DICTIONARY) {
+  int num_pages = static_cast<int>(values_per_page.size());
+  vector<shared_ptr<Buffer>> rle_indices;
+  shared_ptr<DictionaryPage> dict_page =
+      MakeDictPage<Type>(d, values, values_per_page, encoding, rle_indices);
+  pages.push_back(dict_page);
+  int def_level_start = 0;
+  int def_level_end = 0;
+  int rep_level_start = 0;
+  int rep_level_end = 0;
+  for (int i = 0; i < num_pages; i++) {
+    if (max_def_level > 0) {
+      def_level_start = i * num_levels_per_page;
+      def_level_end = (i + 1) * num_levels_per_page;
+    }
+    if (max_rep_level > 0) {
+      rep_level_start = i * num_levels_per_page;
+      rep_level_end = (i + 1) * num_levels_per_page;
+    }
+    shared_ptr<DataPage> data_page = MakeDataPage<Int32Type>(d, {}, values_per_page[i],
+        encoding, rle_indices[i]->data(), static_cast<int>(rle_indices[i]->size()),
+        slice(def_levels, def_level_start, def_level_end), max_def_level,
+        slice(rep_levels, rep_level_start, rep_level_end), max_rep_level);
+    pages.push_back(data_page);
+  }
+}
+
+// Given def/rep levels and values create multiple plain pages
+template <typename Type>
+static void PaginatePlain(const ColumnDescriptor* d,
+    const vector<typename Type::c_type>& values, const vector<int16_t>& def_levels,
+    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level,
+    int num_levels_per_page, const vector<int>& values_per_page,
+    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::PLAIN) {
+  int num_pages = static_cast<int>(values_per_page.size());
+  int def_level_start = 0;
+  int def_level_end = 0;
+  int rep_level_start = 0;
+  int rep_level_end = 0;
+  int value_start = 0;
+  for (int i = 0; i < num_pages; i++) {
+    if (max_def_level > 0) {
+      def_level_start = i * num_levels_per_page;
+      def_level_end = (i + 1) * num_levels_per_page;
+    }
+    if (max_rep_level > 0) {
+      rep_level_start = i * num_levels_per_page;
+      rep_level_end = (i + 1) * num_levels_per_page;
+    }
+    shared_ptr<DataPage> page = MakeDataPage<Type>(d,
+        slice(values, value_start, value_start + values_per_page[i]), values_per_page[i],
+        encoding, NULL, 0, slice(def_levels, def_level_start, def_level_end),
+        max_def_level, slice(rep_levels, rep_level_start, rep_level_end), max_rep_level);
+    pages.push_back(page);
+    value_start += values_per_page[i];
+  }
+}
+
+// Generates pages from randomly generated data
+template <typename Type>
+static int MakePages(const ColumnDescriptor* d, int num_pages, int levels_per_page,
+    vector<int16_t>& def_levels, vector<int16_t>& rep_levels,
+    vector<typename Type::c_type>& values, vector<uint8_t>& buffer,
+    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::PLAIN) {
+  int num_levels = levels_per_page * num_pages;
+  int num_values = 0;
+  uint32_t seed = 0;
+  int16_t zero = 0;
+  int16_t max_def_level = d->max_definition_level();
+  int16_t max_rep_level = d->max_repetition_level();
+  vector<int> values_per_page(num_pages, levels_per_page);
+  // Create definition levels
+  if (max_def_level > 0) {
+    def_levels.resize(num_levels);
+    random_numbers(num_levels, seed, zero, max_def_level, def_levels.data());
+    for (int p = 0; p < num_pages; p++) {
+      int num_values_per_page = 0;
+      for (int i = 0; i < levels_per_page; i++) {
+        if (def_levels[i + p * levels_per_page] == max_def_level) {
+          num_values_per_page++;
+          num_values++;
+        }
+      }
+      values_per_page[p] = num_values_per_page;
+    }
+  } else {
+    num_values = num_levels;
+  }
+  // Create repitition levels
+  if (max_rep_level > 0) {
+    rep_levels.resize(num_levels);
+    random_numbers(num_levels, seed, zero, max_rep_level, rep_levels.data());
+  }
+  // Create values
+  values.resize(num_values);
+  if (encoding == Encoding::PLAIN) {
+    InitValues<typename Type::c_type>(num_values, values, buffer);
+    PaginatePlain<Type>(d, values, def_levels, max_def_level, rep_levels, max_rep_level,
+        levels_per_page, values_per_page, pages);
+  } else if (encoding == Encoding::RLE_DICTIONARY ||
+             encoding == Encoding::PLAIN_DICTIONARY) {
+    // Calls InitValues and repeats the data
+    InitDictValues<typename Type::c_type>(num_values, levels_per_page, values, buffer);
+    PaginateDict<Type>(d, values, def_levels, max_def_level, rep_levels, max_rep_level,
+        levels_per_page, values_per_page, pages);
+  }
+
+  return num_values;
+}
+
+}  // namespace test
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_TEST_UTIL_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/util/memory.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/memory.h b/src/parquet/util/memory.h
index 4f780c4..2e7eb0f 100644
--- a/src/parquet/util/memory.h
+++ b/src/parquet/util/memory.h
@@ -46,20 +46,16 @@ static inline std::unique_ptr<::arrow::Codec> GetCodecFromArrow(Compression::typ
     case Compression::UNCOMPRESSED:
       break;
     case Compression::SNAPPY:
-      PARQUET_THROW_NOT_OK(
-          ::arrow::Codec::Create(::arrow::Compression::SNAPPY, &result));
+      PARQUET_THROW_NOT_OK(::arrow::Codec::Create(::arrow::Compression::SNAPPY, &result));
       break;
     case Compression::GZIP:
-      PARQUET_THROW_NOT_OK(
-          ::arrow::Codec::Create(::arrow::Compression::GZIP, &result));
+      PARQUET_THROW_NOT_OK(::arrow::Codec::Create(::arrow::Compression::GZIP, &result));
       break;
     case Compression::LZO:
-      PARQUET_THROW_NOT_OK(
-          ::arrow::Codec::Create(::arrow::Compression::LZO, &result));
+      PARQUET_THROW_NOT_OK(::arrow::Codec::Create(::arrow::Compression::LZO, &result));
       break;
     case Compression::BROTLI:
-      PARQUET_THROW_NOT_OK(
-          ::arrow::Codec::Create(::arrow::Compression::BROTLI, &result));
+      PARQUET_THROW_NOT_OK(::arrow::Codec::Create(::arrow::Compression::BROTLI, &result));
       break;
     default:
       break;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/util/schema-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/schema-util.h b/src/parquet/util/schema-util.h
index 618d21e..e199c21 100644
--- a/src/parquet/util/schema-util.h
+++ b/src/parquet/util/schema-util.h
@@ -19,8 +19,8 @@
 #define PARQUET_SCHEMA_UTIL_H
 
 #include <string>
-#include <vector>
 #include <unordered_set>
+#include <vector>
 
 #include "parquet/exception.h"
 #include "parquet/schema.h"
@@ -43,8 +43,7 @@ inline bool str_endswith_tuple(const std::string& str) {
 //   If the name is array or ends in _tuple, this should be a list of struct
 //   even for single child elements.
 inline bool HasStructListName(const GroupNode& node) {
-  return (node.name() == "array" ||
-          str_endswith_tuple(node.name()));
+  return (node.name() == "array" || str_endswith_tuple(node.name()));
 }
 
 // TODO(itaiin): This aux. function is to be deleted once repeated structs are supported
@@ -53,8 +52,8 @@ inline bool IsSimpleStruct(const NodePtr& node) {
   if (node->is_repeated()) return false;
   if (node->logical_type() == LogicalType::LIST) return false;
   // Special case mentioned in the format spec:
-    //   If the name is array or ends in _tuple, this should be a list of struct
-    //   even for single child elements.
+  //   If the name is array or ends in _tuple, this should be a list of struct
+  //   even for single child elements.
   auto group = static_cast<const GroupNode*>(node.get());
   if (group->field_count() == 1 && HasStructListName(*group)) return false;
 
@@ -71,9 +70,7 @@ inline bool ColumnIndicesToFieldIndices(const SchemaDescriptor& descr,
   for (auto& column_idx : column_indices) {
     auto field_node = descr.GetColumnRoot(column_idx);
     auto field_idx = group->FieldIndex(field_node->name());
-    if (field_idx < 0) {
-      return false;
-    }
+    if (field_idx < 0) { return false; }
     auto insertion = already_added.insert(field_idx);
     if (insertion.second) { out->push_back(field_idx); }
   }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/util/visibility.h
----------------------------------------------------------------------
diff --git a/src/parquet/util/visibility.h b/src/parquet/util/visibility.h
index 1601335..984fac2 100644
--- a/src/parquet/util/visibility.h
+++ b/src/parquet/util/visibility.h
@@ -21,7 +21,8 @@
 #if defined(_WIN32) || defined(__CYGWIN__)
 #ifdef _MSC_VER
 #pragma warning(push)
-// Disable warning for STL types usage in DLL interface https://web.archive.org/web/20130317015847/http://connect.microsoft.com/VisualStudio/feedback/details/696593/vc-10-vs-2010-basic-string-exports
+// Disable warning for STL types usage in DLL interface
+// https://web.archive.org/web/20130317015847/http://connect.microsoft.com/VisualStudio/feedback/details/696593/vc-10-vs-2010-basic-string-exports
 #pragma warning(disable : 4275 4251)
 // Disable diamond inheritance warnings
 #pragma warning(disable : 4250)


[2/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Posted by uw...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_writer-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_writer-test.cc b/src/parquet/column_writer-test.cc
new file mode 100644
index 0000000..798c7ba
--- /dev/null
+++ b/src/parquet/column_writer-test.cc
@@ -0,0 +1,729 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/file/reader-internal.h"
+#include "parquet/file/writer-internal.h"
+#include "parquet/test-specialization.h"
+#include "parquet/test-util.h"
+#include "parquet/types.h"
+#include "parquet/util/comparison.h"
+#include "parquet/util/memory.h"
+
+namespace parquet {
+
+using schema::NodePtr;
+using schema::PrimitiveNode;
+
+namespace test {
+
+// The default size used in most tests.
+const int SMALL_SIZE = 100;
+// Larger size to test some corner cases, only used in some specific cases.
+const int LARGE_SIZE = 100000;
+// Very large size to test dictionary fallback.
+const int VERY_LARGE_SIZE = 400000;
+
+template <typename TestType>
+class TestPrimitiveWriter : public PrimitiveTypedTest<TestType> {
+ public:
+  typedef typename TestType::c_type T;
+
+  void SetUp() {
+    this->SetupValuesOut(SMALL_SIZE);
+    writer_properties_ = default_writer_properties();
+    definition_levels_out_.resize(SMALL_SIZE);
+    repetition_levels_out_.resize(SMALL_SIZE);
+
+    this->SetUpSchema(Repetition::REQUIRED);
+
+    descr_ = this->schema_.Column(0);
+  }
+
+  Type::type type_num() { return TestType::type_num; }
+
+  void BuildReader(
+      int64_t num_rows, Compression::type compression = Compression::UNCOMPRESSED) {
+    auto buffer = sink_->GetBuffer();
+    std::unique_ptr<InMemoryInputStream> source(new InMemoryInputStream(buffer));
+    std::unique_ptr<SerializedPageReader> page_reader(
+        new SerializedPageReader(std::move(source), num_rows, compression));
+    reader_.reset(new TypedColumnReader<TestType>(this->descr_, std::move(page_reader)));
+  }
+
+  std::shared_ptr<TypedColumnWriter<TestType>> BuildWriter(
+      int64_t output_size = SMALL_SIZE,
+      const ColumnProperties& column_properties = ColumnProperties()) {
+    sink_.reset(new InMemoryOutputStream());
+    metadata_ = ColumnChunkMetaDataBuilder::Make(
+        writer_properties_, this->descr_, reinterpret_cast<uint8_t*>(&thrift_metadata_));
+    std::unique_ptr<SerializedPageWriter> pager(
+        new SerializedPageWriter(sink_.get(), column_properties.codec, metadata_.get()));
+    WriterProperties::Builder wp_builder;
+    if (column_properties.encoding == Encoding::PLAIN_DICTIONARY ||
+        column_properties.encoding == Encoding::RLE_DICTIONARY) {
+      wp_builder.enable_dictionary();
+    } else {
+      wp_builder.disable_dictionary();
+      wp_builder.encoding(column_properties.encoding);
+    }
+    writer_properties_ = wp_builder.build();
+    std::shared_ptr<ColumnWriter> writer = ColumnWriter::Make(
+        metadata_.get(), std::move(pager), output_size, writer_properties_.get());
+    return std::static_pointer_cast<TypedColumnWriter<TestType>>(writer);
+  }
+
+  void ReadColumn(Compression::type compression = Compression::UNCOMPRESSED) {
+    BuildReader(static_cast<int64_t>(this->values_out_.size()), compression);
+    reader_->ReadBatch(static_cast<int>(this->values_out_.size()),
+        definition_levels_out_.data(), repetition_levels_out_.data(),
+        this->values_out_ptr_, &values_read_);
+    this->SyncValuesOut();
+  }
+
+  void ReadColumnFully(Compression::type compression = Compression::UNCOMPRESSED);
+
+  void TestRequiredWithEncoding(Encoding::type encoding) {
+    return TestRequiredWithSettings(encoding, Compression::UNCOMPRESSED, false, false);
+  }
+
+  void TestRequiredWithSettings(Encoding::type encoding, Compression::type compression,
+      bool enable_dictionary, bool enable_statistics, int64_t num_rows = SMALL_SIZE) {
+    this->GenerateData(num_rows);
+
+    this->WriteRequiredWithSettings(
+        encoding, compression, enable_dictionary, enable_statistics, num_rows);
+    this->ReadAndCompare(compression, num_rows);
+
+    this->WriteRequiredWithSettingsSpaced(
+        encoding, compression, enable_dictionary, enable_statistics, num_rows);
+    this->ReadAndCompare(compression, num_rows);
+  }
+
+  void WriteRequiredWithSettings(Encoding::type encoding, Compression::type compression,
+      bool enable_dictionary, bool enable_statistics, int64_t num_rows) {
+    ColumnProperties column_properties(
+        encoding, compression, enable_dictionary, enable_statistics);
+    std::shared_ptr<TypedColumnWriter<TestType>> writer =
+        this->BuildWriter(num_rows, column_properties);
+    writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
+    // The behaviour should be independent from the number of Close() calls
+    writer->Close();
+    writer->Close();
+  }
+
+  void WriteRequiredWithSettingsSpaced(Encoding::type encoding,
+      Compression::type compression, bool enable_dictionary, bool enable_statistics,
+      int64_t num_rows) {
+    std::vector<uint8_t> valid_bits(
+        BitUtil::RoundUpNumBytes(static_cast<uint32_t>(this->values_.size())) + 1, 255);
+    ColumnProperties column_properties(
+        encoding, compression, enable_dictionary, enable_statistics);
+    std::shared_ptr<TypedColumnWriter<TestType>> writer =
+        this->BuildWriter(num_rows, column_properties);
+    writer->WriteBatchSpaced(
+        this->values_.size(), nullptr, nullptr, valid_bits.data(), 0, this->values_ptr_);
+    // The behaviour should be independent from the number of Close() calls
+    writer->Close();
+    writer->Close();
+  }
+
+  void ReadAndCompare(Compression::type compression, int64_t num_rows) {
+    this->SetupValuesOut(num_rows);
+    this->ReadColumnFully(compression);
+    Compare<T> compare(this->descr_);
+    for (size_t i = 0; i < this->values_.size(); i++) {
+      if (compare(this->values_[i], this->values_out_[i]) ||
+          compare(this->values_out_[i], this->values_[i])) {
+        std::cout << "Failed at " << i << std::endl;
+      }
+      ASSERT_FALSE(compare(this->values_[i], this->values_out_[i]));
+      ASSERT_FALSE(compare(this->values_out_[i], this->values_[i]));
+    }
+    ASSERT_EQ(this->values_, this->values_out_);
+  }
+
+  int64_t metadata_num_values() {
+    // Metadata accessor must be created lazily.
+    // This is because the ColumnChunkMetaData semantics dictate the metadata object is
+    // complete (no changes to the metadata buffer can be made after instantiation)
+    auto metadata_accessor = ColumnChunkMetaData::Make(
+        reinterpret_cast<const uint8_t*>(&thrift_metadata_), this->descr_);
+    return metadata_accessor->num_values();
+  }
+
+  std::vector<Encoding::type> metadata_encodings() {
+    // Metadata accessor must be created lazily.
+    // This is because the ColumnChunkMetaData semantics dictate the metadata object is
+    // complete (no changes to the metadata buffer can be made after instantiation)
+    auto metadata_accessor = ColumnChunkMetaData::Make(
+        reinterpret_cast<const uint8_t*>(&thrift_metadata_), this->descr_);
+    return metadata_accessor->encodings();
+  }
+
+ protected:
+  int64_t values_read_;
+  // Keep the reader alive as for ByteArray the lifetime of the ByteArray
+  // content is bound to the reader.
+  std::unique_ptr<TypedColumnReader<TestType>> reader_;
+
+  std::vector<int16_t> definition_levels_out_;
+  std::vector<int16_t> repetition_levels_out_;
+
+  const ColumnDescriptor* descr_;
+
+ private:
+  format::ColumnChunk thrift_metadata_;
+  std::unique_ptr<ColumnChunkMetaDataBuilder> metadata_;
+  std::unique_ptr<InMemoryOutputStream> sink_;
+  std::shared_ptr<WriterProperties> writer_properties_;
+  std::vector<std::vector<uint8_t>> data_buffer_;
+};
+
+template <typename TestType>
+void TestPrimitiveWriter<TestType>::ReadColumnFully(Compression::type compression) {
+  int64_t total_values = static_cast<int64_t>(this->values_out_.size());
+  BuildReader(total_values, compression);
+  values_read_ = 0;
+  while (values_read_ < total_values) {
+    int64_t values_read_recently = 0;
+    reader_->ReadBatch(
+        static_cast<int>(this->values_out_.size()) - static_cast<int>(values_read_),
+        definition_levels_out_.data() + values_read_,
+        repetition_levels_out_.data() + values_read_,
+        this->values_out_ptr_ + values_read_, &values_read_recently);
+    values_read_ += values_read_recently;
+  }
+  this->SyncValuesOut();
+}
+
+template <>
+void TestPrimitiveWriter<FLBAType>::ReadColumnFully(Compression::type compression) {
+  int64_t total_values = static_cast<int64_t>(this->values_out_.size());
+  BuildReader(total_values, compression);
+  this->data_buffer_.clear();
+
+  values_read_ = 0;
+  while (values_read_ < total_values) {
+    int64_t values_read_recently = 0;
+    reader_->ReadBatch(
+        static_cast<int>(this->values_out_.size()) - static_cast<int>(values_read_),
+        definition_levels_out_.data() + values_read_,
+        repetition_levels_out_.data() + values_read_,
+        this->values_out_ptr_ + values_read_, &values_read_recently);
+
+    // Copy contents of the pointers
+    std::vector<uint8_t> data(values_read_recently * this->descr_->type_length());
+    uint8_t* data_ptr = data.data();
+    for (int64_t i = 0; i < values_read_recently; i++) {
+      memcpy(data_ptr + this->descr_->type_length() * i,
+          this->values_out_[i + values_read_].ptr, this->descr_->type_length());
+      this->values_out_[i + values_read_].ptr =
+          data_ptr + this->descr_->type_length() * i;
+    }
+    data_buffer_.emplace_back(std::move(data));
+
+    values_read_ += values_read_recently;
+  }
+  this->SyncValuesOut();
+}
+
+typedef ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
+    BooleanType, ByteArrayType, FLBAType>
+    TestTypes;
+
+TYPED_TEST_CASE(TestPrimitiveWriter, TestTypes);
+
+using TestNullValuesWriter = TestPrimitiveWriter<Int32Type>;
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlain) {
+  this->TestRequiredWithEncoding(Encoding::PLAIN);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredDictionary) {
+  this->TestRequiredWithEncoding(Encoding::PLAIN_DICTIONARY);
+}
+
+/*
+TYPED_TEST(TestPrimitiveWriter, RequiredRLE) {
+  this->TestRequiredWithEncoding(Encoding::RLE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredBitPacked) {
+  this->TestRequiredWithEncoding(Encoding::BIT_PACKED);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredDeltaBinaryPacked) {
+  this->TestRequiredWithEncoding(Encoding::DELTA_BINARY_PACKED);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredDeltaLengthByteArray) {
+  this->TestRequiredWithEncoding(Encoding::DELTA_LENGTH_BYTE_ARRAY);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredDeltaByteArray) {
+  this->TestRequiredWithEncoding(Encoding::DELTA_BYTE_ARRAY);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredRLEDictionary) {
+  this->TestRequiredWithEncoding(Encoding::RLE_DICTIONARY);
+}
+*/
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithSnappyCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::SNAPPY, false, false, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithBrotliCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::BROTLI, false, false, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithGzipCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::GZIP, false, false, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStats) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::UNCOMPRESSED, false, true, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndSnappyCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::SNAPPY, false, true, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndBrotliCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::BROTLI, false, true, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndGzipCompression) {
+  this->TestRequiredWithSettings(
+      Encoding::PLAIN, Compression::GZIP, false, true, LARGE_SIZE);
+}
+
+TYPED_TEST(TestPrimitiveWriter, Optional) {
+  // Optional and non-repeated, with definition levels
+  // but no repetition levels
+  this->SetUpSchema(Repetition::OPTIONAL);
+
+  this->GenerateData(SMALL_SIZE);
+  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
+  definition_levels[1] = 0;
+
+  auto writer = this->BuildWriter();
+  writer->WriteBatch(
+      this->values_.size(), definition_levels.data(), nullptr, this->values_ptr_);
+  writer->Close();
+
+  // PARQUET-703
+  ASSERT_EQ(100, this->metadata_num_values());
+
+  this->ReadColumn();
+  ASSERT_EQ(99, this->values_read_);
+  this->values_out_.resize(99);
+  this->values_.resize(99);
+  ASSERT_EQ(this->values_, this->values_out_);
+}
+
+TYPED_TEST(TestPrimitiveWriter, OptionalSpaced) {
+  // Optional and non-repeated, with definition levels
+  // but no repetition levels
+  this->SetUpSchema(Repetition::OPTIONAL);
+
+  this->GenerateData(SMALL_SIZE);
+  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
+  std::vector<uint8_t> valid_bits(::arrow::BitUtil::BytesForBits(SMALL_SIZE), 255);
+
+  definition_levels[SMALL_SIZE - 1] = 0;
+  ::arrow::BitUtil::ClearBit(valid_bits.data(), SMALL_SIZE - 1);
+  definition_levels[1] = 0;
+  ::arrow::BitUtil::ClearBit(valid_bits.data(), 1);
+
+  auto writer = this->BuildWriter();
+  writer->WriteBatchSpaced(this->values_.size(), definition_levels.data(), nullptr,
+      valid_bits.data(), 0, this->values_ptr_);
+  writer->Close();
+
+  // PARQUET-703
+  ASSERT_EQ(100, this->metadata_num_values());
+
+  this->ReadColumn();
+  ASSERT_EQ(98, this->values_read_);
+  this->values_out_.resize(98);
+  this->values_.resize(99);
+  this->values_.erase(this->values_.begin() + 1);
+  ASSERT_EQ(this->values_, this->values_out_);
+}
+
+TYPED_TEST(TestPrimitiveWriter, Repeated) {
+  // Optional and repeated, so definition and repetition levels
+  this->SetUpSchema(Repetition::REPEATED);
+
+  this->GenerateData(SMALL_SIZE);
+  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
+  definition_levels[1] = 0;
+  std::vector<int16_t> repetition_levels(SMALL_SIZE, 0);
+
+  auto writer = this->BuildWriter();
+  writer->WriteBatch(this->values_.size(), definition_levels.data(),
+      repetition_levels.data(), this->values_ptr_);
+  writer->Close();
+
+  this->ReadColumn();
+  ASSERT_EQ(SMALL_SIZE - 1, this->values_read_);
+  this->values_out_.resize(SMALL_SIZE - 1);
+  this->values_.resize(SMALL_SIZE - 1);
+  ASSERT_EQ(this->values_, this->values_out_);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredTooFewRows) {
+  this->GenerateData(SMALL_SIZE - 1);
+
+  auto writer = this->BuildWriter();
+  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
+  ASSERT_THROW(writer->Close(), ParquetException);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredTooMany) {
+  this->GenerateData(2 * SMALL_SIZE);
+
+  auto writer = this->BuildWriter();
+  ASSERT_THROW(
+      writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_),
+      ParquetException);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RepeatedTooFewRows) {
+  // Optional and repeated, so definition and repetition levels
+  this->SetUpSchema(Repetition::REPEATED);
+
+  this->GenerateData(SMALL_SIZE);
+  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
+  definition_levels[1] = 0;
+  std::vector<int16_t> repetition_levels(SMALL_SIZE, 0);
+  repetition_levels[3] = 1;
+
+  auto writer = this->BuildWriter();
+  writer->WriteBatch(this->values_.size(), definition_levels.data(),
+      repetition_levels.data(), this->values_ptr_);
+  ASSERT_THROW(writer->Close(), ParquetException);
+}
+
+TYPED_TEST(TestPrimitiveWriter, RequiredLargeChunk) {
+  this->GenerateData(LARGE_SIZE);
+
+  // Test case 1: required and non-repeated, so no definition or repetition levels
+  auto writer = this->BuildWriter(LARGE_SIZE);
+  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
+  writer->Close();
+
+  // Just read the first SMALL_SIZE rows to ensure we could read it back in
+  this->ReadColumn();
+  ASSERT_EQ(SMALL_SIZE, this->values_read_);
+  this->values_.resize(SMALL_SIZE);
+  ASSERT_EQ(this->values_, this->values_out_);
+}
+
+// Test case for dictionary fallback encoding
+TYPED_TEST(TestPrimitiveWriter, RequiredVeryLargeChunk) {
+  this->GenerateData(VERY_LARGE_SIZE);
+
+  auto writer = this->BuildWriter(VERY_LARGE_SIZE, Encoding::PLAIN_DICTIONARY);
+  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
+  writer->Close();
+
+  // Read all rows so we are sure that also the non-dictionary pages are read correctly
+  this->SetupValuesOut(VERY_LARGE_SIZE);
+  this->ReadColumnFully();
+  ASSERT_EQ(VERY_LARGE_SIZE, this->values_read_);
+  this->values_.resize(VERY_LARGE_SIZE);
+  ASSERT_EQ(this->values_, this->values_out_);
+  std::vector<Encoding::type> encodings = this->metadata_encodings();
+  // There are 3 encodings (RLE, PLAIN_DICTIONARY, PLAIN) in a fallback case
+  // Dictionary encoding is not allowed for boolean type
+  // There are 2 encodings (RLE, PLAIN) in a non dictionary encoding case
+  if (this->type_num() != Type::BOOLEAN) {
+    ASSERT_EQ(Encoding::PLAIN_DICTIONARY, encodings[0]);
+    ASSERT_EQ(Encoding::PLAIN, encodings[1]);
+    ASSERT_EQ(Encoding::RLE, encodings[2]);
+  } else {
+    ASSERT_EQ(Encoding::PLAIN, encodings[0]);
+    ASSERT_EQ(Encoding::RLE, encodings[1]);
+  }
+}
+
+// PARQUET-719
+// Test case for NULL values
+TEST_F(TestNullValuesWriter, OptionalNullValueChunk) {
+  this->SetUpSchema(Repetition::OPTIONAL);
+
+  this->GenerateData(LARGE_SIZE);
+
+  std::vector<int16_t> definition_levels(LARGE_SIZE, 0);
+  std::vector<int16_t> repetition_levels(LARGE_SIZE, 0);
+
+  auto writer = this->BuildWriter(LARGE_SIZE);
+  // All values being written are NULL
+  writer->WriteBatch(
+      this->values_.size(), definition_levels.data(), repetition_levels.data(), NULL);
+  writer->Close();
+
+  // Just read the first SMALL_SIZE rows to ensure we could read it back in
+  this->ReadColumn();
+  ASSERT_EQ(0, this->values_read_);
+}
+
+// PARQUET-764
+// Correct bitpacking for boolean write at non-byte boundaries
+using TestBooleanValuesWriter = TestPrimitiveWriter<BooleanType>;
+TEST_F(TestBooleanValuesWriter, AlternateBooleanValues) {
+  this->SetUpSchema(Repetition::REQUIRED);
+  auto writer = this->BuildWriter();
+  for (int i = 0; i < SMALL_SIZE; i++) {
+    bool value = (i % 2 == 0) ? true : false;
+    writer->WriteBatch(1, nullptr, nullptr, &value);
+  }
+  writer->Close();
+  this->ReadColumn();
+  for (int i = 0; i < SMALL_SIZE; i++) {
+    ASSERT_EQ((i % 2 == 0) ? true : false, this->values_out_[i]) << i;
+  }
+}
+
+void GenerateLevels(int min_repeat_factor, int max_repeat_factor, int max_level,
+    std::vector<int16_t>& input_levels) {
+  // for each repetition count upto max_repeat_factor
+  for (int repeat = min_repeat_factor; repeat <= max_repeat_factor; repeat++) {
+    // repeat count increases by a factor of 2 for every iteration
+    int repeat_count = (1 << repeat);
+    // generate levels for repetition count upto the maximum level
+    int value = 0;
+    int bwidth = 0;
+    while (value <= max_level) {
+      for (int i = 0; i < repeat_count; i++) {
+        input_levels.push_back(value);
+      }
+      value = (2 << bwidth) - 1;
+      bwidth++;
+    }
+  }
+}
+
+void EncodeLevels(Encoding::type encoding, int max_level, int num_levels,
+    const int16_t* input_levels, std::vector<uint8_t>& bytes) {
+  LevelEncoder encoder;
+  int levels_count = 0;
+  bytes.resize(2 * num_levels);
+  ASSERT_EQ(2 * num_levels, static_cast<int>(bytes.size()));
+  // encode levels
+  if (encoding == Encoding::RLE) {
+    // leave space to write the rle length value
+    encoder.Init(encoding, max_level, num_levels, bytes.data() + sizeof(int32_t),
+        static_cast<int>(bytes.size()));
+
+    levels_count = encoder.Encode(num_levels, input_levels);
+    (reinterpret_cast<int32_t*>(bytes.data()))[0] = encoder.len();
+  } else {
+    encoder.Init(
+        encoding, max_level, num_levels, bytes.data(), static_cast<int>(bytes.size()));
+    levels_count = encoder.Encode(num_levels, input_levels);
+  }
+  ASSERT_EQ(num_levels, levels_count);
+}
+
+void VerifyDecodingLevels(Encoding::type encoding, int max_level,
+    std::vector<int16_t>& input_levels, std::vector<uint8_t>& bytes) {
+  LevelDecoder decoder;
+  int levels_count = 0;
+  std::vector<int16_t> output_levels;
+  int num_levels = static_cast<int>(input_levels.size());
+
+  output_levels.resize(num_levels);
+  ASSERT_EQ(num_levels, static_cast<int>(output_levels.size()));
+
+  // Decode levels and test with multiple decode calls
+  decoder.SetData(encoding, max_level, num_levels, bytes.data());
+  int decode_count = 4;
+  int num_inner_levels = num_levels / decode_count;
+  // Try multiple decoding on a single SetData call
+  for (int ct = 0; ct < decode_count; ct++) {
+    int offset = ct * num_inner_levels;
+    levels_count = decoder.Decode(num_inner_levels, output_levels.data());
+    ASSERT_EQ(num_inner_levels, levels_count);
+    for (int i = 0; i < num_inner_levels; i++) {
+      EXPECT_EQ(input_levels[i + offset], output_levels[i]);
+    }
+  }
+  // check the remaining levels
+  int num_levels_completed = decode_count * (num_levels / decode_count);
+  int num_remaining_levels = num_levels - num_levels_completed;
+  if (num_remaining_levels > 0) {
+    levels_count = decoder.Decode(num_remaining_levels, output_levels.data());
+    ASSERT_EQ(num_remaining_levels, levels_count);
+    for (int i = 0; i < num_remaining_levels; i++) {
+      EXPECT_EQ(input_levels[i + num_levels_completed], output_levels[i]);
+    }
+  }
+  // Test zero Decode values
+  ASSERT_EQ(0, decoder.Decode(1, output_levels.data()));
+}
+
+void VerifyDecodingMultipleSetData(Encoding::type encoding, int max_level,
+    std::vector<int16_t>& input_levels, std::vector<std::vector<uint8_t>>& bytes) {
+  LevelDecoder decoder;
+  int levels_count = 0;
+  std::vector<int16_t> output_levels;
+
+  // Decode levels and test with multiple SetData calls
+  int setdata_count = static_cast<int>(bytes.size());
+  int num_levels = static_cast<int>(input_levels.size()) / setdata_count;
+  output_levels.resize(num_levels);
+  // Try multiple SetData
+  for (int ct = 0; ct < setdata_count; ct++) {
+    int offset = ct * num_levels;
+    ASSERT_EQ(num_levels, static_cast<int>(output_levels.size()));
+    decoder.SetData(encoding, max_level, num_levels, bytes[ct].data());
+    levels_count = decoder.Decode(num_levels, output_levels.data());
+    ASSERT_EQ(num_levels, levels_count);
+    for (int i = 0; i < num_levels; i++) {
+      EXPECT_EQ(input_levels[i + offset], output_levels[i]);
+    }
+  }
+}
+
+// Test levels with maximum bit-width from 1 to 8
+// increase the repetition count for each iteration by a factor of 2
+TEST(TestLevels, TestLevelsDecodeMultipleBitWidth) {
+  int min_repeat_factor = 0;
+  int max_repeat_factor = 7;  // 128
+  int max_bit_width = 8;
+  std::vector<int16_t> input_levels;
+  std::vector<uint8_t> bytes;
+  Encoding::type encodings[2] = {Encoding::RLE, Encoding::BIT_PACKED};
+
+  // for each encoding
+  for (int encode = 0; encode < 2; encode++) {
+    Encoding::type encoding = encodings[encode];
+    // BIT_PACKED requires a sequence of atleast 8
+    if (encoding == Encoding::BIT_PACKED) min_repeat_factor = 3;
+    // for each maximum bit-width
+    for (int bit_width = 1; bit_width <= max_bit_width; bit_width++) {
+      // find the maximum level for the current bit_width
+      int max_level = (1 << bit_width) - 1;
+      // Generate levels
+      GenerateLevels(min_repeat_factor, max_repeat_factor, max_level, input_levels);
+      EncodeLevels(encoding, max_level, static_cast<int>(input_levels.size()),
+          input_levels.data(), bytes);
+      VerifyDecodingLevels(encoding, max_level, input_levels, bytes);
+      input_levels.clear();
+    }
+  }
+}
+
+// Test multiple decoder SetData calls
+TEST(TestLevels, TestLevelsDecodeMultipleSetData) {
+  int min_repeat_factor = 3;
+  int max_repeat_factor = 7;  // 128
+  int bit_width = 8;
+  int max_level = (1 << bit_width) - 1;
+  std::vector<int16_t> input_levels;
+  std::vector<std::vector<uint8_t>> bytes;
+  Encoding::type encodings[2] = {Encoding::RLE, Encoding::BIT_PACKED};
+  GenerateLevels(min_repeat_factor, max_repeat_factor, max_level, input_levels);
+  int num_levels = static_cast<int>(input_levels.size());
+  int setdata_factor = 8;
+  int split_level_size = num_levels / setdata_factor;
+  bytes.resize(setdata_factor);
+
+  // for each encoding
+  for (int encode = 0; encode < 2; encode++) {
+    Encoding::type encoding = encodings[encode];
+    for (int rf = 0; rf < setdata_factor; rf++) {
+      int offset = rf * split_level_size;
+      EncodeLevels(encoding, max_level, split_level_size,
+          reinterpret_cast<int16_t*>(input_levels.data()) + offset, bytes[rf]);
+    }
+    VerifyDecodingMultipleSetData(encoding, max_level, input_levels, bytes);
+  }
+}
+
+TEST(TestLevelEncoder, MinimumBufferSize) {
+  // PARQUET-676, PARQUET-698
+  const int kNumToEncode = 1024;
+
+  std::vector<int16_t> levels;
+  for (int i = 0; i < kNumToEncode; ++i) {
+    if (i % 9 == 0) {
+      levels.push_back(0);
+    } else {
+      levels.push_back(1);
+    }
+  }
+
+  std::vector<uint8_t> output(
+      LevelEncoder::MaxBufferSize(Encoding::RLE, 1, kNumToEncode));
+
+  LevelEncoder encoder;
+  encoder.Init(
+      Encoding::RLE, 1, kNumToEncode, output.data(), static_cast<int>(output.size()));
+  int encode_count = encoder.Encode(kNumToEncode, levels.data());
+
+  ASSERT_EQ(kNumToEncode, encode_count);
+}
+
+TEST(TestLevelEncoder, MinimumBufferSize2) {
+  // PARQUET-708
+  // Test the worst case for bit_width=2 consisting of
+  // LiteralRun(size=8)
+  // RepeatedRun(size=8)
+  // LiteralRun(size=8)
+  // ...
+  const int kNumToEncode = 1024;
+
+  std::vector<int16_t> levels;
+  for (int i = 0; i < kNumToEncode; ++i) {
+    // This forces a literal run of 00000001
+    // followed by eight 1s
+    if ((i % 16) < 7) {
+      levels.push_back(0);
+    } else {
+      levels.push_back(1);
+    }
+  }
+
+  for (int bit_width = 1; bit_width <= 8; bit_width++) {
+    std::vector<uint8_t> output(
+        LevelEncoder::MaxBufferSize(Encoding::RLE, bit_width, kNumToEncode));
+
+    LevelEncoder encoder;
+    encoder.Init(Encoding::RLE, bit_width, kNumToEncode, output.data(),
+        static_cast<int>(output.size()));
+    int encode_count = encoder.Encode(kNumToEncode, levels.data());
+
+    ASSERT_EQ(kNumToEncode, encode_count);
+  }
+}
+
+}  // namespace test
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_writer.cc b/src/parquet/column_writer.cc
new file mode 100644
index 0000000..c13d4a0
--- /dev/null
+++ b/src/parquet/column_writer.cc
@@ -0,0 +1,597 @@
+// 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.
+
+#include "parquet/column_writer.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 {
+
+LevelEncoder::LevelEncoder() {}
+LevelEncoder::~LevelEncoder() {}
+
+void LevelEncoder::Init(Encoding::type encoding, int16_t max_level,
+    int num_buffered_values, uint8_t* data, int data_size) {
+  bit_width_ = BitUtil::Log2(max_level + 1);
+  encoding_ = encoding;
+  switch (encoding) {
+    case Encoding::RLE: {
+      rle_encoder_.reset(new RleEncoder(data, data_size, bit_width_));
+      break;
+    }
+    case Encoding::BIT_PACKED: {
+      int num_bytes =
+          static_cast<int>(BitUtil::Ceil(num_buffered_values * bit_width_, 8));
+      bit_packed_encoder_.reset(new BitWriter(data, num_bytes));
+      break;
+    }
+    default:
+      throw ParquetException("Unknown encoding type for levels.");
+  }
+}
+
+int LevelEncoder::MaxBufferSize(
+    Encoding::type encoding, int16_t max_level, int num_buffered_values) {
+  int bit_width = BitUtil::Log2(max_level + 1);
+  int num_bytes = 0;
+  switch (encoding) {
+    case Encoding::RLE: {
+      // TODO: Due to the way we currently check if the buffer is full enough,
+      // we need to have MinBufferSize as head room.
+      num_bytes = RleEncoder::MaxBufferSize(bit_width, num_buffered_values) +
+                  RleEncoder::MinBufferSize(bit_width);
+      break;
+    }
+    case Encoding::BIT_PACKED: {
+      num_bytes = static_cast<int>(BitUtil::Ceil(num_buffered_values * bit_width, 8));
+      break;
+    }
+    default:
+      throw ParquetException("Unknown encoding type for levels.");
+  }
+  return num_bytes;
+}
+
+int LevelEncoder::Encode(int batch_size, const int16_t* levels) {
+  int num_encoded = 0;
+  if (!rle_encoder_ && !bit_packed_encoder_) {
+    throw ParquetException("Level encoders are not initialized.");
+  }
+
+  if (encoding_ == Encoding::RLE) {
+    for (int i = 0; i < batch_size; ++i) {
+      if (!rle_encoder_->Put(*(levels + i))) { break; }
+      ++num_encoded;
+    }
+    rle_encoder_->Flush();
+    rle_length_ = rle_encoder_->len();
+  } else {
+    for (int i = 0; i < batch_size; ++i) {
+      if (!bit_packed_encoder_->PutValue(*(levels + i), bit_width_)) { break; }
+      ++num_encoded;
+    }
+    bit_packed_encoder_->Flush();
+  }
+  return num_encoded;
+}
+
+// ----------------------------------------------------------------------
+// ColumnWriter
+
+std::shared_ptr<WriterProperties> default_writer_properties() {
+  static std::shared_ptr<WriterProperties> default_writer_properties =
+      WriterProperties::Builder().build();
+  return default_writer_properties;
+}
+
+ColumnWriter::ColumnWriter(ColumnChunkMetaDataBuilder* metadata,
+    std::unique_ptr<PageWriter> pager, int64_t expected_rows, bool has_dictionary,
+    Encoding::type encoding, const WriterProperties* properties)
+    : metadata_(metadata),
+      descr_(metadata->descr()),
+      pager_(std::move(pager)),
+      expected_rows_(expected_rows),
+      has_dictionary_(has_dictionary),
+      encoding_(encoding),
+      properties_(properties),
+      allocator_(properties->memory_pool()),
+      pool_(properties->memory_pool()),
+      num_buffered_values_(0),
+      num_buffered_encoded_values_(0),
+      num_rows_(0),
+      total_bytes_written_(0),
+      closed_(false),
+      fallback_(false) {
+  definition_levels_sink_.reset(new InMemoryOutputStream(allocator_));
+  repetition_levels_sink_.reset(new InMemoryOutputStream(allocator_));
+  definition_levels_rle_ =
+      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
+  repetition_levels_rle_ =
+      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
+  uncompressed_data_ =
+      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
+  if (pager_->has_compressor()) {
+    compressed_data_ =
+        std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
+  }
+}
+
+void ColumnWriter::InitSinks() {
+  definition_levels_sink_->Clear();
+  repetition_levels_sink_->Clear();
+}
+
+void ColumnWriter::WriteDefinitionLevels(int64_t num_levels, const int16_t* levels) {
+  DCHECK(!closed_);
+  definition_levels_sink_->Write(
+      reinterpret_cast<const uint8_t*>(levels), sizeof(int16_t) * num_levels);
+}
+
+void ColumnWriter::WriteRepetitionLevels(int64_t num_levels, const int16_t* levels) {
+  DCHECK(!closed_);
+  repetition_levels_sink_->Write(
+      reinterpret_cast<const uint8_t*>(levels), sizeof(int16_t) * num_levels);
+}
+
+// return the size of the encoded buffer
+int64_t ColumnWriter::RleEncodeLevels(
+    const Buffer& src_buffer, ResizableBuffer* dest_buffer, int16_t max_level) {
+  // TODO: This only works with due to some RLE specifics
+  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level,
+                         static_cast<int>(num_buffered_values_)) +
+                     sizeof(int32_t);
+
+  // Use Arrow::Buffer::shrink_to_fit = false
+  // underlying buffer only keeps growing. Resize to a smaller size does not reallocate.
+  PARQUET_THROW_NOT_OK(dest_buffer->Resize(rle_size, false));
+
+  level_encoder_.Init(Encoding::RLE, max_level, static_cast<int>(num_buffered_values_),
+      dest_buffer->mutable_data() + sizeof(int32_t),
+      static_cast<int>(dest_buffer->size()) - sizeof(int32_t));
+  int encoded = level_encoder_.Encode(static_cast<int>(num_buffered_values_),
+      reinterpret_cast<const int16_t*>(src_buffer.data()));
+  DCHECK_EQ(encoded, num_buffered_values_);
+  reinterpret_cast<int32_t*>(dest_buffer->mutable_data())[0] = level_encoder_.len();
+  int64_t encoded_size = level_encoder_.len() + sizeof(int32_t);
+  return encoded_size;
+}
+
+void ColumnWriter::AddDataPage() {
+  int64_t definition_levels_rle_size = 0;
+  int64_t repetition_levels_rle_size = 0;
+
+  std::shared_ptr<Buffer> values = GetValuesBuffer();
+
+  if (descr_->max_definition_level() > 0) {
+    definition_levels_rle_size = RleEncodeLevels(definition_levels_sink_->GetBufferRef(),
+        definition_levels_rle_.get(), descr_->max_definition_level());
+  }
+
+  if (descr_->max_repetition_level() > 0) {
+    repetition_levels_rle_size = RleEncodeLevels(repetition_levels_sink_->GetBufferRef(),
+        repetition_levels_rle_.get(), descr_->max_repetition_level());
+  }
+
+  int64_t uncompressed_size =
+      definition_levels_rle_size + repetition_levels_rle_size + values->size();
+
+  // Use Arrow::Buffer::shrink_to_fit = false
+  // underlying buffer only keeps growing. Resize to a smaller size does not reallocate.
+  PARQUET_THROW_NOT_OK(uncompressed_data_->Resize(uncompressed_size, false));
+
+  // Concatenate data into a single buffer
+  uint8_t* uncompressed_ptr = uncompressed_data_->mutable_data();
+  memcpy(uncompressed_ptr, repetition_levels_rle_->data(), repetition_levels_rle_size);
+  uncompressed_ptr += repetition_levels_rle_size;
+  memcpy(uncompressed_ptr, definition_levels_rle_->data(), definition_levels_rle_size);
+  uncompressed_ptr += definition_levels_rle_size;
+  memcpy(uncompressed_ptr, values->data(), values->size());
+
+  EncodedStatistics page_stats = GetPageStatistics();
+  ResetPageStatistics();
+
+  std::shared_ptr<Buffer> compressed_data;
+  if (pager_->has_compressor()) {
+    pager_->Compress(*(uncompressed_data_.get()), compressed_data_.get());
+    compressed_data = compressed_data_;
+  } else {
+    compressed_data = uncompressed_data_;
+  }
+
+  // Write the page to OutputStream eagerly if there is no dictionary or
+  // if dictionary encoding has fallen back to PLAIN
+  if (has_dictionary_ && !fallback_) {  // Save pages until end of dictionary encoding
+    std::shared_ptr<Buffer> compressed_data_copy;
+    PARQUET_THROW_NOT_OK(compressed_data->Copy(
+        0, compressed_data->size(), allocator_, &compressed_data_copy));
+    CompressedDataPage page(compressed_data_copy,
+        static_cast<int32_t>(num_buffered_values_), encoding_, Encoding::RLE,
+        Encoding::RLE, uncompressed_size, page_stats);
+    data_pages_.push_back(std::move(page));
+  } else {  // Eagerly write pages
+    CompressedDataPage page(compressed_data, static_cast<int32_t>(num_buffered_values_),
+        encoding_, Encoding::RLE, Encoding::RLE, uncompressed_size, page_stats);
+    WriteDataPage(page);
+  }
+
+  // Re-initialize the sinks for next Page.
+  InitSinks();
+  num_buffered_values_ = 0;
+  num_buffered_encoded_values_ = 0;
+}
+
+void ColumnWriter::WriteDataPage(const CompressedDataPage& page) {
+  total_bytes_written_ += pager_->WriteDataPage(page);
+}
+
+int64_t ColumnWriter::Close() {
+  if (!closed_) {
+    closed_ = true;
+    if (has_dictionary_ && !fallback_) { WriteDictionaryPage(); }
+
+    FlushBufferedDataPages();
+
+    EncodedStatistics chunk_statistics = GetChunkStatistics();
+    if (chunk_statistics.is_set()) metadata_->SetStatistics(chunk_statistics);
+    pager_->Close(has_dictionary_, fallback_);
+  }
+
+  if (num_rows_ != expected_rows_) {
+    std::stringstream ss;
+    ss << "Written rows: " << num_rows_ << " != expected rows: " << expected_rows_
+       << "in the current column chunk";
+    throw ParquetException(ss.str());
+  }
+
+  return total_bytes_written_;
+}
+
+void ColumnWriter::FlushBufferedDataPages() {
+  // Write all outstanding data to a new page
+  if (num_buffered_values_ > 0) { AddDataPage(); }
+  for (size_t i = 0; i < data_pages_.size(); i++) {
+    WriteDataPage(data_pages_[i]);
+  }
+  data_pages_.clear();
+}
+
+// ----------------------------------------------------------------------
+// TypedColumnWriter
+
+template <typename Type>
+TypedColumnWriter<Type>::TypedColumnWriter(ColumnChunkMetaDataBuilder* metadata,
+    std::unique_ptr<PageWriter> pager, int64_t expected_rows, Encoding::type encoding,
+    const WriterProperties* properties)
+    : ColumnWriter(metadata, std::move(pager), expected_rows,
+          (encoding == Encoding::PLAIN_DICTIONARY ||
+              encoding == Encoding::RLE_DICTIONARY),
+          encoding, properties) {
+  switch (encoding) {
+    case Encoding::PLAIN:
+      current_encoder_.reset(new PlainEncoder<Type>(descr_, properties->memory_pool()));
+      break;
+    case Encoding::PLAIN_DICTIONARY:
+    case Encoding::RLE_DICTIONARY:
+      current_encoder_.reset(
+          new DictEncoder<Type>(descr_, &pool_, properties->memory_pool()));
+      break;
+    default:
+      ParquetException::NYI("Selected encoding is not supported");
+  }
+
+  if (properties->statistics_enabled(descr_->path())) {
+    page_statistics_ = std::unique_ptr<TypedStats>(new TypedStats(descr_, allocator_));
+    chunk_statistics_ = std::unique_ptr<TypedStats>(new TypedStats(descr_, allocator_));
+  }
+}
+
+// Only one Dictionary Page is written.
+// Fallback to PLAIN if dictionary page limit is reached.
+template <typename Type>
+void TypedColumnWriter<Type>::CheckDictionarySizeLimit() {
+  auto dict_encoder = static_cast<DictEncoder<Type>*>(current_encoder_.get());
+  if (dict_encoder->dict_encoded_size() >= properties_->dictionary_pagesize_limit()) {
+    WriteDictionaryPage();
+    // Serialize the buffered Dictionary Indicies
+    FlushBufferedDataPages();
+    fallback_ = true;
+    // Only PLAIN encoding is supported for fallback in V1
+    current_encoder_.reset(new PlainEncoder<Type>(descr_, properties_->memory_pool()));
+    encoding_ = Encoding::PLAIN;
+  }
+}
+
+template <typename Type>
+void TypedColumnWriter<Type>::WriteDictionaryPage() {
+  auto dict_encoder = static_cast<DictEncoder<Type>*>(current_encoder_.get());
+  std::shared_ptr<PoolBuffer> buffer =
+      AllocateBuffer(properties_->memory_pool(), dict_encoder->dict_encoded_size());
+  dict_encoder->WriteDict(buffer->mutable_data());
+  // TODO Get rid of this deep call
+  dict_encoder->mem_pool()->FreeAll();
+
+  DictionaryPage page(
+      buffer, dict_encoder->num_entries(), properties_->dictionary_index_encoding());
+  total_bytes_written_ += pager_->WriteDictionaryPage(page);
+}
+
+template <typename Type>
+EncodedStatistics TypedColumnWriter<Type>::GetPageStatistics() {
+  EncodedStatistics result;
+  if (page_statistics_) result = page_statistics_->Encode();
+  return result;
+}
+
+template <typename Type>
+EncodedStatistics TypedColumnWriter<Type>::GetChunkStatistics() {
+  EncodedStatistics result;
+  if (chunk_statistics_) result = chunk_statistics_->Encode();
+  return result;
+}
+
+template <typename Type>
+void TypedColumnWriter<Type>::ResetPageStatistics() {
+  if (chunk_statistics_ != nullptr) {
+    chunk_statistics_->Merge(*page_statistics_);
+    page_statistics_->Reset();
+  }
+}
+
+// ----------------------------------------------------------------------
+// Dynamic column writer constructor
+
+std::shared_ptr<ColumnWriter> ColumnWriter::Make(ColumnChunkMetaDataBuilder* metadata,
+    std::unique_ptr<PageWriter> pager, int64_t expected_rows,
+    const WriterProperties* properties) {
+  const ColumnDescriptor* descr = metadata->descr();
+  Encoding::type encoding = properties->encoding(descr->path());
+  if (properties->dictionary_enabled(descr->path()) &&
+      descr->physical_type() != Type::BOOLEAN) {
+    encoding = properties->dictionary_page_encoding();
+  }
+  switch (descr->physical_type()) {
+    case Type::BOOLEAN:
+      return std::make_shared<BoolWriter>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::INT32:
+      return std::make_shared<Int32Writer>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::INT64:
+      return std::make_shared<Int64Writer>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::INT96:
+      return std::make_shared<Int96Writer>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::FLOAT:
+      return std::make_shared<FloatWriter>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::DOUBLE:
+      return std::make_shared<DoubleWriter>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::BYTE_ARRAY:
+      return std::make_shared<ByteArrayWriter>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    case Type::FIXED_LEN_BYTE_ARRAY:
+      return std::make_shared<FixedLenByteArrayWriter>(
+          metadata, std::move(pager), expected_rows, encoding, properties);
+    default:
+      ParquetException::NYI("type reader not implemented");
+  }
+  // Unreachable code, but supress compiler warning
+  return std::shared_ptr<ColumnWriter>(nullptr);
+}
+
+// ----------------------------------------------------------------------
+// Instantiate templated classes
+
+template <typename DType>
+inline int64_t TypedColumnWriter<DType>::WriteMiniBatch(int64_t num_values,
+    const int16_t* def_levels, const int16_t* rep_levels, const T* values) {
+  int64_t values_to_write = 0;
+  // If the field is required and non-repeated, there are no definition levels
+  if (descr_->max_definition_level() > 0) {
+    for (int64_t i = 0; i < num_values; ++i) {
+      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_write; }
+    }
+
+    WriteDefinitionLevels(num_values, def_levels);
+  } else {
+    // Required field, write all values
+    values_to_write = num_values;
+  }
+
+  // Not present for non-repeated fields
+  if (descr_->max_repetition_level() > 0) {
+    // A row could include more than one value
+    // Count the occasions where we start a new row
+    for (int64_t i = 0; i < num_values; ++i) {
+      if (rep_levels[i] == 0) { num_rows_++; }
+    }
+
+    WriteRepetitionLevels(num_values, rep_levels);
+  } else {
+    // Each value is exactly one row
+    num_rows_ += static_cast<int>(num_values);
+  }
+
+  if (num_rows_ > expected_rows_) {
+    throw ParquetException("More rows were written in the column chunk than expected");
+  }
+
+  // PARQUET-780
+  if (values_to_write > 0) { DCHECK(nullptr != values) << "Values ptr cannot be NULL"; }
+
+  WriteValues(values_to_write, values);
+
+  if (page_statistics_ != nullptr) {
+    page_statistics_->Update(values, values_to_write, num_values - values_to_write);
+  }
+
+  num_buffered_values_ += num_values;
+  num_buffered_encoded_values_ += values_to_write;
+
+  if (current_encoder_->EstimatedDataEncodedSize() >= properties_->data_pagesize()) {
+    AddDataPage();
+  }
+  if (has_dictionary_ && !fallback_) { CheckDictionarySizeLimit(); }
+
+  return values_to_write;
+}
+
+template <typename DType>
+inline int64_t TypedColumnWriter<DType>::WriteMiniBatchSpaced(int64_t num_values,
+    const int16_t* def_levels, const int16_t* rep_levels, const uint8_t* valid_bits,
+    int64_t valid_bits_offset, const T* values, int64_t* num_spaced_written) {
+  int64_t values_to_write = 0;
+  int64_t spaced_values_to_write = 0;
+  // If the field is required and non-repeated, there are no definition levels
+  if (descr_->max_definition_level() > 0) {
+    // Minimal definition level for which spaced values are written
+    int16_t min_spaced_def_level = descr_->max_definition_level();
+    if (descr_->schema_node()->is_optional()) { min_spaced_def_level--; }
+    for (int64_t i = 0; i < num_values; ++i) {
+      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_write; }
+      if (def_levels[i] >= min_spaced_def_level) { ++spaced_values_to_write; }
+    }
+
+    WriteDefinitionLevels(num_values, def_levels);
+  } else {
+    // Required field, write all values
+    values_to_write = num_values;
+    spaced_values_to_write = num_values;
+  }
+
+  // Not present for non-repeated fields
+  if (descr_->max_repetition_level() > 0) {
+    // A row could include more than one value
+    // Count the occasions where we start a new row
+    for (int64_t i = 0; i < num_values; ++i) {
+      if (rep_levels[i] == 0) { num_rows_++; }
+    }
+
+    WriteRepetitionLevels(num_values, rep_levels);
+  } else {
+    // Each value is exactly one row
+    num_rows_ += static_cast<int>(num_values);
+  }
+
+  if (num_rows_ > expected_rows_) {
+    throw ParquetException("More rows were written in the column chunk than expected");
+  }
+
+  if (descr_->schema_node()->is_optional()) {
+    WriteValuesSpaced(spaced_values_to_write, valid_bits, valid_bits_offset, values);
+  } else {
+    WriteValues(values_to_write, values);
+  }
+  *num_spaced_written = spaced_values_to_write;
+
+  if (page_statistics_ != nullptr) {
+    page_statistics_->UpdateSpaced(values, valid_bits, valid_bits_offset, values_to_write,
+        num_values - values_to_write);
+  }
+
+  num_buffered_values_ += num_values;
+  num_buffered_encoded_values_ += values_to_write;
+
+  if (current_encoder_->EstimatedDataEncodedSize() >= properties_->data_pagesize()) {
+    AddDataPage();
+  }
+  if (has_dictionary_ && !fallback_) { CheckDictionarySizeLimit(); }
+
+  return values_to_write;
+}
+
+template <typename DType>
+void TypedColumnWriter<DType>::WriteBatch(int64_t num_values, const int16_t* def_levels,
+    const int16_t* rep_levels, const T* values) {
+  // We check for DataPage limits only after we have inserted the values. If a user
+  // writes a large number of values, the DataPage size can be much above the limit.
+  // The purpose of this chunking is to bound this. Even if a user writes large number
+  // of values, the chunking will ensure the AddDataPage() is called at a reasonable
+  // pagesize limit
+  int64_t write_batch_size = properties_->write_batch_size();
+  int num_batches = static_cast<int>(num_values / write_batch_size);
+  int64_t num_remaining = num_values % write_batch_size;
+  int64_t value_offset = 0;
+  for (int round = 0; round < num_batches; round++) {
+    int64_t offset = round * write_batch_size;
+    int64_t num_values = WriteMiniBatch(write_batch_size, &def_levels[offset],
+        &rep_levels[offset], &values[value_offset]);
+    value_offset += num_values;
+  }
+  // Write the remaining values
+  int64_t offset = num_batches * write_batch_size;
+  WriteMiniBatch(
+      num_remaining, &def_levels[offset], &rep_levels[offset], &values[value_offset]);
+}
+
+template <typename DType>
+void TypedColumnWriter<DType>::WriteBatchSpaced(int64_t num_values,
+    const int16_t* def_levels, const int16_t* rep_levels, const uint8_t* valid_bits,
+    int64_t valid_bits_offset, const T* values) {
+  // We check for DataPage limits only after we have inserted the values. If a user
+  // writes a large number of values, the DataPage size can be much above the limit.
+  // The purpose of this chunking is to bound this. Even if a user writes large number
+  // of values, the chunking will ensure the AddDataPage() is called at a reasonable
+  // pagesize limit
+  int64_t write_batch_size = properties_->write_batch_size();
+  int num_batches = static_cast<int>(num_values / write_batch_size);
+  int64_t num_remaining = num_values % write_batch_size;
+  int64_t num_spaced_written = 0;
+  int64_t values_offset = 0;
+  for (int round = 0; round < num_batches; round++) {
+    int64_t offset = round * write_batch_size;
+    WriteMiniBatchSpaced(write_batch_size, &def_levels[offset], &rep_levels[offset],
+        valid_bits, valid_bits_offset + values_offset, values + values_offset,
+        &num_spaced_written);
+    values_offset += num_spaced_written;
+  }
+  // Write the remaining values
+  int64_t offset = num_batches * write_batch_size;
+  WriteMiniBatchSpaced(num_remaining, &def_levels[offset], &rep_levels[offset],
+      valid_bits, valid_bits_offset + values_offset, values + values_offset,
+      &num_spaced_written);
+}
+
+template <typename DType>
+void TypedColumnWriter<DType>::WriteValues(int64_t num_values, const T* values) {
+  current_encoder_->Put(values, static_cast<int>(num_values));
+}
+
+template <typename DType>
+void TypedColumnWriter<DType>::WriteValuesSpaced(int64_t num_values,
+    const uint8_t* valid_bits, int64_t valid_bits_offset, const T* values) {
+  current_encoder_->PutSpaced(
+      values, static_cast<int>(num_values), valid_bits, valid_bits_offset);
+}
+
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<BooleanType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int32Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int64Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int96Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<FloatType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<DoubleType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<ByteArrayType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<FLBAType>;
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_writer.h b/src/parquet/column_writer.h
new file mode 100644
index 0000000..4e113de
--- /dev/null
+++ b/src/parquet/column_writer.h
@@ -0,0 +1,282 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef PARQUET_COLUMN_WRITER_H
+#define PARQUET_COLUMN_WRITER_H
+
+#include <vector>
+
+#include "parquet/column_page.h"
+#include "parquet/encoding.h"
+#include "parquet/file/metadata.h"
+#include "parquet/properties.h"
+#include "parquet/schema.h"
+#include "parquet/statistics.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/visibility.h"
+
+namespace parquet {
+
+class BitWriter;
+class RleEncoder;
+
+class PARQUET_EXPORT LevelEncoder {
+ public:
+  LevelEncoder();
+  ~LevelEncoder();
+
+  static int MaxBufferSize(
+      Encoding::type encoding, int16_t max_level, int num_buffered_values);
+
+  // Initialize the LevelEncoder.
+  void Init(Encoding::type encoding, int16_t max_level, int num_buffered_values,
+      uint8_t* data, int data_size);
+
+  // Encodes a batch of levels from an array and returns the number of levels encoded
+  int Encode(int batch_size, const int16_t* levels);
+
+  int32_t len() {
+    if (encoding_ != Encoding::RLE) {
+      throw ParquetException("Only implemented for RLE encoding");
+    }
+    return rle_length_;
+  }
+
+ private:
+  int bit_width_;
+  int rle_length_;
+  Encoding::type encoding_;
+  std::unique_ptr<RleEncoder> rle_encoder_;
+  std::unique_ptr<BitWriter> bit_packed_encoder_;
+};
+
+static constexpr int WRITE_BATCH_SIZE = 1000;
+class PARQUET_EXPORT ColumnWriter {
+ public:
+  ColumnWriter(ColumnChunkMetaDataBuilder*, std::unique_ptr<PageWriter>,
+      int64_t expected_rows, bool has_dictionary, Encoding::type encoding,
+      const WriterProperties* properties);
+
+  static std::shared_ptr<ColumnWriter> Make(ColumnChunkMetaDataBuilder*,
+      std::unique_ptr<PageWriter>, int64_t expected_rows,
+      const WriterProperties* properties);
+
+  Type::type type() const { return descr_->physical_type(); }
+
+  const ColumnDescriptor* descr() const { return descr_; }
+
+  /**
+   * Closes the ColumnWriter, commits any buffered values to pages.
+   *
+   * @return Total size of the column in bytes
+   */
+  int64_t Close();
+
+ protected:
+  virtual std::shared_ptr<Buffer> GetValuesBuffer() = 0;
+
+  // Serializes Dictionary Page if enabled
+  virtual void WriteDictionaryPage() = 0;
+
+  // Checks if the Dictionary Page size limit is reached
+  // If the limit is reached, the Dictionary and Data Pages are serialized
+  // The encoding is switched to PLAIN
+
+  virtual void CheckDictionarySizeLimit() = 0;
+
+  // Plain-encoded statistics of the current page
+  virtual EncodedStatistics GetPageStatistics() = 0;
+
+  // Plain-encoded statistics of the whole chunk
+  virtual EncodedStatistics GetChunkStatistics() = 0;
+
+  // Merges page statistics into chunk statistics, then resets the values
+  virtual void ResetPageStatistics() = 0;
+
+  // Adds Data Pages to an in memory buffer in dictionary encoding mode
+  // Serializes the Data Pages in other encoding modes
+  void AddDataPage();
+
+  // Serializes Data Pages
+  void WriteDataPage(const CompressedDataPage& page);
+
+  // Write multiple definition levels
+  void WriteDefinitionLevels(int64_t num_levels, const int16_t* levels);
+
+  // Write multiple repetition levels
+  void WriteRepetitionLevels(int64_t num_levels, const int16_t* levels);
+
+  // RLE encode the src_buffer into dest_buffer and return the encoded size
+  int64_t RleEncodeLevels(
+      const Buffer& src_buffer, ResizableBuffer* dest_buffer, int16_t max_level);
+
+  // Serialize the buffered Data Pages
+  void FlushBufferedDataPages();
+
+  ColumnChunkMetaDataBuilder* metadata_;
+  const ColumnDescriptor* descr_;
+
+  std::unique_ptr<PageWriter> pager_;
+
+  // The number of rows that should be written in this column chunk.
+  int64_t expected_rows_;
+  bool has_dictionary_;
+  Encoding::type encoding_;
+  const WriterProperties* properties_;
+
+  LevelEncoder level_encoder_;
+
+  ::arrow::MemoryPool* allocator_;
+  ChunkedAllocator pool_;
+
+  // The total number of values stored in the data page. This is the maximum of
+  // the number of encoded definition levels or encoded values. For
+  // non-repeated, required columns, this is equal to the number of encoded
+  // values. For repeated or optional values, there may be fewer data values
+  // than levels, and this tells you how many encoded levels there are in that
+  // case.
+  int64_t num_buffered_values_;
+
+  // The total number of stored values. For repeated or optional values, this
+  // number may be lower than num_buffered_values_.
+  int64_t num_buffered_encoded_values_;
+
+  // Total number of rows written with this ColumnWriter
+  int num_rows_;
+
+  // Records the total number of bytes written by the serializer
+  int64_t total_bytes_written_;
+
+  // Flag to check if the Writer has been closed
+  bool closed_;
+
+  // Flag to infer if dictionary encoding has fallen back to PLAIN
+  bool fallback_;
+
+  std::unique_ptr<InMemoryOutputStream> definition_levels_sink_;
+  std::unique_ptr<InMemoryOutputStream> repetition_levels_sink_;
+
+  std::shared_ptr<ResizableBuffer> definition_levels_rle_;
+  std::shared_ptr<ResizableBuffer> repetition_levels_rle_;
+
+  std::shared_ptr<ResizableBuffer> uncompressed_data_;
+  std::shared_ptr<ResizableBuffer> compressed_data_;
+
+  std::vector<CompressedDataPage> data_pages_;
+
+ private:
+  void InitSinks();
+};
+
+// API to write values to a single column. This is the main client facing API.
+template <typename DType>
+class PARQUET_EXPORT TypedColumnWriter : public ColumnWriter {
+ public:
+  typedef typename DType::c_type T;
+
+  TypedColumnWriter(ColumnChunkMetaDataBuilder* metadata,
+      std::unique_ptr<PageWriter> pager, int64_t expected_rows, Encoding::type encoding,
+      const WriterProperties* properties);
+
+  // Write a batch of repetition levels, definition levels, and values to the
+  // column.
+  void WriteBatch(int64_t num_values, const int16_t* def_levels,
+      const int16_t* rep_levels, const T* values);
+
+  /// Write a batch of repetition levels, definition levels, and values to the
+  /// column.
+  ///
+  /// In comparision to WriteBatch the length of repetition and definition levels
+  /// is the same as of the number of values read for max_definition_level == 1.
+  /// In the case of max_definition_level > 1, the repetition and definition
+  /// levels are larger than the values but the values include the null entries
+  /// with definition_level == (max_definition_level - 1). Thus we have to differentiate
+  /// in the parameters of this function if the input has the length of num_values or the
+  /// _number of rows in the lowest nesting level_.
+  ///
+  /// In the case that the most inner node in the Parquet is required, the _number of rows
+  /// in the lowest nesting level_ is equal to the number of non-null values. If the
+  /// inner-most schema node is optional, the _number of rows in the lowest nesting level_
+  /// also includes all values with definition_level == (max_definition_level - 1).
+  ///
+  /// @param num_values number of levels to write.
+  /// @param def_levels The Parquet definiton levels, length is num_values
+  /// @param rep_levels The Parquet repetition levels, length is num_values
+  /// @param valid_bits Bitmap that indicates if the row is null on the lowest nesting
+  ///   level. The length is number of rows in the lowest nesting level.
+  /// @param valid_bits_offset The offset in bits of the valid_bits where the
+  ///   first relevant bit resides.
+  /// @param values The values in the lowest nested level including
+  ///   spacing for nulls on the lowest levels; input has the length
+  ///   of the number of rows on the lowest nesting level.
+  void WriteBatchSpaced(int64_t num_values, const int16_t* def_levels,
+      const int16_t* rep_levels, const uint8_t* valid_bits, int64_t valid_bits_offset,
+      const T* values);
+
+ protected:
+  std::shared_ptr<Buffer> GetValuesBuffer() override {
+    return current_encoder_->FlushValues();
+  }
+  void WriteDictionaryPage() override;
+  void CheckDictionarySizeLimit() override;
+  EncodedStatistics GetPageStatistics() override;
+  EncodedStatistics GetChunkStatistics() override;
+  void ResetPageStatistics() override;
+
+ private:
+  int64_t WriteMiniBatch(int64_t num_values, const int16_t* def_levels,
+      const int16_t* rep_levels, const T* values);
+
+  int64_t WriteMiniBatchSpaced(int64_t num_values, const int16_t* def_levels,
+      const int16_t* rep_levels, const uint8_t* valid_bits, int64_t valid_bits_offset,
+      const T* values, int64_t* num_spaced_written);
+
+  typedef Encoder<DType> EncoderType;
+
+  // Write values to a temporary buffer before they are encoded into pages
+  void WriteValues(int64_t num_values, const T* values);
+  void WriteValuesSpaced(int64_t num_values, const uint8_t* valid_bits,
+      int64_t valid_bits_offset, const T* values);
+  std::unique_ptr<EncoderType> current_encoder_;
+
+  typedef TypedRowGroupStatistics<DType> TypedStats;
+  std::unique_ptr<TypedStats> page_statistics_;
+  std::unique_ptr<TypedStats> chunk_statistics_;
+};
+
+typedef TypedColumnWriter<BooleanType> BoolWriter;
+typedef TypedColumnWriter<Int32Type> Int32Writer;
+typedef TypedColumnWriter<Int64Type> Int64Writer;
+typedef TypedColumnWriter<Int96Type> Int96Writer;
+typedef TypedColumnWriter<FloatType> FloatWriter;
+typedef TypedColumnWriter<DoubleType> DoubleWriter;
+typedef TypedColumnWriter<ByteArrayType> ByteArrayWriter;
+typedef TypedColumnWriter<FLBAType> FixedLenByteArrayWriter;
+
+extern template class PARQUET_EXPORT TypedColumnWriter<BooleanType>;
+extern template class PARQUET_EXPORT TypedColumnWriter<Int32Type>;
+extern template class PARQUET_EXPORT TypedColumnWriter<Int64Type>;
+extern template class PARQUET_EXPORT TypedColumnWriter<Int96Type>;
+extern template class PARQUET_EXPORT TypedColumnWriter<FloatType>;
+extern template class PARQUET_EXPORT TypedColumnWriter<DoubleType>;
+extern template class PARQUET_EXPORT TypedColumnWriter<ByteArrayType>;
+extern template class PARQUET_EXPORT TypedColumnWriter<FLBAType>;
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_READER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/encoding-internal.h
----------------------------------------------------------------------
diff --git a/src/parquet/encoding-internal.h b/src/parquet/encoding-internal.h
index 7e78123..61b8e24 100644
--- a/src/parquet/encoding-internal.h
+++ b/src/parquet/encoding-internal.h
@@ -464,9 +464,8 @@ 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(
-               bit_width(), static_cast<int>(buffered_indices_.size())) +
+    return 1 + RleEncoder::MaxBufferSize(
+                   bit_width(), static_cast<int>(buffered_indices_.size())) +
            RleEncoder::MinBufferSize(bit_width());
   }
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/file-deserialize-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/file-deserialize-test.cc b/src/parquet/file/file-deserialize-test.cc
index 7c4690e..59d2051 100644
--- a/src/parquet/file/file-deserialize-test.cc
+++ b/src/parquet/file/file-deserialize-test.cc
@@ -26,7 +26,7 @@
 #include <string>
 #include <vector>
 
-#include "parquet/column/page.h"
+#include "parquet/column_page.h"
 #include "parquet/exception.h"
 #include "parquet/file/reader-internal.h"
 #include "parquet/parquet_types.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/file-metadata-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/file-metadata-test.cc b/src/parquet/file/file-metadata-test.cc
index a4a2016..10ce40c 100644
--- a/src/parquet/file/file-metadata-test.cc
+++ b/src/parquet/file/file-metadata-test.cc
@@ -15,9 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "parquet/column/statistics.h"
 #include "parquet/file/metadata.h"
 #include "parquet/schema.h"
+#include "parquet/statistics.h"
 #include "parquet/types.h"
 #include <gtest/gtest.h>
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/file-serialize-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/file-serialize-test.cc b/src/parquet/file/file-serialize-test.cc
index 7a90eeb..5736fa1 100644
--- a/src/parquet/file/file-serialize-test.cc
+++ b/src/parquet/file/file-serialize-test.cc
@@ -17,12 +17,12 @@
 
 #include <gtest/gtest.h>
 
-#include "parquet/column/reader.h"
-#include "parquet/column/test-specialization.h"
-#include "parquet/column/test-util.h"
-#include "parquet/column/writer.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
 #include "parquet/file/reader.h"
 #include "parquet/file/writer.h"
+#include "parquet/test-specialization.h"
+#include "parquet/test-util.h"
 #include "parquet/types.h"
 #include "parquet/util/memory.h"
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/metadata.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/metadata.h b/src/parquet/file/metadata.h
index 50d2114..2dc50d1 100644
--- a/src/parquet/file/metadata.h
+++ b/src/parquet/file/metadata.h
@@ -24,9 +24,9 @@
 
 #include "arrow/util/key_value_metadata.h"
 
-#include "parquet/column/properties.h"
-#include "parquet/column/statistics.h"
+#include "parquet/properties.h"
 #include "parquet/schema.h"
+#include "parquet/statistics.h"
 #include "parquet/types.h"
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/printer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/printer.cc b/src/parquet/file/printer.cc
index e398c3a..52b2598 100644
--- a/src/parquet/file/printer.cc
+++ b/src/parquet/file/printer.cc
@@ -20,7 +20,7 @@
 #include <string>
 #include <vector>
 
-#include "parquet/column/scanner.h"
+#include "parquet/column_scanner.h"
 
 using std::string;
 using std::vector;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/reader-internal.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader-internal.cc b/src/parquet/file/reader-internal.cc
index 1d9ab47..c39d3eb 100644
--- a/src/parquet/file/reader-internal.cc
+++ b/src/parquet/file/reader-internal.cc
@@ -26,7 +26,7 @@
 
 #include "arrow/util/compression.h"
 
-#include "parquet/column/page.h"
+#include "parquet/column_page.h"
 #include "parquet/exception.h"
 #include "parquet/schema.h"
 #include "parquet/thrift.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/reader-internal.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader-internal.h b/src/parquet/file/reader-internal.h
index 1ac2384..2667fa8 100644
--- a/src/parquet/file/reader-internal.h
+++ b/src/parquet/file/reader-internal.h
@@ -22,11 +22,11 @@
 #include <memory>
 #include <vector>
 
-#include "parquet/column/page.h"
-#include "parquet/column/properties.h"
+#include "parquet/column_page.h"
 #include "parquet/file/metadata.h"
 #include "parquet/file/reader.h"
 #include "parquet/parquet_types.h"
+#include "parquet/properties.h"
 #include "parquet/types.h"
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader.cc b/src/parquet/file/reader.cc
index 7bf2c76..d3247cb 100644
--- a/src/parquet/file/reader.cc
+++ b/src/parquet/file/reader.cc
@@ -25,9 +25,9 @@
 
 #include "arrow/io/file.h"
 
-#include "parquet/column/page.h"
-#include "parquet/column/reader.h"
-#include "parquet/column/scanner.h"
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_scanner.h"
 #include "parquet/exception.h"
 #include "parquet/file/reader-internal.h"
 #include "parquet/types.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/reader.h b/src/parquet/file/reader.h
index 7d3c3f9..1cd287c 100644
--- a/src/parquet/file/reader.h
+++ b/src/parquet/file/reader.h
@@ -25,11 +25,11 @@
 #include <string>
 #include <vector>
 
-#include "parquet/column/page.h"
-#include "parquet/column/properties.h"
-#include "parquet/column/statistics.h"
+#include "parquet/column_page.h"
 #include "parquet/file/metadata.h"
+#include "parquet/properties.h"
 #include "parquet/schema.h"
+#include "parquet/statistics.h"
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/writer-internal.cc
----------------------------------------------------------------------
diff --git a/src/parquet/file/writer-internal.cc b/src/parquet/file/writer-internal.cc
index bb24737..1cceb95 100644
--- a/src/parquet/file/writer-internal.cc
+++ b/src/parquet/file/writer-internal.cc
@@ -22,7 +22,7 @@
 
 #include "arrow/util/compression.h"
 
-#include "parquet/column/writer.h"
+#include "parquet/column_writer.h"
 #include "parquet/schema-internal.h"
 #include "parquet/schema.h"
 #include "parquet/thrift.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/writer-internal.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/writer-internal.h b/src/parquet/file/writer-internal.h
index 6ac7927..447579a 100644
--- a/src/parquet/file/writer-internal.h
+++ b/src/parquet/file/writer-internal.h
@@ -21,7 +21,7 @@
 #include <memory>
 #include <vector>
 
-#include "parquet/column/page.h"
+#include "parquet/column_page.h"
 #include "parquet/file/metadata.h"
 #include "parquet/file/writer.h"
 #include "parquet/parquet_types.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/file/writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/file/writer.h b/src/parquet/file/writer.h
index 7d48720..b22281a 100644
--- a/src/parquet/file/writer.h
+++ b/src/parquet/file/writer.h
@@ -21,8 +21,8 @@
 #include <cstdint>
 #include <memory>
 
-#include "parquet/column/properties.h"
 #include "parquet/file/metadata.h"
+#include "parquet/properties.h"
 #include "parquet/schema.h"
 #include "parquet/util/memory.h"
 #include "parquet/util/visibility.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/properties-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/properties-test.cc b/src/parquet/properties-test.cc
new file mode 100644
index 0000000..0e6d725
--- /dev/null
+++ b/src/parquet/properties-test.cc
@@ -0,0 +1,64 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include <string>
+
+#include "parquet/file/reader.h"
+#include "parquet/properties.h"
+
+namespace parquet {
+
+using schema::ColumnPath;
+
+namespace test {
+
+TEST(TestReaderProperties, Basics) {
+  ReaderProperties props;
+
+  ASSERT_EQ(DEFAULT_BUFFER_SIZE, props.buffer_size());
+  ASSERT_EQ(DEFAULT_USE_BUFFERED_STREAM, props.is_buffered_stream_enabled());
+}
+
+TEST(TestWriterProperties, Basics) {
+  std::shared_ptr<WriterProperties> props = WriterProperties::Builder().build();
+
+  ASSERT_EQ(DEFAULT_PAGE_SIZE, props->data_pagesize());
+  ASSERT_EQ(DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT, props->dictionary_pagesize_limit());
+  ASSERT_EQ(DEFAULT_WRITER_VERSION, props->version());
+}
+
+TEST(TestWriterProperties, AdvancedHandling) {
+  WriterProperties::Builder builder;
+  builder.compression("gzip", Compression::GZIP);
+  builder.compression(Compression::SNAPPY);
+  builder.encoding(Encoding::DELTA_BINARY_PACKED);
+  builder.encoding("delta-length", Encoding::DELTA_LENGTH_BYTE_ARRAY);
+  std::shared_ptr<WriterProperties> props = builder.build();
+
+  ASSERT_EQ(Compression::GZIP, props->compression(ColumnPath::FromDotString("gzip")));
+  ASSERT_EQ(
+      Compression::SNAPPY, props->compression(ColumnPath::FromDotString("delta-length")));
+  ASSERT_EQ(
+      Encoding::DELTA_BINARY_PACKED, props->encoding(ColumnPath::FromDotString("gzip")));
+  ASSERT_EQ(Encoding::DELTA_LENGTH_BYTE_ARRAY,
+      props->encoding(ColumnPath::FromDotString("delta-length")));
+}
+
+}  // namespace test
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/properties.h
----------------------------------------------------------------------
diff --git a/src/parquet/properties.h b/src/parquet/properties.h
new file mode 100644
index 0000000..3ebc3b7
--- /dev/null
+++ b/src/parquet/properties.h
@@ -0,0 +1,385 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef PARQUET_COLUMN_PROPERTIES_H
+#define PARQUET_COLUMN_PROPERTIES_H
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+
+#include "parquet/exception.h"
+#include "parquet/parquet_version.h"
+#include "parquet/schema.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/visibility.h"
+
+namespace parquet {
+
+struct ParquetVersion {
+  enum type { PARQUET_1_0, PARQUET_2_0 };
+};
+
+static int64_t DEFAULT_BUFFER_SIZE = 0;
+static bool DEFAULT_USE_BUFFERED_STREAM = false;
+
+class PARQUET_EXPORT ReaderProperties {
+ public:
+  explicit ReaderProperties(::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
+      : pool_(pool) {
+    buffered_stream_enabled_ = DEFAULT_USE_BUFFERED_STREAM;
+    buffer_size_ = DEFAULT_BUFFER_SIZE;
+  }
+
+  ::arrow::MemoryPool* memory_pool() const { return pool_; }
+
+  std::unique_ptr<InputStream> GetStream(
+      RandomAccessSource* source, int64_t start, int64_t num_bytes) {
+    std::unique_ptr<InputStream> stream;
+    if (buffered_stream_enabled_) {
+      stream.reset(
+          new BufferedInputStream(pool_, buffer_size_, source, start, num_bytes));
+    } else {
+      stream.reset(new InMemoryInputStream(source, start, num_bytes));
+    }
+    return stream;
+  }
+
+  bool is_buffered_stream_enabled() const { return buffered_stream_enabled_; }
+
+  void enable_buffered_stream() { buffered_stream_enabled_ = true; }
+
+  void disable_buffered_stream() { buffered_stream_enabled_ = false; }
+
+  void set_buffer_size(int64_t buf_size) { buffer_size_ = buf_size; }
+
+  int64_t buffer_size() const { return buffer_size_; }
+
+ private:
+  ::arrow::MemoryPool* pool_;
+  int64_t buffer_size_;
+  bool buffered_stream_enabled_;
+};
+
+ReaderProperties PARQUET_EXPORT default_reader_properties();
+
+static constexpr int64_t DEFAULT_PAGE_SIZE = 1024 * 1024;
+static constexpr bool DEFAULT_IS_DICTIONARY_ENABLED = true;
+static constexpr int64_t DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT = DEFAULT_PAGE_SIZE;
+static constexpr int64_t DEFAULT_WRITE_BATCH_SIZE = 1024;
+static constexpr bool DEFAULT_ARE_STATISTICS_ENABLED = true;
+static constexpr Encoding::type DEFAULT_ENCODING = Encoding::PLAIN;
+static constexpr ParquetVersion::type DEFAULT_WRITER_VERSION =
+    ParquetVersion::PARQUET_1_0;
+static const char DEFAULT_CREATED_BY[] = CREATED_BY_VERSION;
+static constexpr Compression::type DEFAULT_COMPRESSION_TYPE = Compression::UNCOMPRESSED;
+
+class PARQUET_EXPORT ColumnProperties {
+ public:
+  ColumnProperties(Encoding::type encoding = DEFAULT_ENCODING,
+      Compression::type codec = DEFAULT_COMPRESSION_TYPE,
+      bool dictionary_enabled = DEFAULT_IS_DICTIONARY_ENABLED,
+      bool statistics_enabled = DEFAULT_ARE_STATISTICS_ENABLED)
+      : encoding(encoding),
+        codec(codec),
+        dictionary_enabled(dictionary_enabled),
+        statistics_enabled(statistics_enabled) {}
+
+  Encoding::type encoding;
+  Compression::type codec;
+  bool dictionary_enabled;
+  bool statistics_enabled;
+};
+
+class PARQUET_EXPORT WriterProperties {
+ public:
+  class Builder {
+   public:
+    Builder()
+        : pool_(::arrow::default_memory_pool()),
+          dictionary_pagesize_limit_(DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT),
+          write_batch_size_(DEFAULT_WRITE_BATCH_SIZE),
+          pagesize_(DEFAULT_PAGE_SIZE),
+          version_(DEFAULT_WRITER_VERSION),
+          created_by_(DEFAULT_CREATED_BY) {}
+    virtual ~Builder() {}
+
+    Builder* memory_pool(::arrow::MemoryPool* pool) {
+      pool_ = pool;
+      return this;
+    }
+
+    Builder* enable_dictionary() {
+      default_column_properties_.dictionary_enabled = true;
+      return this;
+    }
+
+    Builder* disable_dictionary() {
+      default_column_properties_.dictionary_enabled = false;
+      return this;
+    }
+
+    Builder* enable_dictionary(const std::string& path) {
+      dictionary_enabled_[path] = true;
+      return this;
+    }
+
+    Builder* enable_dictionary(const std::shared_ptr<schema::ColumnPath>& path) {
+      return this->enable_dictionary(path->ToDotString());
+    }
+
+    Builder* disable_dictionary(const std::string& path) {
+      dictionary_enabled_[path] = false;
+      return this;
+    }
+
+    Builder* disable_dictionary(const std::shared_ptr<schema::ColumnPath>& path) {
+      return this->disable_dictionary(path->ToDotString());
+    }
+
+    Builder* dictionary_pagesize_limit(int64_t dictionary_psize_limit) {
+      dictionary_pagesize_limit_ = dictionary_psize_limit;
+      return this;
+    }
+
+    Builder* write_batch_size(int64_t write_batch_size) {
+      write_batch_size_ = write_batch_size;
+      return this;
+    }
+
+    Builder* data_pagesize(int64_t pg_size) {
+      pagesize_ = pg_size;
+      return this;
+    }
+
+    Builder* version(ParquetVersion::type version) {
+      version_ = version;
+      return this;
+    }
+
+    Builder* created_by(const std::string& created_by) {
+      created_by_ = created_by;
+      return this;
+    }
+
+    /**
+     * Define the encoding that is used when we don't utilise dictionary encoding.
+     *
+     * This either apply if dictionary encoding is disabled or if we fallback
+     * as the dictionary grew too large.
+     */
+    Builder* encoding(Encoding::type encoding_type) {
+      if (encoding_type == Encoding::PLAIN_DICTIONARY ||
+          encoding_type == Encoding::RLE_DICTIONARY) {
+        throw ParquetException("Can't use dictionary encoding as fallback encoding");
+      }
+
+      default_column_properties_.encoding = encoding_type;
+      return this;
+    }
+
+    /**
+     * Define the encoding that is used when we don't utilise dictionary encoding.
+     *
+     * This either apply if dictionary encoding is disabled or if we fallback
+     * as the dictionary grew too large.
+     */
+    Builder* encoding(const std::string& path, Encoding::type encoding_type) {
+      if (encoding_type == Encoding::PLAIN_DICTIONARY ||
+          encoding_type == Encoding::RLE_DICTIONARY) {
+        throw ParquetException("Can't use dictionary encoding as fallback encoding");
+      }
+
+      encodings_[path] = encoding_type;
+      return this;
+    }
+
+    /**
+     * Define the encoding that is used when we don't utilise dictionary encoding.
+     *
+     * This either apply if dictionary encoding is disabled or if we fallback
+     * as the dictionary grew too large.
+     */
+    Builder* encoding(
+        const std::shared_ptr<schema::ColumnPath>& path, Encoding::type encoding_type) {
+      return this->encoding(path->ToDotString(), encoding_type);
+    }
+
+    Builder* compression(Compression::type codec) {
+      default_column_properties_.codec = codec;
+      return this;
+    }
+
+    Builder* compression(const std::string& path, Compression::type codec) {
+      codecs_[path] = codec;
+      return this;
+    }
+
+    Builder* compression(
+        const std::shared_ptr<schema::ColumnPath>& path, Compression::type codec) {
+      return this->compression(path->ToDotString(), codec);
+    }
+
+    Builder* enable_statistics() {
+      default_column_properties_.statistics_enabled = true;
+      return this;
+    }
+
+    Builder* disable_statistics() {
+      default_column_properties_.statistics_enabled = false;
+      return this;
+    }
+
+    Builder* enable_statistics(const std::string& path) {
+      statistics_enabled_[path] = true;
+      return this;
+    }
+
+    Builder* enable_statistics(const std::shared_ptr<schema::ColumnPath>& path) {
+      return this->enable_statistics(path->ToDotString());
+    }
+
+    Builder* disable_statistics(const std::string& path) {
+      statistics_enabled_[path] = false;
+      return this;
+    }
+
+    Builder* disable_statistics(const std::shared_ptr<schema::ColumnPath>& path) {
+      return this->disable_statistics(path->ToDotString());
+    }
+
+    std::shared_ptr<WriterProperties> build() {
+      std::unordered_map<std::string, ColumnProperties> column_properties;
+      auto get = [&](const std::string& key) -> ColumnProperties& {
+        auto it = column_properties.find(key);
+        if (it == column_properties.end())
+          return column_properties[key] = default_column_properties_;
+        else
+          return it->second;
+      };
+
+      for (const auto& item : encodings_)
+        get(item.first).encoding = item.second;
+      for (const auto& item : codecs_)
+        get(item.first).codec = item.second;
+      for (const auto& item : dictionary_enabled_)
+        get(item.first).dictionary_enabled = item.second;
+      for (const auto& item : statistics_enabled_)
+        get(item.first).statistics_enabled = item.second;
+
+      return std::shared_ptr<WriterProperties>(new WriterProperties(pool_,
+          dictionary_pagesize_limit_, write_batch_size_, pagesize_, version_, created_by_,
+          default_column_properties_, column_properties));
+    }
+
+   private:
+    ::arrow::MemoryPool* pool_;
+    int64_t dictionary_pagesize_limit_;
+    int64_t write_batch_size_;
+    int64_t pagesize_;
+    ParquetVersion::type version_;
+    std::string created_by_;
+
+    // Settings used for each column unless overridden in any of the maps below
+    ColumnProperties default_column_properties_;
+    std::unordered_map<std::string, Encoding::type> encodings_;
+    std::unordered_map<std::string, Compression::type> codecs_;
+    std::unordered_map<std::string, bool> dictionary_enabled_;
+    std::unordered_map<std::string, bool> statistics_enabled_;
+  };
+
+  inline ::arrow::MemoryPool* memory_pool() const { return pool_; }
+
+  inline int64_t dictionary_pagesize_limit() const { return dictionary_pagesize_limit_; }
+
+  inline int64_t write_batch_size() const { return write_batch_size_; }
+
+  inline int64_t data_pagesize() const { return pagesize_; }
+
+  inline ParquetVersion::type version() const { return parquet_version_; }
+
+  inline std::string created_by() const { return parquet_created_by_; }
+
+  inline Encoding::type dictionary_index_encoding() const {
+    if (parquet_version_ == ParquetVersion::PARQUET_1_0) {
+      return Encoding::PLAIN_DICTIONARY;
+    } else {
+      return Encoding::RLE_DICTIONARY;
+    }
+  }
+
+  inline Encoding::type dictionary_page_encoding() const {
+    if (parquet_version_ == ParquetVersion::PARQUET_1_0) {
+      return Encoding::PLAIN_DICTIONARY;
+    } else {
+      return Encoding::PLAIN;
+    }
+  }
+
+  const ColumnProperties& column_properties(
+      const std::shared_ptr<schema::ColumnPath>& path) const {
+    auto it = column_properties_.find(path->ToDotString());
+    if (it != column_properties_.end()) return it->second;
+    return default_column_properties_;
+  }
+
+  Encoding::type encoding(const std::shared_ptr<schema::ColumnPath>& path) const {
+    return column_properties(path).encoding;
+  }
+
+  Compression::type compression(const std::shared_ptr<schema::ColumnPath>& path) const {
+    return column_properties(path).codec;
+  }
+
+  bool dictionary_enabled(const std::shared_ptr<schema::ColumnPath>& path) const {
+    return column_properties(path).dictionary_enabled;
+  }
+
+  bool statistics_enabled(const std::shared_ptr<schema::ColumnPath>& path) const {
+    return column_properties(path).statistics_enabled;
+  }
+
+ private:
+  explicit WriterProperties(::arrow::MemoryPool* pool, int64_t dictionary_pagesize_limit,
+      int64_t write_batch_size, int64_t pagesize, ParquetVersion::type version,
+      const std::string& created_by, const ColumnProperties& default_column_properties,
+      const std::unordered_map<std::string, ColumnProperties>& column_properties)
+      : pool_(pool),
+        dictionary_pagesize_limit_(dictionary_pagesize_limit),
+        write_batch_size_(write_batch_size),
+        pagesize_(pagesize),
+        parquet_version_(version),
+        parquet_created_by_(created_by),
+        default_column_properties_(default_column_properties),
+        column_properties_(column_properties) {}
+
+  ::arrow::MemoryPool* pool_;
+  int64_t dictionary_pagesize_limit_;
+  int64_t write_batch_size_;
+  int64_t pagesize_;
+  ParquetVersion::type parquet_version_;
+  std::string parquet_created_by_;
+  ColumnProperties default_column_properties_;
+  std::unordered_map<std::string, ColumnProperties> column_properties_;
+};
+
+std::shared_ptr<WriterProperties> PARQUET_EXPORT default_writer_properties();
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_PROPERTIES_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/reader-test.cc b/src/parquet/reader-test.cc
index fcce38b..cb40abb 100644
--- a/src/parquet/reader-test.cc
+++ b/src/parquet/reader-test.cc
@@ -25,8 +25,8 @@
 
 #include "arrow/io/file.h"
 
-#include "parquet/column/reader.h"
-#include "parquet/column/scanner.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_scanner.h"
 #include "parquet/file/printer.h"
 #include "parquet/file/reader-internal.h"
 #include "parquet/file/reader.h"


[5/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Posted by uw...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/level-benchmark.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/level-benchmark.cc b/src/parquet/column/level-benchmark.cc
deleted file mode 100644
index 34c7218..0000000
--- a/src/parquet/column/level-benchmark.cc
+++ /dev/null
@@ -1,78 +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.
-
-#include "benchmark/benchmark.h"
-
-#include "parquet/column/levels.h"
-#include "parquet/util/memory.h"
-
-namespace parquet {
-
-namespace benchmark {
-
-static void BM_RleEncoding(::benchmark::State& state) {
-  std::vector<int16_t> levels(state.range(0), 0);
-  int64_t n = 0;
-  std::generate(
-      levels.begin(), levels.end(), [&state, &n] { return (n++ % state.range(1)) == 0; });
-  int16_t max_level = 1;
-  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level, levels.size());
-  auto buffer_rle = std::make_shared<PoolBuffer>();
-  PARQUET_THROW_NOT_OK(buffer_rle->Resize(rle_size));
-
-  while (state.KeepRunning()) {
-    LevelEncoder level_encoder;
-    level_encoder.Init(Encoding::RLE, max_level, levels.size(),
-        buffer_rle->mutable_data(), buffer_rle->size());
-    level_encoder.Encode(levels.size(), levels.data());
-  }
-  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
-  state.SetItemsProcessed(state.iterations() * state.range(0));
-}
-
-BENCHMARK(BM_RleEncoding)->RangePair(1024, 65536, 1, 16);
-
-static void BM_RleDecoding(::benchmark::State& state) {
-  LevelEncoder level_encoder;
-  std::vector<int16_t> levels(state.range(0), 0);
-  int64_t n = 0;
-  std::generate(
-      levels.begin(), levels.end(), [&state, &n] { return (n++ % state.range(1)) == 0; });
-  int16_t max_level = 1;
-  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level, levels.size());
-  auto buffer_rle = std::make_shared<PoolBuffer>();
-  PARQUET_THROW_NOT_OK(buffer_rle->Resize(rle_size + sizeof(int32_t)));
-  level_encoder.Init(Encoding::RLE, max_level, levels.size(),
-      buffer_rle->mutable_data() + sizeof(int32_t), rle_size);
-  level_encoder.Encode(levels.size(), levels.data());
-  reinterpret_cast<int32_t*>(buffer_rle->mutable_data())[0] = level_encoder.len();
-
-  while (state.KeepRunning()) {
-    LevelDecoder level_decoder;
-    level_decoder.SetData(Encoding::RLE, max_level, levels.size(), buffer_rle->data());
-    level_decoder.Decode(state.range(0), levels.data());
-  }
-
-  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
-  state.SetItemsProcessed(state.iterations() * state.range(0));
-}
-
-BENCHMARK(BM_RleDecoding)->RangePair(1024, 65536, 1, 16);
-
-}  // namespace benchmark
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/levels-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/levels-test.cc b/src/parquet/column/levels-test.cc
deleted file mode 100644
index a6284a9..0000000
--- a/src/parquet/column/levels-test.cc
+++ /dev/null
@@ -1,245 +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.
-
-#include <cstdint>
-#include <gtest/gtest.h>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "parquet/column/levels.h"
-#include "parquet/types.h"
-
-using std::string;
-
-namespace parquet {
-
-void GenerateLevels(int min_repeat_factor, int max_repeat_factor, int max_level,
-    std::vector<int16_t>& input_levels) {
-  // for each repetition count upto max_repeat_factor
-  for (int repeat = min_repeat_factor; repeat <= max_repeat_factor; repeat++) {
-    // repeat count increases by a factor of 2 for every iteration
-    int repeat_count = (1 << repeat);
-    // generate levels for repetition count upto the maximum level
-    int value = 0;
-    int bwidth = 0;
-    while (value <= max_level) {
-      for (int i = 0; i < repeat_count; i++) {
-        input_levels.push_back(value);
-      }
-      value = (2 << bwidth) - 1;
-      bwidth++;
-    }
-  }
-}
-
-void EncodeLevels(Encoding::type encoding, int max_level, int num_levels,
-    const int16_t* input_levels, std::vector<uint8_t>& bytes) {
-  LevelEncoder encoder;
-  int levels_count = 0;
-  bytes.resize(2 * num_levels);
-  ASSERT_EQ(2 * num_levels, static_cast<int>(bytes.size()));
-  // encode levels
-  if (encoding == Encoding::RLE) {
-    // leave space to write the rle length value
-    encoder.Init(encoding, max_level, num_levels, bytes.data() + sizeof(int32_t),
-        static_cast<int>(bytes.size()));
-
-    levels_count = encoder.Encode(num_levels, input_levels);
-    (reinterpret_cast<int32_t*>(bytes.data()))[0] = encoder.len();
-  } else {
-    encoder.Init(
-        encoding, max_level, num_levels, bytes.data(), static_cast<int>(bytes.size()));
-    levels_count = encoder.Encode(num_levels, input_levels);
-  }
-  ASSERT_EQ(num_levels, levels_count);
-}
-
-void VerifyDecodingLevels(Encoding::type encoding, int max_level,
-    std::vector<int16_t>& input_levels, std::vector<uint8_t>& bytes) {
-  LevelDecoder decoder;
-  int levels_count = 0;
-  std::vector<int16_t> output_levels;
-  int num_levels = static_cast<int>(input_levels.size());
-
-  output_levels.resize(num_levels);
-  ASSERT_EQ(num_levels, static_cast<int>(output_levels.size()));
-
-  // Decode levels and test with multiple decode calls
-  decoder.SetData(encoding, max_level, num_levels, bytes.data());
-  int decode_count = 4;
-  int num_inner_levels = num_levels / decode_count;
-  // Try multiple decoding on a single SetData call
-  for (int ct = 0; ct < decode_count; ct++) {
-    int offset = ct * num_inner_levels;
-    levels_count = decoder.Decode(num_inner_levels, output_levels.data());
-    ASSERT_EQ(num_inner_levels, levels_count);
-    for (int i = 0; i < num_inner_levels; i++) {
-      EXPECT_EQ(input_levels[i + offset], output_levels[i]);
-    }
-  }
-  // check the remaining levels
-  int num_levels_completed = decode_count * (num_levels / decode_count);
-  int num_remaining_levels = num_levels - num_levels_completed;
-  if (num_remaining_levels > 0) {
-    levels_count = decoder.Decode(num_remaining_levels, output_levels.data());
-    ASSERT_EQ(num_remaining_levels, levels_count);
-    for (int i = 0; i < num_remaining_levels; i++) {
-      EXPECT_EQ(input_levels[i + num_levels_completed], output_levels[i]);
-    }
-  }
-  // Test zero Decode values
-  ASSERT_EQ(0, decoder.Decode(1, output_levels.data()));
-}
-
-void VerifyDecodingMultipleSetData(Encoding::type encoding, int max_level,
-    std::vector<int16_t>& input_levels, std::vector<std::vector<uint8_t>>& bytes) {
-  LevelDecoder decoder;
-  int levels_count = 0;
-  std::vector<int16_t> output_levels;
-
-  // Decode levels and test with multiple SetData calls
-  int setdata_count = static_cast<int>(bytes.size());
-  int num_levels = static_cast<int>(input_levels.size()) / setdata_count;
-  output_levels.resize(num_levels);
-  // Try multiple SetData
-  for (int ct = 0; ct < setdata_count; ct++) {
-    int offset = ct * num_levels;
-    ASSERT_EQ(num_levels, static_cast<int>(output_levels.size()));
-    decoder.SetData(encoding, max_level, num_levels, bytes[ct].data());
-    levels_count = decoder.Decode(num_levels, output_levels.data());
-    ASSERT_EQ(num_levels, levels_count);
-    for (int i = 0; i < num_levels; i++) {
-      EXPECT_EQ(input_levels[i + offset], output_levels[i]);
-    }
-  }
-}
-
-// Test levels with maximum bit-width from 1 to 8
-// increase the repetition count for each iteration by a factor of 2
-TEST(TestLevels, TestLevelsDecodeMultipleBitWidth) {
-  int min_repeat_factor = 0;
-  int max_repeat_factor = 7;  // 128
-  int max_bit_width = 8;
-  std::vector<int16_t> input_levels;
-  std::vector<uint8_t> bytes;
-  Encoding::type encodings[2] = {Encoding::RLE, Encoding::BIT_PACKED};
-
-  // for each encoding
-  for (int encode = 0; encode < 2; encode++) {
-    Encoding::type encoding = encodings[encode];
-    // BIT_PACKED requires a sequence of atleast 8
-    if (encoding == Encoding::BIT_PACKED) min_repeat_factor = 3;
-    // for each maximum bit-width
-    for (int bit_width = 1; bit_width <= max_bit_width; bit_width++) {
-      // find the maximum level for the current bit_width
-      int max_level = (1 << bit_width) - 1;
-      // Generate levels
-      GenerateLevels(min_repeat_factor, max_repeat_factor, max_level, input_levels);
-      EncodeLevels(encoding, max_level, static_cast<int>(input_levels.size()),
-          input_levels.data(), bytes);
-      VerifyDecodingLevels(encoding, max_level, input_levels, bytes);
-      input_levels.clear();
-    }
-  }
-}
-
-// Test multiple decoder SetData calls
-TEST(TestLevels, TestLevelsDecodeMultipleSetData) {
-  int min_repeat_factor = 3;
-  int max_repeat_factor = 7;  // 128
-  int bit_width = 8;
-  int max_level = (1 << bit_width) - 1;
-  std::vector<int16_t> input_levels;
-  std::vector<std::vector<uint8_t>> bytes;
-  Encoding::type encodings[2] = {Encoding::RLE, Encoding::BIT_PACKED};
-  GenerateLevels(min_repeat_factor, max_repeat_factor, max_level, input_levels);
-  int num_levels = static_cast<int>(input_levels.size());
-  int setdata_factor = 8;
-  int split_level_size = num_levels / setdata_factor;
-  bytes.resize(setdata_factor);
-
-  // for each encoding
-  for (int encode = 0; encode < 2; encode++) {
-    Encoding::type encoding = encodings[encode];
-    for (int rf = 0; rf < setdata_factor; rf++) {
-      int offset = rf * split_level_size;
-      EncodeLevels(encoding, max_level, split_level_size,
-          reinterpret_cast<int16_t*>(input_levels.data()) + offset, bytes[rf]);
-    }
-    VerifyDecodingMultipleSetData(encoding, max_level, input_levels, bytes);
-  }
-}
-
-TEST(TestLevelEncoder, MinimumBufferSize) {
-  // PARQUET-676, PARQUET-698
-  const int kNumToEncode = 1024;
-
-  std::vector<int16_t> levels;
-  for (int i = 0; i < kNumToEncode; ++i) {
-    if (i % 9 == 0) {
-      levels.push_back(0);
-    } else {
-      levels.push_back(1);
-    }
-  }
-
-  std::vector<uint8_t> output(
-      LevelEncoder::MaxBufferSize(Encoding::RLE, 1, kNumToEncode));
-
-  LevelEncoder encoder;
-  encoder.Init(
-      Encoding::RLE, 1, kNumToEncode, output.data(), static_cast<int>(output.size()));
-  int encode_count = encoder.Encode(kNumToEncode, levels.data());
-
-  ASSERT_EQ(kNumToEncode, encode_count);
-}
-
-TEST(TestLevelEncoder, MinimumBufferSize2) {
-  // PARQUET-708
-  // Test the worst case for bit_width=2 consisting of
-  // LiteralRun(size=8)
-  // RepeatedRun(size=8)
-  // LiteralRun(size=8)
-  // ...
-  const int kNumToEncode = 1024;
-
-  std::vector<int16_t> levels;
-  for (int i = 0; i < kNumToEncode; ++i) {
-    // This forces a literal run of 00000001
-    // followed by eight 1s
-    if ((i % 16) < 7) {
-      levels.push_back(0);
-    } else {
-      levels.push_back(1);
-    }
-  }
-
-  for (int bit_width = 1; bit_width <= 8; bit_width++) {
-    std::vector<uint8_t> output(
-        LevelEncoder::MaxBufferSize(Encoding::RLE, bit_width, kNumToEncode));
-
-    LevelEncoder encoder;
-    encoder.Init(Encoding::RLE, bit_width, kNumToEncode, output.data(),
-        static_cast<int>(output.size()));
-    int encode_count = encoder.Encode(kNumToEncode, levels.data());
-
-    ASSERT_EQ(kNumToEncode, encode_count);
-  }
-}
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/levels.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/levels.cc b/src/parquet/column/levels.cc
deleted file mode 100644
index fd25420..0000000
--- a/src/parquet/column/levels.cc
+++ /dev/null
@@ -1,144 +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.
-
-#include "parquet/column/levels.h"
-
-#include <cstdint>
-
-#include "parquet/util/rle-encoding.h"
-
-namespace parquet {
-
-LevelEncoder::LevelEncoder() {}
-LevelEncoder::~LevelEncoder() {}
-
-void LevelEncoder::Init(Encoding::type encoding, int16_t max_level,
-    int num_buffered_values, uint8_t* data, int data_size) {
-  bit_width_ = BitUtil::Log2(max_level + 1);
-  encoding_ = encoding;
-  switch (encoding) {
-    case Encoding::RLE: {
-      rle_encoder_.reset(new RleEncoder(data, data_size, bit_width_));
-      break;
-    }
-    case Encoding::BIT_PACKED: {
-      int num_bytes =
-          static_cast<int>(BitUtil::Ceil(num_buffered_values * bit_width_, 8));
-      bit_packed_encoder_.reset(new BitWriter(data, num_bytes));
-      break;
-    }
-    default:
-      throw ParquetException("Unknown encoding type for levels.");
-  }
-}
-
-int LevelEncoder::MaxBufferSize(
-    Encoding::type encoding, int16_t max_level, int num_buffered_values) {
-  int bit_width = BitUtil::Log2(max_level + 1);
-  int num_bytes = 0;
-  switch (encoding) {
-    case Encoding::RLE: {
-      // TODO: Due to the way we currently check if the buffer is full enough,
-      // we need to have MinBufferSize as head room.
-      num_bytes = RleEncoder::MaxBufferSize(bit_width, num_buffered_values) +
-                  RleEncoder::MinBufferSize(bit_width);
-      break;
-    }
-    case Encoding::BIT_PACKED: {
-      num_bytes = static_cast<int>(BitUtil::Ceil(num_buffered_values * bit_width, 8));
-      break;
-    }
-    default:
-      throw ParquetException("Unknown encoding type for levels.");
-  }
-  return num_bytes;
-}
-
-int LevelEncoder::Encode(int batch_size, const int16_t* levels) {
-  int num_encoded = 0;
-  if (!rle_encoder_ && !bit_packed_encoder_) {
-    throw ParquetException("Level encoders are not initialized.");
-  }
-
-  if (encoding_ == Encoding::RLE) {
-    for (int i = 0; i < batch_size; ++i) {
-      if (!rle_encoder_->Put(*(levels + i))) { break; }
-      ++num_encoded;
-    }
-    rle_encoder_->Flush();
-    rle_length_ = rle_encoder_->len();
-  } else {
-    for (int i = 0; i < batch_size; ++i) {
-      if (!bit_packed_encoder_->PutValue(*(levels + i), bit_width_)) { break; }
-      ++num_encoded;
-    }
-    bit_packed_encoder_->Flush();
-  }
-  return num_encoded;
-}
-
-LevelDecoder::LevelDecoder() : num_values_remaining_(0) {}
-
-LevelDecoder::~LevelDecoder() {}
-
-int LevelDecoder::SetData(Encoding::type encoding, int16_t max_level,
-    int num_buffered_values, const uint8_t* data) {
-  int32_t num_bytes = 0;
-  encoding_ = encoding;
-  num_values_remaining_ = num_buffered_values;
-  bit_width_ = BitUtil::Log2(max_level + 1);
-  switch (encoding) {
-    case Encoding::RLE: {
-      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_));
-      } else {
-        rle_decoder_->Reset(decoder_data, num_bytes, bit_width_);
-      }
-      return sizeof(int32_t) + num_bytes;
-    }
-    case Encoding::BIT_PACKED: {
-      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));
-      } else {
-        bit_packed_decoder_->Reset(data, num_bytes);
-      }
-      return num_bytes;
-    }
-    default:
-      throw ParquetException("Unknown encoding type for levels.");
-  }
-  return -1;
-}
-
-int LevelDecoder::Decode(int batch_size, int16_t* levels) {
-  int num_decoded = 0;
-
-  int num_values = std::min(num_values_remaining_, batch_size);
-  if (encoding_ == Encoding::RLE) {
-    num_decoded = rle_decoder_->GetBatch(levels, num_values);
-  } else {
-    num_decoded = bit_packed_decoder_->GetBatch(bit_width_, levels, num_values);
-  }
-  num_values_remaining_ -= num_decoded;
-  return num_decoded;
-}
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/levels.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/levels.h b/src/parquet/column/levels.h
deleted file mode 100644
index 63e325e..0000000
--- a/src/parquet/column/levels.h
+++ /dev/null
@@ -1,86 +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.
-
-#ifndef PARQUET_COLUMN_LEVELS_H
-#define PARQUET_COLUMN_LEVELS_H
-
-#include <algorithm>
-#include <memory>
-
-#include "parquet/exception.h"
-#include "parquet/types.h"
-
-namespace parquet {
-
-class BitReader;
-class BitWriter;
-class RleDecoder;
-class RleEncoder;
-
-class PARQUET_EXPORT LevelEncoder {
- public:
-  LevelEncoder();
-  ~LevelEncoder();
-
-  static int MaxBufferSize(
-      Encoding::type encoding, int16_t max_level, int num_buffered_values);
-
-  // Initialize the LevelEncoder.
-  void Init(Encoding::type encoding, int16_t max_level, int num_buffered_values,
-      uint8_t* data, int data_size);
-
-  // Encodes a batch of levels from an array and returns the number of levels encoded
-  int Encode(int batch_size, const int16_t* levels);
-
-  int32_t len() {
-    if (encoding_ != Encoding::RLE) {
-      throw ParquetException("Only implemented for RLE encoding");
-    }
-    return rle_length_;
-  }
-
- private:
-  int bit_width_;
-  int rle_length_;
-  Encoding::type encoding_;
-  std::unique_ptr<RleEncoder> rle_encoder_;
-  std::unique_ptr<BitWriter> bit_packed_encoder_;
-};
-
-class PARQUET_EXPORT LevelDecoder {
- public:
-  LevelDecoder();
-  ~LevelDecoder();
-
-  // Initialize the LevelDecoder state with new data
-  // and return the number of bytes consumed
-  int SetData(Encoding::type encoding, int16_t max_level, int num_buffered_values,
-      const uint8_t* data);
-
-  // Decodes a batch of levels into an array and returns the number of levels decoded
-  int Decode(int batch_size, int16_t* levels);
-
- private:
-  int bit_width_;
-  int num_values_remaining_;
-  Encoding::type encoding_;
-  std::unique_ptr<RleDecoder> rle_decoder_;
-  std::unique_ptr<BitReader> bit_packed_decoder_;
-};
-
-}  // namespace parquet
-#endif  // PARQUET_COLUMN_LEVELS_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/page.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/page.h b/src/parquet/column/page.h
deleted file mode 100644
index a3813c5..0000000
--- a/src/parquet/column/page.h
+++ /dev/null
@@ -1,201 +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.
-
-// This module defines an abstract interface for iterating through pages in a
-// Parquet column chunk within a row group. It could be extended in the future
-// to iterate through all data pages in all chunks in a file.
-
-#ifndef PARQUET_COLUMN_PAGE_H
-#define PARQUET_COLUMN_PAGE_H
-
-#include <cstdint>
-#include <memory>
-#include <string>
-
-#include "parquet/column/statistics.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-
-namespace parquet {
-
-// TODO: Parallel processing is not yet safe because of memory-ownership
-// semantics (the PageReader may or may not own the memory referenced by a
-// page)
-//
-// TODO(wesm): In the future Parquet implementations may store the crc code
-// in format::PageHeader. parquet-mr currently does not, so we also skip it
-// here, both on the read and write path
-class Page {
- public:
-  Page(const std::shared_ptr<Buffer>& buffer, PageType::type type)
-      : buffer_(buffer), type_(type) {}
-
-  PageType::type type() const { return type_; }
-
-  std::shared_ptr<Buffer> buffer() const { return buffer_; }
-
-  // @returns: a pointer to the page's data
-  const uint8_t* data() const { return buffer_->data(); }
-
-  // @returns: the total size in bytes of the page's data buffer
-  int32_t size() const { return static_cast<int32_t>(buffer_->size()); }
-
- private:
-  std::shared_ptr<Buffer> buffer_;
-  PageType::type type_;
-};
-
-class DataPage : public Page {
- public:
-  DataPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
-      Encoding::type encoding, Encoding::type definition_level_encoding,
-      Encoding::type repetition_level_encoding,
-      const EncodedStatistics& statistics = EncodedStatistics())
-      : Page(buffer, PageType::DATA_PAGE),
-        num_values_(num_values),
-        encoding_(encoding),
-        definition_level_encoding_(definition_level_encoding),
-        repetition_level_encoding_(repetition_level_encoding),
-        statistics_(statistics) {}
-
-  int32_t num_values() const { return num_values_; }
-
-  Encoding::type encoding() const { return encoding_; }
-
-  Encoding::type repetition_level_encoding() const { return repetition_level_encoding_; }
-
-  Encoding::type definition_level_encoding() const { return definition_level_encoding_; }
-
-  const EncodedStatistics& statistics() const { return statistics_; }
-
- private:
-  int32_t num_values_;
-  Encoding::type encoding_;
-  Encoding::type definition_level_encoding_;
-  Encoding::type repetition_level_encoding_;
-  EncodedStatistics statistics_;
-};
-
-class CompressedDataPage : public DataPage {
- public:
-  CompressedDataPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
-      Encoding::type encoding, Encoding::type definition_level_encoding,
-      Encoding::type repetition_level_encoding, int64_t uncompressed_size,
-      const EncodedStatistics& statistics = EncodedStatistics())
-      : DataPage(buffer, num_values, encoding, definition_level_encoding,
-            repetition_level_encoding, statistics),
-        uncompressed_size_(uncompressed_size) {}
-
-  int64_t uncompressed_size() const { return uncompressed_size_; }
-
- private:
-  int64_t uncompressed_size_;
-};
-
-class DataPageV2 : public Page {
- public:
-  DataPageV2(const std::shared_ptr<Buffer>& buffer, int32_t num_values, int32_t num_nulls,
-      int32_t num_rows, Encoding::type encoding, int32_t definition_levels_byte_length,
-      int32_t repetition_levels_byte_length, bool is_compressed = false)
-      : Page(buffer, PageType::DATA_PAGE_V2),
-        num_values_(num_values),
-        num_nulls_(num_nulls),
-        num_rows_(num_rows),
-        encoding_(encoding),
-        definition_levels_byte_length_(definition_levels_byte_length),
-        repetition_levels_byte_length_(repetition_levels_byte_length),
-        is_compressed_(is_compressed) {}
-
-  int32_t num_values() const { return num_values_; }
-
-  int32_t num_nulls() const { return num_nulls_; }
-
-  int32_t num_rows() const { return num_rows_; }
-
-  Encoding::type encoding() const { return encoding_; }
-
-  int32_t definition_levels_byte_length() const { return definition_levels_byte_length_; }
-
-  int32_t repetition_levels_byte_length() const { return repetition_levels_byte_length_; }
-
-  bool is_compressed() const { return is_compressed_; }
-
- private:
-  int32_t num_values_;
-  int32_t num_nulls_;
-  int32_t num_rows_;
-  Encoding::type encoding_;
-  int32_t definition_levels_byte_length_;
-  int32_t repetition_levels_byte_length_;
-  bool is_compressed_;
-
-  // TODO(wesm): format::DataPageHeaderV2.statistics
-};
-
-class DictionaryPage : public Page {
- public:
-  DictionaryPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
-      Encoding::type encoding, bool is_sorted = false)
-      : Page(buffer, PageType::DICTIONARY_PAGE),
-        num_values_(num_values),
-        encoding_(encoding),
-        is_sorted_(is_sorted) {}
-
-  int32_t num_values() const { return num_values_; }
-
-  Encoding::type encoding() const { return encoding_; }
-
-  bool is_sorted() const { return is_sorted_; }
-
- private:
-  int32_t num_values_;
-  Encoding::type encoding_;
-  bool is_sorted_;
-};
-
-// Abstract page iterator interface. This way, we can feed column pages to the
-// ColumnReader through whatever mechanism we choose
-class PageReader {
- public:
-  virtual ~PageReader() {}
-
-  // @returns: shared_ptr<Page>(nullptr) on EOS, std::shared_ptr<Page>
-  // containing new Page otherwise
-  virtual std::shared_ptr<Page> NextPage() = 0;
-};
-
-class PageWriter {
- public:
-  virtual ~PageWriter() {}
-
-  // The Column Writer decides if dictionary encoding is used if set and
-  // if the dictionary encoding has fallen back to default encoding on reaching dictionary
-  // page limit
-  virtual void Close(bool has_dictionary, bool fallback) = 0;
-
-  virtual int64_t WriteDataPage(const CompressedDataPage& page) = 0;
-
-  virtual int64_t WriteDictionaryPage(const DictionaryPage& page) = 0;
-
-  virtual bool has_compressor() = 0;
-
-  virtual void Compress(const Buffer& src_buffer, ResizableBuffer* dest_buffer) = 0;
-};
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_PAGE_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/properties-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/properties-test.cc b/src/parquet/column/properties-test.cc
deleted file mode 100644
index 07247cf..0000000
--- a/src/parquet/column/properties-test.cc
+++ /dev/null
@@ -1,64 +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.
-
-#include <gtest/gtest.h>
-
-#include <string>
-
-#include "parquet/column/properties.h"
-#include "parquet/file/reader.h"
-
-namespace parquet {
-
-using schema::ColumnPath;
-
-namespace test {
-
-TEST(TestReaderProperties, Basics) {
-  ReaderProperties props;
-
-  ASSERT_EQ(DEFAULT_BUFFER_SIZE, props.buffer_size());
-  ASSERT_EQ(DEFAULT_USE_BUFFERED_STREAM, props.is_buffered_stream_enabled());
-}
-
-TEST(TestWriterProperties, Basics) {
-  std::shared_ptr<WriterProperties> props = WriterProperties::Builder().build();
-
-  ASSERT_EQ(DEFAULT_PAGE_SIZE, props->data_pagesize());
-  ASSERT_EQ(DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT, props->dictionary_pagesize_limit());
-  ASSERT_EQ(DEFAULT_WRITER_VERSION, props->version());
-}
-
-TEST(TestWriterProperties, AdvancedHandling) {
-  WriterProperties::Builder builder;
-  builder.compression("gzip", Compression::GZIP);
-  builder.compression(Compression::SNAPPY);
-  builder.encoding(Encoding::DELTA_BINARY_PACKED);
-  builder.encoding("delta-length", Encoding::DELTA_LENGTH_BYTE_ARRAY);
-  std::shared_ptr<WriterProperties> props = builder.build();
-
-  ASSERT_EQ(Compression::GZIP, props->compression(ColumnPath::FromDotString("gzip")));
-  ASSERT_EQ(
-      Compression::SNAPPY, props->compression(ColumnPath::FromDotString("delta-length")));
-  ASSERT_EQ(
-      Encoding::DELTA_BINARY_PACKED, props->encoding(ColumnPath::FromDotString("gzip")));
-  ASSERT_EQ(Encoding::DELTA_LENGTH_BYTE_ARRAY,
-      props->encoding(ColumnPath::FromDotString("delta-length")));
-}
-
-}  // namespace test
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/properties.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/properties.h b/src/parquet/column/properties.h
deleted file mode 100644
index 3ebc3b7..0000000
--- a/src/parquet/column/properties.h
+++ /dev/null
@@ -1,385 +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.
-
-#ifndef PARQUET_COLUMN_PROPERTIES_H
-#define PARQUET_COLUMN_PROPERTIES_H
-
-#include <memory>
-#include <string>
-#include <unordered_map>
-
-#include "parquet/exception.h"
-#include "parquet/parquet_version.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/visibility.h"
-
-namespace parquet {
-
-struct ParquetVersion {
-  enum type { PARQUET_1_0, PARQUET_2_0 };
-};
-
-static int64_t DEFAULT_BUFFER_SIZE = 0;
-static bool DEFAULT_USE_BUFFERED_STREAM = false;
-
-class PARQUET_EXPORT ReaderProperties {
- public:
-  explicit ReaderProperties(::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
-      : pool_(pool) {
-    buffered_stream_enabled_ = DEFAULT_USE_BUFFERED_STREAM;
-    buffer_size_ = DEFAULT_BUFFER_SIZE;
-  }
-
-  ::arrow::MemoryPool* memory_pool() const { return pool_; }
-
-  std::unique_ptr<InputStream> GetStream(
-      RandomAccessSource* source, int64_t start, int64_t num_bytes) {
-    std::unique_ptr<InputStream> stream;
-    if (buffered_stream_enabled_) {
-      stream.reset(
-          new BufferedInputStream(pool_, buffer_size_, source, start, num_bytes));
-    } else {
-      stream.reset(new InMemoryInputStream(source, start, num_bytes));
-    }
-    return stream;
-  }
-
-  bool is_buffered_stream_enabled() const { return buffered_stream_enabled_; }
-
-  void enable_buffered_stream() { buffered_stream_enabled_ = true; }
-
-  void disable_buffered_stream() { buffered_stream_enabled_ = false; }
-
-  void set_buffer_size(int64_t buf_size) { buffer_size_ = buf_size; }
-
-  int64_t buffer_size() const { return buffer_size_; }
-
- private:
-  ::arrow::MemoryPool* pool_;
-  int64_t buffer_size_;
-  bool buffered_stream_enabled_;
-};
-
-ReaderProperties PARQUET_EXPORT default_reader_properties();
-
-static constexpr int64_t DEFAULT_PAGE_SIZE = 1024 * 1024;
-static constexpr bool DEFAULT_IS_DICTIONARY_ENABLED = true;
-static constexpr int64_t DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT = DEFAULT_PAGE_SIZE;
-static constexpr int64_t DEFAULT_WRITE_BATCH_SIZE = 1024;
-static constexpr bool DEFAULT_ARE_STATISTICS_ENABLED = true;
-static constexpr Encoding::type DEFAULT_ENCODING = Encoding::PLAIN;
-static constexpr ParquetVersion::type DEFAULT_WRITER_VERSION =
-    ParquetVersion::PARQUET_1_0;
-static const char DEFAULT_CREATED_BY[] = CREATED_BY_VERSION;
-static constexpr Compression::type DEFAULT_COMPRESSION_TYPE = Compression::UNCOMPRESSED;
-
-class PARQUET_EXPORT ColumnProperties {
- public:
-  ColumnProperties(Encoding::type encoding = DEFAULT_ENCODING,
-      Compression::type codec = DEFAULT_COMPRESSION_TYPE,
-      bool dictionary_enabled = DEFAULT_IS_DICTIONARY_ENABLED,
-      bool statistics_enabled = DEFAULT_ARE_STATISTICS_ENABLED)
-      : encoding(encoding),
-        codec(codec),
-        dictionary_enabled(dictionary_enabled),
-        statistics_enabled(statistics_enabled) {}
-
-  Encoding::type encoding;
-  Compression::type codec;
-  bool dictionary_enabled;
-  bool statistics_enabled;
-};
-
-class PARQUET_EXPORT WriterProperties {
- public:
-  class Builder {
-   public:
-    Builder()
-        : pool_(::arrow::default_memory_pool()),
-          dictionary_pagesize_limit_(DEFAULT_DICTIONARY_PAGE_SIZE_LIMIT),
-          write_batch_size_(DEFAULT_WRITE_BATCH_SIZE),
-          pagesize_(DEFAULT_PAGE_SIZE),
-          version_(DEFAULT_WRITER_VERSION),
-          created_by_(DEFAULT_CREATED_BY) {}
-    virtual ~Builder() {}
-
-    Builder* memory_pool(::arrow::MemoryPool* pool) {
-      pool_ = pool;
-      return this;
-    }
-
-    Builder* enable_dictionary() {
-      default_column_properties_.dictionary_enabled = true;
-      return this;
-    }
-
-    Builder* disable_dictionary() {
-      default_column_properties_.dictionary_enabled = false;
-      return this;
-    }
-
-    Builder* enable_dictionary(const std::string& path) {
-      dictionary_enabled_[path] = true;
-      return this;
-    }
-
-    Builder* enable_dictionary(const std::shared_ptr<schema::ColumnPath>& path) {
-      return this->enable_dictionary(path->ToDotString());
-    }
-
-    Builder* disable_dictionary(const std::string& path) {
-      dictionary_enabled_[path] = false;
-      return this;
-    }
-
-    Builder* disable_dictionary(const std::shared_ptr<schema::ColumnPath>& path) {
-      return this->disable_dictionary(path->ToDotString());
-    }
-
-    Builder* dictionary_pagesize_limit(int64_t dictionary_psize_limit) {
-      dictionary_pagesize_limit_ = dictionary_psize_limit;
-      return this;
-    }
-
-    Builder* write_batch_size(int64_t write_batch_size) {
-      write_batch_size_ = write_batch_size;
-      return this;
-    }
-
-    Builder* data_pagesize(int64_t pg_size) {
-      pagesize_ = pg_size;
-      return this;
-    }
-
-    Builder* version(ParquetVersion::type version) {
-      version_ = version;
-      return this;
-    }
-
-    Builder* created_by(const std::string& created_by) {
-      created_by_ = created_by;
-      return this;
-    }
-
-    /**
-     * Define the encoding that is used when we don't utilise dictionary encoding.
-     *
-     * This either apply if dictionary encoding is disabled or if we fallback
-     * as the dictionary grew too large.
-     */
-    Builder* encoding(Encoding::type encoding_type) {
-      if (encoding_type == Encoding::PLAIN_DICTIONARY ||
-          encoding_type == Encoding::RLE_DICTIONARY) {
-        throw ParquetException("Can't use dictionary encoding as fallback encoding");
-      }
-
-      default_column_properties_.encoding = encoding_type;
-      return this;
-    }
-
-    /**
-     * Define the encoding that is used when we don't utilise dictionary encoding.
-     *
-     * This either apply if dictionary encoding is disabled or if we fallback
-     * as the dictionary grew too large.
-     */
-    Builder* encoding(const std::string& path, Encoding::type encoding_type) {
-      if (encoding_type == Encoding::PLAIN_DICTIONARY ||
-          encoding_type == Encoding::RLE_DICTIONARY) {
-        throw ParquetException("Can't use dictionary encoding as fallback encoding");
-      }
-
-      encodings_[path] = encoding_type;
-      return this;
-    }
-
-    /**
-     * Define the encoding that is used when we don't utilise dictionary encoding.
-     *
-     * This either apply if dictionary encoding is disabled or if we fallback
-     * as the dictionary grew too large.
-     */
-    Builder* encoding(
-        const std::shared_ptr<schema::ColumnPath>& path, Encoding::type encoding_type) {
-      return this->encoding(path->ToDotString(), encoding_type);
-    }
-
-    Builder* compression(Compression::type codec) {
-      default_column_properties_.codec = codec;
-      return this;
-    }
-
-    Builder* compression(const std::string& path, Compression::type codec) {
-      codecs_[path] = codec;
-      return this;
-    }
-
-    Builder* compression(
-        const std::shared_ptr<schema::ColumnPath>& path, Compression::type codec) {
-      return this->compression(path->ToDotString(), codec);
-    }
-
-    Builder* enable_statistics() {
-      default_column_properties_.statistics_enabled = true;
-      return this;
-    }
-
-    Builder* disable_statistics() {
-      default_column_properties_.statistics_enabled = false;
-      return this;
-    }
-
-    Builder* enable_statistics(const std::string& path) {
-      statistics_enabled_[path] = true;
-      return this;
-    }
-
-    Builder* enable_statistics(const std::shared_ptr<schema::ColumnPath>& path) {
-      return this->enable_statistics(path->ToDotString());
-    }
-
-    Builder* disable_statistics(const std::string& path) {
-      statistics_enabled_[path] = false;
-      return this;
-    }
-
-    Builder* disable_statistics(const std::shared_ptr<schema::ColumnPath>& path) {
-      return this->disable_statistics(path->ToDotString());
-    }
-
-    std::shared_ptr<WriterProperties> build() {
-      std::unordered_map<std::string, ColumnProperties> column_properties;
-      auto get = [&](const std::string& key) -> ColumnProperties& {
-        auto it = column_properties.find(key);
-        if (it == column_properties.end())
-          return column_properties[key] = default_column_properties_;
-        else
-          return it->second;
-      };
-
-      for (const auto& item : encodings_)
-        get(item.first).encoding = item.second;
-      for (const auto& item : codecs_)
-        get(item.first).codec = item.second;
-      for (const auto& item : dictionary_enabled_)
-        get(item.first).dictionary_enabled = item.second;
-      for (const auto& item : statistics_enabled_)
-        get(item.first).statistics_enabled = item.second;
-
-      return std::shared_ptr<WriterProperties>(new WriterProperties(pool_,
-          dictionary_pagesize_limit_, write_batch_size_, pagesize_, version_, created_by_,
-          default_column_properties_, column_properties));
-    }
-
-   private:
-    ::arrow::MemoryPool* pool_;
-    int64_t dictionary_pagesize_limit_;
-    int64_t write_batch_size_;
-    int64_t pagesize_;
-    ParquetVersion::type version_;
-    std::string created_by_;
-
-    // Settings used for each column unless overridden in any of the maps below
-    ColumnProperties default_column_properties_;
-    std::unordered_map<std::string, Encoding::type> encodings_;
-    std::unordered_map<std::string, Compression::type> codecs_;
-    std::unordered_map<std::string, bool> dictionary_enabled_;
-    std::unordered_map<std::string, bool> statistics_enabled_;
-  };
-
-  inline ::arrow::MemoryPool* memory_pool() const { return pool_; }
-
-  inline int64_t dictionary_pagesize_limit() const { return dictionary_pagesize_limit_; }
-
-  inline int64_t write_batch_size() const { return write_batch_size_; }
-
-  inline int64_t data_pagesize() const { return pagesize_; }
-
-  inline ParquetVersion::type version() const { return parquet_version_; }
-
-  inline std::string created_by() const { return parquet_created_by_; }
-
-  inline Encoding::type dictionary_index_encoding() const {
-    if (parquet_version_ == ParquetVersion::PARQUET_1_0) {
-      return Encoding::PLAIN_DICTIONARY;
-    } else {
-      return Encoding::RLE_DICTIONARY;
-    }
-  }
-
-  inline Encoding::type dictionary_page_encoding() const {
-    if (parquet_version_ == ParquetVersion::PARQUET_1_0) {
-      return Encoding::PLAIN_DICTIONARY;
-    } else {
-      return Encoding::PLAIN;
-    }
-  }
-
-  const ColumnProperties& column_properties(
-      const std::shared_ptr<schema::ColumnPath>& path) const {
-    auto it = column_properties_.find(path->ToDotString());
-    if (it != column_properties_.end()) return it->second;
-    return default_column_properties_;
-  }
-
-  Encoding::type encoding(const std::shared_ptr<schema::ColumnPath>& path) const {
-    return column_properties(path).encoding;
-  }
-
-  Compression::type compression(const std::shared_ptr<schema::ColumnPath>& path) const {
-    return column_properties(path).codec;
-  }
-
-  bool dictionary_enabled(const std::shared_ptr<schema::ColumnPath>& path) const {
-    return column_properties(path).dictionary_enabled;
-  }
-
-  bool statistics_enabled(const std::shared_ptr<schema::ColumnPath>& path) const {
-    return column_properties(path).statistics_enabled;
-  }
-
- private:
-  explicit WriterProperties(::arrow::MemoryPool* pool, int64_t dictionary_pagesize_limit,
-      int64_t write_batch_size, int64_t pagesize, ParquetVersion::type version,
-      const std::string& created_by, const ColumnProperties& default_column_properties,
-      const std::unordered_map<std::string, ColumnProperties>& column_properties)
-      : pool_(pool),
-        dictionary_pagesize_limit_(dictionary_pagesize_limit),
-        write_batch_size_(write_batch_size),
-        pagesize_(pagesize),
-        parquet_version_(version),
-        parquet_created_by_(created_by),
-        default_column_properties_(default_column_properties),
-        column_properties_(column_properties) {}
-
-  ::arrow::MemoryPool* pool_;
-  int64_t dictionary_pagesize_limit_;
-  int64_t write_batch_size_;
-  int64_t pagesize_;
-  ParquetVersion::type parquet_version_;
-  std::string parquet_created_by_;
-  ColumnProperties default_column_properties_;
-  std::unordered_map<std::string, ColumnProperties> column_properties_;
-};
-
-std::shared_ptr<WriterProperties> PARQUET_EXPORT default_writer_properties();
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_PROPERTIES_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.cc b/src/parquet/column/reader.cc
deleted file mode 100644
index bc4e4a0..0000000
--- a/src/parquet/column/reader.cc
+++ /dev/null
@@ -1,238 +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.
-
-#include "parquet/column/reader.h"
-
-#include <algorithm>
-#include <cstdint>
-#include <memory>
-
-#include "parquet/column/page.h"
-#include "parquet/column/properties.h"
-#include "parquet/encoding-internal.h"
-
-using arrow::MemoryPool;
-
-namespace parquet {
-
-ReaderProperties default_reader_properties() {
-  static ReaderProperties default_reader_properties;
-  return default_reader_properties;
-}
-
-ColumnReader::ColumnReader(
-    const ColumnDescriptor* descr, std::unique_ptr<PageReader> pager, MemoryPool* pool)
-    : descr_(descr),
-      pager_(std::move(pager)),
-      num_buffered_values_(0),
-      num_decoded_values_(0),
-      pool_(pool) {}
-
-ColumnReader::~ColumnReader() {}
-
-template <typename DType>
-void TypedColumnReader<DType>::ConfigureDictionary(const DictionaryPage* page) {
-  int encoding = static_cast<int>(page->encoding());
-  if (page->encoding() == Encoding::PLAIN_DICTIONARY ||
-      page->encoding() == Encoding::PLAIN) {
-    encoding = static_cast<int>(Encoding::RLE_DICTIONARY);
-  }
-
-  auto it = decoders_.find(encoding);
-  if (it != decoders_.end()) {
-    throw ParquetException("Column cannot have more than one dictionary.");
-  }
-
-  if (page->encoding() == Encoding::PLAIN_DICTIONARY ||
-      page->encoding() == Encoding::PLAIN) {
-    PlainDecoder<DType> dictionary(descr_);
-    dictionary.SetData(page->num_values(), page->data(), page->size());
-
-    // The dictionary is fully decoded during DictionaryDecoder::Init, so the
-    // DictionaryPage buffer is no longer required after this step
-    //
-    // TODO(wesm): investigate whether this all-or-nothing decoding of the
-    // dictionary makes sense and whether performance can be improved
-
-    auto decoder = std::make_shared<DictionaryDecoder<DType>>(descr_, pool_);
-    decoder->SetDict(&dictionary);
-    decoders_[encoding] = decoder;
-  } else {
-    ParquetException::NYI("only plain dictionary encoding has been implemented");
-  }
-
-  current_decoder_ = decoders_[encoding].get();
-}
-
-// PLAIN_DICTIONARY is deprecated but used to be used as a dictionary index
-// encoding.
-static bool IsDictionaryIndexEncoding(const Encoding::type& e) {
-  return e == Encoding::RLE_DICTIONARY || e == Encoding::PLAIN_DICTIONARY;
-}
-
-template <typename DType>
-bool TypedColumnReader<DType>::ReadNewPage() {
-  // Loop until we find the next data page.
-  const uint8_t* buffer;
-
-  while (true) {
-    current_page_ = pager_->NextPage();
-    if (!current_page_) {
-      // EOS
-      return false;
-    }
-
-    if (current_page_->type() == PageType::DICTIONARY_PAGE) {
-      ConfigureDictionary(static_cast<const DictionaryPage*>(current_page_.get()));
-      continue;
-    } else if (current_page_->type() == PageType::DATA_PAGE) {
-      const DataPage* page = static_cast<const DataPage*>(current_page_.get());
-
-      // Read a data page.
-      num_buffered_values_ = page->num_values();
-
-      // Have not decoded any values from the data page yet
-      num_decoded_values_ = 0;
-
-      buffer = page->data();
-
-      // If the data page includes repetition and definition levels, we
-      // initialize the level decoder and subtract the encoded level bytes from
-      // the page size to determine the number of bytes in the encoded data.
-      int64_t data_size = page->size();
-
-      // Data page Layout: Repetition Levels - Definition Levels - encoded values.
-      // Levels are encoded as rle or bit-packed.
-      // Init repetition levels
-      if (descr_->max_repetition_level() > 0) {
-        int64_t rep_levels_bytes = repetition_level_decoder_.SetData(
-            page->repetition_level_encoding(), descr_->max_repetition_level(),
-            static_cast<int>(num_buffered_values_), buffer);
-        buffer += rep_levels_bytes;
-        data_size -= rep_levels_bytes;
-      }
-      // TODO figure a way to set max_definition_level_ to 0
-      // if the initial value is invalid
-
-      // Init definition levels
-      if (descr_->max_definition_level() > 0) {
-        int64_t def_levels_bytes = definition_level_decoder_.SetData(
-            page->definition_level_encoding(), descr_->max_definition_level(),
-            static_cast<int>(num_buffered_values_), buffer);
-        buffer += def_levels_bytes;
-        data_size -= def_levels_bytes;
-      }
-
-      // Get a decoder object for this page or create a new decoder if this is the
-      // first page with this encoding.
-      Encoding::type encoding = page->encoding();
-
-      if (IsDictionaryIndexEncoding(encoding)) { encoding = Encoding::RLE_DICTIONARY; }
-
-      auto it = decoders_.find(static_cast<int>(encoding));
-      if (it != decoders_.end()) {
-        if (encoding == Encoding::RLE_DICTIONARY) {
-          DCHECK(current_decoder_->encoding() == Encoding::RLE_DICTIONARY);
-        }
-        current_decoder_ = it->second.get();
-      } else {
-        switch (encoding) {
-          case Encoding::PLAIN: {
-            std::shared_ptr<DecoderType> decoder(new PlainDecoder<DType>(descr_));
-            decoders_[static_cast<int>(encoding)] = decoder;
-            current_decoder_ = decoder.get();
-            break;
-          }
-          case Encoding::RLE_DICTIONARY:
-            throw ParquetException("Dictionary page must be before data page.");
-
-          case Encoding::DELTA_BINARY_PACKED:
-          case Encoding::DELTA_LENGTH_BYTE_ARRAY:
-          case Encoding::DELTA_BYTE_ARRAY:
-            ParquetException::NYI("Unsupported encoding");
-
-          default:
-            throw ParquetException("Unknown encoding type.");
-        }
-      }
-      current_decoder_->SetData(
-          static_cast<int>(num_buffered_values_), buffer, static_cast<int>(data_size));
-      return true;
-    } else {
-      // We don't know what this page type is. We're allowed to skip non-data
-      // pages.
-      continue;
-    }
-  }
-  return true;
-}
-
-// ----------------------------------------------------------------------
-// Batch read APIs
-
-int64_t ColumnReader::ReadDefinitionLevels(int64_t batch_size, int16_t* levels) {
-  if (descr_->max_definition_level() == 0) { return 0; }
-  return definition_level_decoder_.Decode(static_cast<int>(batch_size), levels);
-}
-
-int64_t ColumnReader::ReadRepetitionLevels(int64_t batch_size, int16_t* levels) {
-  if (descr_->max_repetition_level() == 0) { return 0; }
-  return repetition_level_decoder_.Decode(static_cast<int>(batch_size), levels);
-}
-
-// ----------------------------------------------------------------------
-// Dynamic column reader constructor
-
-std::shared_ptr<ColumnReader> ColumnReader::Make(
-    const ColumnDescriptor* descr, std::unique_ptr<PageReader> pager, MemoryPool* pool) {
-  switch (descr->physical_type()) {
-    case Type::BOOLEAN:
-      return std::make_shared<BoolReader>(descr, std::move(pager), pool);
-    case Type::INT32:
-      return std::make_shared<Int32Reader>(descr, std::move(pager), pool);
-    case Type::INT64:
-      return std::make_shared<Int64Reader>(descr, std::move(pager), pool);
-    case Type::INT96:
-      return std::make_shared<Int96Reader>(descr, std::move(pager), pool);
-    case Type::FLOAT:
-      return std::make_shared<FloatReader>(descr, std::move(pager), pool);
-    case Type::DOUBLE:
-      return std::make_shared<DoubleReader>(descr, std::move(pager), pool);
-    case Type::BYTE_ARRAY:
-      return std::make_shared<ByteArrayReader>(descr, std::move(pager), pool);
-    case Type::FIXED_LEN_BYTE_ARRAY:
-      return std::make_shared<FixedLenByteArrayReader>(descr, std::move(pager), pool);
-    default:
-      ParquetException::NYI("type reader not implemented");
-  }
-  // Unreachable code, but supress compiler warning
-  return std::shared_ptr<ColumnReader>(nullptr);
-}
-
-// ----------------------------------------------------------------------
-// Instantiate templated classes
-
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<BooleanType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int32Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int64Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int96Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<FloatType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<DoubleType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<ByteArrayType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<FLBAType>;
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.h b/src/parquet/column/reader.h
deleted file mode 100644
index 724773d..0000000
--- a/src/parquet/column/reader.h
+++ /dev/null
@@ -1,453 +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.
-
-#ifndef PARQUET_COLUMN_READER_H
-#define PARQUET_COLUMN_READER_H
-
-#include <algorithm>
-#include <cstdint>
-#include <cstring>
-#include <memory>
-#include <unordered_map>
-#include <vector>
-#include <iostream>
-
-#include <arrow/util/bit-util.h>
-
-#include "parquet/column/levels.h"
-#include "parquet/column/page.h"
-#include "parquet/encoding.h"
-#include "parquet/exception.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/visibility.h"
-
-namespace parquet {
-
-class PARQUET_EXPORT ColumnReader {
- public:
-  ColumnReader(const ColumnDescriptor*, std::unique_ptr<PageReader>,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
-  virtual ~ColumnReader();
-
-  static std::shared_ptr<ColumnReader> Make(const ColumnDescriptor* descr,
-      std::unique_ptr<PageReader> pager,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
-
-  // Returns true if there are still values in this column.
-  bool HasNext() {
-    // Either there is no data page available yet, or the data page has been
-    // exhausted
-    if (num_buffered_values_ == 0 || num_decoded_values_ == num_buffered_values_) {
-      if (!ReadNewPage() || num_buffered_values_ == 0) { return false; }
-    }
-    return true;
-  }
-
-  Type::type type() const { return descr_->physical_type(); }
-
-  const ColumnDescriptor* descr() const { return descr_; }
-
- protected:
-  virtual bool ReadNewPage() = 0;
-
-  // Read multiple definition levels into preallocated memory
-  //
-  // Returns the number of decoded definition levels
-  int64_t ReadDefinitionLevels(int64_t batch_size, int16_t* levels);
-
-  // Read multiple repetition levels into preallocated memory
-  // Returns the number of decoded repetition levels
-  int64_t ReadRepetitionLevels(int64_t batch_size, int16_t* levels);
-
-  const ColumnDescriptor* descr_;
-
-  std::unique_ptr<PageReader> pager_;
-  std::shared_ptr<Page> current_page_;
-
-  // Not set if full schema for this field has no optional or repeated elements
-  LevelDecoder definition_level_decoder_;
-
-  // Not set for flat schemas.
-  LevelDecoder repetition_level_decoder_;
-
-  // The total number of values stored in the data page. This is the maximum of
-  // the number of encoded definition levels or encoded values. For
-  // non-repeated, required columns, this is equal to the number of encoded
-  // values. For repeated or optional values, there may be fewer data values
-  // than levels, and this tells you how many encoded levels there are in that
-  // case.
-  int64_t num_buffered_values_;
-
-  // The number of values from the current data page that have been decoded
-  // into memory
-  int64_t num_decoded_values_;
-
-  ::arrow::MemoryPool* pool_;
-};
-
-// API to read values from a single column. This is the main client facing API.
-template <typename DType>
-class PARQUET_EXPORT TypedColumnReader : public ColumnReader {
- public:
-  typedef typename DType::c_type T;
-
-  TypedColumnReader(const ColumnDescriptor* schema, std::unique_ptr<PageReader> pager,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
-      : ColumnReader(schema, std::move(pager), pool), current_decoder_(NULL) {}
-  virtual ~TypedColumnReader() {}
-
-  // Read a batch of repetition levels, definition levels, and values from the
-  // column.
-  //
-  // Since null values are not stored in the values, the number of values read
-  // may be less than the number of repetition and definition levels. With
-  // nested data this is almost certainly true.
-  //
-  // Set def_levels or rep_levels to nullptr if you want to skip reading them.
-  // This is only safe if you know through some other source that there are no
-  // undefined values.
-  //
-  // To fully exhaust a row group, you must read batches until the number of
-  // values read reaches the number of stored values according to the metadata.
-  //
-  // This API is the same for both V1 and V2 of the DataPage
-  //
-  // @returns: actual number of levels read (see values_read for number of values read)
-  int64_t ReadBatch(int64_t batch_size, int16_t* def_levels, int16_t* rep_levels,
-      T* values, int64_t* values_read);
-
-  /// Read a batch of repetition levels, definition levels, and values from the
-  /// column and leave spaces for null entries on the lowest level in the values
-  /// buffer.
-  ///
-  /// In comparision to ReadBatch the length of repetition and definition levels
-  /// is the same as of the number of values read for max_definition_level == 1.
-  /// In the case of max_definition_level > 1, the repetition and definition
-  /// levels are larger than the values but the values include the null entries
-  /// with definition_level == (max_definition_level - 1).
-  ///
-  /// To fully exhaust a row group, you must read batches until the number of
-  /// values read reaches the number of stored values according to the metadata.
-  ///
-  /// @param batch_size the number of levels to read
-  /// @param[out] def_levels The Parquet definition levels, output has
-  ///   the length levels_read.
-  /// @param[out] rep_levels The Parquet repetition levels, output has
-  ///   the length levels_read.
-  /// @param[out] values The values in the lowest nested level including
-  ///   spacing for nulls on the lowest levels; output has the length
-  ///   values_read.
-  /// @param[out] valid_bits Memory allocated for a bitmap that indicates if
-  ///   the row is null or on the maximum definition level. For performance
-  ///   reasons the underlying buffer should be able to store 1 bit more than
-  ///   required. If this requires an additional byte, this byte is only read
-  ///   but never written to.
-  /// @param valid_bits_offset The offset in bits of the valid_bits where the
-  ///   first relevant bit resides.
-  /// @param[out] levels_read The number of repetition/definition levels that were read.
-  /// @param[out] values_read The number of values read, this includes all
-  ///   non-null entries as well as all null-entries on the lowest level
-  ///   (i.e. definition_level == max_definition_level - 1)
-  /// @param[out] null_count The number of nulls on the lowest levels.
-  ///   (i.e. (values_read - null_count) is total number of non-null entries)
-  int64_t ReadBatchSpaced(int64_t batch_size, int16_t* def_levels, int16_t* rep_levels,
-      T* values, uint8_t* valid_bits, int64_t valid_bits_offset, int64_t* levels_read,
-      int64_t* values_read, int64_t* null_count);
-
-  // Skip reading levels
-  // Returns the number of levels skipped
-  int64_t Skip(int64_t num_rows_to_skip);
-
- private:
-  typedef Decoder<DType> DecoderType;
-
-  // Advance to the next data page
-  virtual bool ReadNewPage();
-
-  // Read up to batch_size values from the current data page into the
-  // pre-allocated memory T*
-  //
-  // @returns: the number of values read into the out buffer
-  int64_t ReadValues(int64_t batch_size, T* out);
-
-  // Read up to batch_size values from the current data page into the
-  // pre-allocated memory T*, leaving spaces for null entries according
-  // to the def_levels.
-  //
-  // @returns: the number of values read into the out buffer
-  int64_t ReadValuesSpaced(int64_t batch_size, T* out, int null_count,
-      uint8_t* valid_bits, int64_t valid_bits_offset);
-
-  // Map of encoding type to the respective decoder object. For example, a
-  // column chunk's data pages may include both dictionary-encoded and
-  // plain-encoded data.
-  std::unordered_map<int, std::shared_ptr<DecoderType>> decoders_;
-
-  void ConfigureDictionary(const DictionaryPage* page);
-
-  DecoderType* current_decoder_;
-};
-
-template <typename DType>
-inline int64_t TypedColumnReader<DType>::ReadValues(int64_t batch_size, T* out) {
-  int64_t num_decoded = current_decoder_->Decode(out, static_cast<int>(batch_size));
-  return num_decoded;
-}
-
-template <typename DType>
-inline int64_t TypedColumnReader<DType>::ReadValuesSpaced(int64_t batch_size, T* out,
-    int null_count, uint8_t* valid_bits, int64_t valid_bits_offset) {
-  return current_decoder_->DecodeSpaced(
-      out, static_cast<int>(batch_size), null_count, valid_bits, valid_bits_offset);
-}
-
-template <typename DType>
-inline int64_t TypedColumnReader<DType>::ReadBatch(int64_t batch_size,
-    int16_t* def_levels, int16_t* rep_levels, T* values, int64_t* values_read) {
-  // HasNext invokes ReadNewPage
-  if (!HasNext()) {
-    *values_read = 0;
-    return 0;
-  }
-
-  // TODO(wesm): keep reading data pages until batch_size is reached, or the
-  // row group is finished
-  batch_size = std::min(batch_size, num_buffered_values_ - num_decoded_values_);
-
-  int64_t num_def_levels = 0;
-  int64_t num_rep_levels = 0;
-
-  int64_t values_to_read = 0;
-
-  // If the field is required and non-repeated, there are no definition levels
-  if (descr_->max_definition_level() > 0 && def_levels) {
-    num_def_levels = ReadDefinitionLevels(batch_size, def_levels);
-    // TODO(wesm): this tallying of values-to-decode can be performed with better
-    // cache-efficiency if fused with the level decoding.
-    for (int64_t i = 0; i < num_def_levels; ++i) {
-      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_read; }
-    }
-  } else {
-    // Required field, read all values
-    values_to_read = batch_size;
-  }
-
-  // Not present for non-repeated fields
-  if (descr_->max_repetition_level() > 0 && rep_levels) {
-    num_rep_levels = ReadRepetitionLevels(batch_size, rep_levels);
-    if (def_levels && num_def_levels != num_rep_levels) {
-      throw ParquetException("Number of decoded rep / def levels did not match");
-    }
-  }
-
-  *values_read = ReadValues(values_to_read, values);
-  int64_t total_values = std::max(num_def_levels, *values_read);
-  num_decoded_values_ += total_values;
-
-  return total_values;
-}
-
-inline void DefinitionLevelsToBitmap(const int16_t* def_levels, int64_t num_def_levels,
-    int16_t max_definition_level,  int16_t max_repetition_level,
-    int64_t* values_read, int64_t* null_count,
-    uint8_t* valid_bits, int64_t valid_bits_offset) {
-  int byte_offset = static_cast<int>(valid_bits_offset) / 8;
-  int bit_offset = static_cast<int>(valid_bits_offset) % 8;
-  uint8_t bitset = valid_bits[byte_offset];
-
-  // TODO(itaiin): As an interim solution we are splitting the code path here
-  // between repeated+flat column reads, and non-repeated+nested reads.
-  // Those paths need to be merged in the future
-  for (int i = 0; i < num_def_levels; ++i) {
-    if (def_levels[i] == max_definition_level) {
-      bitset |= (1 << bit_offset);
-    } else if (max_repetition_level > 0) {
-      // repetition+flat case
-      if (def_levels[i] == (max_definition_level - 1)) {
-        bitset &= ~(1 << bit_offset);
-        *null_count += 1;
-      } else {
-        continue;
-      }
-    } else {
-      // non-repeated+nested case
-      if (def_levels[i] < max_definition_level) {
-        bitset &= ~(1 << bit_offset);
-        *null_count += 1;
-      } else {
-        throw ParquetException("definition level exceeds maximum");
-      }
-    }
-
-    bit_offset++;
-    if (bit_offset == 8) {
-      bit_offset = 0;
-      valid_bits[byte_offset] = bitset;
-      byte_offset++;
-      // TODO: Except for the last byte, this shouldn't be needed
-      bitset = valid_bits[byte_offset];
-    }
-  }
-  if (bit_offset != 0) { valid_bits[byte_offset] = bitset; }
-  *values_read = (bit_offset + byte_offset * 8 - valid_bits_offset);
-}
-
-template <typename DType>
-inline int64_t TypedColumnReader<DType>::ReadBatchSpaced(int64_t batch_size,
-    int16_t* def_levels, int16_t* rep_levels, T* values, uint8_t* valid_bits,
-    int64_t valid_bits_offset, int64_t* levels_read, int64_t* values_read,
-    int64_t* null_count_out) {
-  // HasNext invokes ReadNewPage
-  if (!HasNext()) {
-    *levels_read = 0;
-    *values_read = 0;
-    *null_count_out = 0;
-    return 0;
-  }
-
-  int64_t total_values;
-  // TODO(wesm): keep reading data pages until batch_size is reached, or the
-  // row group is finished
-  batch_size = std::min(batch_size, num_buffered_values_ - num_decoded_values_);
-
-  // If the field is required and non-repeated, there are no definition levels
-  if (descr_->max_definition_level() > 0) {
-    int64_t num_def_levels = ReadDefinitionLevels(batch_size, def_levels);
-
-    // Not present for non-repeated fields
-    if (descr_->max_repetition_level() > 0) {
-      int64_t num_rep_levels = ReadRepetitionLevels(batch_size, rep_levels);
-      if (num_def_levels != num_rep_levels) {
-        throw ParquetException("Number of decoded rep / def levels did not match");
-      }
-    }
-
-    // TODO(itaiin): another code path split to merge when the general case is done
-    bool has_spaced_values;
-    if (descr_->max_repetition_level() > 0) {
-      // repeated+flat case
-      has_spaced_values = !descr_->schema_node()->is_required();
-    } else {
-      // non-repeated+nested case
-      // Find if a node forces nulls in the lowest level along the hierarchy
-      const schema::Node* node = descr_->schema_node().get();
-      has_spaced_values = false;
-      while (node) {
-        auto parent = node->parent();
-        if (node->is_optional()) {
-          has_spaced_values = true;
-          break;
-        }
-        node = parent;
-      }
-    }
-
-    int64_t null_count = 0;
-    if (!has_spaced_values) {
-      int values_to_read = 0;
-      for (int64_t i = 0; i < num_def_levels; ++i) {
-        if (def_levels[i] == descr_->max_definition_level()) { ++values_to_read; }
-      }
-      total_values = ReadValues(values_to_read, values);
-      for (int64_t i = 0; i < total_values; i++) {
-        ::arrow::BitUtil::SetBit(valid_bits, valid_bits_offset + i);
-      }
-      *values_read = total_values;
-    } else {
-      int16_t max_definition_level = descr_->max_definition_level();
-      int16_t max_repetition_level = descr_->max_repetition_level();
-      DefinitionLevelsToBitmap(def_levels, num_def_levels, max_definition_level,
-          max_repetition_level, values_read, &null_count, valid_bits, valid_bits_offset);
-      total_values = ReadValuesSpaced(*values_read, values, static_cast<int>(null_count),
-          valid_bits, valid_bits_offset);
-    }
-    *levels_read = num_def_levels;
-    *null_count_out = null_count;
-
-  } else {
-    // Required field, read all values
-    total_values = ReadValues(batch_size, values);
-    for (int64_t i = 0; i < total_values; i++) {
-      ::arrow::BitUtil::SetBit(valid_bits, valid_bits_offset + i);
-    }
-    *null_count_out = 0;
-    *levels_read = total_values;
-  }
-
-  num_decoded_values_ += *levels_read;
-  return total_values;
-}
-
-template <typename DType>
-inline int64_t TypedColumnReader<DType>::Skip(int64_t num_rows_to_skip) {
-  int64_t rows_to_skip = num_rows_to_skip;
-  while (HasNext() && rows_to_skip > 0) {
-    // If the number of rows to skip is more than the number of undecoded values, skip the
-    // Page.
-    if (rows_to_skip > (num_buffered_values_ - num_decoded_values_)) {
-      rows_to_skip -= num_buffered_values_ - num_decoded_values_;
-      num_decoded_values_ = num_buffered_values_;
-    } else {
-      // We need to read this Page
-      // Jump to the right offset in the Page
-      int64_t batch_size = 1024;  // ReadBatch with a smaller memory footprint
-      int64_t values_read = 0;
-
-      std::shared_ptr<PoolBuffer> vals = AllocateBuffer(
-          this->pool_, batch_size * type_traits<DType::type_num>::value_byte_size);
-      std::shared_ptr<PoolBuffer> def_levels =
-          AllocateBuffer(this->pool_, batch_size * sizeof(int16_t));
-
-      std::shared_ptr<PoolBuffer> rep_levels =
-          AllocateBuffer(this->pool_, batch_size * sizeof(int16_t));
-
-      do {
-        batch_size = std::min(batch_size, rows_to_skip);
-        values_read = ReadBatch(static_cast<int>(batch_size),
-            reinterpret_cast<int16_t*>(def_levels->mutable_data()),
-            reinterpret_cast<int16_t*>(rep_levels->mutable_data()),
-            reinterpret_cast<T*>(vals->mutable_data()), &values_read);
-        rows_to_skip -= values_read;
-      } while (values_read > 0 && rows_to_skip > 0);
-    }
-  }
-  return num_rows_to_skip - rows_to_skip;
-}
-
-typedef TypedColumnReader<BooleanType> BoolReader;
-typedef TypedColumnReader<Int32Type> Int32Reader;
-typedef TypedColumnReader<Int64Type> Int64Reader;
-typedef TypedColumnReader<Int96Type> Int96Reader;
-typedef TypedColumnReader<FloatType> FloatReader;
-typedef TypedColumnReader<DoubleType> DoubleReader;
-typedef TypedColumnReader<ByteArrayType> ByteArrayReader;
-typedef TypedColumnReader<FLBAType> FixedLenByteArrayReader;
-
-extern template class PARQUET_EXPORT TypedColumnReader<BooleanType>;
-extern template class PARQUET_EXPORT TypedColumnReader<Int32Type>;
-extern template class PARQUET_EXPORT TypedColumnReader<Int64Type>;
-extern template class PARQUET_EXPORT TypedColumnReader<Int96Type>;
-extern template class PARQUET_EXPORT TypedColumnReader<FloatType>;
-extern template class PARQUET_EXPORT TypedColumnReader<DoubleType>;
-extern template class PARQUET_EXPORT TypedColumnReader<ByteArrayType>;
-extern template class PARQUET_EXPORT TypedColumnReader<FLBAType>;
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_READER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/scan-all.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/scan-all.cc b/src/parquet/column/scan-all.cc
deleted file mode 100644
index 36a7689..0000000
--- a/src/parquet/column/scan-all.cc
+++ /dev/null
@@ -1,56 +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.
-
-#include "parquet/column/scan-all.h"
-
-namespace parquet {
-
-int64_t ScanAllValues(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
-    uint8_t* values, int64_t* values_buffered, parquet::ColumnReader* reader) {
-  switch (reader->type()) {
-    case parquet::Type::BOOLEAN:
-      return ScanAll<parquet::BoolReader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::INT32:
-      return ScanAll<parquet::Int32Reader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::INT64:
-      return ScanAll<parquet::Int64Reader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::INT96:
-      return ScanAll<parquet::Int96Reader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::FLOAT:
-      return ScanAll<parquet::FloatReader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::DOUBLE:
-      return ScanAll<parquet::DoubleReader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::BYTE_ARRAY:
-      return ScanAll<parquet::ByteArrayReader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    case parquet::Type::FIXED_LEN_BYTE_ARRAY:
-      return ScanAll<parquet::FixedLenByteArrayReader>(
-          batch_size, def_levels, rep_levels, values, values_buffered, reader);
-    default:
-      parquet::ParquetException::NYI("type reader not implemented");
-  }
-  // Unreachable code, but supress compiler warning
-  return 0;
-}
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/scan-all.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/scan-all.h b/src/parquet/column/scan-all.h
deleted file mode 100644
index b701c17..0000000
--- a/src/parquet/column/scan-all.h
+++ /dev/null
@@ -1,41 +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.
-
-#ifndef PARQUET_SCAN_ALL_H
-#define PARQUET_SCAN_ALL_H
-
-#include "parquet/column/reader.h"
-
-namespace parquet {
-
-template <typename RType>
-int64_t ScanAll(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
-    uint8_t* values, int64_t* values_buffered, parquet::ColumnReader* reader) {
-  typedef typename RType::T Type;
-  auto typed_reader = static_cast<RType*>(reader);
-  auto vals = reinterpret_cast<Type*>(&values[0]);
-  return typed_reader->ReadBatch(
-      batch_size, def_levels, rep_levels, vals, values_buffered);
-}
-
-int64_t PARQUET_EXPORT ScanAllValues(int32_t batch_size, int16_t* def_levels,
-    int16_t* rep_levels, uint8_t* values, int64_t* values_buffered,
-    parquet::ColumnReader* reader);
-
-}  // namespace parquet
-
-#endif  // PARQUET_SCAN_ALL_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/scanner-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner-test.cc b/src/parquet/column/scanner-test.cc
deleted file mode 100644
index 5d137b7..0000000
--- a/src/parquet/column/scanner-test.cc
+++ /dev/null
@@ -1,232 +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.
-
-#include <gtest/gtest.h>
-
-#include <algorithm>
-#include <cstdint>
-#include <cstdlib>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "parquet/column/page.h"
-#include "parquet/column/scanner.h"
-#include "parquet/column/test-specialization.h"
-#include "parquet/column/test-util.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/test-common.h"
-
-using std::string;
-using std::vector;
-using std::shared_ptr;
-
-namespace parquet {
-
-using schema::NodePtr;
-
-namespace test {
-
-template <>
-void InitDictValues<bool>(
-    int num_values, int dict_per_page, vector<bool>& values, vector<uint8_t>& buffer) {
-  // No op for bool
-}
-
-template <typename Type>
-class TestFlatScanner : public ::testing::Test {
- public:
-  typedef typename Type::c_type T;
-
-  void InitScanner(const ColumnDescriptor* d) {
-    std::unique_ptr<PageReader> pager(new test::MockPageReader(pages_));
-    scanner_ = Scanner::Make(ColumnReader::Make(d, std::move(pager)));
-  }
-
-  void CheckResults(int batch_size, const ColumnDescriptor* d) {
-    TypedScanner<Type>* scanner = reinterpret_cast<TypedScanner<Type>*>(scanner_.get());
-    T val;
-    bool is_null = false;
-    int16_t def_level;
-    int16_t rep_level;
-    int j = 0;
-    scanner->SetBatchSize(batch_size);
-    for (int i = 0; i < num_levels_; i++) {
-      ASSERT_TRUE(scanner->Next(&val, &def_level, &rep_level, &is_null)) << i << j;
-      if (!is_null) {
-        ASSERT_EQ(values_[j], val) << i << "V" << j;
-        j++;
-      }
-      if (d->max_definition_level() > 0) {
-        ASSERT_EQ(def_levels_[i], def_level) << i << "D" << j;
-      }
-      if (d->max_repetition_level() > 0) {
-        ASSERT_EQ(rep_levels_[i], rep_level) << i << "R" << j;
-      }
-    }
-    ASSERT_EQ(num_values_, j);
-    ASSERT_FALSE(scanner->Next(&val, &def_level, &rep_level, &is_null));
-  }
-
-  void Clear() {
-    pages_.clear();
-    values_.clear();
-    def_levels_.clear();
-    rep_levels_.clear();
-  }
-
-  void Execute(int num_pages, int levels_per_page, int batch_size,
-      const ColumnDescriptor* d, Encoding::type encoding) {
-    num_values_ = MakePages<Type>(d, num_pages, levels_per_page, def_levels_, rep_levels_,
-        values_, data_buffer_, pages_, encoding);
-    num_levels_ = num_pages * levels_per_page;
-    InitScanner(d);
-    CheckResults(batch_size, d);
-    Clear();
-  }
-
-  void InitDescriptors(std::shared_ptr<ColumnDescriptor>& d1,
-      std::shared_ptr<ColumnDescriptor>& d2, std::shared_ptr<ColumnDescriptor>& d3,
-      int length) {
-    NodePtr type;
-    type = schema::PrimitiveNode::Make(
-        "c1", Repetition::REQUIRED, Type::type_num, LogicalType::NONE, length);
-    d1.reset(new ColumnDescriptor(type, 0, 0));
-    type = schema::PrimitiveNode::Make(
-        "c2", Repetition::OPTIONAL, Type::type_num, LogicalType::NONE, length);
-    d2.reset(new ColumnDescriptor(type, 4, 0));
-    type = schema::PrimitiveNode::Make(
-        "c3", Repetition::REPEATED, Type::type_num, LogicalType::NONE, length);
-    d3.reset(new ColumnDescriptor(type, 4, 2));
-  }
-
-  void ExecuteAll(int num_pages, int num_levels, int batch_size, int type_length,
-      Encoding::type encoding = Encoding::PLAIN) {
-    std::shared_ptr<ColumnDescriptor> d1;
-    std::shared_ptr<ColumnDescriptor> d2;
-    std::shared_ptr<ColumnDescriptor> d3;
-    InitDescriptors(d1, d2, d3, type_length);
-    // evaluate REQUIRED pages
-    Execute(num_pages, num_levels, batch_size, d1.get(), encoding);
-    // evaluate OPTIONAL pages
-    Execute(num_pages, num_levels, batch_size, d2.get(), encoding);
-    // evaluate REPEATED pages
-    Execute(num_pages, num_levels, batch_size, d3.get(), encoding);
-  }
-
- protected:
-  int num_levels_;
-  int num_values_;
-  vector<shared_ptr<Page>> pages_;
-  std::shared_ptr<Scanner> scanner_;
-  vector<T> values_;
-  vector<int16_t> def_levels_;
-  vector<int16_t> rep_levels_;
-  vector<uint8_t> data_buffer_;  // For BA and FLBA
-};
-
-static int num_levels_per_page = 100;
-static int num_pages = 20;
-static int batch_size = 32;
-
-typedef ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
-    ByteArrayType>
-    TestTypes;
-
-using TestBooleanFlatScanner = TestFlatScanner<BooleanType>;
-using TestFLBAFlatScanner = TestFlatScanner<FLBAType>;
-
-TYPED_TEST_CASE(TestFlatScanner, TestTypes);
-
-TYPED_TEST(TestFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0, Encoding::PLAIN);
-}
-
-TYPED_TEST(TestFlatScanner, TestDictScanner) {
-  this->ExecuteAll(
-      num_pages, num_levels_per_page, batch_size, 0, Encoding::RLE_DICTIONARY);
-}
-
-TEST_F(TestBooleanFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0);
-}
-
-TEST_F(TestFLBAFlatScanner, TestPlainScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH);
-}
-
-TEST_F(TestFLBAFlatScanner, TestDictScanner) {
-  this->ExecuteAll(
-      num_pages, num_levels_per_page, batch_size, FLBA_LENGTH, Encoding::RLE_DICTIONARY);
-}
-
-TEST_F(TestFLBAFlatScanner, TestPlainDictScanner) {
-  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH,
-      Encoding::PLAIN_DICTIONARY);
-}
-
-// PARQUET 502
-TEST_F(TestFLBAFlatScanner, TestSmallBatch) {
-  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::REQUIRED,
-      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
-  const ColumnDescriptor d(type, 0, 0);
-  num_values_ = MakePages<FLBAType>(
-      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
-  num_levels_ = 1 * 100;
-  InitScanner(&d);
-  CheckResults(1, &d);
-}
-
-TEST_F(TestFLBAFlatScanner, TestDescriptorAPI) {
-  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::OPTIONAL,
-      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
-  const ColumnDescriptor d(type, 4, 0);
-  num_values_ = MakePages<FLBAType>(
-      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
-  num_levels_ = 1 * 100;
-  InitScanner(&d);
-  TypedScanner<FLBAType>* scanner =
-      reinterpret_cast<TypedScanner<FLBAType>*>(scanner_.get());
-  ASSERT_EQ(10, scanner->descr()->type_precision());
-  ASSERT_EQ(2, scanner->descr()->type_scale());
-  ASSERT_EQ(FLBA_LENGTH, scanner->descr()->type_length());
-}
-
-TEST_F(TestFLBAFlatScanner, TestFLBAPrinterNext) {
-  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::OPTIONAL,
-      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
-  const ColumnDescriptor d(type, 4, 0);
-  num_values_ = MakePages<FLBAType>(
-      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
-  num_levels_ = 1 * 100;
-  InitScanner(&d);
-  TypedScanner<FLBAType>* scanner =
-      reinterpret_cast<TypedScanner<FLBAType>*>(scanner_.get());
-  scanner->SetBatchSize(batch_size);
-  std::stringstream ss_fail;
-  for (int i = 0; i < num_levels_; i++) {
-    std::stringstream ss;
-    scanner->PrintNext(ss, 17);
-    std::string result = ss.str();
-    ASSERT_LE(17, result.size()) << i;
-  }
-  ASSERT_THROW(scanner->PrintNext(ss_fail, 17), ParquetException);
-}
-
-}  // namespace test
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/scanner.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner.cc b/src/parquet/column/scanner.cc
deleted file mode 100644
index 0295315..0000000
--- a/src/parquet/column/scanner.cc
+++ /dev/null
@@ -1,56 +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.
-
-#include "parquet/column/scanner.h"
-
-#include <cstdint>
-#include <memory>
-
-#include "parquet/column/reader.h"
-#include "parquet/util/memory.h"
-
-using arrow::MemoryPool;
-
-namespace parquet {
-
-std::shared_ptr<Scanner> Scanner::Make(
-    std::shared_ptr<ColumnReader> col_reader, int64_t batch_size, MemoryPool* pool) {
-  switch (col_reader->type()) {
-    case Type::BOOLEAN:
-      return std::make_shared<BoolScanner>(col_reader, batch_size, pool);
-    case Type::INT32:
-      return std::make_shared<Int32Scanner>(col_reader, batch_size, pool);
-    case Type::INT64:
-      return std::make_shared<Int64Scanner>(col_reader, batch_size, pool);
-    case Type::INT96:
-      return std::make_shared<Int96Scanner>(col_reader, batch_size, pool);
-    case Type::FLOAT:
-      return std::make_shared<FloatScanner>(col_reader, batch_size, pool);
-    case Type::DOUBLE:
-      return std::make_shared<DoubleScanner>(col_reader, batch_size, pool);
-    case Type::BYTE_ARRAY:
-      return std::make_shared<ByteArrayScanner>(col_reader, batch_size, pool);
-    case Type::FIXED_LEN_BYTE_ARRAY:
-      return std::make_shared<FixedLenByteArrayScanner>(col_reader, batch_size, pool);
-    default:
-      ParquetException::NYI("type reader not implemented");
-  }
-  // Unreachable code, but supress compiler warning
-  return std::shared_ptr<Scanner>(nullptr);
-}
-
-}  // namespace parquet


[4/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Posted by uw...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/scanner.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner.h b/src/parquet/column/scanner.h
deleted file mode 100644
index a9b83c1..0000000
--- a/src/parquet/column/scanner.h
+++ /dev/null
@@ -1,232 +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.
-
-#ifndef PARQUET_COLUMN_SCANNER_H
-#define PARQUET_COLUMN_SCANNER_H
-
-#include <cstdint>
-#include <memory>
-#include <ostream>
-#include <stdio.h>
-#include <string>
-#include <vector>
-
-#include "parquet/column/reader.h"
-#include "parquet/exception.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/visibility.h"
-
-namespace parquet {
-
-static constexpr int64_t DEFAULT_SCANNER_BATCH_SIZE = 128;
-
-class PARQUET_EXPORT Scanner {
- public:
-  explicit Scanner(std::shared_ptr<ColumnReader> reader,
-      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
-      : batch_size_(batch_size),
-        level_offset_(0),
-        levels_buffered_(0),
-        value_buffer_(std::make_shared<PoolBuffer>(pool)),
-        value_offset_(0),
-        values_buffered_(0),
-        reader_(reader) {
-    def_levels_.resize(descr()->max_definition_level() > 0 ? batch_size_ : 0);
-    rep_levels_.resize(descr()->max_repetition_level() > 0 ? batch_size_ : 0);
-  }
-
-  virtual ~Scanner() {}
-
-  static std::shared_ptr<Scanner> Make(std::shared_ptr<ColumnReader> col_reader,
-      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
-
-  virtual void PrintNext(std::ostream& out, int width) = 0;
-
-  bool HasNext() { return level_offset_ < levels_buffered_ || reader_->HasNext(); }
-
-  const ColumnDescriptor* descr() const { return reader_->descr(); }
-
-  int64_t batch_size() const { return batch_size_; }
-
-  void SetBatchSize(int64_t batch_size) { batch_size_ = batch_size; }
-
- protected:
-  int64_t batch_size_;
-
-  std::vector<int16_t> def_levels_;
-  std::vector<int16_t> rep_levels_;
-  int level_offset_;
-  int levels_buffered_;
-
-  std::shared_ptr<PoolBuffer> value_buffer_;
-  int value_offset_;
-  int64_t values_buffered_;
-
- private:
-  std::shared_ptr<ColumnReader> reader_;
-};
-
-template <typename DType>
-class PARQUET_EXPORT TypedScanner : public Scanner {
- public:
-  typedef typename DType::c_type T;
-
-  explicit TypedScanner(std::shared_ptr<ColumnReader> reader,
-      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
-      : Scanner(reader, batch_size, pool) {
-    typed_reader_ = static_cast<TypedColumnReader<DType>*>(reader.get());
-    int value_byte_size = type_traits<DType::type_num>::value_byte_size;
-    PARQUET_THROW_NOT_OK(value_buffer_->Resize(batch_size_ * value_byte_size));
-    values_ = reinterpret_cast<T*>(value_buffer_->mutable_data());
-  }
-
-  virtual ~TypedScanner() {}
-
-  bool NextLevels(int16_t* def_level, int16_t* rep_level) {
-    if (level_offset_ == levels_buffered_) {
-      levels_buffered_ =
-          static_cast<int>(typed_reader_->ReadBatch(static_cast<int>(batch_size_),
-              def_levels_.data(), rep_levels_.data(), values_, &values_buffered_));
-
-      value_offset_ = 0;
-      level_offset_ = 0;
-      if (!levels_buffered_) { return false; }
-    }
-    *def_level = descr()->max_definition_level() > 0 ? def_levels_[level_offset_] : 0;
-    *rep_level = descr()->max_repetition_level() > 0 ? rep_levels_[level_offset_] : 0;
-    level_offset_++;
-    return true;
-  }
-
-  bool Next(T* val, int16_t* def_level, int16_t* rep_level, bool* is_null) {
-    if (level_offset_ == levels_buffered_) {
-      if (!HasNext()) {
-        // Out of data pages
-        return false;
-      }
-    }
-
-    NextLevels(def_level, rep_level);
-    *is_null = *def_level < descr()->max_definition_level();
-
-    if (*is_null) { return true; }
-
-    if (value_offset_ == values_buffered_) {
-      throw ParquetException("Value was non-null, but has not been buffered");
-    }
-    *val = values_[value_offset_++];
-    return true;
-  }
-
-  // Returns true if there is a next value
-  bool NextValue(T* val, bool* is_null) {
-    if (level_offset_ == levels_buffered_) {
-      if (!HasNext()) {
-        // Out of data pages
-        return false;
-      }
-    }
-
-    // Out of values
-    int16_t def_level = -1;
-    int16_t rep_level = -1;
-    NextLevels(&def_level, &rep_level);
-    *is_null = def_level < descr()->max_definition_level();
-
-    if (*is_null) { return true; }
-
-    if (value_offset_ == values_buffered_) {
-      throw ParquetException("Value was non-null, but has not been buffered");
-    }
-    *val = values_[value_offset_++];
-    return true;
-  }
-
-  virtual void PrintNext(std::ostream& out, int width) {
-    T val;
-    bool is_null = false;
-    char buffer[25];
-
-    if (!NextValue(&val, &is_null)) { throw ParquetException("No more values buffered"); }
-
-    if (is_null) {
-      std::string null_fmt = format_fwf<ByteArrayType>(width);
-      snprintf(buffer, sizeof(buffer), null_fmt.c_str(), "NULL");
-    } else {
-      FormatValue(&val, buffer, sizeof(buffer), width);
-    }
-    out << buffer;
-  }
-
- private:
-  // The ownership of this object is expressed through the reader_ variable in the base
-  TypedColumnReader<DType>* typed_reader_;
-
-  inline void FormatValue(void* val, char* buffer, int bufsize, int width);
-
-  T* values_;
-};
-
-template <typename DType>
-inline void TypedScanner<DType>::FormatValue(
-    void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<DType>(width);
-  snprintf(buffer, bufsize, fmt.c_str(), *reinterpret_cast<T*>(val));
-}
-
-template <>
-inline void TypedScanner<Int96Type>::FormatValue(
-    void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<Int96Type>(width);
-  std::string result = Int96ToString(*reinterpret_cast<Int96*>(val));
-  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
-}
-
-template <>
-inline void TypedScanner<ByteArrayType>::FormatValue(
-    void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<ByteArrayType>(width);
-  std::string result = ByteArrayToString(*reinterpret_cast<ByteArray*>(val));
-  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
-}
-
-template <>
-inline void TypedScanner<FLBAType>::FormatValue(
-    void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<FLBAType>(width);
-  std::string result = FixedLenByteArrayToString(
-      *reinterpret_cast<FixedLenByteArray*>(val), descr()->type_length());
-  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
-}
-
-typedef TypedScanner<BooleanType> BoolScanner;
-typedef TypedScanner<Int32Type> Int32Scanner;
-typedef TypedScanner<Int64Type> Int64Scanner;
-typedef TypedScanner<Int96Type> Int96Scanner;
-typedef TypedScanner<FloatType> FloatScanner;
-typedef TypedScanner<DoubleType> DoubleScanner;
-typedef TypedScanner<ByteArrayType> ByteArrayScanner;
-typedef TypedScanner<FLBAType> FixedLenByteArrayScanner;
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_SCANNER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/statistics-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/statistics-test.cc b/src/parquet/column/statistics-test.cc
deleted file mode 100644
index e656f81..0000000
--- a/src/parquet/column/statistics-test.cc
+++ /dev/null
@@ -1,358 +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.
-
-#include <gtest/gtest.h>
-
-#include <algorithm>
-#include <array>
-#include <cstdint>
-#include <cstring>
-#include <memory>
-#include <vector>
-
-#include "parquet/column/reader.h"
-#include "parquet/column/statistics.h"
-#include "parquet/column/test-specialization.h"
-#include "parquet/column/test-util.h"
-#include "parquet/column/writer.h"
-#include "parquet/file/reader.h"
-#include "parquet/file/writer.h"
-#include "parquet/schema.h"
-#include "parquet/thrift.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-
-using arrow::default_memory_pool;
-using arrow::MemoryPool;
-
-namespace parquet {
-
-using schema::NodePtr;
-using schema::PrimitiveNode;
-using schema::GroupNode;
-
-namespace test {
-
-template <typename TestType>
-class TestRowGroupStatistics : public PrimitiveTypedTest<TestType> {
- public:
-  using T = typename TestType::c_type;
-  using TypedStats = TypedRowGroupStatistics<TestType>;
-
-  std::vector<T> GetDeepCopy(
-      const std::vector<T>&);  // allocates new memory for FLBA/ByteArray
-
-  T* GetValuesPointer(std::vector<T>&);
-  void DeepFree(std::vector<T>&);
-
-  void TestMinMaxEncode() {
-    this->GenerateData(1000);
-
-    TypedStats statistics1(this->schema_.Column(0));
-    statistics1.Update(this->values_ptr_, this->values_.size(), 0);
-    std::string encoded_min = statistics1.EncodeMin();
-    std::string encoded_max = statistics1.EncodeMax();
-
-    TypedStats statistics2(this->schema_.Column(0), encoded_min, encoded_max,
-        this->values_.size(), 0, 0, true);
-
-    TypedStats statistics3(this->schema_.Column(0));
-    std::vector<uint8_t> valid_bits(
-        BitUtil::RoundUpNumBytes(static_cast<uint32_t>(this->values_.size())) + 1, 255);
-    statistics3.UpdateSpaced(
-        this->values_ptr_, valid_bits.data(), 0, this->values_.size(), 0);
-    std::string encoded_min_spaced = statistics3.EncodeMin();
-    std::string encoded_max_spaced = statistics3.EncodeMax();
-
-    ASSERT_EQ(encoded_min, statistics2.EncodeMin());
-    ASSERT_EQ(encoded_max, statistics2.EncodeMax());
-    ASSERT_EQ(statistics1.min(), statistics2.min());
-    ASSERT_EQ(statistics1.max(), statistics2.max());
-    ASSERT_EQ(encoded_min_spaced, statistics2.EncodeMin());
-    ASSERT_EQ(encoded_max_spaced, statistics2.EncodeMax());
-    ASSERT_EQ(statistics3.min(), statistics2.min());
-    ASSERT_EQ(statistics3.max(), statistics2.max());
-  }
-
-  void TestReset() {
-    this->GenerateData(1000);
-
-    TypedStats statistics(this->schema_.Column(0));
-    statistics.Update(this->values_ptr_, this->values_.size(), 0);
-    ASSERT_EQ(this->values_.size(), statistics.num_values());
-
-    statistics.Reset();
-    ASSERT_EQ(0, statistics.null_count());
-    ASSERT_EQ(0, statistics.num_values());
-    ASSERT_EQ("", statistics.EncodeMin());
-    ASSERT_EQ("", statistics.EncodeMax());
-  }
-
-  void TestMerge() {
-    int num_null[2];
-    random_numbers(2, 42, 0, 100, num_null);
-
-    TypedStats statistics1(this->schema_.Column(0));
-    this->GenerateData(1000);
-    statistics1.Update(
-        this->values_ptr_, this->values_.size() - num_null[0], num_null[0]);
-
-    TypedStats statistics2(this->schema_.Column(0));
-    this->GenerateData(1000);
-    statistics2.Update(
-        this->values_ptr_, this->values_.size() - num_null[1], num_null[1]);
-
-    TypedStats total(this->schema_.Column(0));
-    total.Merge(statistics1);
-    total.Merge(statistics2);
-
-    ASSERT_EQ(num_null[0] + num_null[1], total.null_count());
-    ASSERT_EQ(this->values_.size() * 2 - num_null[0] - num_null[1], total.num_values());
-    ASSERT_EQ(total.min(), std::min(statistics1.min(), statistics2.min()));
-    ASSERT_EQ(total.max(), std::max(statistics1.max(), statistics2.max()));
-  }
-
-  void TestFullRoundtrip(int64_t num_values, int64_t null_count) {
-    this->GenerateData(num_values);
-
-    // compute statistics for the whole batch
-    TypedStats expected_stats(this->schema_.Column(0));
-    expected_stats.Update(this->values_ptr_, num_values - null_count, null_count);
-
-    auto sink = std::make_shared<InMemoryOutputStream>();
-    auto gnode = std::static_pointer_cast<GroupNode>(this->node_);
-    std::shared_ptr<WriterProperties> writer_properties =
-        WriterProperties::Builder().enable_statistics("column")->build();
-    auto file_writer = ParquetFileWriter::Open(sink, gnode, writer_properties);
-    auto row_group_writer = file_writer->AppendRowGroup(num_values);
-    auto column_writer =
-        static_cast<TypedColumnWriter<TestType>*>(row_group_writer->NextColumn());
-
-    // simulate the case when data comes from multiple buffers,
-    // in which case special care is necessary for FLBA/ByteArray types
-    for (int i = 0; i < 2; i++) {
-      int64_t batch_num_values = i ? num_values - num_values / 2 : num_values / 2;
-      int64_t batch_null_count = i ? null_count : 0;
-      DCHECK(null_count <= num_values);  // avoid too much headache
-      std::vector<int16_t> definition_levels(batch_null_count, 0);
-      definition_levels.insert(
-          definition_levels.end(), batch_num_values - batch_null_count, 1);
-      auto beg = this->values_.begin() + i * num_values / 2;
-      auto end = beg + batch_num_values;
-      std::vector<T> batch = GetDeepCopy(std::vector<T>(beg, end));
-      T* batch_values_ptr = GetValuesPointer(batch);
-      column_writer->WriteBatch(
-          batch_num_values, definition_levels.data(), nullptr, batch_values_ptr);
-      DeepFree(batch);
-    }
-    column_writer->Close();
-    row_group_writer->Close();
-    file_writer->Close();
-
-    auto buffer = sink->GetBuffer();
-    auto source = std::make_shared<::arrow::io::BufferReader>(buffer);
-    auto file_reader = ParquetFileReader::Open(source);
-    auto rg_reader = file_reader->RowGroup(0);
-    auto column_chunk = rg_reader->metadata()->ColumnChunk(0);
-    if (!column_chunk->is_stats_set()) return;
-    std::shared_ptr<RowGroupStatistics> stats = column_chunk->statistics();
-    // check values after serialization + deserialization
-    ASSERT_EQ(null_count, stats->null_count());
-    ASSERT_EQ(num_values - null_count, stats->num_values());
-    ASSERT_EQ(expected_stats.EncodeMin(), stats->EncodeMin());
-    ASSERT_EQ(expected_stats.EncodeMax(), stats->EncodeMax());
-  }
-};
-
-template <typename TestType>
-typename TestType::c_type* TestRowGroupStatistics<TestType>::GetValuesPointer(
-    std::vector<typename TestType::c_type>& values) {
-  return values.data();
-}
-
-template <>
-bool* TestRowGroupStatistics<BooleanType>::GetValuesPointer(std::vector<bool>& values) {
-  static std::vector<uint8_t> bool_buffer;
-  bool_buffer.clear();
-  bool_buffer.resize(values.size());
-  std::copy(values.begin(), values.end(), bool_buffer.begin());
-  return reinterpret_cast<bool*>(bool_buffer.data());
-}
-
-template <typename TestType>
-typename std::vector<typename TestType::c_type>
-TestRowGroupStatistics<TestType>::GetDeepCopy(
-    const std::vector<typename TestType::c_type>& values) {
-  return values;
-}
-
-template <>
-std::vector<FLBA> TestRowGroupStatistics<FLBAType>::GetDeepCopy(
-    const std::vector<FLBA>& values) {
-  std::vector<FLBA> copy;
-  MemoryPool* pool = ::arrow::default_memory_pool();
-  for (const FLBA& flba : values) {
-    uint8_t* ptr;
-    PARQUET_THROW_NOT_OK(pool->Allocate(FLBA_LENGTH, &ptr));
-    memcpy(ptr, flba.ptr, FLBA_LENGTH);
-    copy.emplace_back(ptr);
-  }
-  return copy;
-}
-
-template <>
-std::vector<ByteArray> TestRowGroupStatistics<ByteArrayType>::GetDeepCopy(
-    const std::vector<ByteArray>& values) {
-  std::vector<ByteArray> copy;
-  MemoryPool* pool = default_memory_pool();
-  for (const ByteArray& ba : values) {
-    uint8_t* ptr;
-    PARQUET_THROW_NOT_OK(pool->Allocate(ba.len, &ptr));
-    memcpy(ptr, ba.ptr, ba.len);
-    copy.emplace_back(ba.len, ptr);
-  }
-  return copy;
-}
-
-template <typename TestType>
-void TestRowGroupStatistics<TestType>::DeepFree(
-    std::vector<typename TestType::c_type>& values) {}
-
-template <>
-void TestRowGroupStatistics<FLBAType>::DeepFree(std::vector<FLBA>& values) {
-  MemoryPool* pool = default_memory_pool();
-  for (FLBA& flba : values) {
-    auto ptr = const_cast<uint8_t*>(flba.ptr);
-    memset(ptr, 0, FLBA_LENGTH);
-    pool->Free(ptr, FLBA_LENGTH);
-  }
-}
-
-template <>
-void TestRowGroupStatistics<ByteArrayType>::DeepFree(std::vector<ByteArray>& values) {
-  MemoryPool* pool = default_memory_pool();
-  for (ByteArray& ba : values) {
-    auto ptr = const_cast<uint8_t*>(ba.ptr);
-    memset(ptr, 0, ba.len);
-    pool->Free(ptr, ba.len);
-  }
-}
-
-template <>
-void TestRowGroupStatistics<ByteArrayType>::TestMinMaxEncode() {
-  this->GenerateData(1000);
-  // Test that we encode min max strings correctly
-  TypedRowGroupStatistics<ByteArrayType> statistics1(this->schema_.Column(0));
-  statistics1.Update(this->values_ptr_, this->values_.size(), 0);
-  std::string encoded_min = statistics1.EncodeMin();
-  std::string encoded_max = statistics1.EncodeMax();
-
-  // encoded is same as unencoded
-  ASSERT_EQ(encoded_min,
-      std::string((const char*)statistics1.min().ptr, statistics1.min().len));
-  ASSERT_EQ(encoded_max,
-      std::string((const char*)statistics1.max().ptr, statistics1.max().len));
-
-  TypedRowGroupStatistics<ByteArrayType> statistics2(this->schema_.Column(0), encoded_min,
-      encoded_max, this->values_.size(), 0, 0, true);
-
-  ASSERT_EQ(encoded_min, statistics2.EncodeMin());
-  ASSERT_EQ(encoded_max, statistics2.EncodeMax());
-  ASSERT_EQ(statistics1.min(), statistics2.min());
-  ASSERT_EQ(statistics1.max(), statistics2.max());
-}
-
-using TestTypes = ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
-    ByteArrayType, FLBAType, BooleanType>;
-
-TYPED_TEST_CASE(TestRowGroupStatistics, TestTypes);
-
-TYPED_TEST(TestRowGroupStatistics, MinMaxEncode) {
-  this->SetUpSchema(Repetition::REQUIRED);
-  this->TestMinMaxEncode();
-}
-
-TYPED_TEST(TestRowGroupStatistics, Reset) {
-  this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestReset();
-}
-
-TYPED_TEST(TestRowGroupStatistics, FullRoundtrip) {
-  this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestFullRoundtrip(100, 31);
-  this->TestFullRoundtrip(1000, 415);
-  this->TestFullRoundtrip(10000, 926);
-}
-
-template <typename TestType>
-class TestNumericRowGroupStatistics : public TestRowGroupStatistics<TestType> {};
-
-using NumericTypes = ::testing::Types<Int32Type, Int64Type, FloatType, DoubleType>;
-
-TYPED_TEST_CASE(TestNumericRowGroupStatistics, NumericTypes);
-
-TYPED_TEST(TestNumericRowGroupStatistics, Merge) {
-  this->SetUpSchema(Repetition::OPTIONAL);
-  this->TestMerge();
-}
-
-TEST(CorruptStatistics, Basics) {
-  ApplicationVersion version("parquet-mr version 1.8.0");
-  SchemaDescriptor schema;
-  schema::NodePtr node;
-  std::vector<schema::NodePtr> fields;
-  // Test Physical Types
-  fields.push_back(schema::PrimitiveNode::Make(
-      "col1", Repetition::OPTIONAL, Type::INT32, LogicalType::NONE));
-  fields.push_back(schema::PrimitiveNode::Make(
-      "col2", Repetition::OPTIONAL, Type::BYTE_ARRAY, LogicalType::NONE));
-  // Test Logical Types
-  fields.push_back(schema::PrimitiveNode::Make(
-      "col3", Repetition::OPTIONAL, Type::INT32, LogicalType::DATE));
-  fields.push_back(schema::PrimitiveNode::Make(
-      "col4", Repetition::OPTIONAL, Type::INT32, LogicalType::UINT_32));
-  fields.push_back(schema::PrimitiveNode::Make("col5", Repetition::OPTIONAL,
-      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::INTERVAL, 12));
-  fields.push_back(schema::PrimitiveNode::Make(
-      "col6", Repetition::OPTIONAL, Type::BYTE_ARRAY, LogicalType::UTF8));
-  node = schema::GroupNode::Make("schema", Repetition::REQUIRED, fields);
-  schema.Init(node);
-
-  format::ColumnChunk col_chunk;
-  col_chunk.meta_data.__isset.statistics = true;
-  auto column_chunk1 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(0), &version);
-  ASSERT_TRUE(column_chunk1->is_stats_set());
-  auto column_chunk2 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(1), &version);
-  ASSERT_FALSE(column_chunk2->is_stats_set());
-  auto column_chunk3 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(2), &version);
-  ASSERT_TRUE(column_chunk3->is_stats_set());
-  auto column_chunk4 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(3), &version);
-  ASSERT_FALSE(column_chunk4->is_stats_set());
-  auto column_chunk5 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(4), &version);
-  ASSERT_FALSE(column_chunk5->is_stats_set());
-  auto column_chunk6 = ColumnChunkMetaData::Make(
-      reinterpret_cast<const uint8_t*>(&col_chunk), schema.Column(5), &version);
-  ASSERT_FALSE(column_chunk6->is_stats_set());
-}
-
-}  // namespace test
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/statistics.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/statistics.cc b/src/parquet/column/statistics.cc
deleted file mode 100644
index 961a2af..0000000
--- a/src/parquet/column/statistics.cc
+++ /dev/null
@@ -1,244 +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.
-
-#include <algorithm>
-#include <cstring>
-
-#include "parquet/column/statistics.h"
-#include "parquet/encoding-internal.h"
-#include "parquet/exception.h"
-#include "parquet/util/comparison.h"
-#include "parquet/util/memory.h"
-
-using arrow::default_memory_pool;
-using arrow::MemoryPool;
-
-namespace parquet {
-
-template <typename DType>
-TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(
-    const ColumnDescriptor* schema, MemoryPool* pool)
-    : pool_(pool),
-      min_buffer_(AllocateBuffer(pool_, 0)),
-      max_buffer_(AllocateBuffer(pool_, 0)) {
-  SetDescr(schema);
-  Reset();
-}
-
-template <typename DType>
-TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(const typename DType::c_type& min,
-    const typename DType::c_type& max, int64_t num_values, int64_t null_count,
-    int64_t distinct_count)
-    : pool_(default_memory_pool()),
-      min_buffer_(AllocateBuffer(pool_, 0)),
-      max_buffer_(AllocateBuffer(pool_, 0)) {
-  IncrementNumValues(num_values);
-  IncrementNullCount(null_count);
-  IncrementDistinctCount(distinct_count);
-
-  Copy(min, &min_, min_buffer_.get());
-  Copy(max, &max_, max_buffer_.get());
-  has_min_max_ = true;
-}
-
-template <typename DType>
-TypedRowGroupStatistics<DType>::TypedRowGroupStatistics(const ColumnDescriptor* schema,
-    const std::string& encoded_min, const std::string& encoded_max, int64_t num_values,
-    int64_t null_count, int64_t distinct_count, bool has_min_max, MemoryPool* pool)
-    : pool_(pool),
-      min_buffer_(AllocateBuffer(pool_, 0)),
-      max_buffer_(AllocateBuffer(pool_, 0)) {
-  IncrementNumValues(num_values);
-  IncrementNullCount(null_count);
-  IncrementDistinctCount(distinct_count);
-
-  SetDescr(schema);
-
-  if (!encoded_min.empty()) { PlainDecode(encoded_min, &min_); }
-  if (!encoded_max.empty()) { PlainDecode(encoded_max, &max_); }
-  has_min_max_ = has_min_max;
-}
-
-template <typename DType>
-bool TypedRowGroupStatistics<DType>::HasMinMax() const {
-  return has_min_max_;
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::Reset() {
-  ResetCounts();
-  has_min_max_ = false;
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::Update(
-    const T* values, int64_t num_not_null, int64_t num_null) {
-  DCHECK(num_not_null >= 0);
-  DCHECK(num_null >= 0);
-
-  IncrementNullCount(num_null);
-  IncrementNumValues(num_not_null);
-  // TODO: support distinct count?
-  if (num_not_null == 0) return;
-
-  Compare<T> compare(descr_);
-  auto batch_minmax = std::minmax_element(values, values + num_not_null, compare);
-  if (!has_min_max_) {
-    has_min_max_ = true;
-    Copy(*batch_minmax.first, &min_, min_buffer_.get());
-    Copy(*batch_minmax.second, &max_, max_buffer_.get());
-  } else {
-    Copy(std::min(min_, *batch_minmax.first, compare), &min_, min_buffer_.get());
-    Copy(std::max(max_, *batch_minmax.second, compare), &max_, max_buffer_.get());
-  }
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::UpdateSpaced(const T* values,
-    const uint8_t* valid_bits, int64_t valid_bits_offset, int64_t num_not_null,
-    int64_t num_null) {
-  DCHECK(num_not_null >= 0);
-  DCHECK(num_null >= 0);
-
-  IncrementNullCount(num_null);
-  IncrementNumValues(num_not_null);
-  // TODO: support distinct count?
-  if (num_not_null == 0) return;
-
-  Compare<T> compare(descr_);
-  INIT_BITSET(valid_bits, static_cast<int>(valid_bits_offset));
-  // Find first valid entry and use that for min/max
-  // As (num_not_null != 0) there must be one
-  int64_t length = num_null + num_not_null;
-  int64_t i = 0;
-  for (; i < length; i++) {
-    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) { break; }
-    READ_NEXT_BITSET(valid_bits);
-  }
-  T min = values[i];
-  T max = values[i];
-  for (; i < length; i++) {
-    if (bitset_valid_bits & (1 << bit_offset_valid_bits)) {
-      if (compare(values[i], min)) {
-        min = values[i];
-      } else if (compare(max, values[i])) {
-        max = values[i];
-      }
-    }
-    READ_NEXT_BITSET(valid_bits);
-  }
-  if (!has_min_max_) {
-    has_min_max_ = true;
-    Copy(min, &min_, min_buffer_.get());
-    Copy(max, &max_, max_buffer_.get());
-  } else {
-    Copy(std::min(min_, min, compare), &min_, min_buffer_.get());
-    Copy(std::max(max_, max, compare), &max_, max_buffer_.get());
-  }
-}
-
-template <typename DType>
-const typename DType::c_type& TypedRowGroupStatistics<DType>::min() const {
-  return min_;
-}
-
-template <typename DType>
-const typename DType::c_type& TypedRowGroupStatistics<DType>::max() const {
-  return max_;
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::Merge(const TypedRowGroupStatistics<DType>& other) {
-  this->MergeCounts(other);
-
-  if (!other.HasMinMax()) return;
-
-  if (!has_min_max_) {
-    Copy(other.min_, &this->min_, min_buffer_.get());
-    Copy(other.max_, &this->max_, max_buffer_.get());
-    has_min_max_ = true;
-    return;
-  }
-
-  Compare<T> compare(descr_);
-  Copy(std::min(this->min_, other.min_, compare), &this->min_, min_buffer_.get());
-  Copy(std::max(this->max_, other.max_, compare), &this->max_, max_buffer_.get());
-}
-
-template <typename DType>
-std::string TypedRowGroupStatistics<DType>::EncodeMin() {
-  std::string s;
-  if (HasMinMax()) this->PlainEncode(min_, &s);
-  return s;
-}
-
-template <typename DType>
-std::string TypedRowGroupStatistics<DType>::EncodeMax() {
-  std::string s;
-  if (HasMinMax()) this->PlainEncode(max_, &s);
-  return s;
-}
-
-template <typename DType>
-EncodedStatistics TypedRowGroupStatistics<DType>::Encode() {
-  EncodedStatistics s;
-  if (HasMinMax()) {
-    s.set_min(this->EncodeMin());
-    s.set_max(this->EncodeMax());
-  }
-  s.set_null_count(this->null_count());
-  return s;
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::PlainEncode(const T& src, std::string* dst) {
-  PlainEncoder<DType> encoder(descr(), pool_);
-  encoder.Put(&src, 1);
-  auto buffer = encoder.FlushValues();
-  auto ptr = reinterpret_cast<const char*>(buffer->data());
-  dst->assign(ptr, buffer->size());
-}
-
-template <typename DType>
-void TypedRowGroupStatistics<DType>::PlainDecode(const std::string& src, T* dst) {
-  PlainDecoder<DType> decoder(descr());
-  decoder.SetData(
-      1, reinterpret_cast<const uint8_t*>(src.c_str()), static_cast<int>(src.size()));
-  decoder.Decode(dst, 1);
-}
-
-template <>
-void TypedRowGroupStatistics<ByteArrayType>::PlainEncode(const T& src, std::string* dst) {
-  dst->assign(reinterpret_cast<const char*>(src.ptr), src.len);
-}
-
-template <>
-void TypedRowGroupStatistics<ByteArrayType>::PlainDecode(const std::string& src, T* dst) {
-  dst->len = static_cast<uint32_t>(src.size());
-  dst->ptr = reinterpret_cast<const uint8_t*>(src.c_str());
-}
-
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<BooleanType>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int32Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int64Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<Int96Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<FloatType>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<DoubleType>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<ByteArrayType>;
-template class PARQUET_TEMPLATE_EXPORT TypedRowGroupStatistics<FLBAType>;
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/statistics.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/statistics.h b/src/parquet/column/statistics.h
deleted file mode 100644
index c6a2487..0000000
--- a/src/parquet/column/statistics.h
+++ /dev/null
@@ -1,234 +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.
-
-#ifndef PARQUET_COLUMN_STATISTICS_H
-#define PARQUET_COLUMN_STATISTICS_H
-
-#include <cstdint>
-#include <memory>
-#include <string>
-
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/visibility.h"
-
-namespace parquet {
-
-class PARQUET_EXPORT EncodedStatistics {
-  std::shared_ptr<std::string> max_, min_;
-
- public:
-  EncodedStatistics()
-      : max_(std::make_shared<std::string>()), min_(std::make_shared<std::string>()) {}
-
-  const std::string& max() const { return *max_; }
-  const std::string& min() const { return *min_; }
-
-  int64_t null_count = 0;
-  int64_t distinct_count = 0;
-
-  bool has_min = false;
-  bool has_max = false;
-  bool has_null_count = false;
-  bool has_distinct_count = false;
-
-  inline bool is_set() const {
-    return has_min || has_max || has_null_count || has_distinct_count;
-  }
-
-  inline EncodedStatistics& set_max(const std::string& value) {
-    *max_ = value;
-    has_max = true;
-    return *this;
-  }
-
-  inline EncodedStatistics& set_min(const std::string& value) {
-    *min_ = value;
-    has_min = true;
-    return *this;
-  }
-
-  inline EncodedStatistics& set_null_count(int64_t value) {
-    null_count = value;
-    has_null_count = true;
-    return *this;
-  }
-
-  inline EncodedStatistics& set_distinct_count(int64_t value) {
-    distinct_count = value;
-    has_distinct_count = true;
-    return *this;
-  }
-};
-
-template <typename DType>
-class PARQUET_EXPORT TypedRowGroupStatistics;
-
-class PARQUET_EXPORT RowGroupStatistics
-    : public std::enable_shared_from_this<RowGroupStatistics> {
- public:
-  int64_t null_count() const { return statistics_.null_count; }
-  int64_t distinct_count() const { return statistics_.distinct_count; }
-  int64_t num_values() const { return num_values_; }
-
-  virtual bool HasMinMax() const = 0;
-  virtual void Reset() = 0;
-
-  // Plain-encoded minimum value
-  virtual std::string EncodeMin() = 0;
-
-  // Plain-encoded maximum value
-  virtual std::string EncodeMax() = 0;
-
-  virtual EncodedStatistics Encode() = 0;
-
-  virtual ~RowGroupStatistics() {}
-
-  Type::type physical_type() const { return descr_->physical_type(); }
-
- protected:
-  const ColumnDescriptor* descr() const { return descr_; }
-  void SetDescr(const ColumnDescriptor* schema) { descr_ = schema; }
-
-  void IncrementNullCount(int64_t n) { statistics_.null_count += n; }
-
-  void IncrementNumValues(int64_t n) { num_values_ += n; }
-
-  void IncrementDistinctCount(int64_t n) { statistics_.distinct_count += n; }
-
-  void MergeCounts(const RowGroupStatistics& other) {
-    this->statistics_.null_count += other.statistics_.null_count;
-    this->statistics_.distinct_count += other.statistics_.distinct_count;
-    this->num_values_ += other.num_values_;
-  }
-
-  void ResetCounts() {
-    this->statistics_.null_count = 0;
-    this->statistics_.distinct_count = 0;
-    this->num_values_ = 0;
-  }
-
-  const ColumnDescriptor* descr_ = nullptr;
-  int64_t num_values_ = 0;
-  EncodedStatistics statistics_;
-};
-
-template <typename DType>
-class TypedRowGroupStatistics : public RowGroupStatistics {
- public:
-  using T = typename DType::c_type;
-
-  TypedRowGroupStatistics(const ColumnDescriptor* schema,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
-
-  TypedRowGroupStatistics(const T& min, const T& max, int64_t num_values,
-      int64_t null_count, int64_t distinct_count);
-
-  TypedRowGroupStatistics(const ColumnDescriptor* schema, const std::string& encoded_min,
-      const std::string& encoded_max, int64_t num_values, int64_t null_count,
-      int64_t distinct_count, bool has_min_max,
-      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
-
-  bool HasMinMax() const override;
-  void Reset() override;
-  void Merge(const TypedRowGroupStatistics<DType>& other);
-
-  void Update(const T* values, int64_t num_not_null, int64_t num_null);
-  void UpdateSpaced(const T* values, const uint8_t* valid_bits, int64_t valid_bits_spaced,
-      int64_t num_not_null, int64_t num_null);
-
-  const T& min() const;
-  const T& max() const;
-
-  std::string EncodeMin() override;
-  std::string EncodeMax() override;
-  EncodedStatistics Encode() override;
-
- private:
-  bool has_min_max_ = false;
-  T min_;
-  T max_;
-  ::arrow::MemoryPool* pool_;
-
-  void PlainEncode(const T& src, std::string* dst);
-  void PlainDecode(const std::string& src, T* dst);
-  void Copy(const T& src, T* dst, PoolBuffer* buffer);
-
-  std::shared_ptr<PoolBuffer> min_buffer_, max_buffer_;
-};
-
-template <typename DType>
-inline void TypedRowGroupStatistics<DType>::Copy(const T& src, T* dst, PoolBuffer*) {
-  *dst = src;
-}
-
-template <>
-inline void TypedRowGroupStatistics<FLBAType>::Copy(
-    const FLBA& src, FLBA* dst, PoolBuffer* buffer) {
-  if (dst->ptr == src.ptr) return;
-  uint32_t len = descr_->type_length();
-  PARQUET_THROW_NOT_OK(buffer->Resize(len, false));
-  std::memcpy(buffer->mutable_data(), src.ptr, len);
-  *dst = FLBA(buffer->data());
-}
-
-template <>
-inline void TypedRowGroupStatistics<ByteArrayType>::Copy(
-    const ByteArray& src, ByteArray* dst, PoolBuffer* buffer) {
-  if (dst->ptr == src.ptr) return;
-  PARQUET_THROW_NOT_OK(buffer->Resize(src.len, false));
-  std::memcpy(buffer->mutable_data(), src.ptr, src.len);
-  *dst = ByteArray(src.len, buffer->data());
-}
-
-template <>
-void TypedRowGroupStatistics<ByteArrayType>::PlainEncode(const T& src, std::string* dst);
-
-template <>
-void TypedRowGroupStatistics<ByteArrayType>::PlainDecode(const std::string& src, T* dst);
-
-typedef TypedRowGroupStatistics<BooleanType> BoolStatistics;
-typedef TypedRowGroupStatistics<Int32Type> Int32Statistics;
-typedef TypedRowGroupStatistics<Int64Type> Int64Statistics;
-typedef TypedRowGroupStatistics<Int96Type> Int96Statistics;
-typedef TypedRowGroupStatistics<FloatType> FloatStatistics;
-typedef TypedRowGroupStatistics<DoubleType> DoubleStatistics;
-typedef TypedRowGroupStatistics<ByteArrayType> ByteArrayStatistics;
-typedef TypedRowGroupStatistics<FLBAType> FLBAStatistics;
-
-#if defined(__GNUC__) && !defined(__clang__)
-#pragma GCC diagnostic push
-#pragma GCC diagnostic ignored "-Wattributes"
-#endif
-
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<BooleanType>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int32Type>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int64Type>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<Int96Type>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<FloatType>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<DoubleType>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<ByteArrayType>;
-PARQUET_EXTERN_TEMPLATE TypedRowGroupStatistics<FLBAType>;
-
-#if defined(__GNUC__) && !defined(__clang__)
-#pragma GCC diagnostic pop
-#endif
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_STATISTICS_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/test-specialization.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/test-specialization.h b/src/parquet/column/test-specialization.h
deleted file mode 100644
index 07767c0..0000000
--- a/src/parquet/column/test-specialization.h
+++ /dev/null
@@ -1,172 +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.
-
-// This module defines an abstract interface for iterating through pages in a
-// Parquet column chunk within a row group. It could be extended in the future
-// to iterate through all data pages in all chunks in a file.
-
-#ifndef PARQUET_COLUMN_TEST_SPECIALIZATION_H
-#define PARQUET_COLUMN_TEST_SPECIALIZATION_H
-
-#include <algorithm>
-#include <limits>
-#include <sstream>
-#include <string>
-#include <vector>
-
-#include "parquet/column/test-util.h"
-
-namespace parquet {
-
-namespace test {
-
-template <>
-void InitValues<bool>(int num_values, vector<bool>& values, vector<uint8_t>& buffer) {
-  values = flip_coins(num_values, 0);
-}
-
-template <>
-void InitValues<ByteArray>(
-    int num_values, vector<ByteArray>& values, vector<uint8_t>& buffer) {
-  int max_byte_array_len = 12;
-  int num_bytes = max_byte_array_len + sizeof(uint32_t);
-  size_t nbytes = num_values * num_bytes;
-  buffer.resize(nbytes);
-  random_byte_array(num_values, 0, buffer.data(), values.data(), max_byte_array_len);
-}
-
-template <>
-void InitValues<FLBA>(int num_values, vector<FLBA>& values, vector<uint8_t>& buffer) {
-  size_t nbytes = num_values * FLBA_LENGTH;
-  buffer.resize(nbytes);
-  random_fixed_byte_array(num_values, 0, buffer.data(), FLBA_LENGTH, values.data());
-}
-
-template <>
-void InitValues<Int96>(int num_values, vector<Int96>& values, vector<uint8_t>& buffer) {
-  random_Int96_numbers(num_values, 0, std::numeric_limits<int32_t>::min(),
-      std::numeric_limits<int32_t>::max(), values.data());
-}
-
-inline std::string TestColumnName(int i) {
-  std::stringstream col_name;
-  col_name << "column_" << i;
-  return col_name.str();
-}
-
-// This class lives here because of its dependency on the InitValues specializations.
-template <typename TestType>
-class PrimitiveTypedTest : public ::testing::Test {
- public:
-  typedef typename TestType::c_type T;
-
-  void SetUpSchema(Repetition::type repetition, int num_columns = 1) {
-    std::vector<schema::NodePtr> fields;
-
-    for (int i = 0; i < num_columns; ++i) {
-      std::string name = TestColumnName(i);
-      fields.push_back(schema::PrimitiveNode::Make(
-          name, repetition, TestType::type_num, LogicalType::NONE, FLBA_LENGTH));
-    }
-    node_ = schema::GroupNode::Make("schema", Repetition::REQUIRED, fields);
-    schema_.Init(node_);
-  }
-
-  void GenerateData(int64_t num_values);
-  void SetupValuesOut(int64_t num_values);
-  void SyncValuesOut();
-
- protected:
-  schema::NodePtr node_;
-  SchemaDescriptor schema_;
-
-  // Input buffers
-  std::vector<T> values_;
-
-  std::vector<int16_t> def_levels_;
-
-  std::vector<uint8_t> buffer_;
-  // Pointer to the values, needed as we cannot use vector<bool>::data()
-  T* values_ptr_;
-  std::vector<uint8_t> bool_buffer_;
-
-  // Output buffers
-  std::vector<T> values_out_;
-  std::vector<uint8_t> bool_buffer_out_;
-  T* values_out_ptr_;
-};
-
-template <typename TestType>
-void PrimitiveTypedTest<TestType>::SyncValuesOut() {}
-
-template <>
-void PrimitiveTypedTest<BooleanType>::SyncValuesOut() {
-    std::vector<uint8_t>::const_iterator source_iterator = bool_buffer_out_.begin();
-    std::vector<T>::iterator destination_iterator = values_out_.begin();
-    while (source_iterator != bool_buffer_out_.end()) {
-        *destination_iterator++ = *source_iterator++ != 0;
-    }
-}
-
-template <typename TestType>
-void PrimitiveTypedTest<TestType>::SetupValuesOut(int64_t num_values) {
-  values_out_.clear();
-  values_out_.resize(num_values);
-  values_out_ptr_ = values_out_.data();
-}
-
-template <>
-void PrimitiveTypedTest<BooleanType>::SetupValuesOut(int64_t num_values) {
-  values_out_.clear();
-  values_out_.resize(num_values);
-
-  bool_buffer_out_.clear();
-  bool_buffer_out_.resize(num_values);
-  // Write once to all values so we can copy it without getting Valgrind errors
-  // about uninitialised values.
-  std::fill(bool_buffer_out_.begin(), bool_buffer_out_.end(), true);
-  values_out_ptr_ = reinterpret_cast<bool*>(bool_buffer_out_.data());
-}
-
-template <typename TestType>
-void PrimitiveTypedTest<TestType>::GenerateData(int64_t num_values) {
-  def_levels_.resize(num_values);
-  values_.resize(num_values);
-
-  InitValues<T>(static_cast<int>(num_values), values_, buffer_);
-  values_ptr_ = values_.data();
-
-  std::fill(def_levels_.begin(), def_levels_.end(), 1);
-}
-
-template <>
-void PrimitiveTypedTest<BooleanType>::GenerateData(int64_t num_values) {
-  def_levels_.resize(num_values);
-  values_.resize(num_values);
-
-  InitValues<T>(static_cast<int>(num_values), values_, buffer_);
-  bool_buffer_.resize(num_values);
-  std::copy(values_.begin(), values_.end(), bool_buffer_.begin());
-  values_ptr_ = reinterpret_cast<bool*>(bool_buffer_.data());
-
-  std::fill(def_levels_.begin(), def_levels_.end(), 1);
-}
-}  // namespace test
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_TEST_SPECIALIZATION_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/test-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/test-util.h b/src/parquet/column/test-util.h
deleted file mode 100644
index c133734..0000000
--- a/src/parquet/column/test-util.h
+++ /dev/null
@@ -1,429 +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.
-
-// This module defines an abstract interface for iterating through pages in a
-// Parquet column chunk within a row group. It could be extended in the future
-// to iterate through all data pages in all chunks in a file.
-
-#ifndef PARQUET_COLUMN_TEST_UTIL_H
-#define PARQUET_COLUMN_TEST_UTIL_H
-
-#include <algorithm>
-#include <limits>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include <gtest/gtest.h>
-
-#include "parquet/column/levels.h"
-#include "parquet/column/page.h"
-#include "parquet/encoding-internal.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/test-common.h"
-
-using std::vector;
-using std::shared_ptr;
-
-namespace parquet {
-
-static int FLBA_LENGTH = 12;
-
-bool operator==(const FixedLenByteArray& a, const FixedLenByteArray& b) {
-  return 0 == memcmp(a.ptr, b.ptr, FLBA_LENGTH);
-}
-
-namespace test {
-
-template <typename T>
-static void InitValues(int num_values, vector<T>& values, vector<uint8_t>& buffer) {
-  random_numbers(num_values, 0, std::numeric_limits<T>::min(),
-      std::numeric_limits<T>::max(), values.data());
-}
-
-template <typename T>
-static void InitDictValues(
-    int num_values, int num_dicts, vector<T>& values, vector<uint8_t>& buffer) {
-  int repeat_factor = num_values / num_dicts;
-  InitValues<T>(num_dicts, values, buffer);
-  // add some repeated values
-  for (int j = 1; j < repeat_factor; ++j) {
-    for (int i = 0; i < num_dicts; ++i) {
-      std::memcpy(&values[num_dicts * j + i], &values[i], sizeof(T));
-    }
-  }
-  // computed only dict_per_page * repeat_factor - 1 values < num_values
-  // compute remaining
-  for (int i = num_dicts * repeat_factor; i < num_values; ++i) {
-    std::memcpy(&values[i], &values[i - num_dicts * repeat_factor], sizeof(T));
-  }
-}
-
-class MockPageReader : public PageReader {
- public:
-  explicit MockPageReader(const vector<shared_ptr<Page>>& pages)
-      : pages_(pages), page_index_(0) {}
-
-  // Implement the PageReader interface
-  virtual shared_ptr<Page> NextPage() {
-    if (page_index_ == static_cast<int>(pages_.size())) {
-      // EOS to consumer
-      return shared_ptr<Page>(nullptr);
-    }
-    return pages_[page_index_++];
-  }
-
- private:
-  vector<shared_ptr<Page>> pages_;
-  int page_index_;
-};
-
-// TODO(wesm): this is only used for testing for now. Refactor to form part of
-// primary file write path
-template <typename Type>
-class DataPageBuilder {
- public:
-  typedef typename Type::c_type T;
-
-  // This class writes data and metadata to the passed inputs
-  explicit DataPageBuilder(InMemoryOutputStream* sink)
-      : sink_(sink),
-        num_values_(0),
-        encoding_(Encoding::PLAIN),
-        definition_level_encoding_(Encoding::RLE),
-        repetition_level_encoding_(Encoding::RLE),
-        have_def_levels_(false),
-        have_rep_levels_(false),
-        have_values_(false) {}
-
-  void AppendDefLevels(const vector<int16_t>& levels, int16_t max_level,
-      Encoding::type encoding = Encoding::RLE) {
-    AppendLevels(levels, max_level, encoding);
-
-    num_values_ = std::max(static_cast<int32_t>(levels.size()), num_values_);
-    definition_level_encoding_ = encoding;
-    have_def_levels_ = true;
-  }
-
-  void AppendRepLevels(const vector<int16_t>& levels, int16_t max_level,
-      Encoding::type encoding = Encoding::RLE) {
-    AppendLevels(levels, max_level, encoding);
-
-    num_values_ = std::max(static_cast<int32_t>(levels.size()), num_values_);
-    repetition_level_encoding_ = encoding;
-    have_rep_levels_ = true;
-  }
-
-  void AppendValues(const ColumnDescriptor* d, const vector<T>& values,
-      Encoding::type encoding = Encoding::PLAIN) {
-    PlainEncoder<Type> encoder(d);
-    encoder.Put(&values[0], static_cast<int>(values.size()));
-    std::shared_ptr<Buffer> values_sink = encoder.FlushValues();
-    sink_->Write(values_sink->data(), values_sink->size());
-
-    num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);
-    encoding_ = encoding;
-    have_values_ = true;
-  }
-
-  int32_t num_values() const { return num_values_; }
-
-  Encoding::type encoding() const { return encoding_; }
-
-  Encoding::type rep_level_encoding() const { return repetition_level_encoding_; }
-
-  Encoding::type def_level_encoding() const { return definition_level_encoding_; }
-
- private:
-  InMemoryOutputStream* sink_;
-
-  int32_t num_values_;
-  Encoding::type encoding_;
-  Encoding::type definition_level_encoding_;
-  Encoding::type repetition_level_encoding_;
-
-  bool have_def_levels_;
-  bool have_rep_levels_;
-  bool have_values_;
-
-  // Used internally for both repetition and definition levels
-  void AppendLevels(
-      const vector<int16_t>& levels, int16_t max_level, Encoding::type encoding) {
-    if (encoding != Encoding::RLE) {
-      ParquetException::NYI("only rle encoding currently implemented");
-    }
-
-    // TODO: compute a more precise maximum size for the encoded levels
-    vector<uint8_t> encode_buffer(levels.size() * 2);
-
-    // We encode into separate memory from the output stream because the
-    // RLE-encoded bytes have to be preceded in the stream by their absolute
-    // size.
-    LevelEncoder encoder;
-    encoder.Init(encoding, max_level, static_cast<int>(levels.size()),
-        encode_buffer.data(), static_cast<int>(encode_buffer.size()));
-
-    encoder.Encode(static_cast<int>(levels.size()), levels.data());
-
-    int32_t rle_bytes = encoder.len();
-    sink_->Write(reinterpret_cast<const uint8_t*>(&rle_bytes), sizeof(int32_t));
-    sink_->Write(encode_buffer.data(), rle_bytes);
-  }
-};
-
-template <>
-void DataPageBuilder<BooleanType>::AppendValues(
-    const ColumnDescriptor* d, const vector<bool>& values, Encoding::type encoding) {
-  if (encoding != Encoding::PLAIN) {
-    ParquetException::NYI("only plain encoding currently implemented");
-  }
-  PlainEncoder<BooleanType> encoder(d);
-  encoder.Put(values, static_cast<int>(values.size()));
-  std::shared_ptr<Buffer> buffer = encoder.FlushValues();
-  sink_->Write(buffer->data(), buffer->size());
-
-  num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);
-  encoding_ = encoding;
-  have_values_ = true;
-}
-
-template <typename Type>
-static shared_ptr<DataPage> MakeDataPage(const ColumnDescriptor* d,
-    const vector<typename Type::c_type>& values, int num_vals, Encoding::type encoding,
-    const uint8_t* indices, int indices_size, const vector<int16_t>& def_levels,
-    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level) {
-  int num_values = 0;
-
-  InMemoryOutputStream page_stream;
-  test::DataPageBuilder<Type> page_builder(&page_stream);
-
-  if (!rep_levels.empty()) { page_builder.AppendRepLevels(rep_levels, max_rep_level); }
-  if (!def_levels.empty()) { page_builder.AppendDefLevels(def_levels, max_def_level); }
-
-  if (encoding == Encoding::PLAIN) {
-    page_builder.AppendValues(d, values, encoding);
-    num_values = page_builder.num_values();
-  } else {  // DICTIONARY PAGES
-    page_stream.Write(indices, indices_size);
-    num_values = std::max(page_builder.num_values(), num_vals);
-  }
-
-  auto buffer = page_stream.GetBuffer();
-
-  return std::make_shared<DataPage>(buffer, num_values, encoding,
-      page_builder.def_level_encoding(), page_builder.rep_level_encoding());
-}
-
-template <typename TYPE>
-class DictionaryPageBuilder {
- public:
-  typedef typename TYPE::c_type TC;
-  static constexpr int TN = TYPE::type_num;
-
-  // This class writes data and metadata to the passed inputs
-  explicit DictionaryPageBuilder(const ColumnDescriptor* d)
-      : num_dict_values_(0), have_values_(false) {
-    encoder_.reset(new DictEncoder<TYPE>(d, &pool_));
-  }
-
-  ~DictionaryPageBuilder() { pool_.FreeAll(); }
-
-  shared_ptr<Buffer> AppendValues(const vector<TC>& values) {
-    int num_values = static_cast<int>(values.size());
-    // Dictionary encoding
-    encoder_->Put(values.data(), num_values);
-    num_dict_values_ = encoder_->num_entries();
-    have_values_ = true;
-    return encoder_->FlushValues();
-  }
-
-  shared_ptr<Buffer> WriteDict() {
-    std::shared_ptr<PoolBuffer> dict_buffer =
-        AllocateBuffer(::arrow::default_memory_pool(), encoder_->dict_encoded_size());
-    encoder_->WriteDict(dict_buffer->mutable_data());
-    return dict_buffer;
-  }
-
-  int32_t num_values() const { return num_dict_values_; }
-
- private:
-  ChunkedAllocator pool_;
-  shared_ptr<DictEncoder<TYPE>> encoder_;
-  int32_t num_dict_values_;
-  bool have_values_;
-};
-
-template <>
-DictionaryPageBuilder<BooleanType>::DictionaryPageBuilder(const ColumnDescriptor* d) {
-  ParquetException::NYI("only plain encoding currently implemented for boolean");
-}
-
-template <>
-shared_ptr<Buffer> DictionaryPageBuilder<BooleanType>::WriteDict() {
-  ParquetException::NYI("only plain encoding currently implemented for boolean");
-  return nullptr;
-}
-
-template <>
-shared_ptr<Buffer> DictionaryPageBuilder<BooleanType>::AppendValues(
-    const vector<TC>& values) {
-  ParquetException::NYI("only plain encoding currently implemented for boolean");
-  return nullptr;
-}
-
-template <typename Type>
-static shared_ptr<DictionaryPage> MakeDictPage(const ColumnDescriptor* d,
-    const vector<typename Type::c_type>& values, const vector<int>& values_per_page,
-    Encoding::type encoding, vector<shared_ptr<Buffer>>& rle_indices) {
-  InMemoryOutputStream page_stream;
-  test::DictionaryPageBuilder<Type> page_builder(d);
-  int num_pages = static_cast<int>(values_per_page.size());
-  int value_start = 0;
-
-  for (int i = 0; i < num_pages; i++) {
-    rle_indices.push_back(page_builder.AppendValues(
-        slice(values, value_start, value_start + values_per_page[i])));
-    value_start += values_per_page[i];
-  }
-
-  auto buffer = page_builder.WriteDict();
-
-  return std::make_shared<DictionaryPage>(
-      buffer, page_builder.num_values(), Encoding::PLAIN);
-}
-
-// Given def/rep levels and values create multiple dict pages
-template <typename Type>
-static void PaginateDict(const ColumnDescriptor* d,
-    const vector<typename Type::c_type>& values, const vector<int16_t>& def_levels,
-    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level,
-    int num_levels_per_page, const vector<int>& values_per_page,
-    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::RLE_DICTIONARY) {
-  int num_pages = static_cast<int>(values_per_page.size());
-  vector<shared_ptr<Buffer>> rle_indices;
-  shared_ptr<DictionaryPage> dict_page =
-      MakeDictPage<Type>(d, values, values_per_page, encoding, rle_indices);
-  pages.push_back(dict_page);
-  int def_level_start = 0;
-  int def_level_end = 0;
-  int rep_level_start = 0;
-  int rep_level_end = 0;
-  for (int i = 0; i < num_pages; i++) {
-    if (max_def_level > 0) {
-      def_level_start = i * num_levels_per_page;
-      def_level_end = (i + 1) * num_levels_per_page;
-    }
-    if (max_rep_level > 0) {
-      rep_level_start = i * num_levels_per_page;
-      rep_level_end = (i + 1) * num_levels_per_page;
-    }
-    shared_ptr<DataPage> data_page = MakeDataPage<Int32Type>(d, {}, values_per_page[i],
-        encoding, rle_indices[i]->data(), static_cast<int>(rle_indices[i]->size()),
-        slice(def_levels, def_level_start, def_level_end), max_def_level,
-        slice(rep_levels, rep_level_start, rep_level_end), max_rep_level);
-    pages.push_back(data_page);
-  }
-}
-
-// Given def/rep levels and values create multiple plain pages
-template <typename Type>
-static void PaginatePlain(const ColumnDescriptor* d,
-    const vector<typename Type::c_type>& values, const vector<int16_t>& def_levels,
-    int16_t max_def_level, const vector<int16_t>& rep_levels, int16_t max_rep_level,
-    int num_levels_per_page, const vector<int>& values_per_page,
-    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::PLAIN) {
-  int num_pages = static_cast<int>(values_per_page.size());
-  int def_level_start = 0;
-  int def_level_end = 0;
-  int rep_level_start = 0;
-  int rep_level_end = 0;
-  int value_start = 0;
-  for (int i = 0; i < num_pages; i++) {
-    if (max_def_level > 0) {
-      def_level_start = i * num_levels_per_page;
-      def_level_end = (i + 1) * num_levels_per_page;
-    }
-    if (max_rep_level > 0) {
-      rep_level_start = i * num_levels_per_page;
-      rep_level_end = (i + 1) * num_levels_per_page;
-    }
-    shared_ptr<DataPage> page = MakeDataPage<Type>(d,
-        slice(values, value_start, value_start + values_per_page[i]), values_per_page[i],
-        encoding, NULL, 0, slice(def_levels, def_level_start, def_level_end),
-        max_def_level, slice(rep_levels, rep_level_start, rep_level_end), max_rep_level);
-    pages.push_back(page);
-    value_start += values_per_page[i];
-  }
-}
-
-// Generates pages from randomly generated data
-template <typename Type>
-static int MakePages(const ColumnDescriptor* d, int num_pages, int levels_per_page,
-    vector<int16_t>& def_levels, vector<int16_t>& rep_levels,
-    vector<typename Type::c_type>& values, vector<uint8_t>& buffer,
-    vector<shared_ptr<Page>>& pages, Encoding::type encoding = Encoding::PLAIN) {
-  int num_levels = levels_per_page * num_pages;
-  int num_values = 0;
-  uint32_t seed = 0;
-  int16_t zero = 0;
-  int16_t max_def_level = d->max_definition_level();
-  int16_t max_rep_level = d->max_repetition_level();
-  vector<int> values_per_page(num_pages, levels_per_page);
-  // Create definition levels
-  if (max_def_level > 0) {
-    def_levels.resize(num_levels);
-    random_numbers(num_levels, seed, zero, max_def_level, def_levels.data());
-    for (int p = 0; p < num_pages; p++) {
-      int num_values_per_page = 0;
-      for (int i = 0; i < levels_per_page; i++) {
-        if (def_levels[i + p * levels_per_page] == max_def_level) {
-          num_values_per_page++;
-          num_values++;
-        }
-      }
-      values_per_page[p] = num_values_per_page;
-    }
-  } else {
-    num_values = num_levels;
-  }
-  // Create repitition levels
-  if (max_rep_level > 0) {
-    rep_levels.resize(num_levels);
-    random_numbers(num_levels, seed, zero, max_rep_level, rep_levels.data());
-  }
-  // Create values
-  values.resize(num_values);
-  if (encoding == Encoding::PLAIN) {
-    InitValues<typename Type::c_type>(num_values, values, buffer);
-    PaginatePlain<Type>(d, values, def_levels, max_def_level, rep_levels, max_rep_level,
-        levels_per_page, values_per_page, pages);
-  } else if (encoding == Encoding::RLE_DICTIONARY ||
-             encoding == Encoding::PLAIN_DICTIONARY) {
-    // Calls InitValues and repeats the data
-    InitDictValues<typename Type::c_type>(num_values, levels_per_page, values, buffer);
-    PaginateDict<Type>(d, values, def_levels, max_def_level, rep_levels, max_rep_level,
-        levels_per_page, values_per_page, pages);
-  }
-
-  return num_values;
-}
-
-}  // namespace test
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_TEST_UTIL_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/writer.cc b/src/parquet/column/writer.cc
deleted file mode 100644
index 59f9999..0000000
--- a/src/parquet/column/writer.cc
+++ /dev/null
@@ -1,528 +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.
-
-#include "parquet/column/writer.h"
-
-#include "parquet/column/properties.h"
-#include "parquet/column/statistics.h"
-#include "parquet/encoding-internal.h"
-#include "parquet/util/logging.h"
-#include "parquet/util/memory.h"
-
-namespace parquet {
-
-// ----------------------------------------------------------------------
-// ColumnWriter
-
-std::shared_ptr<WriterProperties> default_writer_properties() {
-  static std::shared_ptr<WriterProperties> default_writer_properties =
-      WriterProperties::Builder().build();
-  return default_writer_properties;
-}
-
-ColumnWriter::ColumnWriter(ColumnChunkMetaDataBuilder* metadata,
-    std::unique_ptr<PageWriter> pager, int64_t expected_rows, bool has_dictionary,
-    Encoding::type encoding, const WriterProperties* properties)
-    : metadata_(metadata),
-      descr_(metadata->descr()),
-      pager_(std::move(pager)),
-      expected_rows_(expected_rows),
-      has_dictionary_(has_dictionary),
-      encoding_(encoding),
-      properties_(properties),
-      allocator_(properties->memory_pool()),
-      pool_(properties->memory_pool()),
-      num_buffered_values_(0),
-      num_buffered_encoded_values_(0),
-      num_rows_(0),
-      total_bytes_written_(0),
-      closed_(false),
-      fallback_(false) {
-  definition_levels_sink_.reset(new InMemoryOutputStream(allocator_));
-  repetition_levels_sink_.reset(new InMemoryOutputStream(allocator_));
-  definition_levels_rle_ =
-      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
-  repetition_levels_rle_ =
-      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
-  uncompressed_data_ =
-      std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
-  if (pager_->has_compressor()) {
-    compressed_data_ =
-        std::static_pointer_cast<ResizableBuffer>(AllocateBuffer(allocator_, 0));
-  }
-}
-
-void ColumnWriter::InitSinks() {
-  definition_levels_sink_->Clear();
-  repetition_levels_sink_->Clear();
-}
-
-void ColumnWriter::WriteDefinitionLevels(int64_t num_levels, const int16_t* levels) {
-  DCHECK(!closed_);
-  definition_levels_sink_->Write(
-      reinterpret_cast<const uint8_t*>(levels), sizeof(int16_t) * num_levels);
-}
-
-void ColumnWriter::WriteRepetitionLevels(int64_t num_levels, const int16_t* levels) {
-  DCHECK(!closed_);
-  repetition_levels_sink_->Write(
-      reinterpret_cast<const uint8_t*>(levels), sizeof(int16_t) * num_levels);
-}
-
-// return the size of the encoded buffer
-int64_t ColumnWriter::RleEncodeLevels(
-    const Buffer& src_buffer, ResizableBuffer* dest_buffer, int16_t max_level) {
-  // TODO: This only works with due to some RLE specifics
-  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level,
-                         static_cast<int>(num_buffered_values_)) +
-                     sizeof(int32_t);
-
-  // Use Arrow::Buffer::shrink_to_fit = false
-  // underlying buffer only keeps growing. Resize to a smaller size does not reallocate.
-  PARQUET_THROW_NOT_OK(dest_buffer->Resize(rle_size, false));
-
-  level_encoder_.Init(Encoding::RLE, max_level, static_cast<int>(num_buffered_values_),
-      dest_buffer->mutable_data() + sizeof(int32_t),
-      static_cast<int>(dest_buffer->size()) - sizeof(int32_t));
-  int encoded = level_encoder_.Encode(static_cast<int>(num_buffered_values_),
-      reinterpret_cast<const int16_t*>(src_buffer.data()));
-  DCHECK_EQ(encoded, num_buffered_values_);
-  reinterpret_cast<int32_t*>(dest_buffer->mutable_data())[0] = level_encoder_.len();
-  int64_t encoded_size = level_encoder_.len() + sizeof(int32_t);
-  return encoded_size;
-}
-
-void ColumnWriter::AddDataPage() {
-  int64_t definition_levels_rle_size = 0;
-  int64_t repetition_levels_rle_size = 0;
-
-  std::shared_ptr<Buffer> values = GetValuesBuffer();
-
-  if (descr_->max_definition_level() > 0) {
-    definition_levels_rle_size = RleEncodeLevels(definition_levels_sink_->GetBufferRef(),
-        definition_levels_rle_.get(), descr_->max_definition_level());
-  }
-
-  if (descr_->max_repetition_level() > 0) {
-    repetition_levels_rle_size = RleEncodeLevels(repetition_levels_sink_->GetBufferRef(),
-        repetition_levels_rle_.get(), descr_->max_repetition_level());
-  }
-
-  int64_t uncompressed_size =
-      definition_levels_rle_size + repetition_levels_rle_size + values->size();
-
-  // Use Arrow::Buffer::shrink_to_fit = false
-  // underlying buffer only keeps growing. Resize to a smaller size does not reallocate.
-  PARQUET_THROW_NOT_OK(uncompressed_data_->Resize(uncompressed_size, false));
-
-  // Concatenate data into a single buffer
-  uint8_t* uncompressed_ptr = uncompressed_data_->mutable_data();
-  memcpy(uncompressed_ptr, repetition_levels_rle_->data(), repetition_levels_rle_size);
-  uncompressed_ptr += repetition_levels_rle_size;
-  memcpy(uncompressed_ptr, definition_levels_rle_->data(), definition_levels_rle_size);
-  uncompressed_ptr += definition_levels_rle_size;
-  memcpy(uncompressed_ptr, values->data(), values->size());
-
-  EncodedStatistics page_stats = GetPageStatistics();
-  ResetPageStatistics();
-
-  std::shared_ptr<Buffer> compressed_data;
-  if (pager_->has_compressor()) {
-    pager_->Compress(*(uncompressed_data_.get()), compressed_data_.get());
-    compressed_data = compressed_data_;
-  } else {
-    compressed_data = uncompressed_data_;
-  }
-
-  // Write the page to OutputStream eagerly if there is no dictionary or
-  // if dictionary encoding has fallen back to PLAIN
-  if (has_dictionary_ && !fallback_) {  // Save pages until end of dictionary encoding
-    std::shared_ptr<Buffer> compressed_data_copy;
-    PARQUET_THROW_NOT_OK(compressed_data->Copy(
-        0, compressed_data->size(), allocator_, &compressed_data_copy));
-    CompressedDataPage page(compressed_data_copy,
-        static_cast<int32_t>(num_buffered_values_), encoding_, Encoding::RLE,
-        Encoding::RLE, uncompressed_size, page_stats);
-    data_pages_.push_back(std::move(page));
-  } else {  // Eagerly write pages
-    CompressedDataPage page(compressed_data, static_cast<int32_t>(num_buffered_values_),
-        encoding_, Encoding::RLE, Encoding::RLE, uncompressed_size, page_stats);
-    WriteDataPage(page);
-  }
-
-  // Re-initialize the sinks for next Page.
-  InitSinks();
-  num_buffered_values_ = 0;
-  num_buffered_encoded_values_ = 0;
-}
-
-void ColumnWriter::WriteDataPage(const CompressedDataPage& page) {
-  total_bytes_written_ += pager_->WriteDataPage(page);
-}
-
-int64_t ColumnWriter::Close() {
-  if (!closed_) {
-    closed_ = true;
-    if (has_dictionary_ && !fallback_) { WriteDictionaryPage(); }
-
-    FlushBufferedDataPages();
-
-    EncodedStatistics chunk_statistics = GetChunkStatistics();
-    if (chunk_statistics.is_set()) metadata_->SetStatistics(chunk_statistics);
-    pager_->Close(has_dictionary_, fallback_);
-  }
-
-  if (num_rows_ != expected_rows_) {
-    std::stringstream ss;
-    ss << "Written rows: " << num_rows_ << " != expected rows: " << expected_rows_
-       << "in the current column chunk";
-    throw ParquetException(ss.str());
-  }
-
-  return total_bytes_written_;
-}
-
-void ColumnWriter::FlushBufferedDataPages() {
-  // Write all outstanding data to a new page
-  if (num_buffered_values_ > 0) { AddDataPage(); }
-  for (size_t i = 0; i < data_pages_.size(); i++) {
-    WriteDataPage(data_pages_[i]);
-  }
-  data_pages_.clear();
-}
-
-// ----------------------------------------------------------------------
-// TypedColumnWriter
-
-template <typename Type>
-TypedColumnWriter<Type>::TypedColumnWriter(ColumnChunkMetaDataBuilder* metadata,
-    std::unique_ptr<PageWriter> pager, int64_t expected_rows, Encoding::type encoding,
-    const WriterProperties* properties)
-    : ColumnWriter(metadata, std::move(pager), expected_rows,
-          (encoding == Encoding::PLAIN_DICTIONARY ||
-              encoding == Encoding::RLE_DICTIONARY),
-          encoding, properties) {
-  switch (encoding) {
-    case Encoding::PLAIN:
-      current_encoder_.reset(new PlainEncoder<Type>(descr_, properties->memory_pool()));
-      break;
-    case Encoding::PLAIN_DICTIONARY:
-    case Encoding::RLE_DICTIONARY:
-      current_encoder_.reset(
-          new DictEncoder<Type>(descr_, &pool_, properties->memory_pool()));
-      break;
-    default:
-      ParquetException::NYI("Selected encoding is not supported");
-  }
-
-  if (properties->statistics_enabled(descr_->path())) {
-    page_statistics_ = std::unique_ptr<TypedStats>(new TypedStats(descr_, allocator_));
-    chunk_statistics_ = std::unique_ptr<TypedStats>(new TypedStats(descr_, allocator_));
-  }
-}
-
-// Only one Dictionary Page is written.
-// Fallback to PLAIN if dictionary page limit is reached.
-template <typename Type>
-void TypedColumnWriter<Type>::CheckDictionarySizeLimit() {
-  auto dict_encoder = static_cast<DictEncoder<Type>*>(current_encoder_.get());
-  if (dict_encoder->dict_encoded_size() >= properties_->dictionary_pagesize_limit()) {
-    WriteDictionaryPage();
-    // Serialize the buffered Dictionary Indicies
-    FlushBufferedDataPages();
-    fallback_ = true;
-    // Only PLAIN encoding is supported for fallback in V1
-    current_encoder_.reset(new PlainEncoder<Type>(descr_, properties_->memory_pool()));
-    encoding_ = Encoding::PLAIN;
-  }
-}
-
-template <typename Type>
-void TypedColumnWriter<Type>::WriteDictionaryPage() {
-  auto dict_encoder = static_cast<DictEncoder<Type>*>(current_encoder_.get());
-  std::shared_ptr<PoolBuffer> buffer =
-      AllocateBuffer(properties_->memory_pool(), dict_encoder->dict_encoded_size());
-  dict_encoder->WriteDict(buffer->mutable_data());
-  // TODO Get rid of this deep call
-  dict_encoder->mem_pool()->FreeAll();
-
-  DictionaryPage page(
-      buffer, dict_encoder->num_entries(), properties_->dictionary_index_encoding());
-  total_bytes_written_ += pager_->WriteDictionaryPage(page);
-}
-
-template <typename Type>
-EncodedStatistics TypedColumnWriter<Type>::GetPageStatistics() {
-  EncodedStatistics result;
-  if (page_statistics_) result = page_statistics_->Encode();
-  return result;
-}
-
-template <typename Type>
-EncodedStatistics TypedColumnWriter<Type>::GetChunkStatistics() {
-  EncodedStatistics result;
-  if (chunk_statistics_) result = chunk_statistics_->Encode();
-  return result;
-}
-
-template <typename Type>
-void TypedColumnWriter<Type>::ResetPageStatistics() {
-  if (chunk_statistics_ != nullptr) {
-    chunk_statistics_->Merge(*page_statistics_);
-    page_statistics_->Reset();
-  }
-}
-
-// ----------------------------------------------------------------------
-// Dynamic column writer constructor
-
-std::shared_ptr<ColumnWriter> ColumnWriter::Make(ColumnChunkMetaDataBuilder* metadata,
-    std::unique_ptr<PageWriter> pager, int64_t expected_rows,
-    const WriterProperties* properties) {
-  const ColumnDescriptor* descr = metadata->descr();
-  Encoding::type encoding = properties->encoding(descr->path());
-  if (properties->dictionary_enabled(descr->path()) &&
-      descr->physical_type() != Type::BOOLEAN) {
-    encoding = properties->dictionary_page_encoding();
-  }
-  switch (descr->physical_type()) {
-    case Type::BOOLEAN:
-      return std::make_shared<BoolWriter>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::INT32:
-      return std::make_shared<Int32Writer>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::INT64:
-      return std::make_shared<Int64Writer>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::INT96:
-      return std::make_shared<Int96Writer>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::FLOAT:
-      return std::make_shared<FloatWriter>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::DOUBLE:
-      return std::make_shared<DoubleWriter>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::BYTE_ARRAY:
-      return std::make_shared<ByteArrayWriter>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    case Type::FIXED_LEN_BYTE_ARRAY:
-      return std::make_shared<FixedLenByteArrayWriter>(
-          metadata, std::move(pager), expected_rows, encoding, properties);
-    default:
-      ParquetException::NYI("type reader not implemented");
-  }
-  // Unreachable code, but supress compiler warning
-  return std::shared_ptr<ColumnWriter>(nullptr);
-}
-
-// ----------------------------------------------------------------------
-// Instantiate templated classes
-
-template <typename DType>
-inline int64_t TypedColumnWriter<DType>::WriteMiniBatch(int64_t num_values,
-    const int16_t* def_levels, const int16_t* rep_levels, const T* values) {
-  int64_t values_to_write = 0;
-  // If the field is required and non-repeated, there are no definition levels
-  if (descr_->max_definition_level() > 0) {
-    for (int64_t i = 0; i < num_values; ++i) {
-      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_write; }
-    }
-
-    WriteDefinitionLevels(num_values, def_levels);
-  } else {
-    // Required field, write all values
-    values_to_write = num_values;
-  }
-
-  // Not present for non-repeated fields
-  if (descr_->max_repetition_level() > 0) {
-    // A row could include more than one value
-    // Count the occasions where we start a new row
-    for (int64_t i = 0; i < num_values; ++i) {
-      if (rep_levels[i] == 0) { num_rows_++; }
-    }
-
-    WriteRepetitionLevels(num_values, rep_levels);
-  } else {
-    // Each value is exactly one row
-    num_rows_ += static_cast<int>(num_values);
-  }
-
-  if (num_rows_ > expected_rows_) {
-    throw ParquetException("More rows were written in the column chunk than expected");
-  }
-
-  // PARQUET-780
-  if (values_to_write > 0) { DCHECK(nullptr != values) << "Values ptr cannot be NULL"; }
-
-  WriteValues(values_to_write, values);
-
-  if (page_statistics_ != nullptr) {
-    page_statistics_->Update(values, values_to_write, num_values - values_to_write);
-  }
-
-  num_buffered_values_ += num_values;
-  num_buffered_encoded_values_ += values_to_write;
-
-  if (current_encoder_->EstimatedDataEncodedSize() >= properties_->data_pagesize()) {
-    AddDataPage();
-  }
-  if (has_dictionary_ && !fallback_) { CheckDictionarySizeLimit(); }
-
-  return values_to_write;
-}
-
-template <typename DType>
-inline int64_t TypedColumnWriter<DType>::WriteMiniBatchSpaced(int64_t num_values,
-    const int16_t* def_levels, const int16_t* rep_levels, const uint8_t* valid_bits,
-    int64_t valid_bits_offset, const T* values, int64_t* num_spaced_written) {
-  int64_t values_to_write = 0;
-  int64_t spaced_values_to_write = 0;
-  // If the field is required and non-repeated, there are no definition levels
-  if (descr_->max_definition_level() > 0) {
-    // Minimal definition level for which spaced values are written
-    int16_t min_spaced_def_level = descr_->max_definition_level();
-    if (descr_->schema_node()->is_optional()) { min_spaced_def_level--; }
-    for (int64_t i = 0; i < num_values; ++i) {
-      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_write; }
-      if (def_levels[i] >= min_spaced_def_level) { ++spaced_values_to_write; }
-    }
-
-    WriteDefinitionLevels(num_values, def_levels);
-  } else {
-    // Required field, write all values
-    values_to_write = num_values;
-    spaced_values_to_write = num_values;
-  }
-
-  // Not present for non-repeated fields
-  if (descr_->max_repetition_level() > 0) {
-    // A row could include more than one value
-    // Count the occasions where we start a new row
-    for (int64_t i = 0; i < num_values; ++i) {
-      if (rep_levels[i] == 0) { num_rows_++; }
-    }
-
-    WriteRepetitionLevels(num_values, rep_levels);
-  } else {
-    // Each value is exactly one row
-    num_rows_ += static_cast<int>(num_values);
-  }
-
-  if (num_rows_ > expected_rows_) {
-    throw ParquetException("More rows were written in the column chunk than expected");
-  }
-
-  if (descr_->schema_node()->is_optional()) {
-    WriteValuesSpaced(spaced_values_to_write, valid_bits, valid_bits_offset, values);
-  } else {
-    WriteValues(values_to_write, values);
-  }
-  *num_spaced_written = spaced_values_to_write;
-
-  if (page_statistics_ != nullptr) {
-    page_statistics_->UpdateSpaced(values, valid_bits, valid_bits_offset, values_to_write,
-        num_values - values_to_write);
-  }
-
-  num_buffered_values_ += num_values;
-  num_buffered_encoded_values_ += values_to_write;
-
-  if (current_encoder_->EstimatedDataEncodedSize() >= properties_->data_pagesize()) {
-    AddDataPage();
-  }
-  if (has_dictionary_ && !fallback_) { CheckDictionarySizeLimit(); }
-
-  return values_to_write;
-}
-
-template <typename DType>
-void TypedColumnWriter<DType>::WriteBatch(int64_t num_values, const int16_t* def_levels,
-    const int16_t* rep_levels, const T* values) {
-  // We check for DataPage limits only after we have inserted the values. If a user
-  // writes a large number of values, the DataPage size can be much above the limit.
-  // The purpose of this chunking is to bound this. Even if a user writes large number
-  // of values, the chunking will ensure the AddDataPage() is called at a reasonable
-  // pagesize limit
-  int64_t write_batch_size = properties_->write_batch_size();
-  int num_batches = static_cast<int>(num_values / write_batch_size);
-  int64_t num_remaining = num_values % write_batch_size;
-  int64_t value_offset = 0;
-  for (int round = 0; round < num_batches; round++) {
-    int64_t offset = round * write_batch_size;
-    int64_t num_values = WriteMiniBatch(write_batch_size, &def_levels[offset],
-        &rep_levels[offset], &values[value_offset]);
-    value_offset += num_values;
-  }
-  // Write the remaining values
-  int64_t offset = num_batches * write_batch_size;
-  WriteMiniBatch(
-      num_remaining, &def_levels[offset], &rep_levels[offset], &values[value_offset]);
-}
-
-template <typename DType>
-void TypedColumnWriter<DType>::WriteBatchSpaced(int64_t num_values,
-    const int16_t* def_levels, const int16_t* rep_levels, const uint8_t* valid_bits,
-    int64_t valid_bits_offset, const T* values) {
-  // We check for DataPage limits only after we have inserted the values. If a user
-  // writes a large number of values, the DataPage size can be much above the limit.
-  // The purpose of this chunking is to bound this. Even if a user writes large number
-  // of values, the chunking will ensure the AddDataPage() is called at a reasonable
-  // pagesize limit
-  int64_t write_batch_size = properties_->write_batch_size();
-  int num_batches = static_cast<int>(num_values / write_batch_size);
-  int64_t num_remaining = num_values % write_batch_size;
-  int64_t num_spaced_written = 0;
-  int64_t values_offset = 0;
-  for (int round = 0; round < num_batches; round++) {
-    int64_t offset = round * write_batch_size;
-    WriteMiniBatchSpaced(write_batch_size, &def_levels[offset], &rep_levels[offset],
-        valid_bits, valid_bits_offset + values_offset, values + values_offset,
-        &num_spaced_written);
-    values_offset += num_spaced_written;
-  }
-  // Write the remaining values
-  int64_t offset = num_batches * write_batch_size;
-  WriteMiniBatchSpaced(num_remaining, &def_levels[offset], &rep_levels[offset],
-      valid_bits, valid_bits_offset + values_offset, values + values_offset,
-      &num_spaced_written);
-}
-
-template <typename DType>
-void TypedColumnWriter<DType>::WriteValues(int64_t num_values, const T* values) {
-  current_encoder_->Put(values, static_cast<int>(num_values));
-}
-
-template <typename DType>
-void TypedColumnWriter<DType>::WriteValuesSpaced(int64_t num_values,
-    const uint8_t* valid_bits, int64_t valid_bits_offset, const T* values) {
-  current_encoder_->PutSpaced(
-      values, static_cast<int>(num_values), valid_bits, valid_bits_offset);
-}
-
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<BooleanType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int32Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int64Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<Int96Type>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<FloatType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<DoubleType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<ByteArrayType>;
-template class PARQUET_TEMPLATE_EXPORT TypedColumnWriter<FLBAType>;
-
-}  // namespace parquet


[3/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Posted by uw...@apache.org.
http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/writer.h b/src/parquet/column/writer.h
deleted file mode 100644
index 407e808..0000000
--- a/src/parquet/column/writer.h
+++ /dev/null
@@ -1,250 +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.
-
-#ifndef PARQUET_COLUMN_WRITER_H
-#define PARQUET_COLUMN_WRITER_H
-
-#include <vector>
-
-#include "parquet/column/levels.h"
-#include "parquet/column/page.h"
-#include "parquet/column/properties.h"
-#include "parquet/column/statistics.h"
-#include "parquet/encoding.h"
-#include "parquet/file/metadata.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/memory.h"
-#include "parquet/util/visibility.h"
-
-namespace parquet {
-
-static constexpr int WRITE_BATCH_SIZE = 1000;
-class PARQUET_EXPORT ColumnWriter {
- public:
-  ColumnWriter(ColumnChunkMetaDataBuilder*, std::unique_ptr<PageWriter>,
-      int64_t expected_rows, bool has_dictionary, Encoding::type encoding,
-      const WriterProperties* properties);
-
-  static std::shared_ptr<ColumnWriter> Make(ColumnChunkMetaDataBuilder*,
-      std::unique_ptr<PageWriter>, int64_t expected_rows,
-      const WriterProperties* properties);
-
-  Type::type type() const { return descr_->physical_type(); }
-
-  const ColumnDescriptor* descr() const { return descr_; }
-
-  /**
-   * Closes the ColumnWriter, commits any buffered values to pages.
-   *
-   * @return Total size of the column in bytes
-   */
-  int64_t Close();
-
- protected:
-  virtual std::shared_ptr<Buffer> GetValuesBuffer() = 0;
-
-  // Serializes Dictionary Page if enabled
-  virtual void WriteDictionaryPage() = 0;
-
-  // Checks if the Dictionary Page size limit is reached
-  // If the limit is reached, the Dictionary and Data Pages are serialized
-  // The encoding is switched to PLAIN
-
-  virtual void CheckDictionarySizeLimit() = 0;
-
-  // Plain-encoded statistics of the current page
-  virtual EncodedStatistics GetPageStatistics() = 0;
-
-  // Plain-encoded statistics of the whole chunk
-  virtual EncodedStatistics GetChunkStatistics() = 0;
-
-  // Merges page statistics into chunk statistics, then resets the values
-  virtual void ResetPageStatistics() = 0;
-
-  // Adds Data Pages to an in memory buffer in dictionary encoding mode
-  // Serializes the Data Pages in other encoding modes
-  void AddDataPage();
-
-  // Serializes Data Pages
-  void WriteDataPage(const CompressedDataPage& page);
-
-  // Write multiple definition levels
-  void WriteDefinitionLevels(int64_t num_levels, const int16_t* levels);
-
-  // Write multiple repetition levels
-  void WriteRepetitionLevels(int64_t num_levels, const int16_t* levels);
-
-  // RLE encode the src_buffer into dest_buffer and return the encoded size
-  int64_t RleEncodeLevels(
-      const Buffer& src_buffer, ResizableBuffer* dest_buffer, int16_t max_level);
-
-  // Serialize the buffered Data Pages
-  void FlushBufferedDataPages();
-
-  ColumnChunkMetaDataBuilder* metadata_;
-  const ColumnDescriptor* descr_;
-
-  std::unique_ptr<PageWriter> pager_;
-
-  // The number of rows that should be written in this column chunk.
-  int64_t expected_rows_;
-  bool has_dictionary_;
-  Encoding::type encoding_;
-  const WriterProperties* properties_;
-
-  LevelEncoder level_encoder_;
-
-  ::arrow::MemoryPool* allocator_;
-  ChunkedAllocator pool_;
-
-  // The total number of values stored in the data page. This is the maximum of
-  // the number of encoded definition levels or encoded values. For
-  // non-repeated, required columns, this is equal to the number of encoded
-  // values. For repeated or optional values, there may be fewer data values
-  // than levels, and this tells you how many encoded levels there are in that
-  // case.
-  int64_t num_buffered_values_;
-
-  // The total number of stored values. For repeated or optional values, this
-  // number may be lower than num_buffered_values_.
-  int64_t num_buffered_encoded_values_;
-
-  // Total number of rows written with this ColumnWriter
-  int num_rows_;
-
-  // Records the total number of bytes written by the serializer
-  int64_t total_bytes_written_;
-
-  // Flag to check if the Writer has been closed
-  bool closed_;
-
-  // Flag to infer if dictionary encoding has fallen back to PLAIN
-  bool fallback_;
-
-  std::unique_ptr<InMemoryOutputStream> definition_levels_sink_;
-  std::unique_ptr<InMemoryOutputStream> repetition_levels_sink_;
-
-  std::shared_ptr<ResizableBuffer> definition_levels_rle_;
-  std::shared_ptr<ResizableBuffer> repetition_levels_rle_;
-
-  std::shared_ptr<ResizableBuffer> uncompressed_data_;
-  std::shared_ptr<ResizableBuffer> compressed_data_;
-
-  std::vector<CompressedDataPage> data_pages_;
-
- private:
-  void InitSinks();
-};
-
-// API to write values to a single column. This is the main client facing API.
-template <typename DType>
-class PARQUET_EXPORT TypedColumnWriter : public ColumnWriter {
- public:
-  typedef typename DType::c_type T;
-
-  TypedColumnWriter(ColumnChunkMetaDataBuilder* metadata,
-      std::unique_ptr<PageWriter> pager, int64_t expected_rows, Encoding::type encoding,
-      const WriterProperties* properties);
-
-  // Write a batch of repetition levels, definition levels, and values to the
-  // column.
-  void WriteBatch(int64_t num_values, const int16_t* def_levels,
-      const int16_t* rep_levels, const T* values);
-
-  /// Write a batch of repetition levels, definition levels, and values to the
-  /// column.
-  ///
-  /// In comparision to WriteBatch the length of repetition and definition levels
-  /// is the same as of the number of values read for max_definition_level == 1.
-  /// In the case of max_definition_level > 1, the repetition and definition
-  /// levels are larger than the values but the values include the null entries
-  /// with definition_level == (max_definition_level - 1). Thus we have to differentiate
-  /// in the parameters of this function if the input has the length of num_values or the
-  /// _number of rows in the lowest nesting level_.
-  ///
-  /// In the case that the most inner node in the Parquet is required, the _number of rows
-  /// in the lowest nesting level_ is equal to the number of non-null values. If the
-  /// inner-most schema node is optional, the _number of rows in the lowest nesting level_
-  /// also includes all values with definition_level == (max_definition_level - 1).
-  ///
-  /// @param num_values number of levels to write.
-  /// @param def_levels The Parquet definiton levels, length is num_values
-  /// @param rep_levels The Parquet repetition levels, length is num_values
-  /// @param valid_bits Bitmap that indicates if the row is null on the lowest nesting
-  ///   level. The length is number of rows in the lowest nesting level.
-  /// @param valid_bits_offset The offset in bits of the valid_bits where the
-  ///   first relevant bit resides.
-  /// @param values The values in the lowest nested level including
-  ///   spacing for nulls on the lowest levels; input has the length
-  ///   of the number of rows on the lowest nesting level.
-  void WriteBatchSpaced(int64_t num_values, const int16_t* def_levels,
-      const int16_t* rep_levels, const uint8_t* valid_bits, int64_t valid_bits_offset,
-      const T* values);
-
- protected:
-  std::shared_ptr<Buffer> GetValuesBuffer() override {
-    return current_encoder_->FlushValues();
-  }
-  void WriteDictionaryPage() override;
-  void CheckDictionarySizeLimit() override;
-  EncodedStatistics GetPageStatistics() override;
-  EncodedStatistics GetChunkStatistics() override;
-  void ResetPageStatistics() override;
-
- private:
-  int64_t WriteMiniBatch(int64_t num_values, const int16_t* def_levels,
-      const int16_t* rep_levels, const T* values);
-
-  int64_t WriteMiniBatchSpaced(int64_t num_values, const int16_t* def_levels,
-      const int16_t* rep_levels, const uint8_t* valid_bits, int64_t valid_bits_offset,
-      const T* values, int64_t* num_spaced_written);
-
-  typedef Encoder<DType> EncoderType;
-
-  // Write values to a temporary buffer before they are encoded into pages
-  void WriteValues(int64_t num_values, const T* values);
-  void WriteValuesSpaced(int64_t num_values, const uint8_t* valid_bits,
-      int64_t valid_bits_offset, const T* values);
-  std::unique_ptr<EncoderType> current_encoder_;
-
-  typedef TypedRowGroupStatistics<DType> TypedStats;
-  std::unique_ptr<TypedStats> page_statistics_;
-  std::unique_ptr<TypedStats> chunk_statistics_;
-};
-
-typedef TypedColumnWriter<BooleanType> BoolWriter;
-typedef TypedColumnWriter<Int32Type> Int32Writer;
-typedef TypedColumnWriter<Int64Type> Int64Writer;
-typedef TypedColumnWriter<Int96Type> Int96Writer;
-typedef TypedColumnWriter<FloatType> FloatWriter;
-typedef TypedColumnWriter<DoubleType> DoubleWriter;
-typedef TypedColumnWriter<ByteArrayType> ByteArrayWriter;
-typedef TypedColumnWriter<FLBAType> FixedLenByteArrayWriter;
-
-extern template class PARQUET_EXPORT TypedColumnWriter<BooleanType>;
-extern template class PARQUET_EXPORT TypedColumnWriter<Int32Type>;
-extern template class PARQUET_EXPORT TypedColumnWriter<Int64Type>;
-extern template class PARQUET_EXPORT TypedColumnWriter<Int96Type>;
-extern template class PARQUET_EXPORT TypedColumnWriter<FloatType>;
-extern template class PARQUET_EXPORT TypedColumnWriter<DoubleType>;
-extern template class PARQUET_EXPORT TypedColumnWriter<ByteArrayType>;
-extern template class PARQUET_EXPORT TypedColumnWriter<FLBAType>;
-
-}  // namespace parquet
-
-#endif  // PARQUET_COLUMN_READER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_page.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_page.h b/src/parquet/column_page.h
new file mode 100644
index 0000000..7840612
--- /dev/null
+++ b/src/parquet/column_page.h
@@ -0,0 +1,201 @@
+// 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.
+
+// This module defines an abstract interface for iterating through pages in a
+// Parquet column chunk within a row group. It could be extended in the future
+// to iterate through all data pages in all chunks in a file.
+
+#ifndef PARQUET_COLUMN_PAGE_H
+#define PARQUET_COLUMN_PAGE_H
+
+#include <cstdint>
+#include <memory>
+#include <string>
+
+#include "parquet/statistics.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+
+namespace parquet {
+
+// TODO: Parallel processing is not yet safe because of memory-ownership
+// semantics (the PageReader may or may not own the memory referenced by a
+// page)
+//
+// TODO(wesm): In the future Parquet implementations may store the crc code
+// in format::PageHeader. parquet-mr currently does not, so we also skip it
+// here, both on the read and write path
+class Page {
+ public:
+  Page(const std::shared_ptr<Buffer>& buffer, PageType::type type)
+      : buffer_(buffer), type_(type) {}
+
+  PageType::type type() const { return type_; }
+
+  std::shared_ptr<Buffer> buffer() const { return buffer_; }
+
+  // @returns: a pointer to the page's data
+  const uint8_t* data() const { return buffer_->data(); }
+
+  // @returns: the total size in bytes of the page's data buffer
+  int32_t size() const { return static_cast<int32_t>(buffer_->size()); }
+
+ private:
+  std::shared_ptr<Buffer> buffer_;
+  PageType::type type_;
+};
+
+class DataPage : public Page {
+ public:
+  DataPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
+      Encoding::type encoding, Encoding::type definition_level_encoding,
+      Encoding::type repetition_level_encoding,
+      const EncodedStatistics& statistics = EncodedStatistics())
+      : Page(buffer, PageType::DATA_PAGE),
+        num_values_(num_values),
+        encoding_(encoding),
+        definition_level_encoding_(definition_level_encoding),
+        repetition_level_encoding_(repetition_level_encoding),
+        statistics_(statistics) {}
+
+  int32_t num_values() const { return num_values_; }
+
+  Encoding::type encoding() const { return encoding_; }
+
+  Encoding::type repetition_level_encoding() const { return repetition_level_encoding_; }
+
+  Encoding::type definition_level_encoding() const { return definition_level_encoding_; }
+
+  const EncodedStatistics& statistics() const { return statistics_; }
+
+ private:
+  int32_t num_values_;
+  Encoding::type encoding_;
+  Encoding::type definition_level_encoding_;
+  Encoding::type repetition_level_encoding_;
+  EncodedStatistics statistics_;
+};
+
+class CompressedDataPage : public DataPage {
+ public:
+  CompressedDataPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
+      Encoding::type encoding, Encoding::type definition_level_encoding,
+      Encoding::type repetition_level_encoding, int64_t uncompressed_size,
+      const EncodedStatistics& statistics = EncodedStatistics())
+      : DataPage(buffer, num_values, encoding, definition_level_encoding,
+            repetition_level_encoding, statistics),
+        uncompressed_size_(uncompressed_size) {}
+
+  int64_t uncompressed_size() const { return uncompressed_size_; }
+
+ private:
+  int64_t uncompressed_size_;
+};
+
+class DataPageV2 : public Page {
+ public:
+  DataPageV2(const std::shared_ptr<Buffer>& buffer, int32_t num_values, int32_t num_nulls,
+      int32_t num_rows, Encoding::type encoding, int32_t definition_levels_byte_length,
+      int32_t repetition_levels_byte_length, bool is_compressed = false)
+      : Page(buffer, PageType::DATA_PAGE_V2),
+        num_values_(num_values),
+        num_nulls_(num_nulls),
+        num_rows_(num_rows),
+        encoding_(encoding),
+        definition_levels_byte_length_(definition_levels_byte_length),
+        repetition_levels_byte_length_(repetition_levels_byte_length),
+        is_compressed_(is_compressed) {}
+
+  int32_t num_values() const { return num_values_; }
+
+  int32_t num_nulls() const { return num_nulls_; }
+
+  int32_t num_rows() const { return num_rows_; }
+
+  Encoding::type encoding() const { return encoding_; }
+
+  int32_t definition_levels_byte_length() const { return definition_levels_byte_length_; }
+
+  int32_t repetition_levels_byte_length() const { return repetition_levels_byte_length_; }
+
+  bool is_compressed() const { return is_compressed_; }
+
+ private:
+  int32_t num_values_;
+  int32_t num_nulls_;
+  int32_t num_rows_;
+  Encoding::type encoding_;
+  int32_t definition_levels_byte_length_;
+  int32_t repetition_levels_byte_length_;
+  bool is_compressed_;
+
+  // TODO(wesm): format::DataPageHeaderV2.statistics
+};
+
+class DictionaryPage : public Page {
+ public:
+  DictionaryPage(const std::shared_ptr<Buffer>& buffer, int32_t num_values,
+      Encoding::type encoding, bool is_sorted = false)
+      : Page(buffer, PageType::DICTIONARY_PAGE),
+        num_values_(num_values),
+        encoding_(encoding),
+        is_sorted_(is_sorted) {}
+
+  int32_t num_values() const { return num_values_; }
+
+  Encoding::type encoding() const { return encoding_; }
+
+  bool is_sorted() const { return is_sorted_; }
+
+ private:
+  int32_t num_values_;
+  Encoding::type encoding_;
+  bool is_sorted_;
+};
+
+// Abstract page iterator interface. This way, we can feed column pages to the
+// ColumnReader through whatever mechanism we choose
+class PageReader {
+ public:
+  virtual ~PageReader() {}
+
+  // @returns: shared_ptr<Page>(nullptr) on EOS, std::shared_ptr<Page>
+  // containing new Page otherwise
+  virtual std::shared_ptr<Page> NextPage() = 0;
+};
+
+class PageWriter {
+ public:
+  virtual ~PageWriter() {}
+
+  // The Column Writer decides if dictionary encoding is used if set and
+  // if the dictionary encoding has fallen back to default encoding on reaching dictionary
+  // page limit
+  virtual void Close(bool has_dictionary, bool fallback) = 0;
+
+  virtual int64_t WriteDataPage(const CompressedDataPage& page) = 0;
+
+  virtual int64_t WriteDictionaryPage(const DictionaryPage& page) = 0;
+
+  virtual bool has_compressor() = 0;
+
+  virtual void Compress(const Buffer& src_buffer, ResizableBuffer* dest_buffer) = 0;
+};
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_PAGE_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_reader-test.cc b/src/parquet/column_reader-test.cc
new file mode 100644
index 0000000..84d1e37
--- /dev/null
+++ b/src/parquet/column_reader-test.cc
@@ -0,0 +1,371 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <limits>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/column_page.h"
+#include "parquet/column_reader.h"
+#include "parquet/schema.h"
+#include "parquet/test-util.h"
+#include "parquet/types.h"
+#include "parquet/util/test-common.h"
+
+using std::string;
+using std::vector;
+using std::shared_ptr;
+
+namespace parquet {
+
+using schema::NodePtr;
+
+namespace test {
+
+template <typename T>
+static inline bool vector_equal_with_def_levels(const vector<T>& left,
+    const vector<int16_t>& def_levels, int16_t max_def_levels, int16_t max_rep_levels,
+    const vector<T>& right) {
+  size_t i_left = 0;
+  size_t i_right = 0;
+  for (size_t i = 0; i < def_levels.size(); i++) {
+    if (def_levels[i] == max_def_levels) {
+      // Compare
+      if (left[i_left] != right[i_right]) {
+        std::cerr << "index " << i << " left was " << left[i_left] << " right was "
+                  << right[i] << std::endl;
+        return false;
+      }
+      i_left++;
+      i_right++;
+    } else if (def_levels[i] == (max_def_levels - 1)) {
+      // Null entry on the lowest nested level
+      i_right++;
+    } else if (def_levels[i] < (max_def_levels - 1)) {
+      // Null entry on a higher nesting level, only supported for non-repeating data
+      if (max_rep_levels == 0) { i_right++; }
+    }
+  }
+
+  return true;
+}
+
+class TestPrimitiveReader : public ::testing::Test {
+ public:
+  void InitReader(const ColumnDescriptor* d) {
+    std::unique_ptr<PageReader> pager_;
+    pager_.reset(new test::MockPageReader(pages_));
+    reader_ = ColumnReader::Make(d, std::move(pager_));
+  }
+
+  void CheckResults() {
+    vector<int32_t> vresult(num_values_, -1);
+    vector<int16_t> dresult(num_levels_, -1);
+    vector<int16_t> rresult(num_levels_, -1);
+    int64_t values_read = 0;
+    int total_values_read = 0;
+    int batch_actual = 0;
+
+    Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
+    int32_t batch_size = 8;
+    int batch = 0;
+    // This will cover both the cases
+    // 1) batch_size < page_size (multiple ReadBatch from a single page)
+    // 2) batch_size > page_size (BatchRead limits to a single page)
+    do {
+      batch = static_cast<int>(reader->ReadBatch(batch_size, &dresult[0] + batch_actual,
+          &rresult[0] + batch_actual, &vresult[0] + total_values_read, &values_read));
+      total_values_read += static_cast<int>(values_read);
+      batch_actual += batch;
+      batch_size = std::max(batch_size * 2, 4096);
+    } while (batch > 0);
+
+    ASSERT_EQ(num_levels_, batch_actual);
+    ASSERT_EQ(num_values_, total_values_read);
+    ASSERT_TRUE(vector_equal(values_, vresult));
+    if (max_def_level_ > 0) { ASSERT_TRUE(vector_equal(def_levels_, dresult)); }
+    if (max_rep_level_ > 0) { ASSERT_TRUE(vector_equal(rep_levels_, rresult)); }
+    // catch improper writes at EOS
+    batch_actual =
+        static_cast<int>(reader->ReadBatch(5, nullptr, nullptr, nullptr, &values_read));
+    ASSERT_EQ(0, batch_actual);
+    ASSERT_EQ(0, values_read);
+  }
+
+  void CheckResultsSpaced() {
+    vector<int32_t> vresult(num_levels_, -1);
+    vector<int16_t> dresult(num_levels_, -1);
+    vector<int16_t> rresult(num_levels_, -1);
+    vector<uint8_t> valid_bits(num_levels_, 255);
+    int total_values_read = 0;
+    int batch_actual = 0;
+    int levels_actual = 0;
+    int64_t null_count = -1;
+    int64_t levels_read = 0;
+    int64_t values_read;
+
+    Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
+    int32_t batch_size = 8;
+    int batch = 0;
+    // This will cover both the cases
+    // 1) batch_size < page_size (multiple ReadBatch from a single page)
+    // 2) batch_size > page_size (BatchRead limits to a single page)
+    do {
+      batch = static_cast<int>(reader->ReadBatchSpaced(batch_size,
+          dresult.data() + levels_actual, rresult.data() + levels_actual,
+          vresult.data() + batch_actual, valid_bits.data() + batch_actual, 0,
+          &levels_read, &values_read, &null_count));
+      total_values_read += batch - static_cast<int>(null_count);
+      batch_actual += batch;
+      levels_actual += static_cast<int>(levels_read);
+      batch_size = std::max(batch_size * 2, 4096);
+    } while ((batch > 0) || (levels_read > 0));
+
+    ASSERT_EQ(num_levels_, levels_actual);
+    ASSERT_EQ(num_values_, total_values_read);
+    if (max_def_level_ > 0) {
+      ASSERT_TRUE(vector_equal(def_levels_, dresult));
+      ASSERT_TRUE(vector_equal_with_def_levels(
+          values_, dresult, max_def_level_, max_rep_level_, vresult));
+    } else {
+      ASSERT_TRUE(vector_equal(values_, vresult));
+    }
+    if (max_rep_level_ > 0) { ASSERT_TRUE(vector_equal(rep_levels_, rresult)); }
+    // catch improper writes at EOS
+    batch_actual = static_cast<int>(reader->ReadBatchSpaced(5, nullptr, nullptr, nullptr,
+        valid_bits.data(), 0, &levels_read, &values_read, &null_count));
+    ASSERT_EQ(0, batch_actual);
+    ASSERT_EQ(0, null_count);
+  }
+
+  void Clear() {
+    values_.clear();
+    def_levels_.clear();
+    rep_levels_.clear();
+    pages_.clear();
+    reader_.reset();
+  }
+
+  void ExecutePlain(int num_pages, int levels_per_page, const ColumnDescriptor* d) {
+    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
+        rep_levels_, values_, data_buffer_, pages_, Encoding::PLAIN);
+    num_levels_ = num_pages * levels_per_page;
+    InitReader(d);
+    CheckResults();
+    Clear();
+
+    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
+        rep_levels_, values_, data_buffer_, pages_, Encoding::PLAIN);
+    num_levels_ = num_pages * levels_per_page;
+    InitReader(d);
+    CheckResultsSpaced();
+    Clear();
+  }
+
+  void ExecuteDict(int num_pages, int levels_per_page, const ColumnDescriptor* d) {
+    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
+        rep_levels_, values_, data_buffer_, pages_, Encoding::RLE_DICTIONARY);
+    num_levels_ = num_pages * levels_per_page;
+    InitReader(d);
+    CheckResults();
+    Clear();
+
+    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
+        rep_levels_, values_, data_buffer_, pages_, Encoding::RLE_DICTIONARY);
+    num_levels_ = num_pages * levels_per_page;
+    InitReader(d);
+    CheckResultsSpaced();
+    Clear();
+  }
+
+ protected:
+  int num_levels_;
+  int num_values_;
+  int16_t max_def_level_;
+  int16_t max_rep_level_;
+  vector<shared_ptr<Page>> pages_;
+  std::shared_ptr<ColumnReader> reader_;
+  vector<int32_t> values_;
+  vector<int16_t> def_levels_;
+  vector<int16_t> rep_levels_;
+  vector<uint8_t> data_buffer_;  // For BA and FLBA
+};
+
+TEST_F(TestPrimitiveReader, TestInt32FlatRequired) {
+  int levels_per_page = 100;
+  int num_pages = 50;
+  max_def_level_ = 0;
+  max_rep_level_ = 0;
+  NodePtr type = schema::Int32("a", Repetition::REQUIRED);
+  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
+  ExecutePlain(num_pages, levels_per_page, &descr);
+  ExecuteDict(num_pages, levels_per_page, &descr);
+}
+
+TEST_F(TestPrimitiveReader, TestInt32FlatOptional) {
+  int levels_per_page = 100;
+  int num_pages = 50;
+  max_def_level_ = 4;
+  max_rep_level_ = 0;
+  NodePtr type = schema::Int32("b", Repetition::OPTIONAL);
+  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
+  ExecutePlain(num_pages, levels_per_page, &descr);
+  ExecuteDict(num_pages, levels_per_page, &descr);
+}
+
+TEST_F(TestPrimitiveReader, TestInt32FlatRepeated) {
+  int levels_per_page = 100;
+  int num_pages = 50;
+  max_def_level_ = 4;
+  max_rep_level_ = 2;
+  NodePtr type = schema::Int32("c", Repetition::REPEATED);
+  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
+  ExecutePlain(num_pages, levels_per_page, &descr);
+  ExecuteDict(num_pages, levels_per_page, &descr);
+}
+
+TEST_F(TestPrimitiveReader, TestInt32FlatRequiredSkip) {
+  int levels_per_page = 100;
+  int num_pages = 5;
+  max_def_level_ = 0;
+  max_rep_level_ = 0;
+  NodePtr type = schema::Int32("b", Repetition::REQUIRED);
+  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
+  MakePages<Int32Type>(&descr, num_pages, levels_per_page, def_levels_, rep_levels_,
+      values_, data_buffer_, pages_, Encoding::PLAIN);
+  InitReader(&descr);
+  vector<int32_t> vresult(levels_per_page / 2, -1);
+  vector<int16_t> dresult(levels_per_page / 2, -1);
+  vector<int16_t> rresult(levels_per_page / 2, -1);
+
+  Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
+  int64_t values_read = 0;
+
+  // 1) skip_size > page_size (multiple pages skipped)
+  // Skip first 2 pages
+  int64_t levels_skipped = reader->Skip(2 * levels_per_page);
+  ASSERT_EQ(2 * levels_per_page, levels_skipped);
+  // Read half a page
+  reader->ReadBatch(
+      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
+  vector<int32_t> sub_values(values_.begin() + 2 * levels_per_page,
+      values_.begin() + static_cast<int>(2.5 * static_cast<double>(levels_per_page)));
+  ASSERT_TRUE(vector_equal(sub_values, vresult));
+
+  // 2) skip_size == page_size (skip across two pages)
+  levels_skipped = reader->Skip(levels_per_page);
+  ASSERT_EQ(levels_per_page, levels_skipped);
+  // Read half a page
+  reader->ReadBatch(
+      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
+  sub_values.clear();
+  sub_values.insert(sub_values.end(),
+      values_.begin() + static_cast<int>(3.5 * static_cast<double>(levels_per_page)),
+      values_.begin() + 4 * levels_per_page);
+  ASSERT_TRUE(vector_equal(sub_values, vresult));
+
+  // 3) skip_size < page_size (skip limited to a single page)
+  // Skip half a page
+  levels_skipped = reader->Skip(levels_per_page / 2);
+  ASSERT_EQ(0.5 * levels_per_page, levels_skipped);
+  // Read half a page
+  reader->ReadBatch(
+      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
+  sub_values.clear();
+  sub_values.insert(sub_values.end(),
+      values_.begin() + static_cast<int>(4.5 * static_cast<double>(levels_per_page)),
+      values_.end());
+  ASSERT_TRUE(vector_equal(sub_values, vresult));
+
+  values_.clear();
+  def_levels_.clear();
+  rep_levels_.clear();
+  pages_.clear();
+  reader_.reset();
+}
+
+TEST_F(TestPrimitiveReader, TestDictionaryEncodedPages) {
+  max_def_level_ = 0;
+  max_rep_level_ = 0;
+  NodePtr type = schema::Int32("a", Repetition::REQUIRED);
+  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
+  shared_ptr<PoolBuffer> dummy = std::make_shared<PoolBuffer>();
+
+  shared_ptr<DictionaryPage> dict_page =
+      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN);
+  shared_ptr<DataPage> data_page = MakeDataPage<Int32Type>(
+      &descr, {}, 0, Encoding::RLE_DICTIONARY, {}, 0, {}, 0, {}, 0);
+  pages_.push_back(dict_page);
+  pages_.push_back(data_page);
+  InitReader(&descr);
+  // Tests Dict : PLAIN, Data : RLE_DICTIONARY
+  ASSERT_NO_THROW(reader_->HasNext());
+  pages_.clear();
+
+  dict_page = std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN_DICTIONARY);
+  data_page = MakeDataPage<Int32Type>(
+      &descr, {}, 0, Encoding::PLAIN_DICTIONARY, {}, 0, {}, 0, {}, 0);
+  pages_.push_back(dict_page);
+  pages_.push_back(data_page);
+  InitReader(&descr);
+  // Tests Dict : PLAIN_DICTIONARY, Data : PLAIN_DICTIONARY
+  ASSERT_NO_THROW(reader_->HasNext());
+  pages_.clear();
+
+  data_page = MakeDataPage<Int32Type>(
+      &descr, {}, 0, Encoding::RLE_DICTIONARY, {}, 0, {}, 0, {}, 0);
+  pages_.push_back(data_page);
+  InitReader(&descr);
+  // Tests dictionary page must occur before data page
+  ASSERT_THROW(reader_->HasNext(), ParquetException);
+  pages_.clear();
+
+  dict_page = std::make_shared<DictionaryPage>(dummy, 0, Encoding::DELTA_BYTE_ARRAY);
+  pages_.push_back(dict_page);
+  InitReader(&descr);
+  // Tests only RLE_DICTIONARY is supported
+  ASSERT_THROW(reader_->HasNext(), ParquetException);
+  pages_.clear();
+
+  shared_ptr<DictionaryPage> dict_page1 =
+      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN_DICTIONARY);
+  shared_ptr<DictionaryPage> dict_page2 =
+      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN);
+  pages_.push_back(dict_page1);
+  pages_.push_back(dict_page2);
+  InitReader(&descr);
+  // Column cannot have more than one dictionary
+  ASSERT_THROW(reader_->HasNext(), ParquetException);
+  pages_.clear();
+
+  data_page = MakeDataPage<Int32Type>(
+      &descr, {}, 0, Encoding::DELTA_BYTE_ARRAY, {}, 0, {}, 0, {}, 0);
+  pages_.push_back(data_page);
+  InitReader(&descr);
+  // unsupported encoding
+  ASSERT_THROW(reader_->HasNext(), ParquetException);
+  pages_.clear();
+}
+
+}  // namespace test
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_reader.cc b/src/parquet/column_reader.cc
new file mode 100644
index 0000000..f63f6f1
--- /dev/null
+++ b/src/parquet/column_reader.cc
@@ -0,0 +1,289 @@
+// 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.
+
+#include "parquet/column_reader.h"
+
+#include <algorithm>
+#include <cstdint>
+#include <memory>
+
+#include "parquet/column_page.h"
+#include "parquet/encoding-internal.h"
+#include "parquet/properties.h"
+#include "parquet/util/rle-encoding.h"
+
+using arrow::MemoryPool;
+
+namespace parquet {
+
+LevelDecoder::LevelDecoder() : num_values_remaining_(0) {}
+
+LevelDecoder::~LevelDecoder() {}
+
+int LevelDecoder::SetData(Encoding::type encoding, int16_t max_level,
+    int num_buffered_values, const uint8_t* data) {
+  int32_t num_bytes = 0;
+  encoding_ = encoding;
+  num_values_remaining_ = num_buffered_values;
+  bit_width_ = BitUtil::Log2(max_level + 1);
+  switch (encoding) {
+    case Encoding::RLE: {
+      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_));
+      } else {
+        rle_decoder_->Reset(decoder_data, num_bytes, bit_width_);
+      }
+      return sizeof(int32_t) + num_bytes;
+    }
+    case Encoding::BIT_PACKED: {
+      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));
+      } else {
+        bit_packed_decoder_->Reset(data, num_bytes);
+      }
+      return num_bytes;
+    }
+    default:
+      throw ParquetException("Unknown encoding type for levels.");
+  }
+  return -1;
+}
+
+int LevelDecoder::Decode(int batch_size, int16_t* levels) {
+  int num_decoded = 0;
+
+  int num_values = std::min(num_values_remaining_, batch_size);
+  if (encoding_ == Encoding::RLE) {
+    num_decoded = rle_decoder_->GetBatch(levels, num_values);
+  } else {
+    num_decoded = bit_packed_decoder_->GetBatch(bit_width_, levels, num_values);
+  }
+  num_values_remaining_ -= num_decoded;
+  return num_decoded;
+}
+
+ReaderProperties default_reader_properties() {
+  static ReaderProperties default_reader_properties;
+  return default_reader_properties;
+}
+
+ColumnReader::ColumnReader(
+    const ColumnDescriptor* descr, std::unique_ptr<PageReader> pager, MemoryPool* pool)
+    : descr_(descr),
+      pager_(std::move(pager)),
+      num_buffered_values_(0),
+      num_decoded_values_(0),
+      pool_(pool) {}
+
+ColumnReader::~ColumnReader() {}
+
+template <typename DType>
+void TypedColumnReader<DType>::ConfigureDictionary(const DictionaryPage* page) {
+  int encoding = static_cast<int>(page->encoding());
+  if (page->encoding() == Encoding::PLAIN_DICTIONARY ||
+      page->encoding() == Encoding::PLAIN) {
+    encoding = static_cast<int>(Encoding::RLE_DICTIONARY);
+  }
+
+  auto it = decoders_.find(encoding);
+  if (it != decoders_.end()) {
+    throw ParquetException("Column cannot have more than one dictionary.");
+  }
+
+  if (page->encoding() == Encoding::PLAIN_DICTIONARY ||
+      page->encoding() == Encoding::PLAIN) {
+    PlainDecoder<DType> dictionary(descr_);
+    dictionary.SetData(page->num_values(), page->data(), page->size());
+
+    // The dictionary is fully decoded during DictionaryDecoder::Init, so the
+    // DictionaryPage buffer is no longer required after this step
+    //
+    // TODO(wesm): investigate whether this all-or-nothing decoding of the
+    // dictionary makes sense and whether performance can be improved
+
+    auto decoder = std::make_shared<DictionaryDecoder<DType>>(descr_, pool_);
+    decoder->SetDict(&dictionary);
+    decoders_[encoding] = decoder;
+  } else {
+    ParquetException::NYI("only plain dictionary encoding has been implemented");
+  }
+
+  current_decoder_ = decoders_[encoding].get();
+}
+
+// PLAIN_DICTIONARY is deprecated but used to be used as a dictionary index
+// encoding.
+static bool IsDictionaryIndexEncoding(const Encoding::type& e) {
+  return e == Encoding::RLE_DICTIONARY || e == Encoding::PLAIN_DICTIONARY;
+}
+
+template <typename DType>
+bool TypedColumnReader<DType>::ReadNewPage() {
+  // Loop until we find the next data page.
+  const uint8_t* buffer;
+
+  while (true) {
+    current_page_ = pager_->NextPage();
+    if (!current_page_) {
+      // EOS
+      return false;
+    }
+
+    if (current_page_->type() == PageType::DICTIONARY_PAGE) {
+      ConfigureDictionary(static_cast<const DictionaryPage*>(current_page_.get()));
+      continue;
+    } else if (current_page_->type() == PageType::DATA_PAGE) {
+      const DataPage* page = static_cast<const DataPage*>(current_page_.get());
+
+      // Read a data page.
+      num_buffered_values_ = page->num_values();
+
+      // Have not decoded any values from the data page yet
+      num_decoded_values_ = 0;
+
+      buffer = page->data();
+
+      // If the data page includes repetition and definition levels, we
+      // initialize the level decoder and subtract the encoded level bytes from
+      // the page size to determine the number of bytes in the encoded data.
+      int64_t data_size = page->size();
+
+      // Data page Layout: Repetition Levels - Definition Levels - encoded values.
+      // Levels are encoded as rle or bit-packed.
+      // Init repetition levels
+      if (descr_->max_repetition_level() > 0) {
+        int64_t rep_levels_bytes = repetition_level_decoder_.SetData(
+            page->repetition_level_encoding(), descr_->max_repetition_level(),
+            static_cast<int>(num_buffered_values_), buffer);
+        buffer += rep_levels_bytes;
+        data_size -= rep_levels_bytes;
+      }
+      // TODO figure a way to set max_definition_level_ to 0
+      // if the initial value is invalid
+
+      // Init definition levels
+      if (descr_->max_definition_level() > 0) {
+        int64_t def_levels_bytes = definition_level_decoder_.SetData(
+            page->definition_level_encoding(), descr_->max_definition_level(),
+            static_cast<int>(num_buffered_values_), buffer);
+        buffer += def_levels_bytes;
+        data_size -= def_levels_bytes;
+      }
+
+      // Get a decoder object for this page or create a new decoder if this is the
+      // first page with this encoding.
+      Encoding::type encoding = page->encoding();
+
+      if (IsDictionaryIndexEncoding(encoding)) { encoding = Encoding::RLE_DICTIONARY; }
+
+      auto it = decoders_.find(static_cast<int>(encoding));
+      if (it != decoders_.end()) {
+        if (encoding == Encoding::RLE_DICTIONARY) {
+          DCHECK(current_decoder_->encoding() == Encoding::RLE_DICTIONARY);
+        }
+        current_decoder_ = it->second.get();
+      } else {
+        switch (encoding) {
+          case Encoding::PLAIN: {
+            std::shared_ptr<DecoderType> decoder(new PlainDecoder<DType>(descr_));
+            decoders_[static_cast<int>(encoding)] = decoder;
+            current_decoder_ = decoder.get();
+            break;
+          }
+          case Encoding::RLE_DICTIONARY:
+            throw ParquetException("Dictionary page must be before data page.");
+
+          case Encoding::DELTA_BINARY_PACKED:
+          case Encoding::DELTA_LENGTH_BYTE_ARRAY:
+          case Encoding::DELTA_BYTE_ARRAY:
+            ParquetException::NYI("Unsupported encoding");
+
+          default:
+            throw ParquetException("Unknown encoding type.");
+        }
+      }
+      current_decoder_->SetData(
+          static_cast<int>(num_buffered_values_), buffer, static_cast<int>(data_size));
+      return true;
+    } else {
+      // We don't know what this page type is. We're allowed to skip non-data
+      // pages.
+      continue;
+    }
+  }
+  return true;
+}
+
+// ----------------------------------------------------------------------
+// Batch read APIs
+
+int64_t ColumnReader::ReadDefinitionLevels(int64_t batch_size, int16_t* levels) {
+  if (descr_->max_definition_level() == 0) { return 0; }
+  return definition_level_decoder_.Decode(static_cast<int>(batch_size), levels);
+}
+
+int64_t ColumnReader::ReadRepetitionLevels(int64_t batch_size, int16_t* levels) {
+  if (descr_->max_repetition_level() == 0) { return 0; }
+  return repetition_level_decoder_.Decode(static_cast<int>(batch_size), levels);
+}
+
+// ----------------------------------------------------------------------
+// Dynamic column reader constructor
+
+std::shared_ptr<ColumnReader> ColumnReader::Make(
+    const ColumnDescriptor* descr, std::unique_ptr<PageReader> pager, MemoryPool* pool) {
+  switch (descr->physical_type()) {
+    case Type::BOOLEAN:
+      return std::make_shared<BoolReader>(descr, std::move(pager), pool);
+    case Type::INT32:
+      return std::make_shared<Int32Reader>(descr, std::move(pager), pool);
+    case Type::INT64:
+      return std::make_shared<Int64Reader>(descr, std::move(pager), pool);
+    case Type::INT96:
+      return std::make_shared<Int96Reader>(descr, std::move(pager), pool);
+    case Type::FLOAT:
+      return std::make_shared<FloatReader>(descr, std::move(pager), pool);
+    case Type::DOUBLE:
+      return std::make_shared<DoubleReader>(descr, std::move(pager), pool);
+    case Type::BYTE_ARRAY:
+      return std::make_shared<ByteArrayReader>(descr, std::move(pager), pool);
+    case Type::FIXED_LEN_BYTE_ARRAY:
+      return std::make_shared<FixedLenByteArrayReader>(descr, std::move(pager), pool);
+    default:
+      ParquetException::NYI("type reader not implemented");
+  }
+  // Unreachable code, but supress compiler warning
+  return std::shared_ptr<ColumnReader>(nullptr);
+}
+
+// ----------------------------------------------------------------------
+// Instantiate templated classes
+
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<BooleanType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int32Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int64Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<Int96Type>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<FloatType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<DoubleType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<ByteArrayType>;
+template class PARQUET_TEMPLATE_EXPORT TypedColumnReader<FLBAType>;
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_reader.h b/src/parquet/column_reader.h
new file mode 100644
index 0000000..f4b8b02
--- /dev/null
+++ b/src/parquet/column_reader.h
@@ -0,0 +1,475 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef PARQUET_COLUMN_READER_H
+#define PARQUET_COLUMN_READER_H
+
+#include <algorithm>
+#include <cstdint>
+#include <cstring>
+#include <iostream>
+#include <memory>
+#include <unordered_map>
+#include <vector>
+
+#include <arrow/util/bit-util.h>
+
+#include "parquet/column_page.h"
+#include "parquet/encoding.h"
+#include "parquet/exception.h"
+#include "parquet/schema.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/visibility.h"
+
+namespace parquet {
+
+class BitReader;
+class RleDecoder;
+
+class PARQUET_EXPORT LevelDecoder {
+ public:
+  LevelDecoder();
+  ~LevelDecoder();
+
+  // Initialize the LevelDecoder state with new data
+  // and return the number of bytes consumed
+  int SetData(Encoding::type encoding, int16_t max_level, int num_buffered_values,
+      const uint8_t* data);
+
+  // Decodes a batch of levels into an array and returns the number of levels decoded
+  int Decode(int batch_size, int16_t* levels);
+
+ private:
+  int bit_width_;
+  int num_values_remaining_;
+  Encoding::type encoding_;
+  std::unique_ptr<RleDecoder> rle_decoder_;
+  std::unique_ptr<BitReader> bit_packed_decoder_;
+};
+
+class PARQUET_EXPORT ColumnReader {
+ public:
+  ColumnReader(const ColumnDescriptor*, std::unique_ptr<PageReader>,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
+  virtual ~ColumnReader();
+
+  static std::shared_ptr<ColumnReader> Make(const ColumnDescriptor* descr,
+      std::unique_ptr<PageReader> pager,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
+
+  // Returns true if there are still values in this column.
+  bool HasNext() {
+    // Either there is no data page available yet, or the data page has been
+    // exhausted
+    if (num_buffered_values_ == 0 || num_decoded_values_ == num_buffered_values_) {
+      if (!ReadNewPage() || num_buffered_values_ == 0) { return false; }
+    }
+    return true;
+  }
+
+  Type::type type() const { return descr_->physical_type(); }
+
+  const ColumnDescriptor* descr() const { return descr_; }
+
+ protected:
+  virtual bool ReadNewPage() = 0;
+
+  // Read multiple definition levels into preallocated memory
+  //
+  // Returns the number of decoded definition levels
+  int64_t ReadDefinitionLevels(int64_t batch_size, int16_t* levels);
+
+  // Read multiple repetition levels into preallocated memory
+  // Returns the number of decoded repetition levels
+  int64_t ReadRepetitionLevels(int64_t batch_size, int16_t* levels);
+
+  const ColumnDescriptor* descr_;
+
+  std::unique_ptr<PageReader> pager_;
+  std::shared_ptr<Page> current_page_;
+
+  // Not set if full schema for this field has no optional or repeated elements
+  LevelDecoder definition_level_decoder_;
+
+  // Not set for flat schemas.
+  LevelDecoder repetition_level_decoder_;
+
+  // The total number of values stored in the data page. This is the maximum of
+  // the number of encoded definition levels or encoded values. For
+  // non-repeated, required columns, this is equal to the number of encoded
+  // values. For repeated or optional values, there may be fewer data values
+  // than levels, and this tells you how many encoded levels there are in that
+  // case.
+  int64_t num_buffered_values_;
+
+  // The number of values from the current data page that have been decoded
+  // into memory
+  int64_t num_decoded_values_;
+
+  ::arrow::MemoryPool* pool_;
+};
+
+// API to read values from a single column. This is the main client facing API.
+template <typename DType>
+class PARQUET_EXPORT TypedColumnReader : public ColumnReader {
+ public:
+  typedef typename DType::c_type T;
+
+  TypedColumnReader(const ColumnDescriptor* schema, std::unique_ptr<PageReader> pager,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
+      : ColumnReader(schema, std::move(pager), pool), current_decoder_(NULL) {}
+  virtual ~TypedColumnReader() {}
+
+  // Read a batch of repetition levels, definition levels, and values from the
+  // column.
+  //
+  // Since null values are not stored in the values, the number of values read
+  // may be less than the number of repetition and definition levels. With
+  // nested data this is almost certainly true.
+  //
+  // Set def_levels or rep_levels to nullptr if you want to skip reading them.
+  // This is only safe if you know through some other source that there are no
+  // undefined values.
+  //
+  // To fully exhaust a row group, you must read batches until the number of
+  // values read reaches the number of stored values according to the metadata.
+  //
+  // This API is the same for both V1 and V2 of the DataPage
+  //
+  // @returns: actual number of levels read (see values_read for number of values read)
+  int64_t ReadBatch(int64_t batch_size, int16_t* def_levels, int16_t* rep_levels,
+      T* values, int64_t* values_read);
+
+  /// Read a batch of repetition levels, definition levels, and values from the
+  /// column and leave spaces for null entries on the lowest level in the values
+  /// buffer.
+  ///
+  /// In comparision to ReadBatch the length of repetition and definition levels
+  /// is the same as of the number of values read for max_definition_level == 1.
+  /// In the case of max_definition_level > 1, the repetition and definition
+  /// levels are larger than the values but the values include the null entries
+  /// with definition_level == (max_definition_level - 1).
+  ///
+  /// To fully exhaust a row group, you must read batches until the number of
+  /// values read reaches the number of stored values according to the metadata.
+  ///
+  /// @param batch_size the number of levels to read
+  /// @param[out] def_levels The Parquet definition levels, output has
+  ///   the length levels_read.
+  /// @param[out] rep_levels The Parquet repetition levels, output has
+  ///   the length levels_read.
+  /// @param[out] values The values in the lowest nested level including
+  ///   spacing for nulls on the lowest levels; output has the length
+  ///   values_read.
+  /// @param[out] valid_bits Memory allocated for a bitmap that indicates if
+  ///   the row is null or on the maximum definition level. For performance
+  ///   reasons the underlying buffer should be able to store 1 bit more than
+  ///   required. If this requires an additional byte, this byte is only read
+  ///   but never written to.
+  /// @param valid_bits_offset The offset in bits of the valid_bits where the
+  ///   first relevant bit resides.
+  /// @param[out] levels_read The number of repetition/definition levels that were read.
+  /// @param[out] values_read The number of values read, this includes all
+  ///   non-null entries as well as all null-entries on the lowest level
+  ///   (i.e. definition_level == max_definition_level - 1)
+  /// @param[out] null_count The number of nulls on the lowest levels.
+  ///   (i.e. (values_read - null_count) is total number of non-null entries)
+  int64_t ReadBatchSpaced(int64_t batch_size, int16_t* def_levels, int16_t* rep_levels,
+      T* values, uint8_t* valid_bits, int64_t valid_bits_offset, int64_t* levels_read,
+      int64_t* values_read, int64_t* null_count);
+
+  // Skip reading levels
+  // Returns the number of levels skipped
+  int64_t Skip(int64_t num_rows_to_skip);
+
+ private:
+  typedef Decoder<DType> DecoderType;
+
+  // Advance to the next data page
+  virtual bool ReadNewPage();
+
+  // Read up to batch_size values from the current data page into the
+  // pre-allocated memory T*
+  //
+  // @returns: the number of values read into the out buffer
+  int64_t ReadValues(int64_t batch_size, T* out);
+
+  // Read up to batch_size values from the current data page into the
+  // pre-allocated memory T*, leaving spaces for null entries according
+  // to the def_levels.
+  //
+  // @returns: the number of values read into the out buffer
+  int64_t ReadValuesSpaced(int64_t batch_size, T* out, int null_count,
+      uint8_t* valid_bits, int64_t valid_bits_offset);
+
+  // Map of encoding type to the respective decoder object. For example, a
+  // column chunk's data pages may include both dictionary-encoded and
+  // plain-encoded data.
+  std::unordered_map<int, std::shared_ptr<DecoderType>> decoders_;
+
+  void ConfigureDictionary(const DictionaryPage* page);
+
+  DecoderType* current_decoder_;
+};
+
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadValues(int64_t batch_size, T* out) {
+  int64_t num_decoded = current_decoder_->Decode(out, static_cast<int>(batch_size));
+  return num_decoded;
+}
+
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadValuesSpaced(int64_t batch_size, T* out,
+    int null_count, uint8_t* valid_bits, int64_t valid_bits_offset) {
+  return current_decoder_->DecodeSpaced(
+      out, static_cast<int>(batch_size), null_count, valid_bits, valid_bits_offset);
+}
+
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadBatch(int64_t batch_size,
+    int16_t* def_levels, int16_t* rep_levels, T* values, int64_t* values_read) {
+  // HasNext invokes ReadNewPage
+  if (!HasNext()) {
+    *values_read = 0;
+    return 0;
+  }
+
+  // TODO(wesm): keep reading data pages until batch_size is reached, or the
+  // row group is finished
+  batch_size = std::min(batch_size, num_buffered_values_ - num_decoded_values_);
+
+  int64_t num_def_levels = 0;
+  int64_t num_rep_levels = 0;
+
+  int64_t values_to_read = 0;
+
+  // If the field is required and non-repeated, there are no definition levels
+  if (descr_->max_definition_level() > 0 && def_levels) {
+    num_def_levels = ReadDefinitionLevels(batch_size, def_levels);
+    // TODO(wesm): this tallying of values-to-decode can be performed with better
+    // cache-efficiency if fused with the level decoding.
+    for (int64_t i = 0; i < num_def_levels; ++i) {
+      if (def_levels[i] == descr_->max_definition_level()) { ++values_to_read; }
+    }
+  } else {
+    // Required field, read all values
+    values_to_read = batch_size;
+  }
+
+  // Not present for non-repeated fields
+  if (descr_->max_repetition_level() > 0 && rep_levels) {
+    num_rep_levels = ReadRepetitionLevels(batch_size, rep_levels);
+    if (def_levels && num_def_levels != num_rep_levels) {
+      throw ParquetException("Number of decoded rep / def levels did not match");
+    }
+  }
+
+  *values_read = ReadValues(values_to_read, values);
+  int64_t total_values = std::max(num_def_levels, *values_read);
+  num_decoded_values_ += total_values;
+
+  return total_values;
+}
+
+inline void DefinitionLevelsToBitmap(const int16_t* def_levels, int64_t num_def_levels,
+    int16_t max_definition_level, int16_t max_repetition_level, int64_t* values_read,
+    int64_t* null_count, uint8_t* valid_bits, int64_t valid_bits_offset) {
+  int byte_offset = static_cast<int>(valid_bits_offset) / 8;
+  int bit_offset = static_cast<int>(valid_bits_offset) % 8;
+  uint8_t bitset = valid_bits[byte_offset];
+
+  // TODO(itaiin): As an interim solution we are splitting the code path here
+  // between repeated+flat column reads, and non-repeated+nested reads.
+  // Those paths need to be merged in the future
+  for (int i = 0; i < num_def_levels; ++i) {
+    if (def_levels[i] == max_definition_level) {
+      bitset |= (1 << bit_offset);
+    } else if (max_repetition_level > 0) {
+      // repetition+flat case
+      if (def_levels[i] == (max_definition_level - 1)) {
+        bitset &= ~(1 << bit_offset);
+        *null_count += 1;
+      } else {
+        continue;
+      }
+    } else {
+      // non-repeated+nested case
+      if (def_levels[i] < max_definition_level) {
+        bitset &= ~(1 << bit_offset);
+        *null_count += 1;
+      } else {
+        throw ParquetException("definition level exceeds maximum");
+      }
+    }
+
+    bit_offset++;
+    if (bit_offset == 8) {
+      bit_offset = 0;
+      valid_bits[byte_offset] = bitset;
+      byte_offset++;
+      // TODO: Except for the last byte, this shouldn't be needed
+      bitset = valid_bits[byte_offset];
+    }
+  }
+  if (bit_offset != 0) { valid_bits[byte_offset] = bitset; }
+  *values_read = (bit_offset + byte_offset * 8 - valid_bits_offset);
+}
+
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadBatchSpaced(int64_t batch_size,
+    int16_t* def_levels, int16_t* rep_levels, T* values, uint8_t* valid_bits,
+    int64_t valid_bits_offset, int64_t* levels_read, int64_t* values_read,
+    int64_t* null_count_out) {
+  // HasNext invokes ReadNewPage
+  if (!HasNext()) {
+    *levels_read = 0;
+    *values_read = 0;
+    *null_count_out = 0;
+    return 0;
+  }
+
+  int64_t total_values;
+  // TODO(wesm): keep reading data pages until batch_size is reached, or the
+  // row group is finished
+  batch_size = std::min(batch_size, num_buffered_values_ - num_decoded_values_);
+
+  // If the field is required and non-repeated, there are no definition levels
+  if (descr_->max_definition_level() > 0) {
+    int64_t num_def_levels = ReadDefinitionLevels(batch_size, def_levels);
+
+    // Not present for non-repeated fields
+    if (descr_->max_repetition_level() > 0) {
+      int64_t num_rep_levels = ReadRepetitionLevels(batch_size, rep_levels);
+      if (num_def_levels != num_rep_levels) {
+        throw ParquetException("Number of decoded rep / def levels did not match");
+      }
+    }
+
+    // TODO(itaiin): another code path split to merge when the general case is done
+    bool has_spaced_values;
+    if (descr_->max_repetition_level() > 0) {
+      // repeated+flat case
+      has_spaced_values = !descr_->schema_node()->is_required();
+    } else {
+      // non-repeated+nested case
+      // Find if a node forces nulls in the lowest level along the hierarchy
+      const schema::Node* node = descr_->schema_node().get();
+      has_spaced_values = false;
+      while (node) {
+        auto parent = node->parent();
+        if (node->is_optional()) {
+          has_spaced_values = true;
+          break;
+        }
+        node = parent;
+      }
+    }
+
+    int64_t null_count = 0;
+    if (!has_spaced_values) {
+      int values_to_read = 0;
+      for (int64_t i = 0; i < num_def_levels; ++i) {
+        if (def_levels[i] == descr_->max_definition_level()) { ++values_to_read; }
+      }
+      total_values = ReadValues(values_to_read, values);
+      for (int64_t i = 0; i < total_values; i++) {
+        ::arrow::BitUtil::SetBit(valid_bits, valid_bits_offset + i);
+      }
+      *values_read = total_values;
+    } else {
+      int16_t max_definition_level = descr_->max_definition_level();
+      int16_t max_repetition_level = descr_->max_repetition_level();
+      DefinitionLevelsToBitmap(def_levels, num_def_levels, max_definition_level,
+          max_repetition_level, values_read, &null_count, valid_bits, valid_bits_offset);
+      total_values = ReadValuesSpaced(*values_read, values, static_cast<int>(null_count),
+          valid_bits, valid_bits_offset);
+    }
+    *levels_read = num_def_levels;
+    *null_count_out = null_count;
+
+  } else {
+    // Required field, read all values
+    total_values = ReadValues(batch_size, values);
+    for (int64_t i = 0; i < total_values; i++) {
+      ::arrow::BitUtil::SetBit(valid_bits, valid_bits_offset + i);
+    }
+    *null_count_out = 0;
+    *levels_read = total_values;
+  }
+
+  num_decoded_values_ += *levels_read;
+  return total_values;
+}
+
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::Skip(int64_t num_rows_to_skip) {
+  int64_t rows_to_skip = num_rows_to_skip;
+  while (HasNext() && rows_to_skip > 0) {
+    // If the number of rows to skip is more than the number of undecoded values, skip the
+    // Page.
+    if (rows_to_skip > (num_buffered_values_ - num_decoded_values_)) {
+      rows_to_skip -= num_buffered_values_ - num_decoded_values_;
+      num_decoded_values_ = num_buffered_values_;
+    } else {
+      // We need to read this Page
+      // Jump to the right offset in the Page
+      int64_t batch_size = 1024;  // ReadBatch with a smaller memory footprint
+      int64_t values_read = 0;
+
+      std::shared_ptr<PoolBuffer> vals = AllocateBuffer(
+          this->pool_, batch_size * type_traits<DType::type_num>::value_byte_size);
+      std::shared_ptr<PoolBuffer> def_levels =
+          AllocateBuffer(this->pool_, batch_size * sizeof(int16_t));
+
+      std::shared_ptr<PoolBuffer> rep_levels =
+          AllocateBuffer(this->pool_, batch_size * sizeof(int16_t));
+
+      do {
+        batch_size = std::min(batch_size, rows_to_skip);
+        values_read = ReadBatch(static_cast<int>(batch_size),
+            reinterpret_cast<int16_t*>(def_levels->mutable_data()),
+            reinterpret_cast<int16_t*>(rep_levels->mutable_data()),
+            reinterpret_cast<T*>(vals->mutable_data()), &values_read);
+        rows_to_skip -= values_read;
+      } while (values_read > 0 && rows_to_skip > 0);
+    }
+  }
+  return num_rows_to_skip - rows_to_skip;
+}
+
+typedef TypedColumnReader<BooleanType> BoolReader;
+typedef TypedColumnReader<Int32Type> Int32Reader;
+typedef TypedColumnReader<Int64Type> Int64Reader;
+typedef TypedColumnReader<Int96Type> Int96Reader;
+typedef TypedColumnReader<FloatType> FloatReader;
+typedef TypedColumnReader<DoubleType> DoubleReader;
+typedef TypedColumnReader<ByteArrayType> ByteArrayReader;
+typedef TypedColumnReader<FLBAType> FixedLenByteArrayReader;
+
+extern template class PARQUET_EXPORT TypedColumnReader<BooleanType>;
+extern template class PARQUET_EXPORT TypedColumnReader<Int32Type>;
+extern template class PARQUET_EXPORT TypedColumnReader<Int64Type>;
+extern template class PARQUET_EXPORT TypedColumnReader<Int96Type>;
+extern template class PARQUET_EXPORT TypedColumnReader<FloatType>;
+extern template class PARQUET_EXPORT TypedColumnReader<DoubleType>;
+extern template class PARQUET_EXPORT TypedColumnReader<ByteArrayType>;
+extern template class PARQUET_EXPORT TypedColumnReader<FLBAType>;
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_READER_H

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_scanner-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_scanner-test.cc b/src/parquet/column_scanner-test.cc
new file mode 100644
index 0000000..086722b
--- /dev/null
+++ b/src/parquet/column_scanner-test.cc
@@ -0,0 +1,232 @@
+// 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.
+
+#include <gtest/gtest.h>
+
+#include <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "parquet/column_page.h"
+#include "parquet/column_scanner.h"
+#include "parquet/schema.h"
+#include "parquet/test-specialization.h"
+#include "parquet/test-util.h"
+#include "parquet/types.h"
+#include "parquet/util/test-common.h"
+
+using std::string;
+using std::vector;
+using std::shared_ptr;
+
+namespace parquet {
+
+using schema::NodePtr;
+
+namespace test {
+
+template <>
+void InitDictValues<bool>(
+    int num_values, int dict_per_page, vector<bool>& values, vector<uint8_t>& buffer) {
+  // No op for bool
+}
+
+template <typename Type>
+class TestFlatScanner : public ::testing::Test {
+ public:
+  typedef typename Type::c_type T;
+
+  void InitScanner(const ColumnDescriptor* d) {
+    std::unique_ptr<PageReader> pager(new test::MockPageReader(pages_));
+    scanner_ = Scanner::Make(ColumnReader::Make(d, std::move(pager)));
+  }
+
+  void CheckResults(int batch_size, const ColumnDescriptor* d) {
+    TypedScanner<Type>* scanner = reinterpret_cast<TypedScanner<Type>*>(scanner_.get());
+    T val;
+    bool is_null = false;
+    int16_t def_level;
+    int16_t rep_level;
+    int j = 0;
+    scanner->SetBatchSize(batch_size);
+    for (int i = 0; i < num_levels_; i++) {
+      ASSERT_TRUE(scanner->Next(&val, &def_level, &rep_level, &is_null)) << i << j;
+      if (!is_null) {
+        ASSERT_EQ(values_[j], val) << i << "V" << j;
+        j++;
+      }
+      if (d->max_definition_level() > 0) {
+        ASSERT_EQ(def_levels_[i], def_level) << i << "D" << j;
+      }
+      if (d->max_repetition_level() > 0) {
+        ASSERT_EQ(rep_levels_[i], rep_level) << i << "R" << j;
+      }
+    }
+    ASSERT_EQ(num_values_, j);
+    ASSERT_FALSE(scanner->Next(&val, &def_level, &rep_level, &is_null));
+  }
+
+  void Clear() {
+    pages_.clear();
+    values_.clear();
+    def_levels_.clear();
+    rep_levels_.clear();
+  }
+
+  void Execute(int num_pages, int levels_per_page, int batch_size,
+      const ColumnDescriptor* d, Encoding::type encoding) {
+    num_values_ = MakePages<Type>(d, num_pages, levels_per_page, def_levels_, rep_levels_,
+        values_, data_buffer_, pages_, encoding);
+    num_levels_ = num_pages * levels_per_page;
+    InitScanner(d);
+    CheckResults(batch_size, d);
+    Clear();
+  }
+
+  void InitDescriptors(std::shared_ptr<ColumnDescriptor>& d1,
+      std::shared_ptr<ColumnDescriptor>& d2, std::shared_ptr<ColumnDescriptor>& d3,
+      int length) {
+    NodePtr type;
+    type = schema::PrimitiveNode::Make(
+        "c1", Repetition::REQUIRED, Type::type_num, LogicalType::NONE, length);
+    d1.reset(new ColumnDescriptor(type, 0, 0));
+    type = schema::PrimitiveNode::Make(
+        "c2", Repetition::OPTIONAL, Type::type_num, LogicalType::NONE, length);
+    d2.reset(new ColumnDescriptor(type, 4, 0));
+    type = schema::PrimitiveNode::Make(
+        "c3", Repetition::REPEATED, Type::type_num, LogicalType::NONE, length);
+    d3.reset(new ColumnDescriptor(type, 4, 2));
+  }
+
+  void ExecuteAll(int num_pages, int num_levels, int batch_size, int type_length,
+      Encoding::type encoding = Encoding::PLAIN) {
+    std::shared_ptr<ColumnDescriptor> d1;
+    std::shared_ptr<ColumnDescriptor> d2;
+    std::shared_ptr<ColumnDescriptor> d3;
+    InitDescriptors(d1, d2, d3, type_length);
+    // evaluate REQUIRED pages
+    Execute(num_pages, num_levels, batch_size, d1.get(), encoding);
+    // evaluate OPTIONAL pages
+    Execute(num_pages, num_levels, batch_size, d2.get(), encoding);
+    // evaluate REPEATED pages
+    Execute(num_pages, num_levels, batch_size, d3.get(), encoding);
+  }
+
+ protected:
+  int num_levels_;
+  int num_values_;
+  vector<shared_ptr<Page>> pages_;
+  std::shared_ptr<Scanner> scanner_;
+  vector<T> values_;
+  vector<int16_t> def_levels_;
+  vector<int16_t> rep_levels_;
+  vector<uint8_t> data_buffer_;  // For BA and FLBA
+};
+
+static int num_levels_per_page = 100;
+static int num_pages = 20;
+static int batch_size = 32;
+
+typedef ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
+    ByteArrayType>
+    TestTypes;
+
+using TestBooleanFlatScanner = TestFlatScanner<BooleanType>;
+using TestFLBAFlatScanner = TestFlatScanner<FLBAType>;
+
+TYPED_TEST_CASE(TestFlatScanner, TestTypes);
+
+TYPED_TEST(TestFlatScanner, TestPlainScanner) {
+  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0, Encoding::PLAIN);
+}
+
+TYPED_TEST(TestFlatScanner, TestDictScanner) {
+  this->ExecuteAll(
+      num_pages, num_levels_per_page, batch_size, 0, Encoding::RLE_DICTIONARY);
+}
+
+TEST_F(TestBooleanFlatScanner, TestPlainScanner) {
+  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, 0);
+}
+
+TEST_F(TestFLBAFlatScanner, TestPlainScanner) {
+  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH);
+}
+
+TEST_F(TestFLBAFlatScanner, TestDictScanner) {
+  this->ExecuteAll(
+      num_pages, num_levels_per_page, batch_size, FLBA_LENGTH, Encoding::RLE_DICTIONARY);
+}
+
+TEST_F(TestFLBAFlatScanner, TestPlainDictScanner) {
+  this->ExecuteAll(num_pages, num_levels_per_page, batch_size, FLBA_LENGTH,
+      Encoding::PLAIN_DICTIONARY);
+}
+
+// PARQUET 502
+TEST_F(TestFLBAFlatScanner, TestSmallBatch) {
+  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::REQUIRED,
+      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
+  const ColumnDescriptor d(type, 0, 0);
+  num_values_ = MakePages<FLBAType>(
+      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
+  num_levels_ = 1 * 100;
+  InitScanner(&d);
+  CheckResults(1, &d);
+}
+
+TEST_F(TestFLBAFlatScanner, TestDescriptorAPI) {
+  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::OPTIONAL,
+      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
+  const ColumnDescriptor d(type, 4, 0);
+  num_values_ = MakePages<FLBAType>(
+      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
+  num_levels_ = 1 * 100;
+  InitScanner(&d);
+  TypedScanner<FLBAType>* scanner =
+      reinterpret_cast<TypedScanner<FLBAType>*>(scanner_.get());
+  ASSERT_EQ(10, scanner->descr()->type_precision());
+  ASSERT_EQ(2, scanner->descr()->type_scale());
+  ASSERT_EQ(FLBA_LENGTH, scanner->descr()->type_length());
+}
+
+TEST_F(TestFLBAFlatScanner, TestFLBAPrinterNext) {
+  NodePtr type = schema::PrimitiveNode::Make("c1", Repetition::OPTIONAL,
+      Type::FIXED_LEN_BYTE_ARRAY, LogicalType::DECIMAL, FLBA_LENGTH, 10, 2);
+  const ColumnDescriptor d(type, 4, 0);
+  num_values_ = MakePages<FLBAType>(
+      &d, 1, 100, def_levels_, rep_levels_, values_, data_buffer_, pages_);
+  num_levels_ = 1 * 100;
+  InitScanner(&d);
+  TypedScanner<FLBAType>* scanner =
+      reinterpret_cast<TypedScanner<FLBAType>*>(scanner_.get());
+  scanner->SetBatchSize(batch_size);
+  std::stringstream ss_fail;
+  for (int i = 0; i < num_levels_; i++) {
+    std::stringstream ss;
+    scanner->PrintNext(ss, 17);
+    std::string result = ss.str();
+    ASSERT_LE(17, result.size()) << i;
+  }
+  ASSERT_THROW(scanner->PrintNext(ss_fail, 17), ParquetException);
+}
+
+}  // namespace test
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_scanner.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column_scanner.cc b/src/parquet/column_scanner.cc
new file mode 100644
index 0000000..a67af71
--- /dev/null
+++ b/src/parquet/column_scanner.cc
@@ -0,0 +1,90 @@
+// 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.
+
+#include "parquet/column_scanner.h"
+
+#include <cstdint>
+#include <memory>
+
+#include "parquet/column_reader.h"
+#include "parquet/util/memory.h"
+
+using arrow::MemoryPool;
+
+namespace parquet {
+
+std::shared_ptr<Scanner> Scanner::Make(
+    std::shared_ptr<ColumnReader> col_reader, int64_t batch_size, MemoryPool* pool) {
+  switch (col_reader->type()) {
+    case Type::BOOLEAN:
+      return std::make_shared<BoolScanner>(col_reader, batch_size, pool);
+    case Type::INT32:
+      return std::make_shared<Int32Scanner>(col_reader, batch_size, pool);
+    case Type::INT64:
+      return std::make_shared<Int64Scanner>(col_reader, batch_size, pool);
+    case Type::INT96:
+      return std::make_shared<Int96Scanner>(col_reader, batch_size, pool);
+    case Type::FLOAT:
+      return std::make_shared<FloatScanner>(col_reader, batch_size, pool);
+    case Type::DOUBLE:
+      return std::make_shared<DoubleScanner>(col_reader, batch_size, pool);
+    case Type::BYTE_ARRAY:
+      return std::make_shared<ByteArrayScanner>(col_reader, batch_size, pool);
+    case Type::FIXED_LEN_BYTE_ARRAY:
+      return std::make_shared<FixedLenByteArrayScanner>(col_reader, batch_size, pool);
+    default:
+      ParquetException::NYI("type reader not implemented");
+  }
+  // Unreachable code, but supress compiler warning
+  return std::shared_ptr<Scanner>(nullptr);
+}
+
+int64_t ScanAllValues(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
+    uint8_t* values, int64_t* values_buffered, parquet::ColumnReader* reader) {
+  switch (reader->type()) {
+    case parquet::Type::BOOLEAN:
+      return ScanAll<parquet::BoolReader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::INT32:
+      return ScanAll<parquet::Int32Reader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::INT64:
+      return ScanAll<parquet::Int64Reader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::INT96:
+      return ScanAll<parquet::Int96Reader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::FLOAT:
+      return ScanAll<parquet::FloatReader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::DOUBLE:
+      return ScanAll<parquet::DoubleReader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::BYTE_ARRAY:
+      return ScanAll<parquet::ByteArrayReader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    case parquet::Type::FIXED_LEN_BYTE_ARRAY:
+      return ScanAll<parquet::FixedLenByteArrayReader>(
+          batch_size, def_levels, rep_levels, values, values_buffered, reader);
+    default:
+      parquet::ParquetException::NYI("type reader not implemented");
+  }
+  // Unreachable code, but supress compiler warning
+  return 0;
+}
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column_scanner.h
----------------------------------------------------------------------
diff --git a/src/parquet/column_scanner.h b/src/parquet/column_scanner.h
new file mode 100644
index 0000000..4be0b7f
--- /dev/null
+++ b/src/parquet/column_scanner.h
@@ -0,0 +1,246 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef PARQUET_COLUMN_SCANNER_H
+#define PARQUET_COLUMN_SCANNER_H
+
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <stdio.h>
+#include <string>
+#include <vector>
+
+#include "parquet/column_reader.h"
+#include "parquet/exception.h"
+#include "parquet/schema.h"
+#include "parquet/types.h"
+#include "parquet/util/memory.h"
+#include "parquet/util/visibility.h"
+
+namespace parquet {
+
+static constexpr int64_t DEFAULT_SCANNER_BATCH_SIZE = 128;
+
+class PARQUET_EXPORT Scanner {
+ public:
+  explicit Scanner(std::shared_ptr<ColumnReader> reader,
+      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
+      : batch_size_(batch_size),
+        level_offset_(0),
+        levels_buffered_(0),
+        value_buffer_(std::make_shared<PoolBuffer>(pool)),
+        value_offset_(0),
+        values_buffered_(0),
+        reader_(reader) {
+    def_levels_.resize(descr()->max_definition_level() > 0 ? batch_size_ : 0);
+    rep_levels_.resize(descr()->max_repetition_level() > 0 ? batch_size_ : 0);
+  }
+
+  virtual ~Scanner() {}
+
+  static std::shared_ptr<Scanner> Make(std::shared_ptr<ColumnReader> col_reader,
+      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool());
+
+  virtual void PrintNext(std::ostream& out, int width) = 0;
+
+  bool HasNext() { return level_offset_ < levels_buffered_ || reader_->HasNext(); }
+
+  const ColumnDescriptor* descr() const { return reader_->descr(); }
+
+  int64_t batch_size() const { return batch_size_; }
+
+  void SetBatchSize(int64_t batch_size) { batch_size_ = batch_size; }
+
+ protected:
+  int64_t batch_size_;
+
+  std::vector<int16_t> def_levels_;
+  std::vector<int16_t> rep_levels_;
+  int level_offset_;
+  int levels_buffered_;
+
+  std::shared_ptr<PoolBuffer> value_buffer_;
+  int value_offset_;
+  int64_t values_buffered_;
+
+ private:
+  std::shared_ptr<ColumnReader> reader_;
+};
+
+template <typename DType>
+class PARQUET_EXPORT TypedScanner : public Scanner {
+ public:
+  typedef typename DType::c_type T;
+
+  explicit TypedScanner(std::shared_ptr<ColumnReader> reader,
+      int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
+      ::arrow::MemoryPool* pool = ::arrow::default_memory_pool())
+      : Scanner(reader, batch_size, pool) {
+    typed_reader_ = static_cast<TypedColumnReader<DType>*>(reader.get());
+    int value_byte_size = type_traits<DType::type_num>::value_byte_size;
+    PARQUET_THROW_NOT_OK(value_buffer_->Resize(batch_size_ * value_byte_size));
+    values_ = reinterpret_cast<T*>(value_buffer_->mutable_data());
+  }
+
+  virtual ~TypedScanner() {}
+
+  bool NextLevels(int16_t* def_level, int16_t* rep_level) {
+    if (level_offset_ == levels_buffered_) {
+      levels_buffered_ =
+          static_cast<int>(typed_reader_->ReadBatch(static_cast<int>(batch_size_),
+              def_levels_.data(), rep_levels_.data(), values_, &values_buffered_));
+
+      value_offset_ = 0;
+      level_offset_ = 0;
+      if (!levels_buffered_) { return false; }
+    }
+    *def_level = descr()->max_definition_level() > 0 ? def_levels_[level_offset_] : 0;
+    *rep_level = descr()->max_repetition_level() > 0 ? rep_levels_[level_offset_] : 0;
+    level_offset_++;
+    return true;
+  }
+
+  bool Next(T* val, int16_t* def_level, int16_t* rep_level, bool* is_null) {
+    if (level_offset_ == levels_buffered_) {
+      if (!HasNext()) {
+        // Out of data pages
+        return false;
+      }
+    }
+
+    NextLevels(def_level, rep_level);
+    *is_null = *def_level < descr()->max_definition_level();
+
+    if (*is_null) { return true; }
+
+    if (value_offset_ == values_buffered_) {
+      throw ParquetException("Value was non-null, but has not been buffered");
+    }
+    *val = values_[value_offset_++];
+    return true;
+  }
+
+  // Returns true if there is a next value
+  bool NextValue(T* val, bool* is_null) {
+    if (level_offset_ == levels_buffered_) {
+      if (!HasNext()) {
+        // Out of data pages
+        return false;
+      }
+    }
+
+    // Out of values
+    int16_t def_level = -1;
+    int16_t rep_level = -1;
+    NextLevels(&def_level, &rep_level);
+    *is_null = def_level < descr()->max_definition_level();
+
+    if (*is_null) { return true; }
+
+    if (value_offset_ == values_buffered_) {
+      throw ParquetException("Value was non-null, but has not been buffered");
+    }
+    *val = values_[value_offset_++];
+    return true;
+  }
+
+  virtual void PrintNext(std::ostream& out, int width) {
+    T val;
+    bool is_null = false;
+    char buffer[25];
+
+    if (!NextValue(&val, &is_null)) { throw ParquetException("No more values buffered"); }
+
+    if (is_null) {
+      std::string null_fmt = format_fwf<ByteArrayType>(width);
+      snprintf(buffer, sizeof(buffer), null_fmt.c_str(), "NULL");
+    } else {
+      FormatValue(&val, buffer, sizeof(buffer), width);
+    }
+    out << buffer;
+  }
+
+ private:
+  // The ownership of this object is expressed through the reader_ variable in the base
+  TypedColumnReader<DType>* typed_reader_;
+
+  inline void FormatValue(void* val, char* buffer, int bufsize, int width);
+
+  T* values_;
+};
+
+template <typename DType>
+inline void TypedScanner<DType>::FormatValue(
+    void* val, char* buffer, int bufsize, int width) {
+  std::string fmt = format_fwf<DType>(width);
+  snprintf(buffer, bufsize, fmt.c_str(), *reinterpret_cast<T*>(val));
+}
+
+template <>
+inline void TypedScanner<Int96Type>::FormatValue(
+    void* val, char* buffer, int bufsize, int width) {
+  std::string fmt = format_fwf<Int96Type>(width);
+  std::string result = Int96ToString(*reinterpret_cast<Int96*>(val));
+  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
+}
+
+template <>
+inline void TypedScanner<ByteArrayType>::FormatValue(
+    void* val, char* buffer, int bufsize, int width) {
+  std::string fmt = format_fwf<ByteArrayType>(width);
+  std::string result = ByteArrayToString(*reinterpret_cast<ByteArray*>(val));
+  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
+}
+
+template <>
+inline void TypedScanner<FLBAType>::FormatValue(
+    void* val, char* buffer, int bufsize, int width) {
+  std::string fmt = format_fwf<FLBAType>(width);
+  std::string result = FixedLenByteArrayToString(
+      *reinterpret_cast<FixedLenByteArray*>(val), descr()->type_length());
+  snprintf(buffer, bufsize, fmt.c_str(), result.c_str());
+}
+
+typedef TypedScanner<BooleanType> BoolScanner;
+typedef TypedScanner<Int32Type> Int32Scanner;
+typedef TypedScanner<Int64Type> Int64Scanner;
+typedef TypedScanner<Int96Type> Int96Scanner;
+typedef TypedScanner<FloatType> FloatScanner;
+typedef TypedScanner<DoubleType> DoubleScanner;
+typedef TypedScanner<ByteArrayType> ByteArrayScanner;
+typedef TypedScanner<FLBAType> FixedLenByteArrayScanner;
+
+template <typename RType>
+int64_t ScanAll(int32_t batch_size, int16_t* def_levels, int16_t* rep_levels,
+    uint8_t* values, int64_t* values_buffered, parquet::ColumnReader* reader) {
+  typedef typename RType::T Type;
+  auto typed_reader = static_cast<RType*>(reader);
+  auto vals = reinterpret_cast<Type*>(&values[0]);
+  return typed_reader->ReadBatch(
+      batch_size, def_levels, rep_levels, vals, values_buffered);
+}
+
+int64_t PARQUET_EXPORT ScanAllValues(int32_t batch_size, int16_t* def_levels,
+    int16_t* rep_levels, uint8_t* values, int64_t* values_buffered,
+    parquet::ColumnReader* reader);
+
+}  // namespace parquet
+
+#endif  // PARQUET_COLUMN_SCANNER_H


[6/6] parquet-cpp git commit: PARQUET-858: Flatten column directory, minor code consolidation

Posted by uw...@apache.org.
PARQUET-858: Flatten column directory, minor code consolidation

This strictly moves code around and makes no functional changes; the purpose is to make the codebase easier to navigate. I also ran clang-format

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

Closes #363 from wesm/flatten-column-directory and squashes the following commits:

7bc099f [Wes McKinney] Build fixes, clang-format
b16be99 [Wes McKinney] Initial cut flattening out column directory


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

Branch: refs/heads/master
Commit: 84db929ec14175badbbed7083be57eca81a99c09
Parents: 491182c
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Jun 26 09:05:17 2017 +0200
Committer: Uwe L. Korn <uw...@apache.org>
Committed: Mon Jun 26 09:05:17 2017 +0200

----------------------------------------------------------------------
 CMakeLists.txt                                  |  11 +-
 benchmarks/decode_benchmark.cc                  |   6 +-
 src/parquet/CMakeLists.txt                      |  13 +
 src/parquet/api/reader.h                        |   4 +-
 src/parquet/api/writer.h                        |   2 +-
 .../arrow/arrow-reader-writer-benchmark.cc      |   4 +-
 src/parquet/arrow/arrow-reader-writer-test.cc   |  90 ++-
 src/parquet/arrow/reader.cc                     | 123 ++--
 src/parquet/arrow/reader.h                      |   4 +-
 src/parquet/arrow/writer.cc                     |   3 +-
 src/parquet/column-io-benchmark.cc              | 187 +++++
 src/parquet/column/CMakeLists.txt               |  38 -
 src/parquet/column/column-io-benchmark.cc       | 138 ----
 src/parquet/column/column-reader-test.cc        | 374 ----------
 src/parquet/column/column-writer-test.cc        | 516 -------------
 src/parquet/column/level-benchmark.cc           |  78 --
 src/parquet/column/levels-test.cc               | 245 -------
 src/parquet/column/levels.cc                    | 144 ----
 src/parquet/column/levels.h                     |  86 ---
 src/parquet/column/page.h                       | 201 -----
 src/parquet/column/properties-test.cc           |  64 --
 src/parquet/column/properties.h                 | 385 ----------
 src/parquet/column/reader.cc                    | 238 ------
 src/parquet/column/reader.h                     | 453 ------------
 src/parquet/column/scan-all.cc                  |  56 --
 src/parquet/column/scan-all.h                   |  41 --
 src/parquet/column/scanner-test.cc              | 232 ------
 src/parquet/column/scanner.cc                   |  56 --
 src/parquet/column/scanner.h                    | 232 ------
 src/parquet/column/statistics-test.cc           | 358 ---------
 src/parquet/column/statistics.cc                | 244 -------
 src/parquet/column/statistics.h                 | 234 ------
 src/parquet/column/test-specialization.h        | 172 -----
 src/parquet/column/test-util.h                  | 429 -----------
 src/parquet/column/writer.cc                    | 528 --------------
 src/parquet/column/writer.h                     | 250 -------
 src/parquet/column_page.h                       | 201 +++++
 src/parquet/column_reader-test.cc               | 371 ++++++++++
 src/parquet/column_reader.cc                    | 289 ++++++++
 src/parquet/column_reader.h                     | 475 ++++++++++++
 src/parquet/column_scanner-test.cc              | 232 ++++++
 src/parquet/column_scanner.cc                   |  90 +++
 src/parquet/column_scanner.h                    | 246 +++++++
 src/parquet/column_writer-test.cc               | 729 +++++++++++++++++++
 src/parquet/column_writer.cc                    | 597 +++++++++++++++
 src/parquet/column_writer.h                     | 282 +++++++
 src/parquet/encoding-internal.h                 |   5 +-
 src/parquet/file/file-deserialize-test.cc       |   2 +-
 src/parquet/file/file-metadata-test.cc          |   2 +-
 src/parquet/file/file-serialize-test.cc         |   8 +-
 src/parquet/file/metadata.h                     |   4 +-
 src/parquet/file/printer.cc                     |   2 +-
 src/parquet/file/reader-internal.cc             |   2 +-
 src/parquet/file/reader-internal.h              |   4 +-
 src/parquet/file/reader.cc                      |   6 +-
 src/parquet/file/reader.h                       |   6 +-
 src/parquet/file/writer-internal.cc             |   2 +-
 src/parquet/file/writer-internal.h              |   2 +-
 src/parquet/file/writer.h                       |   2 +-
 src/parquet/properties-test.cc                  |  64 ++
 src/parquet/properties.h                        | 385 ++++++++++
 src/parquet/reader-test.cc                      |   4 +-
 src/parquet/schema-test.cc                      |  16 +-
 src/parquet/statistics-test.cc                  | 358 +++++++++
 src/parquet/statistics.cc                       | 244 +++++++
 src/parquet/statistics.h                        | 234 ++++++
 src/parquet/test-specialization.h               | 172 +++++
 src/parquet/test-util.h                         | 430 +++++++++++
 src/parquet/util/memory.h                       |  12 +-
 src/parquet/util/schema-util.h                  |  13 +-
 src/parquet/util/visibility.h                   |   3 +-
 71 files changed, 5755 insertions(+), 5978 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index d0c1a53..47984e6 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -602,12 +602,9 @@ set(LIBPARQUET_SRCS
   src/parquet/arrow/schema.cc
   src/parquet/arrow/writer.cc
 
-  src/parquet/column/levels.cc
-  src/parquet/column/reader.cc
-  src/parquet/column/writer.cc
-  src/parquet/column/scanner.cc
-  src/parquet/column/scan-all.cc
-  src/parquet/column/statistics.cc
+  src/parquet/column_reader.cc
+  src/parquet/column_scanner.cc
+  src/parquet/column_writer.cc
 
   src/parquet/file/metadata.cc
   src/parquet/file/printer.cc
@@ -617,6 +614,7 @@ set(LIBPARQUET_SRCS
   src/parquet/file/writer-internal.cc
 
   src/parquet/schema.cc
+  src/parquet/statistics.cc
 
   src/parquet/parquet_constants.cpp
   src/parquet/parquet_types.cpp
@@ -669,7 +667,6 @@ ADD_LIB(parquet
 add_subdirectory(src/parquet)
 add_subdirectory(src/parquet/api)
 add_subdirectory(src/parquet/arrow)
-add_subdirectory(src/parquet/column)
 add_subdirectory(src/parquet/file)
 add_subdirectory(src/parquet/util)
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/benchmarks/decode_benchmark.cc
----------------------------------------------------------------------
diff --git a/benchmarks/decode_benchmark.cc b/benchmarks/decode_benchmark.cc
index 65a6afd..44776a8 100644
--- a/benchmarks/decode_benchmark.cc
+++ b/benchmarks/decode_benchmark.cc
@@ -288,8 +288,10 @@ void TestPlainIntCompressed(::arrow::Codec* codec, const std::vector<int64_t>& d
   int max_compressed_size = codec->MaxCompressedLen(uncompressed_len, raw_data);
   uint8_t* compressed_data = new uint8_t[max_compressed_size];
   int64_t compressed_len;
-  DCHECK(codec->Compress(uncompressed_len, raw_data, max_compressed_size,
-          compressed_data, &compressed_len).ok());
+  DCHECK(codec
+             ->Compress(uncompressed_len, raw_data, max_compressed_size, compressed_data,
+                 &compressed_len)
+             .ok());
 
   printf("\n%s:\n  Uncompressed len: %d\n  Compressed len:   %d\n", codec->name(),
       uncompressed_len, static_cast<int>(compressed_len));

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/parquet/CMakeLists.txt b/src/parquet/CMakeLists.txt
index f0eedcf..09a689e 100644
--- a/src/parquet/CMakeLists.txt
+++ b/src/parquet/CMakeLists.txt
@@ -17,9 +17,15 @@
 
 # Headers: top level
 install(FILES
+  column_reader.h
+  column_page.h
+  column_scanner.h
+  column_writer.h
   encoding.h
   exception.h
+  properties.h
   schema.h
+  statistics.h
   types.h
   DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/parquet")
 
@@ -40,9 +46,16 @@ install(FILES
   "${CMAKE_CURRENT_BINARY_DIR}/parquet.pc"
   DESTINATION "${CMAKE_INSTALL_LIBDIR}/pkgconfig/")
 
+ADD_PARQUET_TEST(column_reader-test)
+ADD_PARQUET_TEST(column_scanner-test)
+ADD_PARQUET_TEST(column_writer-test)
+ADD_PARQUET_TEST(properties-test)
+ADD_PARQUET_TEST(statistics-test)
 ADD_PARQUET_TEST(encoding-test)
 ADD_PARQUET_TEST(public-api-test)
 ADD_PARQUET_TEST(types-test)
 ADD_PARQUET_TEST(reader-test)
 ADD_PARQUET_TEST(schema-test)
+
+ADD_PARQUET_BENCHMARK(column-io-benchmark)
 ADD_PARQUET_BENCHMARK(encoding-benchmark)

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/api/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/api/reader.h b/src/parquet/api/reader.h
index f41a429..ba9717a 100644
--- a/src/parquet/api/reader.h
+++ b/src/parquet/api/reader.h
@@ -19,8 +19,8 @@
 #define PARQUET_API_READER_H
 
 // Column reader API
-#include "parquet/column/reader.h"
-#include "parquet/column/scan-all.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_scanner.h"
 #include "parquet/exception.h"
 #include "parquet/file/printer.h"
 #include "parquet/file/reader.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/api/writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/api/writer.h b/src/parquet/api/writer.h
index 9c239b2..cc3ae2a 100644
--- a/src/parquet/api/writer.h
+++ b/src/parquet/api/writer.h
@@ -19,7 +19,7 @@
 #define PARQUET_API_WRITER_H
 
 // Column reader API
-#include "parquet/column/writer.h"
+#include "parquet/column_writer.h"
 #include "parquet/exception.h"
 #include "parquet/file/writer.h"
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/arrow/arrow-reader-writer-benchmark.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/arrow-reader-writer-benchmark.cc b/src/parquet/arrow/arrow-reader-writer-benchmark.cc
index b9aa4a2..677e437 100644
--- a/src/parquet/arrow/arrow-reader-writer-benchmark.cc
+++ b/src/parquet/arrow/arrow-reader-writer-benchmark.cc
@@ -19,8 +19,8 @@
 
 #include "parquet/arrow/reader.h"
 #include "parquet/arrow/writer.h"
-#include "parquet/column/reader.h"
-#include "parquet/column/writer.h"
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
 #include "parquet/file/reader-internal.h"
 #include "parquet/file/writer-internal.h"
 #include "parquet/util/memory.h"

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/arrow/arrow-reader-writer-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/arrow-reader-writer-test.cc b/src/parquet/arrow/arrow-reader-writer-test.cc
index 3beca35..f2a9651 100644
--- a/src/parquet/arrow/arrow-reader-writer-test.cc
+++ b/src/parquet/arrow/arrow-reader-writer-test.cc
@@ -1136,10 +1136,10 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
     values_array_ = std::dynamic_pointer_cast<::arrow::Int32Array>(arr);
   }
 
-  void WriteColumnData(size_t num_rows, int16_t* def_levels,
-      int16_t* rep_levels, int32_t* values) {
-    auto typed_writer = static_cast<TypedColumnWriter<Int32Type>*>(
-      row_group_writer_->NextColumn());
+  void WriteColumnData(
+      size_t num_rows, int16_t* def_levels, int16_t* rep_levels, int32_t* values) {
+    auto typed_writer =
+        static_cast<TypedColumnWriter<Int32Type>*>(row_group_writer_->NextColumn());
     typed_writer->WriteBatch(num_rows, def_levels, rep_levels, values);
   }
 
@@ -1149,22 +1149,17 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
     // Also independently count the nulls
     auto local_null_count = 0;
     for (int i = 0; i < array.length(); i++) {
-      if (array.IsNull(i)) {
-        local_null_count++;
-      }
+      if (array.IsNull(i)) { local_null_count++; }
     }
     ASSERT_EQ(local_null_count, expected_nulls);
   }
 
-  void ValidateColumnArray(const ::arrow::Int32Array& array,
-      size_t expected_nulls) {
+  void ValidateColumnArray(const ::arrow::Int32Array& array, size_t expected_nulls) {
     ValidateArray(array, expected_nulls);
 
     int j = 0;
     for (int i = 0; i < values_array_->length(); i++) {
-      if (array.IsNull(i)) {
-        continue;
-      }
+      if (array.IsNull(i)) { continue; }
       ASSERT_EQ(array.Value(i), values_array_->Value(j));
       j++;
     }
@@ -1196,7 +1191,7 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
 
     parquet_fields.push_back(GroupNode::Make("group1", struct_repetition,
         {PrimitiveNode::Make("leaf1", Repetition::REQUIRED, ParquetType::INT32),
-         PrimitiveNode::Make("leaf2", Repetition::OPTIONAL, ParquetType::INT32)}));
+            PrimitiveNode::Make("leaf2", Repetition::OPTIONAL, ParquetType::INT32)}));
     parquet_fields.push_back(
         PrimitiveNode::Make("leaf3", Repetition::REQUIRED, ParquetType::INT32));
 
@@ -1209,7 +1204,7 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
     std::vector<int16_t> leaf1_def_levels(NUM_SIMPLE_TEST_ROWS);
     std::vector<int16_t> leaf2_def_levels(NUM_SIMPLE_TEST_ROWS);
     std::vector<int16_t> leaf3_def_levels(NUM_SIMPLE_TEST_ROWS);
-    for (int i = 0; i < NUM_SIMPLE_TEST_ROWS; i++)  {
+    for (int i = 0; i < NUM_SIMPLE_TEST_ROWS; i++) {
       // leaf1 is required within the optional group1, so it is only null
       // when the group is null
       leaf1_def_levels[i] = (i % 3 == 0) ? 0 : 1;
@@ -1227,18 +1222,18 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
     int32_t* values = reinterpret_cast<int32_t*>(values_array_->data()->mutable_data());
 
     // Create the actual parquet file
-    InitNewParquetFile(std::static_pointer_cast<GroupNode>(schema_node),
-      NUM_SIMPLE_TEST_ROWS);
+    InitNewParquetFile(
+        std::static_pointer_cast<GroupNode>(schema_node), NUM_SIMPLE_TEST_ROWS);
 
     // leaf1 column
-    WriteColumnData(NUM_SIMPLE_TEST_ROWS, leaf1_def_levels.data(),
-      rep_levels.data(), values);
+    WriteColumnData(
+        NUM_SIMPLE_TEST_ROWS, leaf1_def_levels.data(), rep_levels.data(), values);
     // leaf2 column
-    WriteColumnData(NUM_SIMPLE_TEST_ROWS, leaf2_def_levels.data(),
-      rep_levels.data(), values);
+    WriteColumnData(
+        NUM_SIMPLE_TEST_ROWS, leaf2_def_levels.data(), rep_levels.data(), values);
     // leaf3 column
-    WriteColumnData(NUM_SIMPLE_TEST_ROWS, leaf3_def_levels.data(),
-      rep_levels.data(), values);
+    WriteColumnData(
+        NUM_SIMPLE_TEST_ROWS, leaf3_def_levels.data(), rep_levels.data(), values);
 
     FinalizeParquetFile();
     InitReader();
@@ -1250,11 +1245,10 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
 
     for (int i = 0; i < num_children; i++) {
       if (depth <= 1) {
-        children.push_back(PrimitiveNode::Make("leaf",
-          node_repetition, leaf_type));
+        children.push_back(PrimitiveNode::Make("leaf", node_repetition, leaf_type));
       } else {
-        children.push_back(CreateSingleTypedNestedGroup(i, depth - 1, num_children,
-          node_repetition, leaf_type));
+        children.push_back(CreateSingleTypedNestedGroup(
+            i, depth - 1, num_children, node_repetition, leaf_type));
       }
     }
 
@@ -1264,13 +1258,13 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
   }
 
   // A deeply nested schema
-  void CreateMultiLevelNestedParquet(int num_trees, int tree_depth,
-      int num_children, int num_rows, Repetition::type node_repetition) {
+  void CreateMultiLevelNestedParquet(int num_trees, int tree_depth, int num_children,
+      int num_rows, Repetition::type node_repetition) {
     // Create the schema
     std::vector<NodePtr> parquet_fields;
     for (int i = 0; i < num_trees; i++) {
-      parquet_fields.push_back(CreateSingleTypedNestedGroup(i, tree_depth, num_children,
-        node_repetition, ParquetType::INT32));
+      parquet_fields.push_back(CreateSingleTypedNestedGroup(
+          i, tree_depth, num_children, node_repetition, ParquetType::INT32));
     }
     auto schema_node = GroupNode::Make("schema", Repetition::REQUIRED, parquet_fields);
 
@@ -1280,11 +1274,11 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
     std::vector<int16_t> rep_levels(num_rows);
     for (int i = 0; i < num_rows; i++) {
       if (node_repetition == Repetition::REQUIRED) {
-        def_levels[i] = 0; // all is required
+        def_levels[i] = 0;  // all is required
       } else {
-        def_levels[i] = i % tree_depth; // all is optional
+        def_levels[i] = i % tree_depth;  // all is optional
       }
-      rep_levels[i] = 0; // none is repeated
+      rep_levels[i] = 0;  // none is repeated
     }
 
     // Produce values for the columns
@@ -1303,13 +1297,11 @@ class TestNestedSchemaRead : public ::testing::TestWithParam<Repetition::type> {
 
   class DeepParquetTestVisitor : public ArrayVisitor {
    public:
-    DeepParquetTestVisitor(Repetition::type node_repetition,
-      std::shared_ptr<::arrow::Int32Array> expected) :
-      node_repetition_(node_repetition), expected_(expected) {}
+    DeepParquetTestVisitor(
+        Repetition::type node_repetition, std::shared_ptr<::arrow::Int32Array> expected)
+        : node_repetition_(node_repetition), expected_(expected) {}
 
-    Status Validate(std::shared_ptr<Array> tree) {
-      return tree->Accept(this);
-    }
+    Status Validate(std::shared_ptr<Array> tree) { return tree->Accept(this); }
 
     virtual Status Visit(const ::arrow::Int32Array& array) {
       if (node_repetition_ == Repetition::REQUIRED) {
@@ -1367,14 +1359,14 @@ TEST_F(TestNestedSchemaRead, ReadIntoTableFull) {
   ASSERT_EQ(table->schema()->field(0)->type()->num_children(), 2);
   ValidateTableArrayTypes(*table);
 
-  auto struct_field_array = std::static_pointer_cast<::arrow::StructArray>(
-    table->column(0)->data()->chunk(0));
-  auto leaf1_array = std::static_pointer_cast<::arrow::Int32Array>(
-    struct_field_array->field(0));
-  auto leaf2_array = std::static_pointer_cast<::arrow::Int32Array>(
-    struct_field_array->field(1));
-  auto leaf3_array = std::static_pointer_cast<::arrow::Int32Array>(
-    table->column(1)->data()->chunk(0));
+  auto struct_field_array =
+      std::static_pointer_cast<::arrow::StructArray>(table->column(0)->data()->chunk(0));
+  auto leaf1_array =
+      std::static_pointer_cast<::arrow::Int32Array>(struct_field_array->field(0));
+  auto leaf2_array =
+      std::static_pointer_cast<::arrow::Int32Array>(struct_field_array->field(1));
+  auto leaf3_array =
+      std::static_pointer_cast<::arrow::Int32Array>(table->column(1)->data()->chunk(0));
 
   // validate struct and leaf arrays
 
@@ -1383,7 +1375,7 @@ TEST_F(TestNestedSchemaRead, ReadIntoTableFull) {
   // validate leaf1
   ValidateColumnArray(*leaf1_array, NUM_SIMPLE_TEST_ROWS / 3);
   // validate leaf2
-  ValidateColumnArray(*leaf2_array, NUM_SIMPLE_TEST_ROWS * 2/ 3);
+  ValidateColumnArray(*leaf2_array, NUM_SIMPLE_TEST_ROWS * 2 / 3);
   // validate leaf3
   ValidateColumnArray(*leaf3_array, 0);
 }
@@ -1452,7 +1444,7 @@ TEST_P(TestNestedSchemaRead, DeepNestedSchemaRead) {
 }
 
 INSTANTIATE_TEST_CASE_P(Repetition_type, TestNestedSchemaRead,
-  ::testing::Values(Repetition::REQUIRED, Repetition::OPTIONAL));
+    ::testing::Values(Repetition::REQUIRED, Repetition::OPTIONAL));
 
 TEST(TestArrowReaderAdHoc, Int96BadMemoryAccess) {
   // PARQUET-995

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/arrow/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/reader.cc b/src/parquet/arrow/reader.cc
index ef9ac34..a3a26c9 100644
--- a/src/parquet/arrow/reader.cc
+++ b/src/parquet/arrow/reader.cc
@@ -184,8 +184,8 @@ class FileReader::Impl {
 
   Status GetColumn(int i, std::unique_ptr<ColumnReader>* out);
   Status ReadSchemaField(int i, std::shared_ptr<Array>* out);
-  Status ReadSchemaField(int i, const std::vector<int>& indices,
-      std::shared_ptr<Array>* out);
+  Status ReadSchemaField(
+      int i, const std::vector<int>& indices, std::shared_ptr<Array>* out);
   Status GetReaderForNode(int index, const NodePtr& node, const std::vector<int>& indices,
       int16_t def_level, std::unique_ptr<ColumnReader::Impl>* out);
   Status ReadColumn(int i, std::shared_ptr<Array>* out);
@@ -226,7 +226,7 @@ class ColumnReader::Impl {
 };
 
 // Reader implementation for primitive arrays
-class PrimitiveImpl: public ColumnReader::Impl {
+class PrimitiveImpl : public ColumnReader::Impl {
  public:
   PrimitiveImpl(MemoryPool* pool, std::unique_ptr<FileColumnIterator> input)
       : pool_(pool),
@@ -300,11 +300,13 @@ class PrimitiveImpl: public ColumnReader::Impl {
 };
 
 // Reader implementation for struct array
-class StructImpl: public ColumnReader::Impl {
+class StructImpl : public ColumnReader::Impl {
  public:
   explicit StructImpl(const std::vector<std::shared_ptr<Impl>>& children,
       int16_t struct_def_level, MemoryPool* pool, const NodePtr& node)
-      : children_(children), struct_def_level_(struct_def_level), pool_(pool),
+      : children_(children),
+        struct_def_level_(struct_def_level),
+        pool_(pool),
         def_levels_buffer_(pool) {
     InitField(node, children);
   }
@@ -323,10 +325,9 @@ class StructImpl: public ColumnReader::Impl {
   std::shared_ptr<Field> field_;
   PoolBuffer def_levels_buffer_;
 
-  Status DefLevelsToNullArray(std::shared_ptr<MutableBuffer>* null_bitmap,
-      int64_t* null_count);
-  void InitField(const NodePtr& node,
-      const std::vector<std::shared_ptr<Impl>>& children);
+  Status DefLevelsToNullArray(
+      std::shared_ptr<MutableBuffer>* null_bitmap, int64_t* null_count);
+  void InitField(const NodePtr& node, const std::vector<std::shared_ptr<Impl>>& children);
 };
 
 FileReader::FileReader(MemoryPool* pool, std::unique_ptr<ParquetFileReader> reader)
@@ -337,8 +338,7 @@ FileReader::~FileReader() {}
 Status FileReader::Impl::GetColumn(int i, std::unique_ptr<ColumnReader>* out) {
   std::unique_ptr<FileColumnIterator> input(new AllRowGroupsIterator(i, reader_.get()));
 
-  std::unique_ptr<ColumnReader::Impl> impl(
-      new PrimitiveImpl(pool_, std::move(input)));
+  std::unique_ptr<ColumnReader::Impl> impl(new PrimitiveImpl(pool_, std::move(input)));
   *out = std::unique_ptr<ColumnReader>(new ColumnReader(std::move(impl)));
   return Status::OK();
 }
@@ -346,7 +346,6 @@ Status FileReader::Impl::GetColumn(int i, std::unique_ptr<ColumnReader>* out) {
 Status FileReader::Impl::GetReaderForNode(int index, const NodePtr& node,
     const std::vector<int>& indices, int16_t def_level,
     std::unique_ptr<ColumnReader::Impl>* out) {
-
   *out = nullptr;
 
   if (IsSimpleStruct(node)) {
@@ -357,16 +356,14 @@ Status FileReader::Impl::GetReaderForNode(int index, const NodePtr& node,
       // TODO(itaiin): Remove the -1 index hack when all types of nested reads
       // are supported. This currently just signals the lower level reader resolution
       // to abort
-      RETURN_NOT_OK(GetReaderForNode(index, group->field(i), indices,
-        def_level + 1, &child_reader));
-      if (child_reader != nullptr) {
-        children.push_back(std::move(child_reader));
-      }
+      RETURN_NOT_OK(GetReaderForNode(
+          index, group->field(i), indices, def_level + 1, &child_reader));
+      if (child_reader != nullptr) { children.push_back(std::move(child_reader)); }
     }
 
     if (children.size() > 0) {
       *out = std::unique_ptr<ColumnReader::Impl>(
-        new StructImpl(children, def_level, pool_, node));
+          new StructImpl(children, def_level, pool_, node));
     }
   } else {
     // This should be a flat field case - translate the field index to
@@ -376,8 +373,7 @@ Status FileReader::Impl::GetReaderForNode(int index, const NodePtr& node,
       DCHECK(walker->is_group());
       auto group = static_cast<GroupNode*>(walker.get());
       if (group->field_count() != 1) {
-        return Status::NotImplemented(
-          "lists with structs are not supported.");
+        return Status::NotImplemented("lists with structs are not supported.");
       }
       walker = group->field(0);
     }
@@ -405,8 +401,8 @@ Status FileReader::Impl::ReadSchemaField(int i, std::shared_ptr<Array>* out) {
   return ReadSchemaField(i, indices, out);
 }
 
-Status FileReader::Impl::ReadSchemaField(int i, const std::vector<int>& indices,
-    std::shared_ptr<Array>* out) {
+Status FileReader::Impl::ReadSchemaField(
+    int i, const std::vector<int>& indices, std::shared_ptr<Array>* out) {
   auto parquet_schema = reader_->metadata()->schema();
 
   auto node = parquet_schema->group_node()->field(i);
@@ -461,15 +457,14 @@ Status FileReader::Impl::ReadRowGroup(int row_group_index,
   // TODO(wesm): Refactor to share more code with ReadTable
 
   auto ReadColumnFunc = [&indices, &row_group_index, &schema, &columns, &rg_metadata,
-                            this](int i) {
+      this](int i) {
     int column_index = indices[i];
     int64_t batch_size = rg_metadata->ColumnChunk(column_index)->num_values();
 
     std::unique_ptr<FileColumnIterator> input(
         new SingleRowGroupIterator(column_index, row_group_index, reader_.get()));
 
-    std::unique_ptr<ColumnReader::Impl> impl(
-        new PrimitiveImpl(pool_, std::move(input)));
+    std::unique_ptr<ColumnReader::Impl> impl(new PrimitiveImpl(pool_, std::move(input)));
     ColumnReader flat_column_reader(std::move(impl));
 
     std::shared_ptr<Array> array;
@@ -498,8 +493,8 @@ Status FileReader::Impl::ReadTable(
   // We only need to read schema fields which have columns indicated
   // in the indices vector
   std::vector<int> field_indices;
-  if (!ColumnIndicesToFieldIndices(*reader_->metadata()->schema(),
-        indices, &field_indices)) {
+  if (!ColumnIndicesToFieldIndices(
+          *reader_->metadata()->schema(), indices, &field_indices)) {
     return Status::Invalid("Invalid column index");
   }
 
@@ -628,8 +623,7 @@ const ParquetFileReader* FileReader::parquet_reader() const {
 }
 
 template <typename ArrowType, typename ParquetType>
-Status PrimitiveImpl::ReadNonNullableBatch(
-    TypedColumnReader<ParquetType>* reader,
+Status PrimitiveImpl::ReadNonNullableBatch(TypedColumnReader<ParquetType>* reader,
     int64_t values_to_read, int64_t* levels_read) {
   using ArrowCType = typename ArrowType::c_type;
   using ParquetCType = typename ParquetType::c_type;
@@ -709,8 +703,8 @@ Status PrimitiveImpl::ReadNonNullableBatch<::arrow::Date64Type, Int32Type>(
 }
 
 template <>
-Status PrimitiveImpl::ReadNonNullableBatch<::arrow::BooleanType,
-    BooleanType>(TypedColumnReader<BooleanType>* reader, int64_t values_to_read,
+Status PrimitiveImpl::ReadNonNullableBatch<::arrow::BooleanType, BooleanType>(
+    TypedColumnReader<BooleanType>* reader, int64_t values_to_read,
     int64_t* levels_read) {
   RETURN_NOT_OK(values_buffer_.Resize(values_to_read * sizeof(bool), false));
   auto values = reinterpret_cast<bool*>(values_buffer_.mutable_data());
@@ -727,8 +721,7 @@ Status PrimitiveImpl::ReadNonNullableBatch<::arrow::BooleanType,
 }
 
 template <typename ArrowType, typename ParquetType>
-Status PrimitiveImpl::ReadNullableBatch(
-    TypedColumnReader<ParquetType>* reader,
+Status PrimitiveImpl::ReadNullableBatch(TypedColumnReader<ParquetType>* reader,
     int16_t* def_levels, int16_t* rep_levels, int64_t values_to_read,
     int64_t* levels_read, int64_t* values_read) {
   using ArrowCType = typename ArrowType::c_type;
@@ -998,8 +991,7 @@ Status PrimitiveImpl::WrapIntoListArray(const int16_t* def_levels,
 }
 
 template <typename ArrowType, typename ParquetType>
-Status PrimitiveImpl::TypedReadBatch(
-    int batch_size, std::shared_ptr<Array>* out) {
+Status PrimitiveImpl::TypedReadBatch(int batch_size, std::shared_ptr<Array>* out) {
   using ArrowCType = typename ArrowType::c_type;
 
   int values_to_read = batch_size;
@@ -1127,8 +1119,7 @@ Status PrimitiveImpl::TypedReadBatch<::arrow::BooleanType, BooleanType>(
 }
 
 template <typename ArrowType>
-Status PrimitiveImpl::ReadByteArrayBatch(
-    int batch_size, std::shared_ptr<Array>* out) {
+Status PrimitiveImpl::ReadByteArrayBatch(int batch_size, std::shared_ptr<Array>* out) {
   using BuilderType = typename ::arrow::TypeTraits<ArrowType>::BuilderType;
 
   int total_levels_read = 0;
@@ -1252,8 +1243,7 @@ Status PrimitiveImpl::TypedReadBatch<::arrow::StringType, ByteArrayType>(
     return TypedReadBatch<ArrowType, ParquetType>(batch_size, out); \
     break;
 
-Status PrimitiveImpl::NextBatch(
-    int batch_size, std::shared_ptr<Array>* out) {
+Status PrimitiveImpl::NextBatch(int batch_size, std::shared_ptr<Array>* out) {
   if (!column_reader_) {
     // Exhausted all row groups.
     *out = nullptr;
@@ -1265,21 +1255,21 @@ Status PrimitiveImpl::NextBatch(
       *out = std::make_shared<::arrow::NullArray>(batch_size);
       return Status::OK();
       break;
-    TYPED_BATCH_CASE(BOOL, ::arrow::BooleanType, BooleanType)
-    TYPED_BATCH_CASE(UINT8, ::arrow::UInt8Type, Int32Type)
-    TYPED_BATCH_CASE(INT8, ::arrow::Int8Type, Int32Type)
-    TYPED_BATCH_CASE(UINT16, ::arrow::UInt16Type, Int32Type)
-    TYPED_BATCH_CASE(INT16, ::arrow::Int16Type, Int32Type)
-    TYPED_BATCH_CASE(UINT32, ::arrow::UInt32Type, Int32Type)
-    TYPED_BATCH_CASE(INT32, ::arrow::Int32Type, Int32Type)
-    TYPED_BATCH_CASE(UINT64, ::arrow::UInt64Type, Int64Type)
-    TYPED_BATCH_CASE(INT64, ::arrow::Int64Type, Int64Type)
-    TYPED_BATCH_CASE(FLOAT, ::arrow::FloatType, FloatType)
-    TYPED_BATCH_CASE(DOUBLE, ::arrow::DoubleType, DoubleType)
-    TYPED_BATCH_CASE(STRING, ::arrow::StringType, ByteArrayType)
-    TYPED_BATCH_CASE(BINARY, ::arrow::BinaryType, ByteArrayType)
-    TYPED_BATCH_CASE(DATE32, ::arrow::Date32Type, Int32Type)
-    TYPED_BATCH_CASE(DATE64, ::arrow::Date64Type, Int32Type)
+      TYPED_BATCH_CASE(BOOL, ::arrow::BooleanType, BooleanType)
+      TYPED_BATCH_CASE(UINT8, ::arrow::UInt8Type, Int32Type)
+      TYPED_BATCH_CASE(INT8, ::arrow::Int8Type, Int32Type)
+      TYPED_BATCH_CASE(UINT16, ::arrow::UInt16Type, Int32Type)
+      TYPED_BATCH_CASE(INT16, ::arrow::Int16Type, Int32Type)
+      TYPED_BATCH_CASE(UINT32, ::arrow::UInt32Type, Int32Type)
+      TYPED_BATCH_CASE(INT32, ::arrow::Int32Type, Int32Type)
+      TYPED_BATCH_CASE(UINT64, ::arrow::UInt64Type, Int64Type)
+      TYPED_BATCH_CASE(INT64, ::arrow::Int64Type, Int64Type)
+      TYPED_BATCH_CASE(FLOAT, ::arrow::FloatType, FloatType)
+      TYPED_BATCH_CASE(DOUBLE, ::arrow::DoubleType, DoubleType)
+      TYPED_BATCH_CASE(STRING, ::arrow::StringType, ByteArrayType)
+      TYPED_BATCH_CASE(BINARY, ::arrow::BinaryType, ByteArrayType)
+      TYPED_BATCH_CASE(DATE32, ::arrow::Date32Type, Int32Type)
+      TYPED_BATCH_CASE(DATE64, ::arrow::Date64Type, Int32Type)
     case ::arrow::Type::FIXED_SIZE_BINARY: {
       int32_t byte_width =
           static_cast<::arrow::FixedSizeBinaryType*>(field_->type().get())->byte_width();
@@ -1340,15 +1330,13 @@ Status ColumnReader::NextBatch(int batch_size, std::shared_ptr<Array>* out) {
 // StructImpl methods
 
 Status StructImpl::DefLevelsToNullArray(
-    std::shared_ptr<MutableBuffer>* null_bitmap_out,
-    int64_t* null_count_out) {
+    std::shared_ptr<MutableBuffer>* null_bitmap_out, int64_t* null_count_out) {
   std::shared_ptr<MutableBuffer> null_bitmap;
   auto null_count = 0;
   ValueLevelsPtr def_levels_data;
   size_t def_levels_length;
   RETURN_NOT_OK(GetDefLevels(&def_levels_data, &def_levels_length));
-  RETURN_NOT_OK(GetEmptyBitmap(pool_,
-    def_levels_length, &null_bitmap));
+  RETURN_NOT_OK(GetEmptyBitmap(pool_, def_levels_length, &null_bitmap));
   uint8_t* null_bitmap_ptr = null_bitmap->mutable_data();
   for (size_t i = 0; i < def_levels_length; i++) {
     if (def_levels_data[i] < struct_def_level_) {
@@ -1397,11 +1385,10 @@ Status StructImpl::GetDefLevels(ValueLevelsPtr* data, size_t* length) {
     for (size_t i = 0; i < child_length; i++) {
       // Check that value is either uninitialized, or current
       // and previous children def levels agree on the struct level
-      DCHECK((result_levels[i] == -1) ||
-             ((result_levels[i] >= struct_def_level_) ==
-              (child_def_levels[i] >= struct_def_level_)));
-      result_levels[i] = std::max(result_levels[i],
-        std::min(child_def_levels[i], struct_def_level_));
+      DCHECK((result_levels[i] == -1) || ((result_levels[i] >= struct_def_level_) ==
+                                             (child_def_levels[i] >= struct_def_level_)));
+      result_levels[i] =
+          std::max(result_levels[i], std::min(child_def_levels[i], struct_def_level_));
     }
   }
   *data = reinterpret_cast<ValueLevelsPtr>(def_levels_buffer_.data());
@@ -1409,11 +1396,11 @@ Status StructImpl::GetDefLevels(ValueLevelsPtr* data, size_t* length) {
   return Status::OK();
 }
 
-void StructImpl::InitField(const NodePtr& node,
-    const std::vector<std::shared_ptr<Impl>>& children) {
+void StructImpl::InitField(
+    const NodePtr& node, const std::vector<std::shared_ptr<Impl>>& children) {
   // Make a shallow node to field conversion from the children fields
   std::vector<std::shared_ptr<::arrow::Field>> fields(children.size());
-  for (size_t i  = 0; i < children.size(); i++) {
+  for (size_t i = 0; i < children.size(); i++) {
     fields[i] = children[i]->field();
   }
   auto type = std::make_shared<::arrow::StructType>(fields);
@@ -1440,8 +1427,8 @@ Status StructImpl::NextBatch(int batch_size, std::shared_ptr<Array>* out) {
 
   RETURN_NOT_OK(DefLevelsToNullArray(&null_bitmap, &null_count));
 
-  *out = std::make_shared<StructArray>(field()->type(), batch_size, children_arrays,
-    null_bitmap, null_count);
+  *out = std::make_shared<StructArray>(
+      field()->type(), batch_size, children_arrays, null_bitmap, null_count);
 
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/arrow/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/reader.h b/src/parquet/arrow/reader.h
index 06a64f8..8d9aeb5 100644
--- a/src/parquet/arrow/reader.h
+++ b/src/parquet/arrow/reader.h
@@ -130,8 +130,8 @@ class PARQUET_EXPORT FileReader {
   // i=1 indices={3} will read foo2 column
   // i=1 indices={2} will result in out=nullptr
   // leaf indices which are unrelated to the schema field are ignored
-  ::arrow::Status ReadSchemaField(int i, const std::vector<int>& indices,
-      std::shared_ptr<::arrow::Array>* out);
+  ::arrow::Status ReadSchemaField(
+      int i, const std::vector<int>& indices, std::shared_ptr<::arrow::Array>* out);
 
   // Read a table of columns into a Table
   ::arrow::Status ReadTable(std::shared_ptr<::arrow::Table>* out);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/arrow/writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/arrow/writer.cc b/src/parquet/arrow/writer.cc
index af4f754..b8cb45c 100644
--- a/src/parquet/arrow/writer.cc
+++ b/src/parquet/arrow/writer.cc
@@ -525,8 +525,7 @@ Status FileWriter::Impl::TypedWriteBatch<Int32Type, ::arrow::NullType>(
     const int16_t* def_levels, const int16_t* rep_levels) {
   auto writer = reinterpret_cast<TypedColumnWriter<Int32Type>*>(column_writer);
 
-  PARQUET_CATCH_NOT_OK(
-      writer->WriteBatch(num_levels, def_levels, rep_levels, nullptr));
+  PARQUET_CATCH_NOT_OK(writer->WriteBatch(num_levels, def_levels, rep_levels, nullptr));
   PARQUET_CATCH_NOT_OK(writer->Close());
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column-io-benchmark.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column-io-benchmark.cc b/src/parquet/column-io-benchmark.cc
new file mode 100644
index 0000000..0a60367
--- /dev/null
+++ b/src/parquet/column-io-benchmark.cc
@@ -0,0 +1,187 @@
+// 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.
+
+#include "benchmark/benchmark.h"
+
+#include "parquet/column_reader.h"
+#include "parquet/column_writer.h"
+#include "parquet/file/reader-internal.h"
+#include "parquet/file/writer-internal.h"
+#include "parquet/util/memory.h"
+
+namespace parquet {
+
+using schema::PrimitiveNode;
+
+namespace benchmark {
+
+std::unique_ptr<Int64Writer> BuildWriter(int64_t output_size, OutputStream* dst,
+    ColumnChunkMetaDataBuilder* metadata, ColumnDescriptor* schema,
+    const WriterProperties* properties) {
+  std::unique_ptr<SerializedPageWriter> pager(
+      new SerializedPageWriter(dst, Compression::UNCOMPRESSED, metadata));
+  return std::unique_ptr<Int64Writer>(new Int64Writer(
+      metadata, std::move(pager), output_size, Encoding::PLAIN, properties));
+}
+
+std::shared_ptr<ColumnDescriptor> Int64Schema(Repetition::type repetition) {
+  auto node = PrimitiveNode::Make("int64", repetition, Type::INT64);
+  return std::make_shared<ColumnDescriptor>(
+      node, repetition != Repetition::REQUIRED, repetition == Repetition::REPEATED);
+}
+
+void SetBytesProcessed(::benchmark::State& state, Repetition::type repetition) {
+  int64_t bytes_processed = state.iterations() * state.range(0) * sizeof(int64_t);
+  if (repetition != Repetition::REQUIRED) {
+    bytes_processed += state.iterations() * state.range(0) * sizeof(int16_t);
+  }
+  if (repetition == Repetition::REPEATED) {
+    bytes_processed += state.iterations() * state.range(0) * sizeof(int16_t);
+  }
+  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
+}
+
+template <Repetition::type repetition>
+static void BM_WriteInt64Column(::benchmark::State& state) {
+  format::ColumnChunk thrift_metadata;
+  std::vector<int64_t> values(state.range(0), 128);
+  std::vector<int16_t> definition_levels(state.range(0), 1);
+  std::vector<int16_t> repetition_levels(state.range(0), 0);
+  std::shared_ptr<ColumnDescriptor> schema = Int64Schema(repetition);
+  std::shared_ptr<WriterProperties> properties = default_writer_properties();
+  auto metadata = ColumnChunkMetaDataBuilder::Make(
+      properties, schema.get(), reinterpret_cast<uint8_t*>(&thrift_metadata));
+
+  while (state.KeepRunning()) {
+    InMemoryOutputStream stream;
+    std::unique_ptr<Int64Writer> writer = BuildWriter(
+        state.range(0), &stream, metadata.get(), schema.get(), properties.get());
+    writer->WriteBatch(
+        values.size(), definition_levels.data(), repetition_levels.data(), values.data());
+    writer->Close();
+  }
+  SetBytesProcessed(state, repetition);
+}
+
+BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::REQUIRED)->Range(1024, 65536);
+
+BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::OPTIONAL)->Range(1024, 65536);
+
+BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::REPEATED)->Range(1024, 65536);
+
+std::unique_ptr<Int64Reader> BuildReader(
+    std::shared_ptr<Buffer>& buffer, int64_t num_values, ColumnDescriptor* schema) {
+  std::unique_ptr<InMemoryInputStream> source(new InMemoryInputStream(buffer));
+  std::unique_ptr<SerializedPageReader> page_reader(
+      new SerializedPageReader(std::move(source), num_values, Compression::UNCOMPRESSED));
+  return std::unique_ptr<Int64Reader>(new Int64Reader(schema, std::move(page_reader)));
+}
+
+template <Repetition::type repetition>
+static void BM_ReadInt64Column(::benchmark::State& state) {
+  format::ColumnChunk thrift_metadata;
+  std::vector<int64_t> values(state.range(0), 128);
+  std::vector<int16_t> definition_levels(state.range(0), 1);
+  std::vector<int16_t> repetition_levels(state.range(0), 0);
+  std::shared_ptr<ColumnDescriptor> schema = Int64Schema(repetition);
+  std::shared_ptr<WriterProperties> properties = default_writer_properties();
+  auto metadata = ColumnChunkMetaDataBuilder::Make(
+      properties, schema.get(), reinterpret_cast<uint8_t*>(&thrift_metadata));
+
+  InMemoryOutputStream stream;
+  std::unique_ptr<Int64Writer> writer = BuildWriter(
+      state.range(0), &stream, metadata.get(), schema.get(), properties.get());
+  writer->WriteBatch(
+      values.size(), definition_levels.data(), repetition_levels.data(), values.data());
+  writer->Close();
+
+  std::shared_ptr<Buffer> src = stream.GetBuffer();
+  std::vector<int64_t> values_out(state.range(1));
+  std::vector<int16_t> definition_levels_out(state.range(1));
+  std::vector<int16_t> repetition_levels_out(state.range(1));
+  while (state.KeepRunning()) {
+    std::unique_ptr<Int64Reader> reader = BuildReader(src, state.range(1), schema.get());
+    int64_t values_read = 0;
+    for (size_t i = 0; i < values.size(); i += values_read) {
+      reader->ReadBatch(values_out.size(), definition_levels_out.data(),
+          repetition_levels_out.data(), values_out.data(), &values_read);
+    }
+  }
+  SetBytesProcessed(state, repetition);
+}
+
+BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::REQUIRED)
+    ->RangePair(1024, 65536, 1, 1024);
+
+BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::OPTIONAL)
+    ->RangePair(1024, 65536, 1, 1024);
+
+BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::REPEATED)
+    ->RangePair(1024, 65536, 1, 1024);
+
+static void BM_RleEncoding(::benchmark::State& state) {
+  std::vector<int16_t> levels(state.range(0), 0);
+  int64_t n = 0;
+  std::generate(
+      levels.begin(), levels.end(), [&state, &n] { return (n++ % state.range(1)) == 0; });
+  int16_t max_level = 1;
+  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level, levels.size());
+  auto buffer_rle = std::make_shared<PoolBuffer>();
+  PARQUET_THROW_NOT_OK(buffer_rle->Resize(rle_size));
+
+  while (state.KeepRunning()) {
+    LevelEncoder level_encoder;
+    level_encoder.Init(Encoding::RLE, max_level, levels.size(),
+        buffer_rle->mutable_data(), buffer_rle->size());
+    level_encoder.Encode(levels.size(), levels.data());
+  }
+  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
+  state.SetItemsProcessed(state.iterations() * state.range(0));
+}
+
+BENCHMARK(BM_RleEncoding)->RangePair(1024, 65536, 1, 16);
+
+static void BM_RleDecoding(::benchmark::State& state) {
+  LevelEncoder level_encoder;
+  std::vector<int16_t> levels(state.range(0), 0);
+  int64_t n = 0;
+  std::generate(
+      levels.begin(), levels.end(), [&state, &n] { return (n++ % state.range(1)) == 0; });
+  int16_t max_level = 1;
+  int64_t rle_size = LevelEncoder::MaxBufferSize(Encoding::RLE, max_level, levels.size());
+  auto buffer_rle = std::make_shared<PoolBuffer>();
+  PARQUET_THROW_NOT_OK(buffer_rle->Resize(rle_size + sizeof(int32_t)));
+  level_encoder.Init(Encoding::RLE, max_level, levels.size(),
+      buffer_rle->mutable_data() + sizeof(int32_t), rle_size);
+  level_encoder.Encode(levels.size(), levels.data());
+  reinterpret_cast<int32_t*>(buffer_rle->mutable_data())[0] = level_encoder.len();
+
+  while (state.KeepRunning()) {
+    LevelDecoder level_decoder;
+    level_decoder.SetData(Encoding::RLE, max_level, levels.size(), buffer_rle->data());
+    level_decoder.Decode(state.range(0), levels.data());
+  }
+
+  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
+  state.SetItemsProcessed(state.iterations() * state.range(0));
+}
+
+BENCHMARK(BM_RleDecoding)->RangePair(1024, 65536, 1, 16);
+
+}  // namespace benchmark
+
+}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/parquet/column/CMakeLists.txt b/src/parquet/column/CMakeLists.txt
deleted file mode 100644
index 789248d..0000000
--- a/src/parquet/column/CMakeLists.txt
+++ /dev/null
@@ -1,38 +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.
-
-# Headers: top level
-install(FILES
-  levels.h
-  page.h
-  properties.h
-  reader.h
-  scan-all.h
-  scanner.h
-  writer.h
-  statistics.h
-  DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/parquet/column")
-
-ADD_PARQUET_TEST(column-reader-test)
-ADD_PARQUET_TEST(column-writer-test)
-ADD_PARQUET_TEST(levels-test)
-ADD_PARQUET_TEST(properties-test)
-ADD_PARQUET_TEST(scanner-test)
-ADD_PARQUET_TEST(statistics-test)
-
-ADD_PARQUET_BENCHMARK(column-io-benchmark)
-ADD_PARQUET_BENCHMARK(level-benchmark)

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/column-io-benchmark.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/column-io-benchmark.cc b/src/parquet/column/column-io-benchmark.cc
deleted file mode 100644
index 24afab2..0000000
--- a/src/parquet/column/column-io-benchmark.cc
+++ /dev/null
@@ -1,138 +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.
-
-#include "benchmark/benchmark.h"
-
-#include "parquet/column/reader.h"
-#include "parquet/column/writer.h"
-#include "parquet/file/reader-internal.h"
-#include "parquet/file/writer-internal.h"
-#include "parquet/util/memory.h"
-
-namespace parquet {
-
-using schema::PrimitiveNode;
-
-namespace benchmark {
-
-std::unique_ptr<Int64Writer> BuildWriter(int64_t output_size, OutputStream* dst,
-    ColumnChunkMetaDataBuilder* metadata, ColumnDescriptor* schema,
-    const WriterProperties* properties) {
-  std::unique_ptr<SerializedPageWriter> pager(
-      new SerializedPageWriter(dst, Compression::UNCOMPRESSED, metadata));
-  return std::unique_ptr<Int64Writer>(new Int64Writer(
-      metadata, std::move(pager), output_size, Encoding::PLAIN, properties));
-}
-
-std::shared_ptr<ColumnDescriptor> Int64Schema(Repetition::type repetition) {
-  auto node = PrimitiveNode::Make("int64", repetition, Type::INT64);
-  return std::make_shared<ColumnDescriptor>(
-      node, repetition != Repetition::REQUIRED, repetition == Repetition::REPEATED);
-}
-
-void SetBytesProcessed(::benchmark::State& state, Repetition::type repetition) {
-  int64_t bytes_processed = state.iterations() * state.range(0) * sizeof(int64_t);
-  if (repetition != Repetition::REQUIRED) {
-    bytes_processed += state.iterations() * state.range(0) * sizeof(int16_t);
-  }
-  if (repetition == Repetition::REPEATED) {
-    bytes_processed += state.iterations() * state.range(0) * sizeof(int16_t);
-  }
-  state.SetBytesProcessed(state.iterations() * state.range(0) * sizeof(int16_t));
-}
-
-template <Repetition::type repetition>
-static void BM_WriteInt64Column(::benchmark::State& state) {
-  format::ColumnChunk thrift_metadata;
-  std::vector<int64_t> values(state.range(0), 128);
-  std::vector<int16_t> definition_levels(state.range(0), 1);
-  std::vector<int16_t> repetition_levels(state.range(0), 0);
-  std::shared_ptr<ColumnDescriptor> schema = Int64Schema(repetition);
-  std::shared_ptr<WriterProperties> properties = default_writer_properties();
-  auto metadata = ColumnChunkMetaDataBuilder::Make(
-      properties, schema.get(), reinterpret_cast<uint8_t*>(&thrift_metadata));
-
-  while (state.KeepRunning()) {
-    InMemoryOutputStream stream;
-    std::unique_ptr<Int64Writer> writer = BuildWriter(
-        state.range(0), &stream, metadata.get(), schema.get(), properties.get());
-    writer->WriteBatch(
-        values.size(), definition_levels.data(), repetition_levels.data(), values.data());
-    writer->Close();
-  }
-  SetBytesProcessed(state, repetition);
-}
-
-BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::REQUIRED)->Range(1024, 65536);
-
-BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::OPTIONAL)->Range(1024, 65536);
-
-BENCHMARK_TEMPLATE(BM_WriteInt64Column, Repetition::REPEATED)->Range(1024, 65536);
-
-std::unique_ptr<Int64Reader> BuildReader(
-    std::shared_ptr<Buffer>& buffer, int64_t num_values, ColumnDescriptor* schema) {
-  std::unique_ptr<InMemoryInputStream> source(new InMemoryInputStream(buffer));
-  std::unique_ptr<SerializedPageReader> page_reader(
-      new SerializedPageReader(std::move(source), num_values, Compression::UNCOMPRESSED));
-  return std::unique_ptr<Int64Reader>(new Int64Reader(schema, std::move(page_reader)));
-}
-
-template <Repetition::type repetition>
-static void BM_ReadInt64Column(::benchmark::State& state) {
-  format::ColumnChunk thrift_metadata;
-  std::vector<int64_t> values(state.range(0), 128);
-  std::vector<int16_t> definition_levels(state.range(0), 1);
-  std::vector<int16_t> repetition_levels(state.range(0), 0);
-  std::shared_ptr<ColumnDescriptor> schema = Int64Schema(repetition);
-  std::shared_ptr<WriterProperties> properties = default_writer_properties();
-  auto metadata = ColumnChunkMetaDataBuilder::Make(
-      properties, schema.get(), reinterpret_cast<uint8_t*>(&thrift_metadata));
-
-  InMemoryOutputStream stream;
-  std::unique_ptr<Int64Writer> writer = BuildWriter(
-      state.range(0), &stream, metadata.get(), schema.get(), properties.get());
-  writer->WriteBatch(
-      values.size(), definition_levels.data(), repetition_levels.data(), values.data());
-  writer->Close();
-
-  std::shared_ptr<Buffer> src = stream.GetBuffer();
-  std::vector<int64_t> values_out(state.range(1));
-  std::vector<int16_t> definition_levels_out(state.range(1));
-  std::vector<int16_t> repetition_levels_out(state.range(1));
-  while (state.KeepRunning()) {
-    std::unique_ptr<Int64Reader> reader = BuildReader(src, state.range(1), schema.get());
-    int64_t values_read = 0;
-    for (size_t i = 0; i < values.size(); i += values_read) {
-      reader->ReadBatch(values_out.size(), definition_levels_out.data(),
-          repetition_levels_out.data(), values_out.data(), &values_read);
-    }
-  }
-  SetBytesProcessed(state, repetition);
-}
-
-BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::REQUIRED)
-    ->RangePair(1024, 65536, 1, 1024);
-
-BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::OPTIONAL)
-    ->RangePair(1024, 65536, 1, 1024);
-
-BENCHMARK_TEMPLATE(BM_ReadInt64Column, Repetition::REPEATED)
-    ->RangePair(1024, 65536, 1, 1024);
-
-}  // namespace benchmark
-
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/column-reader-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/column-reader-test.cc b/src/parquet/column/column-reader-test.cc
deleted file mode 100644
index e34ac4c..0000000
--- a/src/parquet/column/column-reader-test.cc
+++ /dev/null
@@ -1,374 +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.
-
-#include <gtest/gtest.h>
-
-#include <algorithm>
-#include <cstdint>
-#include <cstdlib>
-#include <limits>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "parquet/column/page.h"
-#include "parquet/column/reader.h"
-#include "parquet/column/test-util.h"
-#include "parquet/schema.h"
-#include "parquet/types.h"
-#include "parquet/util/test-common.h"
-
-using std::string;
-using std::vector;
-using std::shared_ptr;
-
-namespace parquet {
-
-using schema::NodePtr;
-
-namespace test {
-
-template <typename T>
-static inline bool vector_equal_with_def_levels(const vector<T>& left,
-    const vector<int16_t>& def_levels, int16_t max_def_levels, int16_t max_rep_levels,
-    const vector<T>& right) {
-  size_t i_left = 0;
-  size_t i_right = 0;
-  for (size_t i = 0; i < def_levels.size(); i++) {
-    if (def_levels[i] == max_def_levels) {
-      // Compare
-      if (left[i_left] != right[i_right]) {
-        std::cerr << "index " << i << " left was " << left[i_left] << " right was "
-                  << right[i] << std::endl;
-        return false;
-      }
-      i_left++;
-      i_right++;
-    } else if (def_levels[i] == (max_def_levels -1)) {
-      // Null entry on the lowest nested level
-      i_right++;
-    } else if (def_levels[i] < (max_def_levels - 1)) {
-      // Null entry on a higher nesting level, only supported for non-repeating data
-      if (max_rep_levels == 0) {
-        i_right++;
-      }
-    }
-  }
-
-  return true;
-}
-
-class TestPrimitiveReader : public ::testing::Test {
- public:
-  void InitReader(const ColumnDescriptor* d) {
-    std::unique_ptr<PageReader> pager_;
-    pager_.reset(new test::MockPageReader(pages_));
-    reader_ = ColumnReader::Make(d, std::move(pager_));
-  }
-
-  void CheckResults() {
-    vector<int32_t> vresult(num_values_, -1);
-    vector<int16_t> dresult(num_levels_, -1);
-    vector<int16_t> rresult(num_levels_, -1);
-    int64_t values_read = 0;
-    int total_values_read = 0;
-    int batch_actual = 0;
-
-    Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
-    int32_t batch_size = 8;
-    int batch = 0;
-    // This will cover both the cases
-    // 1) batch_size < page_size (multiple ReadBatch from a single page)
-    // 2) batch_size > page_size (BatchRead limits to a single page)
-    do {
-      batch = static_cast<int>(reader->ReadBatch(batch_size, &dresult[0] + batch_actual,
-          &rresult[0] + batch_actual, &vresult[0] + total_values_read, &values_read));
-      total_values_read += static_cast<int>(values_read);
-      batch_actual += batch;
-      batch_size = std::max(batch_size * 2, 4096);
-    } while (batch > 0);
-
-    ASSERT_EQ(num_levels_, batch_actual);
-    ASSERT_EQ(num_values_, total_values_read);
-    ASSERT_TRUE(vector_equal(values_, vresult));
-    if (max_def_level_ > 0) { ASSERT_TRUE(vector_equal(def_levels_, dresult)); }
-    if (max_rep_level_ > 0) { ASSERT_TRUE(vector_equal(rep_levels_, rresult)); }
-    // catch improper writes at EOS
-    batch_actual =
-        static_cast<int>(reader->ReadBatch(5, nullptr, nullptr, nullptr, &values_read));
-    ASSERT_EQ(0, batch_actual);
-    ASSERT_EQ(0, values_read);
-  }
-
-  void CheckResultsSpaced() {
-    vector<int32_t> vresult(num_levels_, -1);
-    vector<int16_t> dresult(num_levels_, -1);
-    vector<int16_t> rresult(num_levels_, -1);
-    vector<uint8_t> valid_bits(num_levels_, 255);
-    int total_values_read = 0;
-    int batch_actual = 0;
-    int levels_actual = 0;
-    int64_t null_count = -1;
-    int64_t levels_read = 0;
-    int64_t values_read;
-
-    Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
-    int32_t batch_size = 8;
-    int batch = 0;
-    // This will cover both the cases
-    // 1) batch_size < page_size (multiple ReadBatch from a single page)
-    // 2) batch_size > page_size (BatchRead limits to a single page)
-    do {
-      batch = static_cast<int>(reader->ReadBatchSpaced(batch_size,
-          dresult.data() + levels_actual, rresult.data() + levels_actual,
-          vresult.data() + batch_actual, valid_bits.data() + batch_actual, 0,
-          &levels_read, &values_read, &null_count));
-      total_values_read += batch - static_cast<int>(null_count);
-      batch_actual += batch;
-      levels_actual += static_cast<int>(levels_read);
-      batch_size = std::max(batch_size * 2, 4096);
-    } while ((batch > 0) || (levels_read > 0));
-
-    ASSERT_EQ(num_levels_, levels_actual);
-    ASSERT_EQ(num_values_, total_values_read);
-    if (max_def_level_ > 0) {
-      ASSERT_TRUE(vector_equal(def_levels_, dresult));
-      ASSERT_TRUE(
-          vector_equal_with_def_levels(values_, dresult, max_def_level_,
-            max_rep_level_, vresult));
-    } else {
-      ASSERT_TRUE(vector_equal(values_, vresult));
-    }
-    if (max_rep_level_ > 0) { ASSERT_TRUE(vector_equal(rep_levels_, rresult)); }
-    // catch improper writes at EOS
-    batch_actual = static_cast<int>(reader->ReadBatchSpaced(5, nullptr, nullptr, nullptr,
-        valid_bits.data(), 0, &levels_read, &values_read, &null_count));
-    ASSERT_EQ(0, batch_actual);
-    ASSERT_EQ(0, null_count);
-  }
-
-  void Clear() {
-    values_.clear();
-    def_levels_.clear();
-    rep_levels_.clear();
-    pages_.clear();
-    reader_.reset();
-  }
-
-  void ExecutePlain(int num_pages, int levels_per_page, const ColumnDescriptor* d) {
-    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
-        rep_levels_, values_, data_buffer_, pages_, Encoding::PLAIN);
-    num_levels_ = num_pages * levels_per_page;
-    InitReader(d);
-    CheckResults();
-    Clear();
-
-    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
-        rep_levels_, values_, data_buffer_, pages_, Encoding::PLAIN);
-    num_levels_ = num_pages * levels_per_page;
-    InitReader(d);
-    CheckResultsSpaced();
-    Clear();
-  }
-
-  void ExecuteDict(int num_pages, int levels_per_page, const ColumnDescriptor* d) {
-    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
-        rep_levels_, values_, data_buffer_, pages_, Encoding::RLE_DICTIONARY);
-    num_levels_ = num_pages * levels_per_page;
-    InitReader(d);
-    CheckResults();
-    Clear();
-
-    num_values_ = MakePages<Int32Type>(d, num_pages, levels_per_page, def_levels_,
-        rep_levels_, values_, data_buffer_, pages_, Encoding::RLE_DICTIONARY);
-    num_levels_ = num_pages * levels_per_page;
-    InitReader(d);
-    CheckResultsSpaced();
-    Clear();
-  }
-
- protected:
-  int num_levels_;
-  int num_values_;
-  int16_t max_def_level_;
-  int16_t max_rep_level_;
-  vector<shared_ptr<Page>> pages_;
-  std::shared_ptr<ColumnReader> reader_;
-  vector<int32_t> values_;
-  vector<int16_t> def_levels_;
-  vector<int16_t> rep_levels_;
-  vector<uint8_t> data_buffer_;  // For BA and FLBA
-};
-
-TEST_F(TestPrimitiveReader, TestInt32FlatRequired) {
-  int levels_per_page = 100;
-  int num_pages = 50;
-  max_def_level_ = 0;
-  max_rep_level_ = 0;
-  NodePtr type = schema::Int32("a", Repetition::REQUIRED);
-  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
-}
-
-TEST_F(TestPrimitiveReader, TestInt32FlatOptional) {
-  int levels_per_page = 100;
-  int num_pages = 50;
-  max_def_level_ = 4;
-  max_rep_level_ = 0;
-  NodePtr type = schema::Int32("b", Repetition::OPTIONAL);
-  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
-}
-
-TEST_F(TestPrimitiveReader, TestInt32FlatRepeated) {
-  int levels_per_page = 100;
-  int num_pages = 50;
-  max_def_level_ = 4;
-  max_rep_level_ = 2;
-  NodePtr type = schema::Int32("c", Repetition::REPEATED);
-  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  ExecutePlain(num_pages, levels_per_page, &descr);
-  ExecuteDict(num_pages, levels_per_page, &descr);
-}
-
-TEST_F(TestPrimitiveReader, TestInt32FlatRequiredSkip) {
-  int levels_per_page = 100;
-  int num_pages = 5;
-  max_def_level_ = 0;
-  max_rep_level_ = 0;
-  NodePtr type = schema::Int32("b", Repetition::REQUIRED);
-  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  MakePages<Int32Type>(&descr, num_pages, levels_per_page, def_levels_, rep_levels_,
-      values_, data_buffer_, pages_, Encoding::PLAIN);
-  InitReader(&descr);
-  vector<int32_t> vresult(levels_per_page / 2, -1);
-  vector<int16_t> dresult(levels_per_page / 2, -1);
-  vector<int16_t> rresult(levels_per_page / 2, -1);
-
-  Int32Reader* reader = static_cast<Int32Reader*>(reader_.get());
-  int64_t values_read = 0;
-
-  // 1) skip_size > page_size (multiple pages skipped)
-  // Skip first 2 pages
-  int64_t levels_skipped = reader->Skip(2 * levels_per_page);
-  ASSERT_EQ(2 * levels_per_page, levels_skipped);
-  // Read half a page
-  reader->ReadBatch(
-      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
-  vector<int32_t> sub_values(values_.begin() + 2 * levels_per_page,
-      values_.begin() + static_cast<int>(2.5 * static_cast<double>(levels_per_page)));
-  ASSERT_TRUE(vector_equal(sub_values, vresult));
-
-  // 2) skip_size == page_size (skip across two pages)
-  levels_skipped = reader->Skip(levels_per_page);
-  ASSERT_EQ(levels_per_page, levels_skipped);
-  // Read half a page
-  reader->ReadBatch(
-      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
-  sub_values.clear();
-  sub_values.insert(sub_values.end(),
-      values_.begin() + static_cast<int>(3.5 * static_cast<double>(levels_per_page)),
-      values_.begin() + 4 * levels_per_page);
-  ASSERT_TRUE(vector_equal(sub_values, vresult));
-
-  // 3) skip_size < page_size (skip limited to a single page)
-  // Skip half a page
-  levels_skipped = reader->Skip(levels_per_page / 2);
-  ASSERT_EQ(0.5 * levels_per_page, levels_skipped);
-  // Read half a page
-  reader->ReadBatch(
-      levels_per_page / 2, dresult.data(), rresult.data(), vresult.data(), &values_read);
-  sub_values.clear();
-  sub_values.insert(sub_values.end(),
-      values_.begin() + static_cast<int>(4.5 * static_cast<double>(levels_per_page)),
-      values_.end());
-  ASSERT_TRUE(vector_equal(sub_values, vresult));
-
-  values_.clear();
-  def_levels_.clear();
-  rep_levels_.clear();
-  pages_.clear();
-  reader_.reset();
-}
-
-TEST_F(TestPrimitiveReader, TestDictionaryEncodedPages) {
-  max_def_level_ = 0;
-  max_rep_level_ = 0;
-  NodePtr type = schema::Int32("a", Repetition::REQUIRED);
-  const ColumnDescriptor descr(type, max_def_level_, max_rep_level_);
-  shared_ptr<PoolBuffer> dummy = std::make_shared<PoolBuffer>();
-
-  shared_ptr<DictionaryPage> dict_page =
-      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN);
-  shared_ptr<DataPage> data_page = MakeDataPage<Int32Type>(
-      &descr, {}, 0, Encoding::RLE_DICTIONARY, {}, 0, {}, 0, {}, 0);
-  pages_.push_back(dict_page);
-  pages_.push_back(data_page);
-  InitReader(&descr);
-  // Tests Dict : PLAIN, Data : RLE_DICTIONARY
-  ASSERT_NO_THROW(reader_->HasNext());
-  pages_.clear();
-
-  dict_page = std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN_DICTIONARY);
-  data_page = MakeDataPage<Int32Type>(
-      &descr, {}, 0, Encoding::PLAIN_DICTIONARY, {}, 0, {}, 0, {}, 0);
-  pages_.push_back(dict_page);
-  pages_.push_back(data_page);
-  InitReader(&descr);
-  // Tests Dict : PLAIN_DICTIONARY, Data : PLAIN_DICTIONARY
-  ASSERT_NO_THROW(reader_->HasNext());
-  pages_.clear();
-
-  data_page = MakeDataPage<Int32Type>(
-      &descr, {}, 0, Encoding::RLE_DICTIONARY, {}, 0, {}, 0, {}, 0);
-  pages_.push_back(data_page);
-  InitReader(&descr);
-  // Tests dictionary page must occur before data page
-  ASSERT_THROW(reader_->HasNext(), ParquetException);
-  pages_.clear();
-
-  dict_page = std::make_shared<DictionaryPage>(dummy, 0, Encoding::DELTA_BYTE_ARRAY);
-  pages_.push_back(dict_page);
-  InitReader(&descr);
-  // Tests only RLE_DICTIONARY is supported
-  ASSERT_THROW(reader_->HasNext(), ParquetException);
-  pages_.clear();
-
-  shared_ptr<DictionaryPage> dict_page1 =
-      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN_DICTIONARY);
-  shared_ptr<DictionaryPage> dict_page2 =
-      std::make_shared<DictionaryPage>(dummy, 0, Encoding::PLAIN);
-  pages_.push_back(dict_page1);
-  pages_.push_back(dict_page2);
-  InitReader(&descr);
-  // Column cannot have more than one dictionary
-  ASSERT_THROW(reader_->HasNext(), ParquetException);
-  pages_.clear();
-
-  data_page = MakeDataPage<Int32Type>(
-      &descr, {}, 0, Encoding::DELTA_BYTE_ARRAY, {}, 0, {}, 0, {}, 0);
-  pages_.push_back(data_page);
-  InitReader(&descr);
-  // unsupported encoding
-  ASSERT_THROW(reader_->HasNext(), ParquetException);
-  pages_.clear();
-}
-
-}  // namespace test
-}  // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/84db929e/src/parquet/column/column-writer-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/column-writer-test.cc b/src/parquet/column/column-writer-test.cc
deleted file mode 100644
index 1e1da4a..0000000
--- a/src/parquet/column/column-writer-test.cc
+++ /dev/null
@@ -1,516 +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.
-
-#include <gtest/gtest.h>
-
-#include "parquet/column/test-specialization.h"
-#include "parquet/column/test-util.h"
-
-#include "parquet/column/reader.h"
-#include "parquet/column/writer.h"
-#include "parquet/file/reader-internal.h"
-#include "parquet/file/writer-internal.h"
-#include "parquet/types.h"
-#include "parquet/util/comparison.h"
-#include "parquet/util/memory.h"
-
-namespace parquet {
-
-using schema::NodePtr;
-using schema::PrimitiveNode;
-
-namespace test {
-
-// The default size used in most tests.
-const int SMALL_SIZE = 100;
-// Larger size to test some corner cases, only used in some specific cases.
-const int LARGE_SIZE = 100000;
-// Very large size to test dictionary fallback.
-const int VERY_LARGE_SIZE = 400000;
-
-template <typename TestType>
-class TestPrimitiveWriter : public PrimitiveTypedTest<TestType> {
- public:
-  typedef typename TestType::c_type T;
-
-  void SetUp() {
-    this->SetupValuesOut(SMALL_SIZE);
-    writer_properties_ = default_writer_properties();
-    definition_levels_out_.resize(SMALL_SIZE);
-    repetition_levels_out_.resize(SMALL_SIZE);
-
-    this->SetUpSchema(Repetition::REQUIRED);
-
-    descr_ = this->schema_.Column(0);
-  }
-
-  Type::type type_num() { return TestType::type_num; }
-
-  void BuildReader(
-      int64_t num_rows, Compression::type compression = Compression::UNCOMPRESSED) {
-    auto buffer = sink_->GetBuffer();
-    std::unique_ptr<InMemoryInputStream> source(new InMemoryInputStream(buffer));
-    std::unique_ptr<SerializedPageReader> page_reader(
-        new SerializedPageReader(std::move(source), num_rows, compression));
-    reader_.reset(new TypedColumnReader<TestType>(this->descr_, std::move(page_reader)));
-  }
-
-  std::shared_ptr<TypedColumnWriter<TestType>> BuildWriter(
-      int64_t output_size = SMALL_SIZE,
-      const ColumnProperties& column_properties = ColumnProperties()) {
-    sink_.reset(new InMemoryOutputStream());
-    metadata_ = ColumnChunkMetaDataBuilder::Make(
-        writer_properties_, this->descr_, reinterpret_cast<uint8_t*>(&thrift_metadata_));
-    std::unique_ptr<SerializedPageWriter> pager(
-        new SerializedPageWriter(sink_.get(), column_properties.codec, metadata_.get()));
-    WriterProperties::Builder wp_builder;
-    if (column_properties.encoding == Encoding::PLAIN_DICTIONARY ||
-        column_properties.encoding == Encoding::RLE_DICTIONARY) {
-      wp_builder.enable_dictionary();
-    } else {
-      wp_builder.disable_dictionary();
-      wp_builder.encoding(column_properties.encoding);
-    }
-    writer_properties_ = wp_builder.build();
-    std::shared_ptr<ColumnWriter> writer = ColumnWriter::Make(
-        metadata_.get(), std::move(pager), output_size, writer_properties_.get());
-    return std::static_pointer_cast<TypedColumnWriter<TestType>>(writer);
-  }
-
-  void ReadColumn(Compression::type compression = Compression::UNCOMPRESSED) {
-    BuildReader(static_cast<int64_t>(this->values_out_.size()), compression);
-    reader_->ReadBatch(static_cast<int>(this->values_out_.size()),
-        definition_levels_out_.data(), repetition_levels_out_.data(),
-        this->values_out_ptr_, &values_read_);
-    this->SyncValuesOut();
-  }
-
-  void ReadColumnFully(Compression::type compression = Compression::UNCOMPRESSED);
-
-  void TestRequiredWithEncoding(Encoding::type encoding) {
-    return TestRequiredWithSettings(encoding, Compression::UNCOMPRESSED, false, false);
-  }
-
-  void TestRequiredWithSettings(Encoding::type encoding, Compression::type compression,
-      bool enable_dictionary, bool enable_statistics, int64_t num_rows = SMALL_SIZE) {
-    this->GenerateData(num_rows);
-
-    this->WriteRequiredWithSettings(
-        encoding, compression, enable_dictionary, enable_statistics, num_rows);
-    this->ReadAndCompare(compression, num_rows);
-
-    this->WriteRequiredWithSettingsSpaced(
-        encoding, compression, enable_dictionary, enable_statistics, num_rows);
-    this->ReadAndCompare(compression, num_rows);
-  }
-
-  void WriteRequiredWithSettings(Encoding::type encoding, Compression::type compression,
-      bool enable_dictionary, bool enable_statistics, int64_t num_rows) {
-    ColumnProperties column_properties(
-        encoding, compression, enable_dictionary, enable_statistics);
-    std::shared_ptr<TypedColumnWriter<TestType>> writer =
-        this->BuildWriter(num_rows, column_properties);
-    writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
-    // The behaviour should be independent from the number of Close() calls
-    writer->Close();
-    writer->Close();
-  }
-
-  void WriteRequiredWithSettingsSpaced(Encoding::type encoding,
-      Compression::type compression, bool enable_dictionary, bool enable_statistics,
-      int64_t num_rows) {
-    std::vector<uint8_t> valid_bits(
-        BitUtil::RoundUpNumBytes(static_cast<uint32_t>(this->values_.size())) + 1, 255);
-    ColumnProperties column_properties(
-        encoding, compression, enable_dictionary, enable_statistics);
-    std::shared_ptr<TypedColumnWriter<TestType>> writer =
-        this->BuildWriter(num_rows, column_properties);
-    writer->WriteBatchSpaced(
-        this->values_.size(), nullptr, nullptr, valid_bits.data(), 0, this->values_ptr_);
-    // The behaviour should be independent from the number of Close() calls
-    writer->Close();
-    writer->Close();
-  }
-
-  void ReadAndCompare(Compression::type compression, int64_t num_rows) {
-    this->SetupValuesOut(num_rows);
-    this->ReadColumnFully(compression);
-    Compare<T> compare(this->descr_);
-    for (size_t i = 0; i < this->values_.size(); i++) {
-      if (compare(this->values_[i], this->values_out_[i]) ||
-          compare(this->values_out_[i], this->values_[i])) {
-        std::cout << "Failed at " << i << std::endl;
-      }
-      ASSERT_FALSE(compare(this->values_[i], this->values_out_[i]));
-      ASSERT_FALSE(compare(this->values_out_[i], this->values_[i]));
-    }
-    ASSERT_EQ(this->values_, this->values_out_);
-  }
-
-  int64_t metadata_num_values() {
-    // Metadata accessor must be created lazily.
-    // This is because the ColumnChunkMetaData semantics dictate the metadata object is
-    // complete (no changes to the metadata buffer can be made after instantiation)
-    auto metadata_accessor = ColumnChunkMetaData::Make(
-        reinterpret_cast<const uint8_t*>(&thrift_metadata_), this->descr_);
-    return metadata_accessor->num_values();
-  }
-
-  std::vector<Encoding::type> metadata_encodings() {
-    // Metadata accessor must be created lazily.
-    // This is because the ColumnChunkMetaData semantics dictate the metadata object is
-    // complete (no changes to the metadata buffer can be made after instantiation)
-    auto metadata_accessor = ColumnChunkMetaData::Make(
-        reinterpret_cast<const uint8_t*>(&thrift_metadata_), this->descr_);
-    return metadata_accessor->encodings();
-  }
-
- protected:
-  int64_t values_read_;
-  // Keep the reader alive as for ByteArray the lifetime of the ByteArray
-  // content is bound to the reader.
-  std::unique_ptr<TypedColumnReader<TestType>> reader_;
-
-  std::vector<int16_t> definition_levels_out_;
-  std::vector<int16_t> repetition_levels_out_;
-
-  const ColumnDescriptor* descr_;
-
- private:
-  format::ColumnChunk thrift_metadata_;
-  std::unique_ptr<ColumnChunkMetaDataBuilder> metadata_;
-  std::unique_ptr<InMemoryOutputStream> sink_;
-  std::shared_ptr<WriterProperties> writer_properties_;
-  std::vector<std::vector<uint8_t>> data_buffer_;
-};
-
-template <typename TestType>
-void TestPrimitiveWriter<TestType>::ReadColumnFully(Compression::type compression) {
-  int64_t total_values = static_cast<int64_t>(this->values_out_.size());
-  BuildReader(total_values, compression);
-  values_read_ = 0;
-  while (values_read_ < total_values) {
-    int64_t values_read_recently = 0;
-    reader_->ReadBatch(
-        static_cast<int>(this->values_out_.size()) - static_cast<int>(values_read_),
-        definition_levels_out_.data() + values_read_,
-        repetition_levels_out_.data() + values_read_,
-        this->values_out_ptr_ + values_read_, &values_read_recently);
-    values_read_ += values_read_recently;
-  }
-  this->SyncValuesOut();
-}
-
-template <>
-void TestPrimitiveWriter<FLBAType>::ReadColumnFully(Compression::type compression) {
-  int64_t total_values = static_cast<int64_t>(this->values_out_.size());
-  BuildReader(total_values, compression);
-  this->data_buffer_.clear();
-
-  values_read_ = 0;
-  while (values_read_ < total_values) {
-    int64_t values_read_recently = 0;
-    reader_->ReadBatch(
-        static_cast<int>(this->values_out_.size()) - static_cast<int>(values_read_),
-        definition_levels_out_.data() + values_read_,
-        repetition_levels_out_.data() + values_read_,
-        this->values_out_ptr_ + values_read_, &values_read_recently);
-
-    // Copy contents of the pointers
-    std::vector<uint8_t> data(values_read_recently * this->descr_->type_length());
-    uint8_t* data_ptr = data.data();
-    for (int64_t i = 0; i < values_read_recently; i++) {
-      memcpy(data_ptr + this->descr_->type_length() * i,
-          this->values_out_[i + values_read_].ptr, this->descr_->type_length());
-      this->values_out_[i + values_read_].ptr =
-          data_ptr + this->descr_->type_length() * i;
-    }
-    data_buffer_.emplace_back(std::move(data));
-
-    values_read_ += values_read_recently;
-  }
-  this->SyncValuesOut();
-}
-
-typedef ::testing::Types<Int32Type, Int64Type, Int96Type, FloatType, DoubleType,
-    BooleanType, ByteArrayType, FLBAType>
-    TestTypes;
-
-TYPED_TEST_CASE(TestPrimitiveWriter, TestTypes);
-
-using TestNullValuesWriter = TestPrimitiveWriter<Int32Type>;
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlain) {
-  this->TestRequiredWithEncoding(Encoding::PLAIN);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredDictionary) {
-  this->TestRequiredWithEncoding(Encoding::PLAIN_DICTIONARY);
-}
-
-/*
-TYPED_TEST(TestPrimitiveWriter, RequiredRLE) {
-  this->TestRequiredWithEncoding(Encoding::RLE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredBitPacked) {
-  this->TestRequiredWithEncoding(Encoding::BIT_PACKED);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredDeltaBinaryPacked) {
-  this->TestRequiredWithEncoding(Encoding::DELTA_BINARY_PACKED);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredDeltaLengthByteArray) {
-  this->TestRequiredWithEncoding(Encoding::DELTA_LENGTH_BYTE_ARRAY);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredDeltaByteArray) {
-  this->TestRequiredWithEncoding(Encoding::DELTA_BYTE_ARRAY);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredRLEDictionary) {
-  this->TestRequiredWithEncoding(Encoding::RLE_DICTIONARY);
-}
-*/
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithSnappyCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::SNAPPY, false, false, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithBrotliCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::BROTLI, false, false, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithGzipCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::GZIP, false, false, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStats) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::UNCOMPRESSED, false, true, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndSnappyCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::SNAPPY, false, true, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndBrotliCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::BROTLI, false, true, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredPlainWithStatsAndGzipCompression) {
-  this->TestRequiredWithSettings(
-      Encoding::PLAIN, Compression::GZIP, false, true, LARGE_SIZE);
-}
-
-TYPED_TEST(TestPrimitiveWriter, Optional) {
-  // Optional and non-repeated, with definition levels
-  // but no repetition levels
-  this->SetUpSchema(Repetition::OPTIONAL);
-
-  this->GenerateData(SMALL_SIZE);
-  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
-  definition_levels[1] = 0;
-
-  auto writer = this->BuildWriter();
-  writer->WriteBatch(
-      this->values_.size(), definition_levels.data(), nullptr, this->values_ptr_);
-  writer->Close();
-
-  // PARQUET-703
-  ASSERT_EQ(100, this->metadata_num_values());
-
-  this->ReadColumn();
-  ASSERT_EQ(99, this->values_read_);
-  this->values_out_.resize(99);
-  this->values_.resize(99);
-  ASSERT_EQ(this->values_, this->values_out_);
-}
-
-TYPED_TEST(TestPrimitiveWriter, OptionalSpaced) {
-  // Optional and non-repeated, with definition levels
-  // but no repetition levels
-  this->SetUpSchema(Repetition::OPTIONAL);
-
-  this->GenerateData(SMALL_SIZE);
-  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
-  std::vector<uint8_t> valid_bits(::arrow::BitUtil::BytesForBits(SMALL_SIZE), 255);
-
-  definition_levels[SMALL_SIZE - 1] = 0;
-  ::arrow::BitUtil::ClearBit(valid_bits.data(), SMALL_SIZE - 1);
-  definition_levels[1] = 0;
-  ::arrow::BitUtil::ClearBit(valid_bits.data(), 1);
-
-  auto writer = this->BuildWriter();
-  writer->WriteBatchSpaced(this->values_.size(), definition_levels.data(), nullptr,
-      valid_bits.data(), 0, this->values_ptr_);
-  writer->Close();
-
-  // PARQUET-703
-  ASSERT_EQ(100, this->metadata_num_values());
-
-  this->ReadColumn();
-  ASSERT_EQ(98, this->values_read_);
-  this->values_out_.resize(98);
-  this->values_.resize(99);
-  this->values_.erase(this->values_.begin() + 1);
-  ASSERT_EQ(this->values_, this->values_out_);
-}
-
-TYPED_TEST(TestPrimitiveWriter, Repeated) {
-  // Optional and repeated, so definition and repetition levels
-  this->SetUpSchema(Repetition::REPEATED);
-
-  this->GenerateData(SMALL_SIZE);
-  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
-  definition_levels[1] = 0;
-  std::vector<int16_t> repetition_levels(SMALL_SIZE, 0);
-
-  auto writer = this->BuildWriter();
-  writer->WriteBatch(this->values_.size(), definition_levels.data(),
-      repetition_levels.data(), this->values_ptr_);
-  writer->Close();
-
-  this->ReadColumn();
-  ASSERT_EQ(SMALL_SIZE - 1, this->values_read_);
-  this->values_out_.resize(SMALL_SIZE - 1);
-  this->values_.resize(SMALL_SIZE - 1);
-  ASSERT_EQ(this->values_, this->values_out_);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredTooFewRows) {
-  this->GenerateData(SMALL_SIZE - 1);
-
-  auto writer = this->BuildWriter();
-  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
-  ASSERT_THROW(writer->Close(), ParquetException);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredTooMany) {
-  this->GenerateData(2 * SMALL_SIZE);
-
-  auto writer = this->BuildWriter();
-  ASSERT_THROW(
-      writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_),
-      ParquetException);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RepeatedTooFewRows) {
-  // Optional and repeated, so definition and repetition levels
-  this->SetUpSchema(Repetition::REPEATED);
-
-  this->GenerateData(SMALL_SIZE);
-  std::vector<int16_t> definition_levels(SMALL_SIZE, 1);
-  definition_levels[1] = 0;
-  std::vector<int16_t> repetition_levels(SMALL_SIZE, 0);
-  repetition_levels[3] = 1;
-
-  auto writer = this->BuildWriter();
-  writer->WriteBatch(this->values_.size(), definition_levels.data(),
-      repetition_levels.data(), this->values_ptr_);
-  ASSERT_THROW(writer->Close(), ParquetException);
-}
-
-TYPED_TEST(TestPrimitiveWriter, RequiredLargeChunk) {
-  this->GenerateData(LARGE_SIZE);
-
-  // Test case 1: required and non-repeated, so no definition or repetition levels
-  auto writer = this->BuildWriter(LARGE_SIZE);
-  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
-  writer->Close();
-
-  // Just read the first SMALL_SIZE rows to ensure we could read it back in
-  this->ReadColumn();
-  ASSERT_EQ(SMALL_SIZE, this->values_read_);
-  this->values_.resize(SMALL_SIZE);
-  ASSERT_EQ(this->values_, this->values_out_);
-}
-
-// Test case for dictionary fallback encoding
-TYPED_TEST(TestPrimitiveWriter, RequiredVeryLargeChunk) {
-  this->GenerateData(VERY_LARGE_SIZE);
-
-  auto writer = this->BuildWriter(VERY_LARGE_SIZE, Encoding::PLAIN_DICTIONARY);
-  writer->WriteBatch(this->values_.size(), nullptr, nullptr, this->values_ptr_);
-  writer->Close();
-
-  // Read all rows so we are sure that also the non-dictionary pages are read correctly
-  this->SetupValuesOut(VERY_LARGE_SIZE);
-  this->ReadColumnFully();
-  ASSERT_EQ(VERY_LARGE_SIZE, this->values_read_);
-  this->values_.resize(VERY_LARGE_SIZE);
-  ASSERT_EQ(this->values_, this->values_out_);
-  std::vector<Encoding::type> encodings = this->metadata_encodings();
-  // There are 3 encodings (RLE, PLAIN_DICTIONARY, PLAIN) in a fallback case
-  // Dictionary encoding is not allowed for boolean type
-  // There are 2 encodings (RLE, PLAIN) in a non dictionary encoding case
-  if (this->type_num() != Type::BOOLEAN) {
-    ASSERT_EQ(Encoding::PLAIN_DICTIONARY, encodings[0]);
-    ASSERT_EQ(Encoding::PLAIN, encodings[1]);
-    ASSERT_EQ(Encoding::RLE, encodings[2]);
-  } else {
-    ASSERT_EQ(Encoding::PLAIN, encodings[0]);
-    ASSERT_EQ(Encoding::RLE, encodings[1]);
-  }
-}
-
-// PARQUET-719
-// Test case for NULL values
-TEST_F(TestNullValuesWriter, OptionalNullValueChunk) {
-  this->SetUpSchema(Repetition::OPTIONAL);
-
-  this->GenerateData(LARGE_SIZE);
-
-  std::vector<int16_t> definition_levels(LARGE_SIZE, 0);
-  std::vector<int16_t> repetition_levels(LARGE_SIZE, 0);
-
-  auto writer = this->BuildWriter(LARGE_SIZE);
-  // All values being written are NULL
-  writer->WriteBatch(
-      this->values_.size(), definition_levels.data(), repetition_levels.data(), NULL);
-  writer->Close();
-
-  // Just read the first SMALL_SIZE rows to ensure we could read it back in
-  this->ReadColumn();
-  ASSERT_EQ(0, this->values_read_);
-}
-
-// PARQUET-764
-// Correct bitpacking for boolean write at non-byte boundaries
-using TestBooleanValuesWriter = TestPrimitiveWriter<BooleanType>;
-TEST_F(TestBooleanValuesWriter, AlternateBooleanValues) {
-  this->SetUpSchema(Repetition::REQUIRED);
-  auto writer = this->BuildWriter();
-  for (int i = 0; i < SMALL_SIZE; i++) {
-    bool value = (i % 2 == 0) ? true : false;
-    writer->WriteBatch(1, nullptr, nullptr, &value);
-  }
-  writer->Close();
-  this->ReadColumn();
-  for (int i = 0; i < SMALL_SIZE; i++) {
-    ASSERT_EQ((i % 2 == 0) ? true : false, this->values_out_[i]) << i;
-  }
-}
-
-}  // namespace test
-}  // namespace parquet