You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by we...@apache.org on 2016/04/30 20:13:21 UTC

parquet-cpp git commit: PARQUET-547: Refactor templates to all be based on DataType structs

Repository: parquet-cpp
Updated Branches:
  refs/heads/master 5fb7d20a9 -> dc0fc7d49


PARQUET-547: Refactor templates to all be based on DataType structs

Author: Wes McKinney <we...@apache.org>

Closes #91 from wesm/PARQUET-547 and squashes the following commits:

97b8b9a [Wes McKinney] Refactor templates to all be based on DataType subclasses


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

Branch: refs/heads/master
Commit: dc0fc7d49ff2ffa8dcaba618a36f8524b9dbeba4
Parents: 5fb7d20
Author: Wes McKinney <we...@apache.org>
Authored: Sat Apr 30 11:13:34 2016 -0700
Committer: Wes McKinney <we...@apache.org>
Committed: Sat Apr 30 11:13:34 2016 -0700

----------------------------------------------------------------------
 example/decode_benchmark.cc                     |  4 +-
 src/parquet/column/reader.cc                    | 30 ++++++-------
 src/parquet/column/reader.h                     | 30 ++++++-------
 src/parquet/column/scanner-test.cc              | 12 ++---
 src/parquet/column/scanner.h                    | 46 ++++++++++----------
 src/parquet/column/test-util.h                  |  4 +-
 src/parquet/column/writer.cc                    | 23 +++++-----
 src/parquet/column/writer.h                     | 32 +++++++-------
 src/parquet/encodings/decoder.h                 |  6 +--
 src/parquet/encodings/delta-bit-pack-encoding.h | 12 ++---
 .../encodings/delta-byte-array-encoding.h       |  8 ++--
 .../delta-length-byte-array-encoding.h          |  8 ++--
 src/parquet/encodings/dictionary-encoding.h     | 27 ++++++------
 src/parquet/encodings/encoder.h                 |  4 +-
 src/parquet/encodings/encoding-test.cc          | 16 +++----
 src/parquet/encodings/plain-encoding.h          | 40 ++++++++---------
 src/parquet/types.h                             |  4 +-
 17 files changed, 151 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/example/decode_benchmark.cc
----------------------------------------------------------------------
diff --git a/example/decode_benchmark.cc b/example/decode_benchmark.cc
index 3285a71..81eee02 100644
--- a/example/decode_benchmark.cc
+++ b/example/decode_benchmark.cc
@@ -202,7 +202,7 @@ class DeltaByteArrayEncoder {
 
 uint64_t TestPlainIntEncoding(const uint8_t* data, int num_values, int batch_size) {
   uint64_t result = 0;
-  PlainDecoder<Type::INT64> decoder(nullptr);
+  PlainDecoder<Int64Type> decoder(nullptr);
   decoder.SetData(num_values, data, num_values * sizeof(int64_t));
   int64_t values[batch_size];
   for (int i = 0; i < num_values;) {
@@ -225,7 +225,7 @@ uint64_t TestBinaryPackedEncoding(const char* name, const vector<int64_t>& value
   } else {
     mini_block_size = 32;
   }
-  DeltaBitPackDecoder<Type::INT64> decoder(nullptr);
+  DeltaBitPackDecoder<Int64Type> decoder(nullptr);
   DeltaBitPackEncoder encoder(mini_block_size);
   for (size_t i = 0; i < values.size(); ++i) {
     encoder.Add(values[i]);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/reader.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.cc b/src/parquet/column/reader.cc
index f379348..4598dfb 100644
--- a/src/parquet/column/reader.cc
+++ b/src/parquet/column/reader.cc
@@ -36,8 +36,8 @@ ColumnReader::ColumnReader(const ColumnDescriptor* descr,
     num_decoded_values_(0),
     allocator_(allocator) {}
 
-template <int TYPE>
-void TypedColumnReader<TYPE>::ConfigureDictionary(const DictionaryPage* page) {
+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) {
@@ -51,7 +51,7 @@ void TypedColumnReader<TYPE>::ConfigureDictionary(const DictionaryPage* page) {
 
   if (page->encoding() == Encoding::PLAIN_DICTIONARY ||
       page->encoding() == Encoding::PLAIN) {
-    PlainDecoder<TYPE> dictionary(descr_);
+    PlainDecoder<DType> dictionary(descr_);
     dictionary.SetData(page->num_values(), page->data(), page->size());
 
     // The dictionary is fully decoded during DictionaryDecoder::Init, so the
@@ -60,7 +60,7 @@ void TypedColumnReader<TYPE>::ConfigureDictionary(const DictionaryPage* page) {
     // 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<TYPE> >(descr_, allocator_);
+    auto decoder = std::make_shared<DictionaryDecoder<DType> >(descr_, allocator_);
     decoder->SetDict(&dictionary);
     decoders_[encoding] = decoder;
   } else {
@@ -77,8 +77,8 @@ static bool IsDictionaryIndexEncoding(const Encoding::type& e) {
     e == Encoding::PLAIN_DICTIONARY;
 }
 
-template <int TYPE>
-bool TypedColumnReader<TYPE>::ReadNewPage() {
+template <typename DType>
+bool TypedColumnReader<DType>::ReadNewPage() {
   // Loop until we find the next data page.
   const uint8_t* buffer;
 
@@ -147,7 +147,7 @@ bool TypedColumnReader<TYPE>::ReadNewPage() {
       } else {
         switch (encoding) {
           case Encoding::PLAIN: {
-            std::shared_ptr<DecoderType> decoder(new PlainDecoder<TYPE>(descr_));
+            std::shared_ptr<DecoderType> decoder(new PlainDecoder<DType>(descr_));
             decoders_[static_cast<int>(encoding)] = decoder;
             current_decoder_ = decoder.get();
             break;
@@ -227,13 +227,13 @@ std::shared_ptr<ColumnReader> ColumnReader::Make(
 // ----------------------------------------------------------------------
 // Instantiate templated classes
 
-template class TypedColumnReader<Type::BOOLEAN>;
-template class TypedColumnReader<Type::INT32>;
-template class TypedColumnReader<Type::INT64>;
-template class TypedColumnReader<Type::INT96>;
-template class TypedColumnReader<Type::FLOAT>;
-template class TypedColumnReader<Type::DOUBLE>;
-template class TypedColumnReader<Type::BYTE_ARRAY>;
-template class TypedColumnReader<Type::FIXED_LEN_BYTE_ARRAY>;
+template class TypedColumnReader<BooleanType>;
+template class TypedColumnReader<Int32Type>;
+template class TypedColumnReader<Int64Type>;
+template class TypedColumnReader<Int96Type>;
+template class TypedColumnReader<FloatType>;
+template class TypedColumnReader<DoubleType>;
+template class TypedColumnReader<ByteArrayType>;
+template class TypedColumnReader<FLBAType>;
 
 } // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/reader.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/reader.h b/src/parquet/column/reader.h
index 0739e7e..7704c52 100644
--- a/src/parquet/column/reader.h
+++ b/src/parquet/column/reader.h
@@ -102,10 +102,10 @@ class ColumnReader {
 };
 
 // API to read values from a single column. This is the main client facing API.
-template <int TYPE>
+template <typename DType>
 class TypedColumnReader : public ColumnReader {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   TypedColumnReader(const ColumnDescriptor* schema,
       std::unique_ptr<PageReader> pager,
@@ -131,7 +131,7 @@ class TypedColumnReader : public ColumnReader {
       T* values, int64_t* values_read);
 
  private:
-  typedef Decoder<TYPE> DecoderType;
+  typedef Decoder<DType> DecoderType;
 
   // Advance to the next data page
   virtual bool ReadNewPage();
@@ -153,14 +153,14 @@ class TypedColumnReader : public ColumnReader {
 };
 
 
-template <int TYPE>
-inline int64_t TypedColumnReader<TYPE>::ReadValues(int64_t batch_size, T* out) {
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadValues(int64_t batch_size, T* out) {
   int64_t num_decoded = current_decoder_->Decode(out, batch_size);
   return num_decoded;
 }
 
-template <int TYPE>
-inline int64_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_levels,
+template <typename DType>
+inline int64_t TypedColumnReader<DType>::ReadBatch(int batch_size, int16_t* def_levels,
     int16_t* rep_levels, T* values, int64_t* values_read) {
   // HasNext invokes ReadNewPage
   if (!HasNext()) {
@@ -208,14 +208,14 @@ inline int64_t TypedColumnReader<TYPE>::ReadBatch(int batch_size, int16_t* def_l
 }
 
 
-typedef TypedColumnReader<Type::BOOLEAN> BoolReader;
-typedef TypedColumnReader<Type::INT32> Int32Reader;
-typedef TypedColumnReader<Type::INT64> Int64Reader;
-typedef TypedColumnReader<Type::INT96> Int96Reader;
-typedef TypedColumnReader<Type::FLOAT> FloatReader;
-typedef TypedColumnReader<Type::DOUBLE> DoubleReader;
-typedef TypedColumnReader<Type::BYTE_ARRAY> ByteArrayReader;
-typedef TypedColumnReader<Type::FIXED_LEN_BYTE_ARRAY> FixedLenByteArrayReader;
+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;
 
 } // namespace parquet
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/scanner-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner-test.cc b/src/parquet/column/scanner-test.cc
index 983f94e..78bc3c9 100644
--- a/src/parquet/column/scanner-test.cc
+++ b/src/parquet/column/scanner-test.cc
@@ -99,8 +99,8 @@ class TestFlatScanner : public ::testing::Test {
   }
 
   void CheckResults(int batch_size, const ColumnDescriptor *d) {
-    TypedScanner<Type::type_num>* scanner =
-      reinterpret_cast<TypedScanner<Type::type_num>* >(scanner_.get());
+    TypedScanner<Type>* scanner =
+      reinterpret_cast<TypedScanner<Type>* >(scanner_.get());
     T val;
     bool is_null = false;
     int16_t def_level;
@@ -243,8 +243,8 @@ TEST_F(TestFlatFLBAScanner, TestDescriptorAPI) {
       data_buffer_, pages_);
   num_levels_ = 1 * 100;
   InitScanner(&d);
-  TypedScanner<FLBAType::type_num>* scanner =
-    reinterpret_cast<TypedScanner<FLBAType::type_num>* >(scanner_.get());
+  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());
@@ -258,8 +258,8 @@ TEST_F(TestFlatFLBAScanner, TestFLBAPrinterNext) {
       data_buffer_, pages_);
   num_levels_ = 1 * 100;
   InitScanner(&d);
-  TypedScanner<FLBAType::type_num>* scanner =
-    reinterpret_cast<TypedScanner<FLBAType::type_num>* >(scanner_.get());
+  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++) {

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/scanner.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/scanner.h b/src/parquet/column/scanner.h
index 5d04a0e..d52838e 100644
--- a/src/parquet/column/scanner.h
+++ b/src/parquet/column/scanner.h
@@ -91,17 +91,17 @@ class Scanner {
 };
 
 
-template <int TYPE>
+template <typename DType>
 class TypedScanner : public Scanner {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   explicit TypedScanner(std::shared_ptr<ColumnReader> reader,
       int64_t batch_size = DEFAULT_SCANNER_BATCH_SIZE,
       MemoryAllocator* allocator = default_allocator()) :
       Scanner(reader, batch_size, allocator) {
-    typed_reader_ = static_cast<TypedColumnReader<TYPE>*>(reader.get());
-    int value_byte_size = type_traits<TYPE>::value_byte_size;
+    typed_reader_ = static_cast<TypedColumnReader<DType>*>(reader.get());
+    int value_byte_size = type_traits<DType::type_num>::value_byte_size;
     value_buffer_.Resize(batch_size_ * value_byte_size);
     values_ = reinterpret_cast<T*>(&value_buffer_[0]);
   }
@@ -183,7 +183,7 @@ class TypedScanner : public Scanner {
     }
 
     if (is_null) {
-      std::string null_fmt = format_fwf<Type::BYTE_ARRAY>(width);
+      std::string null_fmt = format_fwf<ByteArrayType>(width);
       snprintf(buffer, sizeof(buffer), null_fmt.c_str(), "NULL");
     } else {
       FormatValue(&val, buffer, sizeof(buffer), width);
@@ -193,7 +193,7 @@ class TypedScanner : public Scanner {
 
  private:
   // The ownership of this object is expressed through the reader_ variable in the base
-  TypedColumnReader<TYPE>* typed_reader_;
+  TypedColumnReader<DType>* typed_reader_;
 
   inline void FormatValue(void* val, char* buffer, int bufsize, int width);
 
@@ -201,47 +201,47 @@ class TypedScanner : public Scanner {
 };
 
 
-template <int TYPE>
-inline void TypedScanner<TYPE>::FormatValue(void* val, char* buffer,
+template <typename DType>
+inline void TypedScanner<DType>::FormatValue(void* val, char* buffer,
     int bufsize, int width) {
-  std::string fmt = format_fwf<TYPE>(width);
+  std::string fmt = format_fwf<DType>(width);
   snprintf(buffer, bufsize, fmt.c_str(), *reinterpret_cast<T*>(val));
 }
 
 template <>
-inline void TypedScanner<Type::INT96>::FormatValue(
+inline void TypedScanner<Int96Type>::FormatValue(
     void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<Type::INT96>(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<Type::BYTE_ARRAY>::FormatValue(
+inline void TypedScanner<ByteArrayType>::FormatValue(
     void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<Type::BYTE_ARRAY>(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<Type::FIXED_LEN_BYTE_ARRAY>::FormatValue(
+inline void TypedScanner<FLBAType>::FormatValue(
     void* val, char* buffer, int bufsize, int width) {
-  std::string fmt = format_fwf<Type::FIXED_LEN_BYTE_ARRAY>(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<Type::BOOLEAN> BoolScanner;
-typedef TypedScanner<Type::INT32> Int32Scanner;
-typedef TypedScanner<Type::INT64> Int64Scanner;
-typedef TypedScanner<Type::INT96> Int96Scanner;
-typedef TypedScanner<Type::FLOAT> FloatScanner;
-typedef TypedScanner<Type::DOUBLE> DoubleScanner;
-typedef TypedScanner<Type::BYTE_ARRAY> ByteArrayScanner;
-typedef TypedScanner<Type::FIXED_LEN_BYTE_ARRAY> FixedLenByteArrayScanner;
+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
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/test-util.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/test-util.h b/src/parquet/column/test-util.h
index 2801f3c..95b1981 100644
--- a/src/parquet/column/test-util.h
+++ b/src/parquet/column/test-util.h
@@ -128,7 +128,7 @@ class DataPageBuilder {
 
   void AppendValues(const ColumnDescriptor *d, const vector<T>& values,
       Encoding::type encoding = Encoding::PLAIN) {
-    PlainEncoder<Type::type_num> encoder(d);
+    PlainEncoder<Type> encoder(d);
     encoder.Encode(&values[0], values.size(), sink_);
 
     num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);
@@ -195,7 +195,7 @@ void DataPageBuilder<BooleanType>::AppendValues(const ColumnDescriptor *d,
   if (encoding != Encoding::PLAIN) {
     ParquetException::NYI("only plain encoding currently implemented");
   }
-  PlainEncoder<Type::BOOLEAN> encoder(d);
+  PlainEncoder<BooleanType> encoder(d);
   encoder.Encode(values, values.size(), sink_);
 
   num_values_ = std::max(static_cast<int32_t>(values.size()), num_values_);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/writer.cc
----------------------------------------------------------------------
diff --git a/src/parquet/column/writer.cc b/src/parquet/column/writer.cc
index f851316..4dcb672 100644
--- a/src/parquet/column/writer.cc
+++ b/src/parquet/column/writer.cc
@@ -117,14 +117,14 @@ int64_t ColumnWriter::Close() {
 // ----------------------------------------------------------------------
 // TypedColumnWriter
 
-template <int TYPE>
-TypedColumnWriter<TYPE>::TypedColumnWriter(const ColumnDescriptor* schema,
+template <typename Type>
+TypedColumnWriter<Type>::TypedColumnWriter(const ColumnDescriptor* schema,
       std::unique_ptr<PageWriter> pager, int64_t expected_rows,
       MemoryAllocator* allocator) :
       ColumnWriter(schema, std::move(pager), expected_rows, allocator) {
   // TODO(PARQUET-590) Get decoder type from WriterProperties
   current_encoder_ = std::unique_ptr<EncoderType>(
-      new PlainEncoder<TYPE>(schema, allocator));
+      new PlainEncoder<Type>(schema, allocator));
 }
 
 // ----------------------------------------------------------------------
@@ -170,14 +170,13 @@ std::shared_ptr<ColumnWriter> ColumnWriter::Make(
 // ----------------------------------------------------------------------
 // Instantiate templated classes
 
-template class TypedColumnWriter<Type::BOOLEAN>;
-template class TypedColumnWriter<Type::INT32>;
-template class TypedColumnWriter<Type::INT64>;
-template class TypedColumnWriter<Type::INT96>;
-template class TypedColumnWriter<Type::FLOAT>;
-template class TypedColumnWriter<Type::DOUBLE>;
-template class TypedColumnWriter<Type::BYTE_ARRAY>;
-template class TypedColumnWriter<Type::FIXED_LEN_BYTE_ARRAY>;
-
+template class TypedColumnWriter<BooleanType>;
+template class TypedColumnWriter<Int32Type>;
+template class TypedColumnWriter<Int64Type>;
+template class TypedColumnWriter<Int96Type>;
+template class TypedColumnWriter<FloatType>;
+template class TypedColumnWriter<DoubleType>;
+template class TypedColumnWriter<ByteArrayType>;
+template class TypedColumnWriter<FLBAType>;
 
 } // namespace parquet

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/column/writer.h
----------------------------------------------------------------------
diff --git a/src/parquet/column/writer.h b/src/parquet/column/writer.h
index b9ea265..7ccfe73 100644
--- a/src/parquet/column/writer.h
+++ b/src/parquet/column/writer.h
@@ -101,10 +101,10 @@ class ColumnWriter {
 };
 
 // API to write values to a single column. This is the main client facing API.
-template <int TYPE>
+template <typename DType>
 class TypedColumnWriter : public ColumnWriter {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   TypedColumnWriter(const ColumnDescriptor* schema,
       std::unique_ptr<PageWriter> pager, int64_t expected_rows,
@@ -116,7 +116,7 @@ class TypedColumnWriter : public ColumnWriter {
       T* values);
 
  private:
-  typedef Encoder<TYPE> EncoderType;
+  typedef Encoder<DType> EncoderType;
 
   // Write values to a temporary buffer before they are encoded into pages
   void WriteValues(int64_t num_values, T* values);
@@ -135,8 +135,8 @@ class TypedColumnWriter : public ColumnWriter {
 // See also: parquet-column/../column/impl/ColumnWriteStoreV2.java:sizeCheck
 const int64_t PAGE_VALUE_COUNT = 1000;
 
-template <int TYPE>
-inline void TypedColumnWriter<TYPE>::WriteBatch(int64_t num_values, int16_t* def_levels,
+template <typename DType>
+inline void TypedColumnWriter<DType>::WriteBatch(int64_t num_values, int16_t* def_levels,
     int16_t* rep_levels, T* values) {
   int64_t values_to_write = 0;
 
@@ -185,22 +185,20 @@ inline void TypedColumnWriter<TYPE>::WriteBatch(int64_t num_values, int16_t* def
   }
 }
 
-template <int TYPE>
-void TypedColumnWriter<TYPE>::WriteValues(int64_t num_values, T* values) {
+template <typename DType>
+void TypedColumnWriter<DType>::WriteValues(int64_t num_values, T* values) {
   current_encoder_->Encode(values, num_values, values_sink_.get());
 }
 
-
-typedef TypedColumnWriter<Type::BOOLEAN> BoolWriter;
-typedef TypedColumnWriter<Type::INT32> Int32Writer;
-typedef TypedColumnWriter<Type::INT64> Int64Writer;
-typedef TypedColumnWriter<Type::INT96> Int96Writer;
-typedef TypedColumnWriter<Type::FLOAT> FloatWriter;
-typedef TypedColumnWriter<Type::DOUBLE> DoubleWriter;
-typedef TypedColumnWriter<Type::BYTE_ARRAY> ByteArrayWriter;
-typedef TypedColumnWriter<Type::FIXED_LEN_BYTE_ARRAY> FixedLenByteArrayWriter;
+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;
 
 } // namespace parquet
 
 #endif // PARQUET_COLUMN_READER_H
-

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/decoder.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/decoder.h b/src/parquet/encodings/decoder.h
index f1928ee..36af107 100644
--- a/src/parquet/encodings/decoder.h
+++ b/src/parquet/encodings/decoder.h
@@ -28,11 +28,11 @@ namespace parquet {
 
 class ColumnDescriptor;
 
-// The Decoder template is parameterized on parquet::Type::type
-template <int TYPE>
+// The Decoder template is parameterized on parquet::DataType subclasses
+template <typename DType>
 class Decoder {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   virtual ~Decoder() {}
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/delta-bit-pack-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/delta-bit-pack-encoding.h b/src/parquet/encodings/delta-bit-pack-encoding.h
index a91dcc1..b0a16a7 100644
--- a/src/parquet/encodings/delta-bit-pack-encoding.h
+++ b/src/parquet/encodings/delta-bit-pack-encoding.h
@@ -28,16 +28,16 @@
 
 namespace parquet {
 
-template <int TYPE>
-class DeltaBitPackDecoder : public Decoder<TYPE> {
+template <typename DType>
+class DeltaBitPackDecoder : public Decoder<DType> {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   explicit DeltaBitPackDecoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator())
-      : Decoder<TYPE>(descr, Encoding::DELTA_BINARY_PACKED),
+      : Decoder<DType>(descr, Encoding::DELTA_BINARY_PACKED),
         delta_bit_widths_(0, allocator) {
-    if (TYPE != Type::INT32 && TYPE != Type::INT64) {
+    if (DType::type_num != Type::INT32 && DType::type_num != Type::INT64) {
       throw ParquetException("Delta bit pack encoding should only be for integer data.");
     }
   }
@@ -54,7 +54,7 @@ class DeltaBitPackDecoder : public Decoder<TYPE> {
   }
 
  private:
-  using Decoder<TYPE>::num_values_;
+  using Decoder<DType>::num_values_;
 
   void InitBlock() {
     int32_t block_size;

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/delta-byte-array-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/delta-byte-array-encoding.h b/src/parquet/encodings/delta-byte-array-encoding.h
index d6e7b05..34867e2 100644
--- a/src/parquet/encodings/delta-byte-array-encoding.h
+++ b/src/parquet/encodings/delta-byte-array-encoding.h
@@ -26,11 +26,11 @@
 
 namespace parquet {
 
-class DeltaByteArrayDecoder : public Decoder<Type::BYTE_ARRAY> {
+class DeltaByteArrayDecoder : public Decoder<ByteArrayType> {
  public:
   explicit DeltaByteArrayDecoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator())
-      : Decoder<Type::BYTE_ARRAY>(descr, Encoding::DELTA_BYTE_ARRAY),
+      : Decoder<ByteArrayType>(descr, Encoding::DELTA_BYTE_ARRAY),
       prefix_len_decoder_(nullptr, allocator),
       suffix_decoder_(nullptr, allocator) {
   }
@@ -70,9 +70,9 @@ class DeltaByteArrayDecoder : public Decoder<Type::BYTE_ARRAY> {
   }
 
  private:
-  using Decoder<Type::BYTE_ARRAY>::num_values_;
+  using Decoder<ByteArrayType>::num_values_;
 
-  DeltaBitPackDecoder<Type::INT32> prefix_len_decoder_;
+  DeltaBitPackDecoder<Int32Type> prefix_len_decoder_;
   DeltaLengthByteArrayDecoder suffix_decoder_;
   ByteArray last_value_;
 };

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/delta-length-byte-array-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/delta-length-byte-array-encoding.h b/src/parquet/encodings/delta-length-byte-array-encoding.h
index ee4c802..7a19aa3 100644
--- a/src/parquet/encodings/delta-length-byte-array-encoding.h
+++ b/src/parquet/encodings/delta-length-byte-array-encoding.h
@@ -27,11 +27,11 @@
 
 namespace parquet {
 
-class DeltaLengthByteArrayDecoder : public Decoder<Type::BYTE_ARRAY> {
+class DeltaLengthByteArrayDecoder : public Decoder<ByteArrayType> {
  public:
   explicit DeltaLengthByteArrayDecoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator()) :
-      Decoder<Type::BYTE_ARRAY>(descr, Encoding::DELTA_LENGTH_BYTE_ARRAY),
+      Decoder<ByteArrayType>(descr, Encoding::DELTA_LENGTH_BYTE_ARRAY),
       len_decoder_(nullptr, allocator) {
   }
 
@@ -60,8 +60,8 @@ class DeltaLengthByteArrayDecoder : public Decoder<Type::BYTE_ARRAY> {
   }
 
  private:
-  using Decoder<Type::BYTE_ARRAY>::num_values_;
-  DeltaBitPackDecoder<Type::INT32> len_decoder_;
+  using Decoder<ByteArrayType>::num_values_;
+  DeltaBitPackDecoder<Int32Type> len_decoder_;
   const uint8_t* data_;
   int len_;
 };

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/dictionary-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/dictionary-encoding.h b/src/parquet/encodings/dictionary-encoding.h
index 14f574e..e26ba2d 100644
--- a/src/parquet/encodings/dictionary-encoding.h
+++ b/src/parquet/encodings/dictionary-encoding.h
@@ -36,21 +36,21 @@
 
 namespace parquet {
 
-template <int TYPE>
-class DictionaryDecoder : public Decoder<TYPE> {
+template <typename Type>
+class DictionaryDecoder : public Decoder<Type> {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename Type::c_type T;
 
   // Initializes the dictionary with values from 'dictionary'. The data in
   // dictionary is not guaranteed to persist in memory after this call so the
   // dictionary decoder needs to copy the data out if necessary.
   explicit DictionaryDecoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator()):
-      Decoder<TYPE>(descr, Encoding::RLE_DICTIONARY), dictionary_(0, allocator),
+      Decoder<Type>(descr, Encoding::RLE_DICTIONARY), dictionary_(0, allocator),
       byte_array_data_(0, allocator) {}
 
   // Perform type-specific initiatialization
-  void SetDict(Decoder<TYPE>* dictionary);
+  void SetDict(Decoder<Type>* dictionary);
 
   virtual void SetData(int num_values, const uint8_t* data, int len) {
     num_values_ = num_values;
@@ -70,7 +70,7 @@ class DictionaryDecoder : public Decoder<TYPE> {
   }
 
  private:
-  using Decoder<TYPE>::num_values_;
+  using Decoder<Type>::num_values_;
 
   int index() {
     int idx = 0;
@@ -89,22 +89,22 @@ class DictionaryDecoder : public Decoder<TYPE> {
   RleDecoder idx_decoder_;
 };
 
-template <int TYPE>
-inline void DictionaryDecoder<TYPE>::SetDict(Decoder<TYPE>* dictionary) {
+template <typename Type>
+inline void DictionaryDecoder<Type>::SetDict(Decoder<Type>* dictionary) {
   int num_dictionary_values = dictionary->values_left();
   dictionary_.Resize(num_dictionary_values);
   dictionary->Decode(&dictionary_[0], num_dictionary_values);
 }
 
 template <>
-inline void DictionaryDecoder<Type::BOOLEAN>::SetDict(
-    Decoder<Type::BOOLEAN>* dictionary) {
+inline void DictionaryDecoder<BooleanType>::SetDict(
+    Decoder<BooleanType>* dictionary) {
   ParquetException::NYI("Dictionary encoding is not implemented for boolean values");
 }
 
 template <>
-inline void DictionaryDecoder<Type::BYTE_ARRAY>::SetDict(
-    Decoder<Type::BYTE_ARRAY>* dictionary) {
+inline void DictionaryDecoder<ByteArrayType>::SetDict(
+    Decoder<ByteArrayType>* dictionary) {
   int num_dictionary_values = dictionary->values_left();
   dictionary_.Resize(num_dictionary_values);
   dictionary->Decode(&dictionary_[0], num_dictionary_values);
@@ -123,8 +123,7 @@ inline void DictionaryDecoder<Type::BYTE_ARRAY>::SetDict(
 }
 
 template <>
-inline void DictionaryDecoder<Type::FIXED_LEN_BYTE_ARRAY>::SetDict(
-    Decoder<Type::FIXED_LEN_BYTE_ARRAY>* dictionary) {
+inline void DictionaryDecoder<FLBAType>::SetDict(Decoder<FLBAType>* dictionary) {
   int num_dictionary_values = dictionary->values_left();
   dictionary_.Resize(num_dictionary_values);
   dictionary->Decode(&dictionary_[0], num_dictionary_values);

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/encoder.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/encoder.h b/src/parquet/encodings/encoder.h
index d7af2f9..0d69111 100644
--- a/src/parquet/encodings/encoder.h
+++ b/src/parquet/encodings/encoder.h
@@ -32,10 +32,10 @@ class OutputStream;
 // dictionary encoding) we use a class instance to maintain any state.
 //
 // TODO(wesm): Encode interface API is temporary
-template <int TYPE>
+template <typename DType>
 class Encoder {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   virtual ~Encoder() {}
 

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/encoding-test.cc
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/encoding-test.cc b/src/parquet/encodings/encoding-test.cc
index 18ff5cc..d55de76 100644
--- a/src/parquet/encodings/encoding-test.cc
+++ b/src/parquet/encodings/encoding-test.cc
@@ -47,8 +47,8 @@ TEST(VectorBooleanTest, TestEncodeDecode) {
   // seed the prng so failure is deterministic
   vector<bool> draws = flip_coins_seed(nvalues, 0.5, 0);
 
-  PlainEncoder<Type::BOOLEAN> encoder(nullptr);
-  PlainDecoder<Type::BOOLEAN> decoder(nullptr);
+  PlainEncoder<BooleanType> encoder(nullptr);
+  PlainDecoder<BooleanType> decoder(nullptr);
 
   InMemoryOutputStream dst;
   encoder.Encode(draws, nvalues, &dst);
@@ -218,8 +218,8 @@ class TestPlainEncoding : public TestEncodingBase<Type> {
   static constexpr int TYPE = Type::type_num;
 
   virtual void CheckRoundtrip() {
-    PlainEncoder<TYPE> encoder(descr_.get());
-    PlainDecoder<TYPE> decoder(descr_.get());
+    PlainEncoder<Type> encoder(descr_.get());
+    PlainDecoder<Type> decoder(descr_.get());
     InMemoryOutputStream dst;
     encoder.Encode(draws_, num_values_, &dst);
 
@@ -274,11 +274,11 @@ class TestDictionaryEncoding : public TestEncodingBase<Type> {
         indices->size());
     indices->Resize(actual_bytes);
 
-    PlainDecoder<TYPE> dict_decoder(descr_.get());
+    PlainDecoder<Type> dict_decoder(descr_.get());
     dict_decoder.SetData(encoder.num_entries(), dict_buffer_->data(),
         dict_buffer_->size());
 
-    DictionaryDecoder<TYPE> decoder(descr_.get());
+    DictionaryDecoder<Type> decoder(descr_.get());
     decoder.SetDict(&dict_decoder);
 
     decoder.SetData(num_values_, indices->data(), indices->size());
@@ -303,8 +303,8 @@ TYPED_TEST(TestDictionaryEncoding, BasicRoundTrip) {
 }
 
 TEST(TestDictionaryEncoding, CannotDictDecodeBoolean) {
-  PlainDecoder<Type::BOOLEAN> dict_decoder(nullptr);
-  DictionaryDecoder<Type::BOOLEAN> decoder(nullptr);
+  PlainDecoder<BooleanType> dict_decoder(nullptr);
+  DictionaryDecoder<BooleanType> decoder(nullptr);
 
   ASSERT_THROW(decoder.SetDict(&dict_decoder), ParquetException);
 }

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/encodings/plain-encoding.h
----------------------------------------------------------------------
diff --git a/src/parquet/encodings/plain-encoding.h b/src/parquet/encodings/plain-encoding.h
index eee4463..56243c8 100644
--- a/src/parquet/encodings/plain-encoding.h
+++ b/src/parquet/encodings/plain-encoding.h
@@ -33,14 +33,14 @@ namespace parquet {
 // ----------------------------------------------------------------------
 // Encoding::PLAIN decoder implementation
 
-template <int TYPE>
-class PlainDecoder : public Decoder<TYPE> {
+template <typename DType>
+class PlainDecoder : public Decoder<DType> {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
-  using Decoder<TYPE>::num_values_;
+  typedef typename DType::c_type T;
+  using Decoder<DType>::num_values_;
 
   explicit PlainDecoder(const ColumnDescriptor* descr) :
-      Decoder<TYPE>(descr, Encoding::PLAIN),
+      Decoder<DType>(descr, Encoding::PLAIN),
       data_(NULL), len_(0) {
     if (descr_ && descr_->physical_type() == Type::FIXED_LEN_BYTE_ARRAY) {
       type_length_ = descr_->type_length();
@@ -58,7 +58,7 @@ class PlainDecoder : public Decoder<TYPE> {
   virtual int Decode(T* buffer, int max_values);
 
  private:
-  using Decoder<TYPE>::descr_;
+  using Decoder<DType>::descr_;
   const uint8_t* data_;
   int len_;
   int type_length_;
@@ -112,8 +112,8 @@ inline int DecodePlain<FixedLenByteArray>(const uint8_t* data, int64_t data_size
   return bytes_to_decode;
 }
 
-template <int TYPE>
-inline int PlainDecoder<TYPE>::Decode(T* buffer, int max_values) {
+template <typename DType>
+inline int PlainDecoder<DType>::Decode(T* buffer, int max_values) {
   max_values = std::min(max_values, num_values_);
   int bytes_consumed = DecodePlain<T>(data_, len_, max_values,
       type_length_, buffer);
@@ -124,10 +124,10 @@ inline int PlainDecoder<TYPE>::Decode(T* buffer, int max_values) {
 }
 
 template <>
-class PlainDecoder<Type::BOOLEAN> : public Decoder<Type::BOOLEAN> {
+class PlainDecoder<BooleanType> : public Decoder<BooleanType> {
  public:
   explicit PlainDecoder(const ColumnDescriptor* descr) :
-      Decoder<Type::BOOLEAN>(descr, Encoding::PLAIN) {}
+      Decoder<BooleanType>(descr, Encoding::PLAIN) {}
 
   virtual void SetData(int num_values, const uint8_t* data, int len) {
     num_values_ = num_values;
@@ -168,24 +168,24 @@ class PlainDecoder<Type::BOOLEAN> : public Decoder<Type::BOOLEAN> {
 // ----------------------------------------------------------------------
 // Encoding::PLAIN encoder implementation
 
-template <int TYPE>
-class PlainEncoder : public Encoder<TYPE> {
+template <typename DType>
+class PlainEncoder : public Encoder<DType> {
  public:
-  typedef typename type_traits<TYPE>::value_type T;
+  typedef typename DType::c_type T;
 
   explicit PlainEncoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator()) :
-      Encoder<TYPE>(descr, Encoding::PLAIN, allocator) {}
+      Encoder<DType>(descr, Encoding::PLAIN, allocator) {}
 
   void Encode(const T* src, int num_values, OutputStream* dst) override;
 };
 
 template <>
-class PlainEncoder<Type::BOOLEAN> : public Encoder<Type::BOOLEAN> {
+class PlainEncoder<BooleanType> : public Encoder<BooleanType> {
  public:
   explicit PlainEncoder(const ColumnDescriptor* descr,
       MemoryAllocator* allocator = default_allocator()) :
-      Encoder<Type::BOOLEAN>(descr, Encoding::PLAIN, allocator) {}
+      Encoder<BooleanType>(descr, Encoding::PLAIN, allocator) {}
 
   virtual void Encode(const bool* src, int num_values, OutputStream* dst) {
     int bytes_required = BitUtil::Ceil(num_values, 8);
@@ -221,14 +221,14 @@ class PlainEncoder<Type::BOOLEAN> : public Encoder<Type::BOOLEAN> {
   }
 };
 
-template <int TYPE>
-inline void PlainEncoder<TYPE>::Encode(const T* buffer, int num_values,
+template <typename DType>
+inline void PlainEncoder<DType>::Encode(const T* buffer, int num_values,
     OutputStream* dst) {
   dst->Write(reinterpret_cast<const uint8_t*>(buffer), num_values * sizeof(T));
 }
 
 template <>
-inline void PlainEncoder<Type::BYTE_ARRAY>::Encode(const ByteArray* src,
+inline void PlainEncoder<ByteArrayType>::Encode(const ByteArray* src,
     int num_values, OutputStream* dst) {
   for (int i = 0; i < num_values; ++i) {
     // Write the result to the output stream
@@ -238,7 +238,7 @@ inline void PlainEncoder<Type::BYTE_ARRAY>::Encode(const ByteArray* src,
 }
 
 template <>
-inline void PlainEncoder<Type::FIXED_LEN_BYTE_ARRAY>::Encode(
+inline void PlainEncoder<FLBAType>::Encode(
     const FixedLenByteArray* src, int num_values, OutputStream* dst) {
   for (int i = 0; i < num_values; ++i) {
     // Write the result to the output stream

http://git-wip-us.apache.org/repos/asf/parquet-cpp/blob/dc0fc7d4/src/parquet/types.h
----------------------------------------------------------------------
diff --git a/src/parquet/types.h b/src/parquet/types.h
index 450d1ab..72017f9 100644
--- a/src/parquet/types.h
+++ b/src/parquet/types.h
@@ -277,10 +277,10 @@ typedef DataType<Type::DOUBLE> DoubleType;
 typedef DataType<Type::BYTE_ARRAY> ByteArrayType;
 typedef DataType<Type::FIXED_LEN_BYTE_ARRAY> FLBAType;
 
-template <int TYPE>
+template <typename Type>
 inline std::string format_fwf(int width) {
   std::stringstream ss;
-  ss << "%-" << width << type_traits<TYPE>::printf_code;
+  ss << "%-" << width << type_traits<Type::type_num>::printf_code;
   return ss.str();
 }