You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/04/10 13:25:23 UTC

[2/2] arrow git commit: ARROW-782: [C++] API cleanup, change public member access in DataType classes to functions, use class instead of struct

ARROW-782: [C++] API cleanup, change public member access in DataType classes to functions, use class instead of struct

Breaking APIs isn't ideal, but this one is fairly long overdue. The DataType classes are more than passive data carriers, and Google's C++ guide recommends using class instead of struct for this. That means we should put members in protected or private scope, and access them.

I also renamed a couple of things to help with code clarity

* `DataType::type` is now `DataType::id()`
* `Array::type_enum` is not `Array::type_id`

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

Closes #520 from wesm/ARROW-782 and squashes the following commits:

f8dd131 [Wes McKinney] Revert changes with garrow_data_type_new_raw
40de60e [Wes McKinney] Fix glib usages of changed APIs
0097122 [Wes McKinney] Update post rebase
f725655 [Wes McKinney] cpplint
e77f6a5 [Wes McKinney] Change public member access in DataType classes to functions, use class instead of struct


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

Branch: refs/heads/master
Commit: 793f4e0c51e320247ba71c9ccc7970e3eac1d01e
Parents: d1a9aff
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Apr 10 09:25:17 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Apr 10 09:25:17 2017 -0400

----------------------------------------------------------------------
 c_glib/arrow-glib/array-builder.cpp      |   2 +-
 c_glib/arrow-glib/array.cpp              |   4 +-
 c_glib/arrow-glib/data-type.cpp          |   4 +-
 c_glib/arrow-glib/field.cpp              |   7 +-
 cpp/src/arrow/array-decimal-test.cc      |  42 ------
 cpp/src/arrow/array-test.cc              |  12 +-
 cpp/src/arrow/array.cc                   |   8 +-
 cpp/src/arrow/array.h                    |   4 +-
 cpp/src/arrow/builder.cc                 |   8 +-
 cpp/src/arrow/compare.cc                 |  23 +--
 cpp/src/arrow/compare.h                  |   2 +-
 cpp/src/arrow/ipc/feather-test.cc        |   2 +-
 cpp/src/arrow/ipc/feather.cc             |  16 +--
 cpp/src/arrow/ipc/ipc-read-write-test.cc |   6 +-
 cpp/src/arrow/ipc/json-internal.cc       |  44 +++---
 cpp/src/arrow/ipc/json.cc                |   4 +-
 cpp/src/arrow/ipc/metadata.cc            |  32 ++---
 cpp/src/arrow/ipc/metadata.h             |   4 +-
 cpp/src/arrow/ipc/reader.cc              |   2 +-
 cpp/src/arrow/ipc/test-common.h          |  18 +--
 cpp/src/arrow/ipc/writer.cc              |   4 +-
 cpp/src/arrow/ipc/writer.h               |   2 +-
 cpp/src/arrow/loader.cc                  |  11 +-
 cpp/src/arrow/loader.h                   |   2 +-
 cpp/src/arrow/python/builtin_convert.cc  |   4 +-
 cpp/src/arrow/python/numpy_convert.cc    |   2 +-
 cpp/src/arrow/python/numpy_convert.h     |   2 +-
 cpp/src/arrow/python/pandas_convert.cc   |  52 +++----
 cpp/src/arrow/python/pandas_convert.h    |   2 +-
 cpp/src/arrow/table-test.cc              |   4 +-
 cpp/src/arrow/table.cc                   |   4 +-
 cpp/src/arrow/table.h                    |   4 +-
 cpp/src/arrow/tensor.cc                  |   8 +-
 cpp/src/arrow/tensor.h                   |   2 +-
 cpp/src/arrow/type-test.cc               |  64 +++++++--
 cpp/src/arrow/type.cc                    |  34 ++---
 cpp/src/arrow/type.h                     | 200 +++++++++++++++++---------
 cpp/src/arrow/type_fwd.h                 |  34 ++---
 cpp/src/arrow/visitor_inline.h           |   6 +-
 python/pyarrow/array.pyx                 |   2 +-
 python/pyarrow/includes/libarrow.pxd     |  25 ++--
 python/pyarrow/scalar.pyx                |  24 ++--
 python/pyarrow/schema.pyx                |  22 +--
 43 files changed, 407 insertions(+), 351 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/c_glib/arrow-glib/array-builder.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/array-builder.cpp b/c_glib/arrow-glib/array-builder.cpp
index 0f038c8..aea93d0 100644
--- a/c_glib/arrow-glib/array-builder.cpp
+++ b/c_glib/arrow-glib/array-builder.cpp
@@ -161,7 +161,7 @@ garrow_array_builder_new_raw(std::shared_ptr<arrow::ArrayBuilder> *arrow_builder
 {
   GType type;
 
-  switch ((*arrow_builder)->type()->type) {
+  switch ((*arrow_builder)->type()->id()) {
   case arrow::Type::type::BOOL:
     type = GARROW_TYPE_BOOLEAN_ARRAY_BUILDER;
     break;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/c_glib/arrow-glib/array.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/array.cpp b/c_glib/arrow-glib/array.cpp
index 9d0e101..e016ba9 100644
--- a/c_glib/arrow-glib/array.cpp
+++ b/c_glib/arrow-glib/array.cpp
@@ -216,7 +216,7 @@ GArrowType
 garrow_array_get_value_type(GArrowArray *array)
 {
   auto arrow_array = garrow_array_get_raw(array);
-  return garrow_type_from_raw(arrow_array->type_enum());
+  return garrow_type_from_raw(arrow_array->type_id());
 }
 
 /**
@@ -247,7 +247,7 @@ garrow_array_new_raw(std::shared_ptr<arrow::Array> *arrow_array)
   GType type;
   GArrowArray *array;
 
-  switch ((*arrow_array)->type_enum()) {
+  switch ((*arrow_array)->type_id()) {
   case arrow::Type::type::NA:
     type = GARROW_TYPE_NULL_ARRAY;
     break;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/c_glib/arrow-glib/data-type.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/data-type.cpp b/c_glib/arrow-glib/data-type.cpp
index 2df9e7a..12932a1 100644
--- a/c_glib/arrow-glib/data-type.cpp
+++ b/c_glib/arrow-glib/data-type.cpp
@@ -180,7 +180,7 @@ GArrowType
 garrow_data_type_type(GArrowDataType *data_type)
 {
   const auto arrow_data_type = garrow_data_type_get_raw(data_type);
-  return garrow_type_from_raw(arrow_data_type->type);
+  return garrow_type_from_raw(arrow_data_type->id());
 }
 
 G_END_DECLS
@@ -191,7 +191,7 @@ garrow_data_type_new_raw(std::shared_ptr<arrow::DataType> *arrow_data_type)
   GType type;
   GArrowDataType *data_type;
 
-  switch ((*arrow_data_type)->type) {
+  switch ((*arrow_data_type)->id()) {
   case arrow::Type::type::NA:
     type = GARROW_TYPE_NULL_DATA_TYPE;
     break;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/c_glib/arrow-glib/field.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/field.cpp b/c_glib/arrow-glib/field.cpp
index 0dcaf0a..5fd0c4d 100644
--- a/c_glib/arrow-glib/field.cpp
+++ b/c_glib/arrow-glib/field.cpp
@@ -171,7 +171,7 @@ const gchar *
 garrow_field_get_name(GArrowField *field)
 {
   const auto arrow_field = garrow_field_get_raw(field);
-  return arrow_field->name.c_str();
+  return arrow_field->name().c_str();
 }
 
 /**
@@ -184,7 +184,8 @@ GArrowDataType *
 garrow_field_get_data_type(GArrowField *field)
 {
   const auto arrow_field = garrow_field_get_raw(field);
-  return garrow_data_type_new_raw(&arrow_field->type);
+  auto type = arrow_field->type();
+  return garrow_data_type_new_raw(&type);
 }
 
 /**
@@ -197,7 +198,7 @@ gboolean
 garrow_field_is_nullable(GArrowField *field)
 {
   const auto arrow_field = garrow_field_get_raw(field);
-  return arrow_field->nullable;
+  return arrow_field->nullable();
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/array-decimal-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-decimal-test.cc b/cpp/src/arrow/array-decimal-test.cc
index 8353acc..4bde7ab 100644
--- a/cpp/src/arrow/array-decimal-test.cc
+++ b/cpp/src/arrow/array-decimal-test.cc
@@ -25,48 +25,6 @@
 namespace arrow {
 namespace decimal {
 
-TEST(TypesTest, TestDecimal32Type) {
-  DecimalType t1(8, 4);
-
-  ASSERT_EQ(t1.type, Type::DECIMAL);
-  ASSERT_EQ(t1.precision, 8);
-  ASSERT_EQ(t1.scale, 4);
-
-  ASSERT_EQ(t1.ToString(), std::string("decimal(8, 4)"));
-
-  // Test properties
-  ASSERT_EQ(t1.byte_width(), 4);
-  ASSERT_EQ(t1.bit_width(), 32);
-}
-
-TEST(TypesTest, TestDecimal64Type) {
-  DecimalType t1(12, 5);
-
-  ASSERT_EQ(t1.type, Type::DECIMAL);
-  ASSERT_EQ(t1.precision, 12);
-  ASSERT_EQ(t1.scale, 5);
-
-  ASSERT_EQ(t1.ToString(), std::string("decimal(12, 5)"));
-
-  // Test properties
-  ASSERT_EQ(t1.byte_width(), 8);
-  ASSERT_EQ(t1.bit_width(), 64);
-}
-
-TEST(TypesTest, TestDecimal128Type) {
-  DecimalType t1(27, 7);
-
-  ASSERT_EQ(t1.type, Type::DECIMAL);
-  ASSERT_EQ(t1.precision, 27);
-  ASSERT_EQ(t1.scale, 7);
-
-  ASSERT_EQ(t1.ToString(), std::string("decimal(27, 7)"));
-
-  // Test properties
-  ASSERT_EQ(t1.byte_width(), 16);
-  ASSERT_EQ(t1.bit_width(), 128);
-}
-
 template <typename T>
 class DecimalTestBase {
  public:

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/array-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-test.cc b/cpp/src/arrow/array-test.cc
index e50f4fd..99279f3 100644
--- a/cpp/src/arrow/array-test.cc
+++ b/cpp/src/arrow/array-test.cc
@@ -691,8 +691,8 @@ TEST_F(TestStringArray, TestArrayBasics) {
 TEST_F(TestStringArray, TestType) {
   std::shared_ptr<DataType> type = strings_->type();
 
-  ASSERT_EQ(Type::STRING, type->type);
-  ASSERT_EQ(Type::STRING, strings_->type_enum());
+  ASSERT_EQ(Type::STRING, type->id());
+  ASSERT_EQ(Type::STRING, strings_->type_id());
 }
 
 TEST_F(TestStringArray, TestListFunctions) {
@@ -905,8 +905,8 @@ TEST_F(TestBinaryArray, TestArrayBasics) {
 TEST_F(TestBinaryArray, TestType) {
   std::shared_ptr<DataType> type = strings_->type();
 
-  ASSERT_EQ(Type::BINARY, type->type);
-  ASSERT_EQ(Type::BINARY, strings_->type_enum());
+  ASSERT_EQ(Type::BINARY, type->id());
+  ASSERT_EQ(Type::BINARY, strings_->type_id());
 }
 
 TEST_F(TestBinaryArray, TestListFunctions) {
@@ -1679,8 +1679,8 @@ TEST_F(TestStructBuilder, TestAppendNull) {
   ASSERT_TRUE(result_->field(1)->IsNull(0));
   ASSERT_TRUE(result_->field(1)->IsNull(1));
 
-  ASSERT_EQ(Type::LIST, result_->field(0)->type_enum());
-  ASSERT_EQ(Type::INT32, result_->field(1)->type_enum());
+  ASSERT_EQ(Type::LIST, result_->field(0)->type_id());
+  ASSERT_EQ(Type::INT32, result_->field(1)->type_id());
 }
 
 TEST_F(TestStructBuilder, TestBasics) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/array.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index c4a78f3..e640bbd 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -312,8 +312,8 @@ bool DecimalArray::IsNegative(int64_t i) const {
 
 std::string DecimalArray::FormatValue(int64_t i) const {
   const auto type_ = std::dynamic_pointer_cast<DecimalType>(type());
-  const int precision = type_->precision;
-  const int scale = type_->scale;
+  const int precision = type_->precision();
+  const int scale = type_->scale();
   const int byte_width = byte_width_;
   const uint8_t* bytes = GetValue(i);
   switch (byte_width) {
@@ -453,11 +453,11 @@ DictionaryArray::DictionaryArray(
           indices->offset()),
       dict_type_(static_cast<const DictionaryType*>(type.get())),
       indices_(indices) {
-  DCHECK_EQ(type->type, Type::DICTIONARY);
+  DCHECK_EQ(type->id(), Type::DICTIONARY);
 }
 
 Status DictionaryArray::Validate() const {
-  Type::type index_type_id = indices_->type()->type;
+  Type::type index_type_id = indices_->type()->id();
   if (!is_integer(index_type_id)) {
     return Status::Invalid("Dictionary indices must be integer type");
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 4f8b22e..071d4e3 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -80,7 +80,7 @@ class ARROW_EXPORT Array {
   int64_t null_count() const;
 
   std::shared_ptr<DataType> type() const { return type_; }
-  Type::type type_enum() const { return type_->type; }
+  Type::type type_id() const { return type_->id(); }
 
   /// Buffer for the null bitmap.
   ///
@@ -447,7 +447,7 @@ class ARROW_EXPORT UnionArray : public Array {
   const type_id_t* raw_type_ids() const { return raw_type_ids_ + offset_; }
   const int32_t* raw_value_offsets() const { return raw_value_offsets_ + offset_; }
 
-  UnionMode mode() const { return static_cast<const UnionType&>(*type_.get()).mode; }
+  UnionMode mode() const { return static_cast<const UnionType&>(*type_.get()).mode(); }
 
   std::shared_ptr<Array> child(int pos) const;
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/builder.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index 4281a61..d85eb32 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -253,7 +253,7 @@ BooleanBuilder::BooleanBuilder(MemoryPool* pool)
 
 BooleanBuilder::BooleanBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type)
     : BooleanBuilder(pool) {
-  DCHECK_EQ(Type::BOOL, type->type);
+  DCHECK_EQ(Type::BOOL, type->id());
 }
 
 Status BooleanBuilder::Init(int64_t capacity) {
@@ -602,7 +602,7 @@ std::shared_ptr<ArrayBuilder> StructBuilder::field_builder(int pos) const {
 // TODO(wesm): come up with a less monolithic strategy
 Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
     std::shared_ptr<ArrayBuilder>* out) {
-  switch (type->type) {
+  switch (type->id()) {
     BUILDER_CASE(UINT8, UInt8Builder);
     BUILDER_CASE(INT8, Int8Builder);
     BUILDER_CASE(UINT16, UInt16Builder);
@@ -633,12 +633,12 @@ Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
     }
 
     case Type::STRUCT: {
-      std::vector<FieldPtr>& fields = type->children_;
+      const std::vector<FieldPtr>& fields = type->children();
       std::vector<std::shared_ptr<ArrayBuilder>> values_builder;
 
       for (auto it : fields) {
         std::shared_ptr<ArrayBuilder> builder;
-        RETURN_NOT_OK(MakeBuilder(pool, it->type, &builder));
+        RETURN_NOT_OK(MakeBuilder(pool, it->type(), &builder));
         values_builder.push_back(builder);
       }
       out->reset(new StructBuilder(pool, type, values_builder));

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/compare.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index 2297e4b..e02f3f0 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -151,7 +151,7 @@ class RangeEqualsVisitor {
     // Define a mapping from the type id to child number
     uint8_t max_code = 0;
 
-    const std::vector<uint8_t> type_codes = left_type.type_codes;
+    const std::vector<uint8_t>& type_codes = left_type.type_codes();
     for (size_t i = 0; i < type_codes.size(); ++i) {
       const uint8_t code = type_codes[i];
       if (code > max_code) { max_code = code; }
@@ -532,7 +532,7 @@ class ApproxEqualsVisitor : public ArrayEqualsVisitor {
 
 static bool BaseDataEquals(const Array& left, const Array& right) {
   if (left.length() != right.length() || left.null_count() != right.null_count() ||
-      left.type_enum() != right.type_enum()) {
+      left.type_id() != right.type_id()) {
     return false;
   }
   if (left.null_count() > 0) {
@@ -571,7 +571,7 @@ Status ArrayRangeEquals(const Array& left, const Array& right, int64_t left_star
     int64_t left_end_idx, int64_t right_start_idx, bool* are_equal) {
   if (&left == &right) {
     *are_equal = true;
-  } else if (left.type_enum() != right.type_enum()) {
+  } else if (left.type_id() != right.type_id()) {
     *are_equal = false;
   } else if (left.length() == 0) {
     *are_equal = true;
@@ -615,7 +615,7 @@ Status TensorEquals(const Tensor& left, const Tensor& right, bool* are_equal) {
   // The arrays are the same object
   if (&left == &right) {
     *are_equal = true;
-  } else if (left.type_enum() != right.type_enum()) {
+  } else if (left.type_id() != right.type_id()) {
     *are_equal = false;
   } else if (left.size() == 0) {
     *are_equal = true;
@@ -670,13 +670,13 @@ class TypeEqualsVisitor {
       Status>::type
   Visit(const T& left) {
     const auto& right = static_cast<const T&>(right_);
-    result_ = left.unit == right.unit;
+    result_ = left.unit() == right.unit();
     return Status::OK();
   }
 
   Status Visit(const TimestampType& left) {
     const auto& right = static_cast<const TimestampType&>(right_);
-    result_ = left.unit == right.unit && left.timezone == right.timezone;
+    result_ = left.unit() == right.unit() && left.timezone() == right.timezone();
     return Status::OK();
   }
 
@@ -688,7 +688,7 @@ class TypeEqualsVisitor {
 
   Status Visit(const DecimalType& left) {
     const auto& right = static_cast<const DecimalType&>(right_);
-    result_ = left.precision == right.precision && left.scale == right.scale;
+    result_ = left.precision() == right.precision() && left.scale() == right.scale();
     return Status::OK();
   }
 
@@ -699,13 +699,14 @@ class TypeEqualsVisitor {
   Status Visit(const UnionType& left) {
     const auto& right = static_cast<const UnionType&>(right_);
 
-    if (left.mode != right.mode || left.type_codes.size() != right.type_codes.size()) {
+    if (left.mode() != right.mode() ||
+        left.type_codes().size() != right.type_codes().size()) {
       result_ = false;
       return Status::OK();
     }
 
-    const std::vector<uint8_t> left_codes = left.type_codes;
-    const std::vector<uint8_t> right_codes = right.type_codes;
+    const std::vector<uint8_t>& left_codes = left.type_codes();
+    const std::vector<uint8_t>& right_codes = right.type_codes();
 
     for (size_t i = 0; i < left_codes.size(); ++i) {
       if (left_codes[i] != right_codes[i]) {
@@ -743,7 +744,7 @@ Status TypeEquals(const DataType& left, const DataType& right, bool* are_equal)
   // The arrays are the same object
   if (&left == &right) {
     *are_equal = true;
-  } else if (left.type != right.type) {
+  } else if (left.id() != right.id()) {
     *are_equal = false;
   } else {
     TypeEqualsVisitor visitor(right);

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/compare.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.h b/cpp/src/arrow/compare.h
index 522b11d..96a6435 100644
--- a/cpp/src/arrow/compare.h
+++ b/cpp/src/arrow/compare.h
@@ -27,7 +27,7 @@
 namespace arrow {
 
 class Array;
-struct DataType;
+class DataType;
 class Status;
 class Tensor;
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/feather-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/feather-test.cc b/cpp/src/arrow/ipc/feather-test.cc
index 077a44b..fb26df6 100644
--- a/cpp/src/arrow/ipc/feather-test.cc
+++ b/cpp/src/arrow/ipc/feather-test.cc
@@ -379,7 +379,7 @@ TEST_F(TestTableWriter, TimeTypes) {
 
   for (int i = 1; i < schema->num_fields(); ++i) {
     std::shared_ptr<Array> arr;
-    LoadArray(schema->field(i)->type, fields, buffers, &arr);
+    LoadArray(schema->field(i)->type(), fields, buffers, &arr);
     arrays.push_back(arr);
   }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/feather.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/feather.cc b/cpp/src/arrow/ipc/feather.cc
index e838e1f..5dc0396 100644
--- a/cpp/src/arrow/ipc/feather.cc
+++ b/cpp/src/arrow/ipc/feather.cc
@@ -349,7 +349,7 @@ class TableReader::TableReaderImpl {
       buffers.push_back(nullptr);
     }
 
-    if (is_binary_like(type->type)) {
+    if (is_binary_like(type->id())) {
       int64_t offsets_size = GetOutputLength((meta->length() + 1) * sizeof(int32_t));
       buffers.push_back(SliceBuffer(buffer, offset, offsets_size));
       offset += offsets_size;
@@ -516,13 +516,13 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
   }
 
   Status LoadArrayMetadata(const Array& values, ArrayMetadata* meta) {
-    if (!(is_primitive(values.type_enum()) || is_binary_like(values.type_enum()))) {
+    if (!(is_primitive(values.type_id()) || is_binary_like(values.type_id()))) {
       std::stringstream ss;
       ss << "Array is not primitive type: " << values.type()->ToString();
       return Status::Invalid(ss.str());
     }
 
-    meta->type = ToFlatbufferType(values.type_enum());
+    meta->type = ToFlatbufferType(values.type_id());
 
     RETURN_NOT_OK(stream_->Tell(&meta->offset));
 
@@ -552,7 +552,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
 
     const uint8_t* values_buffer = nullptr;
 
-    if (is_binary_like(values.type_enum())) {
+    if (is_binary_like(values.type_id())) {
       const auto& bin_values = static_cast<const BinaryArray&>(values);
 
       int64_t offset_bytes = sizeof(int32_t) * (values.length() + 1);
@@ -570,7 +570,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
       const auto& prim_values = static_cast<const PrimitiveArray&>(values);
       const auto& fw_type = static_cast<const FixedWidthType&>(*values.type());
 
-      if (values.type_enum() == Type::BOOL) {
+      if (values.type_id() == Type::BOOL) {
         // Booleans are bit-packed
         values_bytes = BitUtil::BytesForBits(values.length());
       } else {
@@ -616,7 +616,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
   Status Visit(const DictionaryArray& values) override {
     const auto& dict_type = static_cast<const DictionaryType&>(*values.type());
 
-    if (!is_integer(values.indices()->type_enum())) {
+    if (!is_integer(values.indices()->type_id())) {
       return Status::Invalid("Category values must be integers");
     }
 
@@ -631,7 +631,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
   Status Visit(const TimestampArray& values) override {
     RETURN_NOT_OK(WritePrimitiveValues(values));
     const auto& ts_type = static_cast<const TimestampType&>(*values.type());
-    current_column_->SetTimestamp(ts_type.unit, ts_type.timezone);
+    current_column_->SetTimestamp(ts_type.unit(), ts_type.timezone());
     return Status::OK();
   }
 
@@ -643,7 +643,7 @@ class TableWriter::TableWriterImpl : public ArrayVisitor {
 
   Status Visit(const Time32Array& values) override {
     RETURN_NOT_OK(WritePrimitiveValues(values));
-    auto unit = static_cast<const Time32Type&>(*values.type()).unit;
+    auto unit = static_cast<const Time32Type&>(*values.type()).unit();
     current_column_->SetTime(unit);
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/ipc-read-write-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc
index 1a91ec3..98a7c3d 100644
--- a/cpp/src/arrow/ipc/ipc-read-write-test.cc
+++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc
@@ -569,13 +569,13 @@ void CheckBatchDictionaries(const RecordBatch& batch) {
   // Check that dictionaries that should be the same are the same
   auto schema = batch.schema();
 
-  const auto& t0 = static_cast<const DictionaryType&>(*schema->field(0)->type);
-  const auto& t1 = static_cast<const DictionaryType&>(*schema->field(1)->type);
+  const auto& t0 = static_cast<const DictionaryType&>(*schema->field(0)->type());
+  const auto& t1 = static_cast<const DictionaryType&>(*schema->field(1)->type());
 
   ASSERT_EQ(t0.dictionary().get(), t1.dictionary().get());
 
   // Same dictionary used for list values
-  const auto& t3 = static_cast<const ListType&>(*schema->field(3)->type);
+  const auto& t3 = static_cast<const ListType&>(*schema->field(3)->type());
   const auto& t3_value = static_cast<const DictionaryType&>(*t3.value_type());
   ASSERT_EQ(t0.dictionary().get(), t3_value.dictionary().get());
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/json-internal.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc
index fe0a7c9..18ee834 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -114,13 +114,13 @@ class JsonSchemaWriter {
     writer_->StartObject();
 
     writer_->Key("name");
-    writer_->String(field.name.c_str());
+    writer_->String(field.name().c_str());
 
     writer_->Key("nullable");
-    writer_->Bool(field.nullable);
+    writer_->Bool(field.nullable());
 
     // Visit the type
-    RETURN_NOT_OK(VisitTypeInline(*field.type, this));
+    RETURN_NOT_OK(VisitTypeInline(*field.type(), this));
     writer_->EndObject();
 
     return Status::OK();
@@ -153,7 +153,7 @@ class JsonSchemaWriter {
 
   void WriteTypeMetadata(const IntervalType& type) {
     writer_->Key("unit");
-    switch (type.unit) {
+    switch (type.unit()) {
       case IntervalType::Unit::YEAR_MONTH:
         writer_->String("YEAR_MONTH");
         break;
@@ -165,23 +165,23 @@ class JsonSchemaWriter {
 
   void WriteTypeMetadata(const TimestampType& type) {
     writer_->Key("unit");
-    writer_->String(GetTimeUnitName(type.unit));
-    if (type.timezone.size() > 0) {
+    writer_->String(GetTimeUnitName(type.unit()));
+    if (type.timezone().size() > 0) {
       writer_->Key("timezone");
-      writer_->String(type.timezone);
+      writer_->String(type.timezone());
     }
   }
 
   void WriteTypeMetadata(const TimeType& type) {
     writer_->Key("unit");
-    writer_->String(GetTimeUnitName(type.unit));
+    writer_->String(GetTimeUnitName(type.unit()));
     writer_->Key("bitWidth");
     writer_->Int(type.bit_width());
   }
 
   void WriteTypeMetadata(const DateType& type) {
     writer_->Key("unit");
-    switch (type.unit) {
+    switch (type.unit()) {
       case DateUnit::DAY:
         writer_->String("DAY");
         break;
@@ -198,14 +198,14 @@ class JsonSchemaWriter {
 
   void WriteTypeMetadata(const DecimalType& type) {
     writer_->Key("precision");
-    writer_->Int(type.precision);
+    writer_->Int(type.precision());
     writer_->Key("scale");
-    writer_->Int(type.scale);
+    writer_->Int(type.scale());
   }
 
   void WriteTypeMetadata(const UnionType& type) {
     writer_->Key("mode");
-    switch (type.mode) {
+    switch (type.mode()) {
       case UnionMode::SPARSE:
         writer_->String("SPARSE");
         break;
@@ -217,8 +217,8 @@ class JsonSchemaWriter {
     // Write type ids
     writer_->Key("typeIds");
     writer_->StartArray();
-    for (size_t i = 0; i < type.type_codes.size(); ++i) {
-      writer_->Uint(type.type_codes[i]);
+    for (size_t i = 0; i < type.type_codes().size(); ++i) {
+      writer_->Uint(type.type_codes()[i]);
     }
     writer_->EndArray();
   }
@@ -461,7 +461,7 @@ class JsonArrayWriter {
     writer_->Key("children");
     writer_->StartArray();
     for (size_t i = 0; i < fields.size(); ++i) {
-      RETURN_NOT_OK(VisitArray(fields[i]->name, *arrays[i].get()));
+      RETURN_NOT_OK(VisitArray(fields[i]->name(), *arrays[i].get()));
     }
     writer_->EndArray();
     return Status::OK();
@@ -513,7 +513,7 @@ class JsonArrayWriter {
     auto type = static_cast<const UnionType*>(array.type().get());
 
     WriteIntegerField("TYPE_ID", array.raw_type_ids(), array.length());
-    if (type->mode == UnionMode::DENSE) {
+    if (type->mode() == UnionMode::DENSE) {
       WriteIntegerField("OFFSET", array.raw_value_offsets(), array.length());
     }
     return WriteChildren(type->children(), array.children());
@@ -1026,7 +1026,7 @@ class JsonArrayReader {
     RETURN_NOT_OK(
         GetIntArray<uint8_t>(json_type_ids->value.GetArray(), length, &type_id_buffer));
 
-    if (union_type.mode == UnionMode::DENSE) {
+    if (union_type.mode() == UnionMode::DENSE) {
       const auto& json_offsets = json_array.FindMember("OFFSET");
       RETURN_NOT_ARRAY("OFFSET", json_offsets, json_array);
       RETURN_NOT_OK(
@@ -1072,9 +1072,9 @@ class JsonArrayReader {
       auto it = json_child.FindMember("name");
       RETURN_NOT_STRING("name", it, json_child);
 
-      DCHECK_EQ(it->value.GetString(), child_field->name);
+      DCHECK_EQ(it->value.GetString(), child_field->name());
       std::shared_ptr<Array> child;
-      RETURN_NOT_OK(GetArray(json_children_arr[i], child_field->type, &child));
+      RETURN_NOT_OK(GetArray(json_children_arr[i], child_field->type(), &child));
       array->emplace_back(child);
     }
 
@@ -1109,7 +1109,7 @@ class JsonArrayReader {
   case TYPE::type_id:   \
     return ReadArray<TYPE>(json_array, length, is_valid, type, array);
 
-    switch (type->type) {
+    switch (type->id()) {
       TYPE_CASE(NullType);
       TYPE_CASE(BooleanType);
       TYPE_CASE(UInt8Type);
@@ -1192,7 +1192,7 @@ Status ReadJsonArray(MemoryPool* pool, const rj::Value& json_array, const Schema
 
   std::shared_ptr<Field> result = nullptr;
   for (const std::shared_ptr<Field>& field : schema.fields()) {
-    if (field->name == name) {
+    if (field->name() == name) {
       result = field;
       break;
     }
@@ -1204,7 +1204,7 @@ Status ReadJsonArray(MemoryPool* pool, const rj::Value& json_array, const Schema
     return Status::KeyError(ss.str());
   }
 
-  return ReadJsonArray(pool, json_array, result->type, array);
+  return ReadJsonArray(pool, json_array, result->type(), array);
 }
 
 }  // namespace ipc

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/json.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/json.cc b/cpp/src/arrow/ipc/json.cc
index 8056b6f..0abd6d7 100644
--- a/cpp/src/arrow/ipc/json.cc
+++ b/cpp/src/arrow/ipc/json.cc
@@ -79,7 +79,7 @@ class JsonWriter::JsonWriterImpl {
       DCHECK_EQ(batch.num_rows(), column->length())
           << "Array length did not match record batch length";
 
-      RETURN_NOT_OK(WriteJsonArray(schema_->field(i)->name, *column, writer_.get()));
+      RETURN_NOT_OK(WriteJsonArray(schema_->field(i)->name(), *column, writer_.get()));
     }
 
     writer_->EndArray();
@@ -158,7 +158,7 @@ class JsonReader::JsonReaderImpl {
 
     std::vector<std::shared_ptr<Array>> columns(json_columns.Size());
     for (int i = 0; i < static_cast<int>(columns.size()); ++i) {
-      const std::shared_ptr<DataType>& type = schema_->field(i)->type;
+      const std::shared_ptr<DataType>& type = schema_->field(i)->type();
       RETURN_NOT_OK(ReadJsonArray(pool_, json_columns[i], type, &columns[i]));
     }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/metadata.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/metadata.cc b/cpp/src/arrow/ipc/metadata.cc
index d902ec2..84f8883 100644
--- a/cpp/src/arrow/ipc/metadata.cc
+++ b/cpp/src/arrow/ipc/metadata.cc
@@ -163,13 +163,13 @@ static Status UnionToFlatBuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
 
   const auto& union_type = static_cast<const UnionType&>(*type);
 
-  flatbuf::UnionMode mode = union_type.mode == UnionMode::SPARSE
+  flatbuf::UnionMode mode = union_type.mode() == UnionMode::SPARSE
                                 ? flatbuf::UnionMode_Sparse
                                 : flatbuf::UnionMode_Dense;
 
   std::vector<int32_t> type_ids;
-  type_ids.reserve(union_type.type_codes.size());
-  for (uint8_t code : union_type.type_codes) {
+  type_ids.reserve(union_type.type_codes().size());
+  for (uint8_t code : union_type.type_codes()) {
     type_ids.push_back(code);
   }
 
@@ -306,7 +306,7 @@ static Status TypeFromFlatbuffer(flatbuf::Type type, const void* type_data,
 static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
     std::vector<FieldOffset>* children, std::vector<VectorLayoutOffset>* layout,
     flatbuf::Type* out_type, DictionaryMemo* dictionary_memo, Offset* offset) {
-  if (type->type == Type::DICTIONARY) {
+  if (type->id() == Type::DICTIONARY) {
     // In this library, the dictionary "type" is a logical construct. Here we
     // pass through to the value type, as we've already captured the index
     // type in the DictionaryEncoding metadata in the parent field
@@ -340,7 +340,7 @@ static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
     layout->push_back(offset);
   }
 
-  switch (type->type) {
+  switch (type->id()) {
     case Type::BOOL:
       *out_type = flatbuf::Type_Bool;
       *offset = flatbuf::CreateBool(fbb).Union();
@@ -393,21 +393,21 @@ static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
     case Type::TIME32: {
       const auto& time_type = static_cast<const Time32Type&>(*type);
       *out_type = flatbuf::Type_Time;
-      *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit), 32).Union();
+      *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit()), 32).Union();
     } break;
     case Type::TIME64: {
       const auto& time_type = static_cast<const Time64Type&>(*type);
       *out_type = flatbuf::Type_Time;
-      *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit), 64).Union();
+      *offset = flatbuf::CreateTime(fbb, ToFlatbufferUnit(time_type.unit()), 64).Union();
     } break;
     case Type::TIMESTAMP: {
       const auto& ts_type = static_cast<const TimestampType&>(*type);
       *out_type = flatbuf::Type_Timestamp;
 
-      flatbuf::TimeUnit fb_unit = ToFlatbufferUnit(ts_type.unit);
+      flatbuf::TimeUnit fb_unit = ToFlatbufferUnit(ts_type.unit());
       FBString fb_timezone = 0;
-      if (ts_type.timezone.size() > 0) {
-        fb_timezone = fbb.CreateString(ts_type.timezone);
+      if (ts_type.timezone().size() > 0) {
+        fb_timezone = fbb.CreateString(ts_type.timezone());
       }
       *offset = flatbuf::CreateTimestamp(fbb, fb_unit, fb_timezone).Union();
     } break;
@@ -431,7 +431,7 @@ static Status TypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
 
 static Status TensorTypeToFlatbuffer(FBB& fbb, const std::shared_ptr<DataType>& type,
     flatbuf::Type* out_type, Offset* offset) {
-  switch (type->type) {
+  switch (type->id()) {
     case Type::UINT8:
       INT_TO_FB_CASE(8, false);
     case Type::INT8:
@@ -486,7 +486,7 @@ static DictionaryOffset GetDictionaryEncoding(
 
 static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr<Field>& field,
     DictionaryMemo* dictionary_memo, FieldOffset* offset) {
-  auto fb_name = fbb.CreateString(field->name);
+  auto fb_name = fbb.CreateString(field->name());
 
   flatbuf::Type type_enum;
   Offset type_offset;
@@ -495,18 +495,18 @@ static Status FieldToFlatbuffer(FBB& fbb, const std::shared_ptr<Field>& field,
   std::vector<VectorLayoutOffset> layout;
 
   RETURN_NOT_OK(TypeToFlatbuffer(
-      fbb, field->type, &children, &layout, &type_enum, dictionary_memo, &type_offset));
+      fbb, field->type(), &children, &layout, &type_enum, dictionary_memo, &type_offset));
   auto fb_children = fbb.CreateVector(children);
   auto fb_layout = fbb.CreateVector(layout);
 
   DictionaryOffset dictionary = 0;
-  if (field->type->type == Type::DICTIONARY) {
+  if (field->type()->id() == Type::DICTIONARY) {
     dictionary = GetDictionaryEncoding(
-        fbb, static_cast<const DictionaryType&>(*field->type), dictionary_memo);
+        fbb, static_cast<const DictionaryType&>(*field->type()), dictionary_memo);
   }
 
   // TODO: produce the list of VectorTypes
-  *offset = flatbuf::CreateField(fbb, fb_name, field->nullable, type_enum, type_offset,
+  *offset = flatbuf::CreateField(fbb, fb_name, field->nullable(), type_enum, type_offset,
       dictionary, fb_children, fb_layout);
 
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/metadata.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/metadata.h b/cpp/src/arrow/ipc/metadata.h
index b042882..84026c4 100644
--- a/cpp/src/arrow/ipc/metadata.h
+++ b/cpp/src/arrow/ipc/metadata.h
@@ -34,8 +34,8 @@ namespace arrow {
 
 class Array;
 class Buffer;
-struct DataType;
-struct Field;
+class DataType;
+class Field;
 class Schema;
 class Status;
 class Tensor;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/reader.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/reader.cc b/cpp/src/arrow/ipc/reader.cc
index a7c4f04..69fde17 100644
--- a/cpp/src/arrow/ipc/reader.cc
+++ b/cpp/src/arrow/ipc/reader.cc
@@ -97,7 +97,7 @@ static Status LoadRecordBatchFromSource(const std::shared_ptr<Schema>& schema,
   context.max_recursion_depth = max_recursion_depth;
 
   for (int i = 0; i < schema->num_fields(); ++i) {
-    RETURN_NOT_OK(LoadArray(schema->field(i)->type, &context, &arrays[i]));
+    RETURN_NOT_OK(LoadArray(schema->field(i)->type(), &context, &arrays[i]));
     DCHECK_EQ(num_rows, arrays[i]->length())
         << "Array length did not match record batch length";
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/test-common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h
index 9e0480d..a17b609 100644
--- a/cpp/src/arrow/ipc/test-common.h
+++ b/cpp/src/arrow/ipc/test-common.h
@@ -552,9 +552,9 @@ Status MakeTimestamps(std::shared_ptr<RecordBatch>* out) {
       1489272000000, 1489272000000, 1489273000000};
 
   std::shared_ptr<Array> a0, a1, a2;
-  ArrayFromVector<TimestampType, int64_t>(f0->type, is_valid, ts_values, &a0);
-  ArrayFromVector<TimestampType, int64_t>(f1->type, is_valid, ts_values, &a1);
-  ArrayFromVector<TimestampType, int64_t>(f2->type, is_valid, ts_values, &a2);
+  ArrayFromVector<TimestampType, int64_t>(f0->type(), is_valid, ts_values, &a0);
+  ArrayFromVector<TimestampType, int64_t>(f1->type(), is_valid, ts_values, &a1);
+  ArrayFromVector<TimestampType, int64_t>(f2->type(), is_valid, ts_values, &a2);
 
   ArrayVector arrays = {a0, a1, a2};
   *out = std::make_shared<RecordBatch>(schema, a0->length(), arrays);
@@ -575,10 +575,10 @@ Status MakeTimes(std::shared_ptr<RecordBatch>* out) {
       1489272000000, 1489272000000, 1489273000000};
 
   std::shared_ptr<Array> a0, a1, a2, a3;
-  ArrayFromVector<Time32Type, int32_t>(f0->type, is_valid, t32_values, &a0);
-  ArrayFromVector<Time64Type, int64_t>(f1->type, is_valid, t64_values, &a1);
-  ArrayFromVector<Time32Type, int32_t>(f2->type, is_valid, t32_values, &a2);
-  ArrayFromVector<Time64Type, int64_t>(f3->type, is_valid, t64_values, &a3);
+  ArrayFromVector<Time32Type, int32_t>(f0->type(), is_valid, t32_values, &a0);
+  ArrayFromVector<Time64Type, int64_t>(f1->type(), is_valid, t64_values, &a1);
+  ArrayFromVector<Time32Type, int32_t>(f2->type(), is_valid, t32_values, &a2);
+  ArrayFromVector<Time64Type, int64_t>(f3->type(), is_valid, t64_values, &a3);
 
   ArrayVector arrays = {a0, a1, a2, a3};
   *out = std::make_shared<RecordBatch>(schema, a0->length(), arrays);
@@ -605,8 +605,8 @@ Status MakeFWBinary(std::shared_ptr<RecordBatch>* out) {
 
   std::shared_ptr<Array> a1, a2;
 
-  FixedSizeBinaryBuilder b1(default_memory_pool(), f0->type);
-  FixedSizeBinaryBuilder b2(default_memory_pool(), f1->type);
+  FixedSizeBinaryBuilder b1(default_memory_pool(), f0->type());
+  FixedSizeBinaryBuilder b2(default_memory_pool(), f1->type());
 
   std::vector<std::string> values1 = {"foo1", "foo2", "foo3", "foo4"};
   AppendValues(is_valid, values1, &b1);

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/writer.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index d38a65c..18a5855 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -364,7 +364,7 @@ class RecordBatchWriter : public ArrayVisitor {
 
       // The Union type codes are not necessary 0-indexed
       uint8_t max_code = 0;
-      for (uint8_t code : type.type_codes) {
+      for (uint8_t code : type.type_codes()) {
         if (code > max_code) { max_code = code; }
       }
 
@@ -406,7 +406,7 @@ class RecordBatchWriter : public ArrayVisitor {
       for (int i = 0; i < type.num_children(); ++i) {
         std::shared_ptr<Array> child = array.child(i);
         if (array.offset() != 0) {
-          const uint8_t code = type.type_codes[i];
+          const uint8_t code = type.type_codes()[i];
           child = child->Slice(child_offsets[code], child_lengths[code]);
         }
         RETURN_NOT_OK(VisitArray(*child));

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/ipc/writer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/writer.h b/cpp/src/arrow/ipc/writer.h
index 0b7a6e1..629bcb9 100644
--- a/cpp/src/arrow/ipc/writer.h
+++ b/cpp/src/arrow/ipc/writer.h
@@ -31,7 +31,7 @@ namespace arrow {
 
 class Array;
 class Buffer;
-struct Field;
+class Field;
 class MemoryPool;
 class RecordBatch;
 class Schema;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/loader.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/loader.cc b/cpp/src/arrow/loader.cc
index f3347f9..f9f6e6f 100644
--- a/cpp/src/arrow/loader.cc
+++ b/cpp/src/arrow/loader.cc
@@ -24,6 +24,7 @@
 
 #include "arrow/array.h"
 #include "arrow/buffer.h"
+#include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
 #include "arrow/util/logging.h"
@@ -32,10 +33,6 @@
 
 namespace arrow {
 
-class Array;
-struct DataType;
-class Status;
-
 class ArrayLoader {
  public:
   ArrayLoader(const std::shared_ptr<DataType>& type, ArrayLoaderContext* context)
@@ -114,7 +111,7 @@ class ArrayLoader {
   }
 
   Status LoadChild(const Field& field, std::shared_ptr<Array>* out) {
-    ArrayLoader loader(field.type, context_);
+    ArrayLoader loader(field.type(), context_);
     --context_->max_recursion_depth;
     RETURN_NOT_OK(loader.Load(out));
     ++context_->max_recursion_depth;
@@ -211,11 +208,11 @@ class ArrayLoader {
     RETURN_NOT_OK(LoadCommon(&field_meta, &null_bitmap));
     if (field_meta.length > 0) {
       RETURN_NOT_OK(GetBuffer(context_->buffer_index, &type_ids));
-      if (type.mode == UnionMode::DENSE) {
+      if (type.mode() == UnionMode::DENSE) {
         RETURN_NOT_OK(GetBuffer(context_->buffer_index + 1, &offsets));
       }
     }
-    context_->buffer_index += type.mode == UnionMode::DENSE ? 2 : 1;
+    context_->buffer_index += type.mode() == UnionMode::DENSE ? 2 : 1;
 
     std::vector<std::shared_ptr<Array>> fields;
     RETURN_NOT_OK(LoadChildren(type.children(), &fields));

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/loader.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/loader.h b/cpp/src/arrow/loader.h
index 9b650e2..f5e3995 100644
--- a/cpp/src/arrow/loader.h
+++ b/cpp/src/arrow/loader.h
@@ -33,7 +33,7 @@ namespace arrow {
 
 class Array;
 class Buffer;
-struct DataType;
+class DataType;
 
 // ARROW-109: We set this number arbitrarily to help catch user mistakes. For
 // deeply nested schemas, it is expected the user will indicate explicitly the

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/python/builtin_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/builtin_convert.cc b/cpp/src/arrow/python/builtin_convert.cc
index 1ae13f3..8cc9876 100644
--- a/cpp/src/arrow/python/builtin_convert.cc
+++ b/cpp/src/arrow/python/builtin_convert.cc
@@ -571,7 +571,7 @@ class DecimalConverter : public TypedConverter<arrow::DecimalBuilder> {
 
 // Dynamic constructor for sequence converters
 std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type) {
-  switch (type->type) {
+  switch (type->id()) {
     case Type::BOOL:
       return std::make_shared<BoolConverter>();
     case Type::INT64:
@@ -637,7 +637,7 @@ Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>
 Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out,
     const std::shared_ptr<DataType>& type, int64_t size) {
   // Handle NA / NullType case
-  if (type->type == Type::NA) {
+  if (type->id() == Type::NA) {
     out->reset(new NullArray(size));
     return Status::OK();
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/python/numpy_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/numpy_convert.cc b/cpp/src/arrow/python/numpy_convert.cc
index 23470fb..ab79e17 100644
--- a/cpp/src/arrow/python/numpy_convert.cc
+++ b/cpp/src/arrow/python/numpy_convert.cc
@@ -118,7 +118,7 @@ Status GetNumPyType(const DataType& type, int* type_num) {
     *type_num = NPY_##NPY_NAME;               \
     break;
 
-  switch (type.type) {
+  switch (type.id()) {
     NUMPY_TYPE_CASE(UINT8, UINT8);
     NUMPY_TYPE_CASE(INT8, INT8);
     NUMPY_TYPE_CASE(UINT16, UINT16);

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/python/numpy_convert.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/numpy_convert.h b/cpp/src/arrow/python/numpy_convert.h
index 685a626..c252640 100644
--- a/cpp/src/arrow/python/numpy_convert.h
+++ b/cpp/src/arrow/python/numpy_convert.h
@@ -31,7 +31,7 @@
 
 namespace arrow {
 
-struct DataType;
+class DataType;
 class MemoryPool;
 class Status;
 class Tensor;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/python/pandas_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_convert.cc b/cpp/src/arrow/python/pandas_convert.cc
index 1a250e8..643c5fb 100644
--- a/cpp/src/arrow/python/pandas_convert.cc
+++ b/cpp/src/arrow/python/pandas_convert.cc
@@ -669,8 +669,8 @@ static Status ConvertDecimals(const ChunkedArray& data, PyObject** out_values) {
   for (int c = 0; c < data.num_chunks(); c++) {
     auto* arr(static_cast<arrow::DecimalArray*>(data.chunk(c).get()));
     auto type(std::dynamic_pointer_cast<arrow::DecimalType>(arr->type()));
-    const int precision = type->precision;
-    const int scale = type->scale;
+    const int precision = type->precision();
+    const int scale = type->scale();
     const int bit_width = type->bit_width();
 
     for (int64_t i = 0; i < arr->length(); ++i) {
@@ -764,7 +764,7 @@ Status PandasConverter::ConvertObjects() {
 
   // This means we received an explicit type from the user
   if (type_) {
-    switch (type_->type) {
+    switch (type_->id()) {
       case Type::STRING:
         return ConvertObjectStrings();
       case Type::FIXED_SIZE_BINARY:
@@ -777,7 +777,7 @@ Status PandasConverter::ConvertObjects() {
         return ConvertDates<Date64Type>();
       case Type::LIST: {
         const auto& list_field = static_cast<const ListType&>(*type_);
-        return ConvertLists(list_field.value_field()->type);
+        return ConvertLists(list_field.value_field()->type());
       }
       case Type::DECIMAL:
         return ConvertDecimals();
@@ -860,7 +860,7 @@ inline Status PandasConverter::ConvertTypedLists(const std::shared_ptr<DataType>
       std::shared_ptr<DataType> inferred_type;
       RETURN_NOT_OK(list_builder.Append(true));
       RETURN_NOT_OK(InferArrowTypeAndSize(objects[i], &size, &inferred_type));
-      if (inferred_type->type != type->type) {
+      if (inferred_type->id() != type->id()) {
         std::stringstream ss;
         ss << inferred_type->ToString() << " cannot be converted to " << type->ToString();
         return Status::TypeError(ss.str());
@@ -909,7 +909,7 @@ inline Status PandasConverter::ConvertTypedLists<NPY_OBJECT, StringType>(
       std::shared_ptr<DataType> inferred_type;
       RETURN_NOT_OK(list_builder.Append(true));
       RETURN_NOT_OK(InferArrowTypeAndSize(objects[i], &size, &inferred_type));
-      if (inferred_type->type != Type::STRING) {
+      if (inferred_type->id() != Type::STRING) {
         std::stringstream ss;
         ss << inferred_type->ToString() << " cannot be converted to STRING.";
         return Status::TypeError(ss.str());
@@ -928,7 +928,7 @@ inline Status PandasConverter::ConvertTypedLists<NPY_OBJECT, StringType>(
   }
 
 Status PandasConverter::ConvertLists(const std::shared_ptr<DataType>& type) {
-  switch (type->type) {
+  switch (type->id()) {
     LIST_CASE(UINT8, NPY_UINT8, UInt8Type)
     LIST_CASE(INT8, NPY_INT8, Int8Type)
     LIST_CASE(UINT16, NPY_UINT16, UInt16Type)
@@ -1300,7 +1300,7 @@ class ObjectBlock : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     PyObject** out_buffer =
         reinterpret_cast<PyObject**>(block_data_) + rel_placement * num_rows_;
@@ -1319,7 +1319,7 @@ class ObjectBlock : public PandasBlock {
       RETURN_NOT_OK(ConvertDecimals(data, out_buffer));
     } else if (type == Type::LIST) {
       auto list_type = std::static_pointer_cast<ListType>(col->type());
-      switch (list_type->value_type()->type) {
+      switch (list_type->value_type()->id()) {
         CONVERTLISTSLIKE_CASE(UInt8Type, UINT8)
         CONVERTLISTSLIKE_CASE(Int8Type, INT8)
         CONVERTLISTSLIKE_CASE(UInt16Type, UINT16)
@@ -1360,7 +1360,7 @@ class IntBlock : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     C_TYPE* out_buffer =
         reinterpret_cast<C_TYPE*>(block_data_) + rel_placement * num_rows_;
@@ -1392,7 +1392,7 @@ class Float32Block : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     if (type != Type::FLOAT) { return Status::NotImplemented(col->type()->ToString()); }
 
@@ -1412,7 +1412,7 @@ class Float64Block : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     double* out_buffer =
         reinterpret_cast<double*>(block_data_) + rel_placement * num_rows_;
@@ -1465,7 +1465,7 @@ class BoolBlock : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     if (type != Type::BOOL) { return Status::NotImplemented(col->type()->ToString()); }
 
@@ -1496,7 +1496,7 @@ class DatetimeBlock : public PandasBlock {
 
   Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
       int64_t rel_placement) override {
-    Type::type type = col->type()->type;
+    Type::type type = col->type()->id();
 
     int64_t* out_buffer =
         reinterpret_cast<int64_t*>(block_data_) + rel_placement * num_rows_;
@@ -1514,13 +1514,13 @@ class DatetimeBlock : public PandasBlock {
     } else if (type == Type::TIMESTAMP) {
       auto ts_type = static_cast<TimestampType*>(col->type().get());
 
-      if (ts_type->unit == TimeUnit::NANO) {
+      if (ts_type->unit() == TimeUnit::NANO) {
         ConvertNumericNullable<int64_t>(data, kPandasTimestampNull, out_buffer);
-      } else if (ts_type->unit == TimeUnit::MICRO) {
+      } else if (ts_type->unit() == TimeUnit::MICRO) {
         ConvertDatetimeNanos<int64_t, 1000L>(data, out_buffer);
-      } else if (ts_type->unit == TimeUnit::MILLI) {
+      } else if (ts_type->unit() == TimeUnit::MILLI) {
         ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
-      } else if (ts_type->unit == TimeUnit::SECOND) {
+      } else if (ts_type->unit() == TimeUnit::SECOND) {
         ConvertDatetimeNanos<int64_t, 1000000000L>(data, out_buffer);
       } else {
         return Status::NotImplemented("Unsupported time unit");
@@ -1661,7 +1661,7 @@ static inline Status MakeCategoricalBlock(const std::shared_ptr<DataType>& type,
     int64_t num_rows, std::shared_ptr<PandasBlock>* block) {
   // All categoricals become a block with a single column
   auto dict_type = static_cast<const DictionaryType*>(type.get());
-  switch (dict_type->index_type()->type) {
+  switch (dict_type->index_type()->id()) {
     case Type::INT8:
       *block = std::make_shared<CategoricalBlock<Type::INT8>>(num_rows);
       break;
@@ -1714,7 +1714,7 @@ class DataFrameBlockCreator {
       std::shared_ptr<Column> col = table_->column(i);
       PandasBlock::type output_type;
 
-      Type::type column_type = col->type()->type;
+      Type::type column_type = col->type()->id();
       switch (column_type) {
         case Type::BOOL:
           output_type = col->null_count() > 0 ? PandasBlock::OBJECT : PandasBlock::BOOL;
@@ -1762,7 +1762,7 @@ class DataFrameBlockCreator {
           break;
         case Type::TIMESTAMP: {
           const auto& ts_type = static_cast<const TimestampType&>(*col->type());
-          if (ts_type.timezone != "") {
+          if (ts_type.timezone() != "") {
             output_type = PandasBlock::DATETIME_WITH_TZ;
           } else {
             output_type = PandasBlock::DATETIME;
@@ -1770,7 +1770,7 @@ class DataFrameBlockCreator {
         } break;
         case Type::LIST: {
           auto list_type = std::static_pointer_cast<ListType>(col->type());
-          if (!ListTypeSupported(list_type->value_type()->type)) {
+          if (!ListTypeSupported(list_type->value_type()->id())) {
             std::stringstream ss;
             ss << "Not implemented type for lists: "
                << list_type->value_type()->ToString();
@@ -1795,7 +1795,7 @@ class DataFrameBlockCreator {
         categorical_blocks_[i] = block;
       } else if (output_type == PandasBlock::DATETIME_WITH_TZ) {
         const auto& ts_type = static_cast<const TimestampType&>(*col->type());
-        block = std::make_shared<DatetimeTZBlock>(ts_type.timezone, table_->num_rows());
+        block = std::make_shared<DatetimeTZBlock>(ts_type.timezone(), table_->num_rows());
         RETURN_NOT_OK(block->Allocate());
         datetimetz_blocks_[i] = block;
       } else {
@@ -1942,10 +1942,10 @@ inline void set_numpy_metadata(int type, DataType* datatype, PyArrayObject* out)
   if (type == NPY_DATETIME) {
     PyArray_Descr* descr = PyArray_DESCR(out);
     auto date_dtype = reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(descr->c_metadata);
-    if (datatype->type == Type::TIMESTAMP) {
+    if (datatype->id() == Type::TIMESTAMP) {
       auto timestamp_type = static_cast<TimestampType*>(datatype);
 
-      switch (timestamp_type->unit) {
+      switch (timestamp_type->unit()) {
         case TimestampType::Unit::SECOND:
           date_dtype->meta.base = NPY_FR_s;
           break;
@@ -2154,7 +2154,7 @@ class ArrowDeserializer {
     RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
     auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
     auto list_type = std::static_pointer_cast<ListType>(col_->type());
-    switch (list_type->value_type()->type) {
+    switch (list_type->value_type()->id()) {
       CONVERTVALUES_LISTSLIKE_CASE(UInt8Type, UINT8)
       CONVERTVALUES_LISTSLIKE_CASE(Int8Type, INT8)
       CONVERTVALUES_LISTSLIKE_CASE(UInt16Type, UINT16)

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/python/pandas_convert.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_convert.h b/cpp/src/arrow/python/pandas_convert.h
index 4d32c8b..fd901d8 100644
--- a/cpp/src/arrow/python/pandas_convert.h
+++ b/cpp/src/arrow/python/pandas_convert.h
@@ -32,7 +32,7 @@ namespace arrow {
 
 class Array;
 class Column;
-struct DataType;
+class DataType;
 class MemoryPool;
 class Status;
 class Table;

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/table-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table-test.cc b/cpp/src/arrow/table-test.cc
index cdc0238..0da4c0f 100644
--- a/cpp/src/arrow/table-test.cc
+++ b/cpp/src/arrow/table-test.cc
@@ -244,8 +244,8 @@ TEST_F(TestTable, Metadata) {
   ASSERT_TRUE(table_->schema()->Equals(*schema_));
 
   auto col = table_->column(0);
-  ASSERT_EQ(schema_->field(0)->name, col->name());
-  ASSERT_EQ(schema_->field(0)->type, col->type());
+  ASSERT_EQ(schema_->field(0)->name(), col->name());
+  ASSERT_EQ(schema_->field(0)->type(), col->type());
 }
 
 TEST_F(TestTable, InvalidColumns) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/table.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index 4c5257b..eabd98b 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -153,7 +153,7 @@ RecordBatch::RecordBatch(const std::shared_ptr<Schema>& schema, int64_t num_rows
     : schema_(schema), num_rows_(num_rows), columns_(std::move(columns)) {}
 
 const std::string& RecordBatch::column_name(int i) const {
-  return schema_->field(i)->name;
+  return schema_->field(i)->name();
 }
 
 bool RecordBatch::Equals(const RecordBatch& other) const {
@@ -204,7 +204,7 @@ Status RecordBatch::Validate() const {
          << " vs " << num_rows_;
       return Status::Invalid(ss.str());
     }
-    const auto& schema_type = *schema_->field(i)->type;
+    const auto& schema_type = *schema_->field(i)->type();
     if (!arr.type()->Equals(schema_type)) {
       std::stringstream ss;
       ss << "Column " << i << " type not match schema: " << arr.type()->ToString()

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/table.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index b15d31b..cfd1f36 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -81,10 +81,10 @@ class ARROW_EXPORT Column {
   std::shared_ptr<Field> field() const { return field_; }
 
   // @returns: the column's name in the passed metadata
-  const std::string& name() const { return field_->name; }
+  const std::string& name() const { return field_->name(); }
 
   // @returns: the column's type according to the metadata
-  std::shared_ptr<DataType> type() const { return field_->type; }
+  std::shared_ptr<DataType> type() const { return field_->type(); }
 
   // @returns: the column's data as a chunked logical array
   std::shared_ptr<ChunkedArray> data() const { return data_; }

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/tensor.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/tensor.cc b/cpp/src/arrow/tensor.cc
index d1c4083..fa3e203 100644
--- a/cpp/src/arrow/tensor.cc
+++ b/cpp/src/arrow/tensor.cc
@@ -61,7 +61,7 @@ Tensor::Tensor(const std::shared_ptr<DataType>& type, const std::shared_ptr<Buff
     const std::vector<int64_t>& shape, const std::vector<int64_t>& strides,
     const std::vector<std::string>& dim_names)
     : type_(type), data_(data), shape_(shape), strides_(strides), dim_names_(dim_names) {
-  DCHECK(is_tensor_supported(type->type));
+  DCHECK(is_tensor_supported(type->id()));
   if (shape.size() > 0 && strides.size() == 0) {
     ComputeRowMajorStrides(static_cast<const FixedWidthType&>(*type_), shape, &strides_);
   }
@@ -107,6 +107,10 @@ bool Tensor::is_column_major() const {
   return strides_ == f_strides;
 }
 
+Type::type Tensor::type_id() const {
+  return type_->id();
+}
+
 bool Tensor::Equals(const Tensor& other) const {
   bool are_equal = false;
   Status error = TensorEquals(*this, other, &are_equal);
@@ -161,7 +165,7 @@ Status ARROW_EXPORT MakeTensor(const std::shared_ptr<DataType>& type,
     const std::shared_ptr<Buffer>& data, const std::vector<int64_t>& shape,
     const std::vector<int64_t>& strides, const std::vector<std::string>& dim_names,
     std::shared_ptr<Tensor>* tensor) {
-  switch (type->type) {
+  switch (type->id()) {
     TENSOR_CASE(INT8, Int8Tensor);
     TENSOR_CASE(INT16, Int16Tensor);
     TENSOR_CASE(INT32, Int32Tensor);

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/tensor.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/tensor.h b/cpp/src/arrow/tensor.h
index 12015f1..7741c30 100644
--- a/cpp/src/arrow/tensor.h
+++ b/cpp/src/arrow/tensor.h
@@ -98,7 +98,7 @@ class ARROW_EXPORT Tensor {
   /// AKA "Fortran order"
   bool is_column_major() const;
 
-  Type::type type_enum() const { return type_->type; }
+  Type::type type_id() const;
 
   bool Equals(const Tensor& other) const;
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/type-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type-test.cc b/cpp/src/arrow/type-test.cc
index 66164e3..dec7268 100644
--- a/cpp/src/arrow/type-test.cc
+++ b/cpp/src/arrow/type-test.cc
@@ -34,11 +34,11 @@ TEST(TestField, Basics) {
   Field f0("f0", int32());
   Field f0_nn("f0", int32(), false);
 
-  ASSERT_EQ(f0.name, "f0");
-  ASSERT_EQ(f0.type->ToString(), int32()->ToString());
+  ASSERT_EQ(f0.name(), "f0");
+  ASSERT_EQ(f0.type()->ToString(), int32()->ToString());
 
-  ASSERT_TRUE(f0.nullable);
-  ASSERT_FALSE(f0_nn.nullable);
+  ASSERT_TRUE(f0.nullable());
+  ASSERT_FALSE(f0_nn.nullable());
 }
 
 TEST(TestField, Equals) {
@@ -121,7 +121,7 @@ TEST_F(TestSchema, GetFieldByName) {
   TEST(TypesTest, TestPrimitive_##ENUM) {        \
     KLASS tp;                                    \
                                                  \
-    ASSERT_EQ(tp.type, Type::ENUM);              \
+    ASSERT_EQ(tp.id(), Type::ENUM);              \
     ASSERT_EQ(tp.ToString(), std::string(NAME)); \
   }
 
@@ -145,19 +145,19 @@ TEST(TestBinaryType, ToString) {
   StringType t2;
   EXPECT_TRUE(t1.Equals(e1));
   EXPECT_FALSE(t1.Equals(t2));
-  ASSERT_EQ(t1.type, Type::BINARY);
+  ASSERT_EQ(t1.id(), Type::BINARY);
   ASSERT_EQ(t1.ToString(), std::string("binary"));
 }
 
 TEST(TestStringType, ToString) {
   StringType str;
-  ASSERT_EQ(str.type, Type::STRING);
+  ASSERT_EQ(str.id(), Type::STRING);
   ASSERT_EQ(str.ToString(), std::string("string"));
 }
 
 TEST(TestFixedSizeBinaryType, ToString) {
   auto t = fixed_size_binary(10);
-  ASSERT_EQ(t->type, Type::FIXED_SIZE_BINARY);
+  ASSERT_EQ(t->id(), Type::FIXED_SIZE_BINARY);
   ASSERT_EQ("fixed_size_binary[10]", t->ToString());
 }
 
@@ -175,13 +175,13 @@ TEST(TestListType, Basics) {
   std::shared_ptr<DataType> vt = std::make_shared<UInt8Type>();
 
   ListType list_type(vt);
-  ASSERT_EQ(list_type.type, Type::LIST);
+  ASSERT_EQ(list_type.id(), Type::LIST);
 
   ASSERT_EQ("list", list_type.name());
   ASSERT_EQ("list<item: uint8>", list_type.ToString());
 
-  ASSERT_EQ(list_type.value_type()->type, vt->type);
-  ASSERT_EQ(list_type.value_type()->type, vt->type);
+  ASSERT_EQ(list_type.value_type()->id(), vt->id());
+  ASSERT_EQ(list_type.value_type()->id(), vt->id());
 
   std::shared_ptr<DataType> st = std::make_shared<StringType>();
   std::shared_ptr<DataType> lt = std::make_shared<ListType>(st);
@@ -315,4 +315,46 @@ TEST(TestStructType, Basics) {
   // TODO(wesm): out of bounds for field(...)
 }
 
+TEST(TypesTest, TestDecimal32Type) {
+  DecimalType t1(8, 4);
+
+  ASSERT_EQ(t1.id(), Type::DECIMAL);
+  ASSERT_EQ(t1.precision(), 8);
+  ASSERT_EQ(t1.scale(), 4);
+
+  ASSERT_EQ(t1.ToString(), std::string("decimal(8, 4)"));
+
+  // Test properties
+  ASSERT_EQ(t1.byte_width(), 4);
+  ASSERT_EQ(t1.bit_width(), 32);
+}
+
+TEST(TypesTest, TestDecimal64Type) {
+  DecimalType t1(12, 5);
+
+  ASSERT_EQ(t1.id(), Type::DECIMAL);
+  ASSERT_EQ(t1.precision(), 12);
+  ASSERT_EQ(t1.scale(), 5);
+
+  ASSERT_EQ(t1.ToString(), std::string("decimal(12, 5)"));
+
+  // Test properties
+  ASSERT_EQ(t1.byte_width(), 8);
+  ASSERT_EQ(t1.bit_width(), 64);
+}
+
+TEST(TypesTest, TestDecimal128Type) {
+  DecimalType t1(27, 7);
+
+  ASSERT_EQ(t1.id(), Type::DECIMAL);
+  ASSERT_EQ(t1.precision(), 27);
+  ASSERT_EQ(t1.scale(), 7);
+
+  ASSERT_EQ(t1.ToString(), std::string("decimal(27, 7)"));
+
+  // Test properties
+  ASSERT_EQ(t1.byte_width(), 16);
+  ASSERT_EQ(t1.bit_width(), 128);
+}
+
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 93cab14..a2300d6 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -32,8 +32,8 @@ namespace arrow {
 
 bool Field::Equals(const Field& other) const {
   return (this == &other) ||
-         (this->name == other.name && this->nullable == other.nullable &&
-             this->type->Equals(*other.type.get()));
+         (this->name_ == other.name_ && this->nullable_ == other.nullable_ &&
+             this->type_->Equals(*other.type_.get()));
 }
 
 bool Field::Equals(const std::shared_ptr<Field>& other) const {
@@ -42,8 +42,8 @@ bool Field::Equals(const std::shared_ptr<Field>& other) const {
 
 std::string Field::ToString() const {
   std::stringstream ss;
-  ss << this->name << ": " << this->type->ToString();
-  if (!this->nullable) { ss << " not null"; }
+  ss << this->name_ << ": " << this->type_->ToString();
+  if (!this->nullable_) { ss << " not null"; }
   return ss.str();
 }
 
@@ -107,7 +107,7 @@ std::string StructType::ToString() const {
   for (int i = 0; i < this->num_children(); ++i) {
     if (i > 0) { s << ", "; }
     std::shared_ptr<Field> field = this->child(i);
-    s << field->name << ": " << field->type->ToString();
+    s << field->name() << ": " << field->type()->ToString();
   }
   s << ">";
   return s.str();
@@ -117,7 +117,7 @@ std::string StructType::ToString() const {
 // Date types
 
 DateType::DateType(Type::type type_id, DateUnit unit)
-    : FixedWidthType(type_id), unit(unit) {}
+    : FixedWidthType(type_id), unit_(unit) {}
 
 Date32Type::Date32Type() : DateType(Type::DATE32, DateUnit::DAY) {}
 
@@ -135,7 +135,7 @@ std::string Date32Type::ToString() const {
 // Time types
 
 TimeType::TimeType(Type::type type_id, TimeUnit unit)
-    : FixedWidthType(type_id), unit(unit) {}
+    : FixedWidthType(type_id), unit_(unit) {}
 
 Time32Type::Time32Type(TimeUnit unit) : TimeType(Type::TIME32, unit) {
   DCHECK(unit == TimeUnit::SECOND || unit == TimeUnit::MILLI)
@@ -144,7 +144,7 @@ Time32Type::Time32Type(TimeUnit unit) : TimeType(Type::TIME32, unit) {
 
 std::string Time32Type::ToString() const {
   std::stringstream ss;
-  ss << "time32[" << this->unit << "]";
+  ss << "time32[" << this->unit_ << "]";
   return ss.str();
 }
 
@@ -155,7 +155,7 @@ Time64Type::Time64Type(TimeUnit unit) : TimeType(Type::TIME64, unit) {
 
 std::string Time64Type::ToString() const {
   std::stringstream ss;
-  ss << "time64[" << this->unit << "]";
+  ss << "time64[" << this->unit_ << "]";
   return ss.str();
 }
 
@@ -164,8 +164,8 @@ std::string Time64Type::ToString() const {
 
 std::string TimestampType::ToString() const {
   std::stringstream ss;
-  ss << "timestamp[" << this->unit;
-  if (this->timezone.size() > 0) { ss << ", tz=" << this->timezone; }
+  ss << "timestamp[" << this->unit_;
+  if (this->timezone_.size() > 0) { ss << ", tz=" << this->timezone_; }
   ss << "]";
   return ss.str();
 }
@@ -175,14 +175,14 @@ std::string TimestampType::ToString() const {
 
 UnionType::UnionType(const std::vector<std::shared_ptr<Field>>& fields,
     const std::vector<uint8_t>& type_codes, UnionMode mode)
-    : NestedType(Type::UNION), mode(mode), type_codes(type_codes) {
+    : NestedType(Type::UNION), mode_(mode), type_codes_(type_codes) {
   children_ = fields;
 }
 
 std::string UnionType::ToString() const {
   std::stringstream s;
 
-  if (mode == UnionMode::SPARSE) {
+  if (mode_ == UnionMode::SPARSE) {
     s << "union[sparse]<";
   } else {
     s << "union[dense]<";
@@ -190,7 +190,7 @@ std::string UnionType::ToString() const {
 
   for (size_t i = 0; i < children_.size(); ++i) {
     if (i) { s << ", "; }
-    s << children_[i]->ToString() << "=" << static_cast<int>(type_codes[i]);
+    s << children_[i]->ToString() << "=" << static_cast<int>(type_codes_[i]);
   }
   s << ">";
   return s.str();
@@ -246,7 +246,7 @@ bool Schema::Equals(const Schema& other) const {
 std::shared_ptr<Field> Schema::GetFieldByName(const std::string& name) {
   if (fields_.size() > 0 && name_to_index_.size() == 0) {
     for (size_t i = 0; i < fields_.size(); ++i) {
-      name_to_index_[fields_[i]->name] = static_cast<int>(i);
+      name_to_index_[fields_[i]->name()] = static_cast<int>(i);
     }
   }
 
@@ -423,7 +423,7 @@ std::vector<BufferDescr> StructType::GetBufferLayout() const {
 }
 
 std::vector<BufferDescr> UnionType::GetBufferLayout() const {
-  if (mode == UnionMode::SPARSE) {
+  if (mode_ == UnionMode::SPARSE) {
     return {kValidityBuffer, kTypeBuffer};
   } else {
     return {kValidityBuffer, kTypeBuffer, kOffsetBuffer};
@@ -432,7 +432,7 @@ std::vector<BufferDescr> UnionType::GetBufferLayout() const {
 
 std::string DecimalType::ToString() const {
   std::stringstream s;
-  s << "decimal(" << precision << ", " << scale << ")";
+  s << "decimal(" << precision_ << ", " << scale_ << ")";
   return s.str();
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 730cbed..6810b35 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -127,13 +127,9 @@ class BufferDescr {
   int bit_width_;
 };
 
-struct ARROW_EXPORT DataType {
-  Type::type type;
-
-  std::vector<std::shared_ptr<Field>> children_;
-
-  explicit DataType(Type::type type) : type(type) {}
-
+class ARROW_EXPORT DataType {
+ public:
+  explicit DataType(Type::type id) : id_(id) {}
   virtual ~DataType();
 
   // Return whether the types are equal
@@ -155,13 +151,20 @@ struct ARROW_EXPORT DataType {
 
   virtual std::vector<BufferDescr> GetBufferLayout() const = 0;
 
+  Type::type id() const { return id_; }
+
+ protected:
+  Type::type id_;
+  std::vector<std::shared_ptr<Field>> children_;
+
  private:
   DISALLOW_COPY_AND_ASSIGN(DataType);
 };
 
 typedef std::shared_ptr<DataType> TypePtr;
 
-struct ARROW_EXPORT FixedWidthType : public DataType {
+class ARROW_EXPORT FixedWidthType : public DataType {
+ public:
   using DataType::DataType;
 
   virtual int bit_width() const = 0;
@@ -169,53 +172,64 @@ struct ARROW_EXPORT FixedWidthType : public DataType {
   std::vector<BufferDescr> GetBufferLayout() const override;
 };
 
-struct ARROW_EXPORT PrimitiveCType : public FixedWidthType {
+class ARROW_EXPORT PrimitiveCType : public FixedWidthType {
+ public:
   using FixedWidthType::FixedWidthType;
 };
 
-struct ARROW_EXPORT Integer : public PrimitiveCType {
+class ARROW_EXPORT Integer : public PrimitiveCType {
+ public:
   using PrimitiveCType::PrimitiveCType;
   virtual bool is_signed() const = 0;
 };
 
-struct ARROW_EXPORT FloatingPoint : public PrimitiveCType {
+class ARROW_EXPORT FloatingPoint : public PrimitiveCType {
+ public:
   using PrimitiveCType::PrimitiveCType;
   enum Precision { HALF, SINGLE, DOUBLE };
   virtual Precision precision() const = 0;
 };
 
-struct ARROW_EXPORT NestedType : public DataType {
+class ARROW_EXPORT NestedType : public DataType {
+ public:
   using DataType::DataType;
 };
 
-struct NoExtraMeta {};
+class NoExtraMeta {};
 
 // A field is a piece of metadata that includes (for now) a name and a data
 // type
-struct ARROW_EXPORT Field {
-  // Field name
-  std::string name;
-
-  // The field's data type
-  std::shared_ptr<DataType> type;
-
-  // Fields can be nullable
-  bool nullable;
-
+class ARROW_EXPORT Field {
+ public:
   Field(const std::string& name, const std::shared_ptr<DataType>& type,
       bool nullable = true)
-      : name(name), type(type), nullable(nullable) {}
+      : name_(name), type_(type), nullable_(nullable) {}
 
   bool Equals(const Field& other) const;
   bool Equals(const std::shared_ptr<Field>& other) const;
 
   std::string ToString() const;
+
+  const std::string& name() const { return name_; }
+  std::shared_ptr<DataType> type() const { return type_; }
+  bool nullable() const { return nullable_; }
+
+ private:
+  // Field name
+  std::string name_;
+
+  // The field's data type
+  std::shared_ptr<DataType> type_;
+
+  // Fields can be nullable
+  bool nullable_;
 };
 
 typedef std::shared_ptr<Field> FieldPtr;
 
 template <typename DERIVED, typename BASE, Type::type TYPE_ID, typename C_TYPE>
-struct ARROW_EXPORT CTypeImpl : public BASE {
+class ARROW_EXPORT CTypeImpl : public BASE {
+ public:
   using c_type = C_TYPE;
   static constexpr Type::type type_id = TYPE_ID;
 
@@ -230,7 +244,8 @@ struct ARROW_EXPORT CTypeImpl : public BASE {
   std::string ToString() const override { return std::string(DERIVED::name()); }
 };
 
-struct ARROW_EXPORT NullType : public DataType, public NoExtraMeta {
+class ARROW_EXPORT NullType : public DataType, public NoExtraMeta {
+ public:
   static constexpr Type::type type_id = Type::NA;
 
   NullType() : DataType(Type::NA) {}
@@ -244,11 +259,12 @@ struct ARROW_EXPORT NullType : public DataType, public NoExtraMeta {
 };
 
 template <typename DERIVED, Type::type TYPE_ID, typename C_TYPE>
-struct IntegerTypeImpl : public CTypeImpl<DERIVED, Integer, TYPE_ID, C_TYPE> {
+class IntegerTypeImpl : public CTypeImpl<DERIVED, Integer, TYPE_ID, C_TYPE> {
   bool is_signed() const override { return std::is_signed<C_TYPE>::value; }
 };
 
-struct ARROW_EXPORT BooleanType : public FixedWidthType, public NoExtraMeta {
+class ARROW_EXPORT BooleanType : public FixedWidthType, public NoExtraMeta {
+ public:
   static constexpr Type::type type_id = Type::BOOL;
 
   BooleanType() : FixedWidthType(Type::BOOL) {}
@@ -260,60 +276,72 @@ struct ARROW_EXPORT BooleanType : public FixedWidthType, public NoExtraMeta {
   static std::string name() { return "bool"; }
 };
 
-struct ARROW_EXPORT UInt8Type : public IntegerTypeImpl<UInt8Type, Type::UINT8, uint8_t> {
+class ARROW_EXPORT UInt8Type : public IntegerTypeImpl<UInt8Type, Type::UINT8, uint8_t> {
+ public:
   static std::string name() { return "uint8"; }
 };
 
-struct ARROW_EXPORT Int8Type : public IntegerTypeImpl<Int8Type, Type::INT8, int8_t> {
+class ARROW_EXPORT Int8Type : public IntegerTypeImpl<Int8Type, Type::INT8, int8_t> {
+ public:
   static std::string name() { return "int8"; }
 };
 
-struct ARROW_EXPORT UInt16Type
+class ARROW_EXPORT UInt16Type
     : public IntegerTypeImpl<UInt16Type, Type::UINT16, uint16_t> {
+ public:
   static std::string name() { return "uint16"; }
 };
 
-struct ARROW_EXPORT Int16Type : public IntegerTypeImpl<Int16Type, Type::INT16, int16_t> {
+class ARROW_EXPORT Int16Type : public IntegerTypeImpl<Int16Type, Type::INT16, int16_t> {
+ public:
   static std::string name() { return "int16"; }
 };
 
-struct ARROW_EXPORT UInt32Type
+class ARROW_EXPORT UInt32Type
     : public IntegerTypeImpl<UInt32Type, Type::UINT32, uint32_t> {
+ public:
   static std::string name() { return "uint32"; }
 };
 
-struct ARROW_EXPORT Int32Type : public IntegerTypeImpl<Int32Type, Type::INT32, int32_t> {
+class ARROW_EXPORT Int32Type : public IntegerTypeImpl<Int32Type, Type::INT32, int32_t> {
+ public:
   static std::string name() { return "int32"; }
 };
 
-struct ARROW_EXPORT UInt64Type
+class ARROW_EXPORT UInt64Type
     : public IntegerTypeImpl<UInt64Type, Type::UINT64, uint64_t> {
+ public:
   static std::string name() { return "uint64"; }
 };
 
-struct ARROW_EXPORT Int64Type : public IntegerTypeImpl<Int64Type, Type::INT64, int64_t> {
+class ARROW_EXPORT Int64Type : public IntegerTypeImpl<Int64Type, Type::INT64, int64_t> {
+ public:
   static std::string name() { return "int64"; }
 };
 
-struct ARROW_EXPORT HalfFloatType
+class ARROW_EXPORT HalfFloatType
     : public CTypeImpl<HalfFloatType, FloatingPoint, Type::HALF_FLOAT, uint16_t> {
+ public:
   Precision precision() const override;
   static std::string name() { return "halffloat"; }
 };
 
-struct ARROW_EXPORT FloatType
+class ARROW_EXPORT FloatType
     : public CTypeImpl<FloatType, FloatingPoint, Type::FLOAT, float> {
+ public:
   Precision precision() const override;
   static std::string name() { return "float"; }
 };
 
-struct ARROW_EXPORT DoubleType
+class ARROW_EXPORT DoubleType
     : public CTypeImpl<DoubleType, FloatingPoint, Type::DOUBLE, double> {
+ public:
   Precision precision() const override;
   static std::string name() { return "double"; }
 };
 
-struct ARROW_EXPORT ListType : public NestedType {
+class ARROW_EXPORT ListType : public NestedType {
+ public:
   static constexpr Type::type type_id = Type::LIST;
 
   // List can contain any other logical value type
@@ -326,7 +354,7 @@ struct ARROW_EXPORT ListType : public NestedType {
 
   std::shared_ptr<Field> value_field() const { return children_[0]; }
 
-  std::shared_ptr<DataType> value_type() const { return children_[0]->type; }
+  std::shared_ptr<DataType> value_type() const { return children_[0]->type(); }
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
@@ -337,7 +365,8 @@ struct ARROW_EXPORT ListType : public NestedType {
 };
 
 // BinaryType type is represents lists of 1-byte values.
-struct ARROW_EXPORT BinaryType : public DataType, public NoExtraMeta {
+class ARROW_EXPORT BinaryType : public DataType, public NoExtraMeta {
+ public:
   static constexpr Type::type type_id = Type::BINARY;
 
   BinaryType() : BinaryType(Type::BINARY) {}
@@ -376,7 +405,8 @@ class ARROW_EXPORT FixedSizeBinaryType : public FixedWidthType {
 };
 
 // UTF-8 encoded strings
-struct ARROW_EXPORT StringType : public BinaryType {
+class ARROW_EXPORT StringType : public BinaryType {
+ public:
   static constexpr Type::type type_id = Type::STRING;
 
   StringType() : BinaryType(Type::STRING) {}
@@ -386,7 +416,8 @@ struct ARROW_EXPORT StringType : public BinaryType {
   static std::string name() { return "utf8"; }
 };
 
-struct ARROW_EXPORT StructType : public NestedType {
+class ARROW_EXPORT StructType : public NestedType {
+ public:
   static constexpr Type::type type_id = Type::STRUCT;
 
   explicit StructType(const std::vector<std::shared_ptr<Field>>& fields)
@@ -412,25 +443,32 @@ static inline int decimal_byte_width(int precision) {
   }
 }
 
-struct ARROW_EXPORT DecimalType : public FixedSizeBinaryType {
+class ARROW_EXPORT DecimalType : public FixedSizeBinaryType {
+ public:
   static constexpr Type::type type_id = Type::DECIMAL;
 
-  explicit DecimalType(int precision_, int scale_)
-      : FixedSizeBinaryType(decimal_byte_width(precision_), Type::DECIMAL),
-        precision(precision_),
-        scale(scale_) {}
+  explicit DecimalType(int precision, int scale)
+      : FixedSizeBinaryType(decimal_byte_width(precision), Type::DECIMAL),
+        precision_(precision),
+        scale_(scale) {}
+
   std::vector<BufferDescr> GetBufferLayout() const override;
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
   static std::string name() { return "decimal"; }
 
-  int precision;
-  int scale;
+  int precision() const { return precision_; }
+  int scale() const { return scale_; }
+
+ private:
+  int precision_;
+  int scale_;
 };
 
 enum class UnionMode : char { SPARSE, DENSE };
 
-struct ARROW_EXPORT UnionType : public NestedType {
+class ARROW_EXPORT UnionType : public NestedType {
+ public:
   static constexpr Type::type type_id = Type::UNION;
 
   UnionType(const std::vector<std::shared_ptr<Field>>& fields,
@@ -442,12 +480,17 @@ struct ARROW_EXPORT UnionType : public NestedType {
 
   std::vector<BufferDescr> GetBufferLayout() const override;
 
-  UnionMode mode;
+  const std::vector<uint8_t>& type_codes() const { return type_codes_; }
+
+  UnionMode mode() const { return mode_; }
+
+ private:
+  UnionMode mode_;
 
   // The type id used in the data to indicate each data type in the union. For
   // example, the first type in the union might be denoted by the id 5 (instead
   // of 0).
-  std::vector<uint8_t> type_codes;
+  std::vector<uint8_t> type_codes_;
 };
 
 // ----------------------------------------------------------------------
@@ -455,16 +498,18 @@ struct ARROW_EXPORT UnionType : public NestedType {
 
 enum class DateUnit : char { DAY = 0, MILLI = 1 };
 
-struct ARROW_EXPORT DateType : public FixedWidthType {
+class ARROW_EXPORT DateType : public FixedWidthType {
  public:
-  DateUnit unit;
+  DateUnit unit() const { return unit_; }
 
  protected:
   DateType(Type::type type_id, DateUnit unit);
+  DateUnit unit_;
 };
 
 /// Date as int32_t days since UNIX epoch
-struct ARROW_EXPORT Date32Type : public DateType {
+class ARROW_EXPORT Date32Type : public DateType {
+ public:
   static constexpr Type::type type_id = Type::DATE32;
 
   using c_type = int32_t;
@@ -478,7 +523,8 @@ struct ARROW_EXPORT Date32Type : public DateType {
 };
 
 /// Date as int64_t milliseconds since UNIX epoch
-struct ARROW_EXPORT Date64Type : public DateType {
+class ARROW_EXPORT Date64Type : public DateType {
+ public:
   static constexpr Type::type type_id = Type::DATE64;
 
   using c_type = int64_t;
@@ -512,15 +558,17 @@ static inline std::ostream& operator<<(std::ostream& os, TimeUnit unit) {
   return os;
 }
 
-struct ARROW_EXPORT TimeType : public FixedWidthType {
+class ARROW_EXPORT TimeType : public FixedWidthType {
  public:
-  TimeUnit unit;
+  TimeUnit unit() const { return unit_; }
 
  protected:
   TimeType(Type::type type_id, TimeUnit unit);
+  TimeUnit unit_;
 };
 
-struct ARROW_EXPORT Time32Type : public TimeType {
+class ARROW_EXPORT Time32Type : public TimeType {
+ public:
   static constexpr Type::type type_id = Type::TIME32;
   using c_type = int32_t;
 
@@ -532,7 +580,8 @@ struct ARROW_EXPORT Time32Type : public TimeType {
   std::string ToString() const override;
 };
 
-struct ARROW_EXPORT Time64Type : public TimeType {
+class ARROW_EXPORT Time64Type : public TimeType {
+ public:
   static constexpr Type::type type_id = Type::TIME64;
   using c_type = int64_t;
 
@@ -544,7 +593,8 @@ struct ARROW_EXPORT Time64Type : public TimeType {
   std::string ToString() const override;
 };
 
-struct ARROW_EXPORT TimestampType : public FixedWidthType {
+class ARROW_EXPORT TimestampType : public FixedWidthType {
+ public:
   using Unit = TimeUnit;
 
   typedef int64_t c_type;
@@ -553,20 +603,25 @@ struct ARROW_EXPORT TimestampType : public FixedWidthType {
   int bit_width() const override { return static_cast<int>(sizeof(int64_t) * CHAR_BIT); }
 
   explicit TimestampType(TimeUnit unit = TimeUnit::MILLI)
-      : FixedWidthType(Type::TIMESTAMP), unit(unit) {}
+      : FixedWidthType(Type::TIMESTAMP), unit_(unit) {}
 
   explicit TimestampType(TimeUnit unit, const std::string& timezone)
-      : FixedWidthType(Type::TIMESTAMP), unit(unit), timezone(timezone) {}
+      : FixedWidthType(Type::TIMESTAMP), unit_(unit), timezone_(timezone) {}
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;
   static std::string name() { return "timestamp"; }
 
-  TimeUnit unit;
-  std::string timezone;
+  TimeUnit unit() const { return unit_; }
+  const std::string& timezone() const { return timezone_; }
+
+ private:
+  TimeUnit unit_;
+  std::string timezone_;
 };
 
-struct ARROW_EXPORT IntervalType : public FixedWidthType {
+class ARROW_EXPORT IntervalType : public FixedWidthType {
+ public:
   enum class Unit : char { YEAR_MONTH = 0, DAY_TIME = 1 };
 
   using c_type = int64_t;
@@ -574,14 +629,17 @@ struct ARROW_EXPORT IntervalType : public FixedWidthType {
 
   int bit_width() const override { return static_cast<int>(sizeof(int64_t) * CHAR_BIT); }
 
-  Unit unit;
-
   explicit IntervalType(Unit unit = Unit::YEAR_MONTH)
-      : FixedWidthType(Type::INTERVAL), unit(unit) {}
+      : FixedWidthType(Type::INTERVAL), unit_(unit) {}
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override { return name(); }
   static std::string name() { return "date"; }
+
+  Unit unit() const { return unit_; }
+
+ private:
+  Unit unit_;
 };
 
 // ----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/type_fwd.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_fwd.h b/cpp/src/arrow/type_fwd.h
index 2bb05f8..99c09bd 100644
--- a/cpp/src/arrow/type_fwd.h
+++ b/cpp/src/arrow/type_fwd.h
@@ -26,10 +26,10 @@ namespace arrow {
 
 class Status;
 
-struct DataType;
+class DataType;
 class Array;
 class ArrayBuilder;
-struct Field;
+class Field;
 class Tensor;
 
 class Buffer;
@@ -40,14 +40,14 @@ class Schema;
 class DictionaryType;
 class DictionaryArray;
 
-struct NullType;
+class NullType;
 class NullArray;
 
-struct BooleanType;
+class BooleanType;
 class BooleanArray;
 class BooleanBuilder;
 
-struct BinaryType;
+class BinaryType;
 class BinaryArray;
 class BinaryBuilder;
 
@@ -55,23 +55,23 @@ class FixedSizeBinaryType;
 class FixedSizeBinaryArray;
 class FixedSizeBinaryBuilder;
 
-struct StringType;
+class StringType;
 class StringArray;
 class StringBuilder;
 
-struct ListType;
+class ListType;
 class ListArray;
 class ListBuilder;
 
-struct StructType;
+class StructType;
 class StructArray;
 class StructBuilder;
 
-struct DecimalType;
+class DecimalType;
 class DecimalArray;
 class DecimalBuilder;
 
-struct UnionType;
+class UnionType;
 class UnionArray;
 
 template <typename TypeClass>
@@ -84,7 +84,7 @@ template <typename TypeClass>
 class NumericTensor;
 
 #define _NUMERIC_TYPE_DECL(KLASS)                     \
-  struct KLASS##Type;                                 \
+  class KLASS##Type;                                  \
   using KLASS##Array = NumericArray<KLASS##Type>;     \
   using KLASS##Builder = NumericBuilder<KLASS##Type>; \
   using KLASS##Tensor = NumericTensor<KLASS##Type>;
@@ -103,27 +103,27 @@ _NUMERIC_TYPE_DECL(Double);
 
 #undef _NUMERIC_TYPE_DECL
 
-struct Date64Type;
+class Date64Type;
 using Date64Array = NumericArray<Date64Type>;
 using Date64Builder = NumericBuilder<Date64Type>;
 
-struct Date32Type;
+class Date32Type;
 using Date32Array = NumericArray<Date32Type>;
 using Date32Builder = NumericBuilder<Date32Type>;
 
-struct Time32Type;
+class Time32Type;
 using Time32Array = NumericArray<Time32Type>;
 using Time32Builder = NumericBuilder<Time32Type>;
 
-struct Time64Type;
+class Time64Type;
 using Time64Array = NumericArray<Time64Type>;
 using Time64Builder = NumericBuilder<Time64Type>;
 
-struct TimestampType;
+class TimestampType;
 using TimestampArray = NumericArray<TimestampType>;
 using TimestampBuilder = NumericBuilder<TimestampType>;
 
-struct IntervalType;
+class IntervalType;
 using IntervalArray = NumericArray<IntervalType>;
 
 // ----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/arrow/blob/793f4e0c/cpp/src/arrow/visitor_inline.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/visitor_inline.h b/cpp/src/arrow/visitor_inline.h
index 29b3db6..bc5f493 100644
--- a/cpp/src/arrow/visitor_inline.h
+++ b/cpp/src/arrow/visitor_inline.h
@@ -33,7 +33,7 @@ namespace arrow {
 
 template <typename VISITOR>
 inline Status VisitTypeInline(const DataType& type, VISITOR* visitor) {
-  switch (type.type) {
+  switch (type.id()) {
     TYPE_VISIT_INLINE(NullType);
     TYPE_VISIT_INLINE(BooleanType);
     TYPE_VISIT_INLINE(Int8Type);
@@ -72,7 +72,7 @@ inline Status VisitTypeInline(const DataType& type, VISITOR* visitor) {
 
 template <typename VISITOR>
 inline Status VisitArrayInline(const Array& array, VISITOR* visitor) {
-  switch (array.type_enum()) {
+  switch (array.type_id()) {
     ARRAY_VISIT_INLINE(NullType);
     ARRAY_VISIT_INLINE(BooleanType);
     ARRAY_VISIT_INLINE(Int8Type);
@@ -111,7 +111,7 @@ inline Status VisitArrayInline(const Array& array, VISITOR* visitor) {
 
 template <typename VISITOR>
 inline Status VisitTensorInline(const Tensor& array, VISITOR* visitor) {
-  switch (array.type_enum()) {
+  switch (array.type_id()) {
     TENSOR_VISIT_INLINE(Int8Type);
     TENSOR_VISIT_INLINE(UInt8Type);
     TENSOR_VISIT_INLINE(Int16Type);