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/03/23 16:38:50 UTC

arrow git commit: ARROW-621: [C++] Start IPC benchmark suite for record batches, implement "inline" visitor. Code reorg

Repository: arrow
Updated Branches:
  refs/heads/master 2a568f093 -> e968ca6e3


ARROW-621: [C++] Start IPC benchmark suite for record batches, implement "inline" visitor. Code reorg

>From the benchmarks, the difference between virtual functions and an "inline" switch statement is very small, but it serves to reduce some boilerplate when many of the visit functions are the same

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

Closes #427 from wesm/ARROW-621 and squashes the following commits:

b975053 [Wes McKinney] cpplint
782636a [Wes McKinney] Mark template inline
3ae494e [Wes McKinney] Inline visitor, remove code duplication in loader.cc in favor of templates / std::enable_if
1b2d253 [Wes McKinney] Tweak benchmark params
b126ca8 [Wes McKinney] Draft IPC roundtrip benchmark for wide record batches. Some test code refactoring


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

Branch: refs/heads/master
Commit: e968ca6e30209abeb90e099eb95de59655be73a8
Parents: 2a568f0
Author: Wes McKinney <we...@twosigma.com>
Authored: Thu Mar 23 12:38:40 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Thu Mar 23 12:38:40 2017 -0400

----------------------------------------------------------------------
 cpp/CMakeLists.txt                            |   1 +
 cpp/src/arrow/CMakeLists.txt                  |   1 +
 cpp/src/arrow/api.h                           |   1 +
 cpp/src/arrow/array-list-test.cc              |   1 +
 cpp/src/arrow/array-primitive-test.cc         |   1 +
 cpp/src/arrow/array-string-test.cc            |   1 +
 cpp/src/arrow/array-struct-test.cc            |   1 +
 cpp/src/arrow/array.cc                        |  36 +-----
 cpp/src/arrow/array.h                         |  33 +----
 cpp/src/arrow/column-test.cc                  |   1 +
 cpp/src/arrow/ipc/CMakeLists.txt              |   4 +
 cpp/src/arrow/ipc/ipc-read-write-benchmark.cc | 134 +++++++++++++++++++++
 cpp/src/arrow/ipc/reader.h                    |   5 +-
 cpp/src/arrow/ipc/writer.cc                   |   8 +-
 cpp/src/arrow/loader.cc                       |  65 +++++-----
 cpp/src/arrow/table-test.cc                   |   1 +
 cpp/src/arrow/test-common.h                   |  84 +++++++++++++
 cpp/src/arrow/test-util.h                     |  45 +------
 cpp/src/arrow/type.cc                         |  36 +-----
 cpp/src/arrow/type.h                          |  33 +----
 cpp/src/arrow/type_fwd.h                      |   2 +
 cpp/src/arrow/visitor.cc                      |  96 +++++++++++++++
 cpp/src/arrow/visitor.h                       |  93 ++++++++++++++
 cpp/src/arrow/visitor_inline.h                |  67 +++++++++++
 24 files changed, 528 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index 61e645d..c04afe4 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -845,6 +845,7 @@ set(ARROW_SRCS
   src/arrow/status.cc
   src/arrow/table.cc
   src/arrow/type.cc
+  src/arrow/visitor.cc
 
   src/arrow/util/bit-util.cc
 )

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index 24a9547..0e83aac 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -34,6 +34,7 @@ install(FILES
   type_fwd.h
   type_traits.h
   test-util.h
+  visitor.h
   DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/arrow")
 
 # pkg-config support

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/api.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/api.h b/cpp/src/arrow/api.h
index 3bc8666..ea818b6 100644
--- a/cpp/src/arrow/api.h
+++ b/cpp/src/arrow/api.h
@@ -32,5 +32,6 @@
 #include "arrow/status.h"
 #include "arrow/table.h"
 #include "arrow/type.h"
+#include "arrow/visitor.h"
 
 #endif  // ARROW_API_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array-list-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-list-test.cc b/cpp/src/arrow/array-list-test.cc
index 87dfdaa..1cfa77f 100644
--- a/cpp/src/arrow/array-list-test.cc
+++ b/cpp/src/arrow/array-list-test.cc
@@ -26,6 +26,7 @@
 #include "arrow/array.h"
 #include "arrow/builder.h"
 #include "arrow/status.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array-primitive-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-primitive-test.cc b/cpp/src/arrow/array-primitive-test.cc
index dfa37a8..6863e58 100644
--- a/cpp/src/arrow/array-primitive-test.cc
+++ b/cpp/src/arrow/array-primitive-test.cc
@@ -26,6 +26,7 @@
 #include "arrow/buffer.h"
 #include "arrow/builder.h"
 #include "arrow/status.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array-string-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-string-test.cc b/cpp/src/arrow/array-string-test.cc
index ed38acd..6c2c151 100644
--- a/cpp/src/arrow/array-string-test.cc
+++ b/cpp/src/arrow/array-string-test.cc
@@ -25,6 +25,7 @@
 
 #include "arrow/array.h"
 #include "arrow/builder.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array-struct-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-struct-test.cc b/cpp/src/arrow/array-struct-test.cc
index f4e7409..4eb1eab 100644
--- a/cpp/src/arrow/array-struct-test.cc
+++ b/cpp/src/arrow/array-struct-test.cc
@@ -24,6 +24,7 @@
 #include "arrow/array.h"
 #include "arrow/builder.h"
 #include "arrow/status.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 4fa2b2b..20b732a 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -28,6 +28,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
 #include "arrow/util/logging.h"
+#include "arrow/visitor.h"
 
 namespace arrow {
 
@@ -468,41 +469,6 @@ Status DictionaryArray::Accept(ArrayVisitor* visitor) const {
   return visitor->Visit(*this);
 }
 
-#define ARRAY_VISITOR_DEFAULT(ARRAY_CLASS)                   \
-  Status ArrayVisitor::Visit(const ARRAY_CLASS& array) {     \
-    return Status::NotImplemented(array.type()->ToString()); \
-  }
-
-ARRAY_VISITOR_DEFAULT(NullArray);
-ARRAY_VISITOR_DEFAULT(BooleanArray);
-ARRAY_VISITOR_DEFAULT(Int8Array);
-ARRAY_VISITOR_DEFAULT(Int16Array);
-ARRAY_VISITOR_DEFAULT(Int32Array);
-ARRAY_VISITOR_DEFAULT(Int64Array);
-ARRAY_VISITOR_DEFAULT(UInt8Array);
-ARRAY_VISITOR_DEFAULT(UInt16Array);
-ARRAY_VISITOR_DEFAULT(UInt32Array);
-ARRAY_VISITOR_DEFAULT(UInt64Array);
-ARRAY_VISITOR_DEFAULT(HalfFloatArray);
-ARRAY_VISITOR_DEFAULT(FloatArray);
-ARRAY_VISITOR_DEFAULT(DoubleArray);
-ARRAY_VISITOR_DEFAULT(BinaryArray);
-ARRAY_VISITOR_DEFAULT(StringArray);
-ARRAY_VISITOR_DEFAULT(FixedWidthBinaryArray);
-ARRAY_VISITOR_DEFAULT(Date32Array);
-ARRAY_VISITOR_DEFAULT(Date64Array);
-ARRAY_VISITOR_DEFAULT(TimeArray);
-ARRAY_VISITOR_DEFAULT(TimestampArray);
-ARRAY_VISITOR_DEFAULT(IntervalArray);
-ARRAY_VISITOR_DEFAULT(ListArray);
-ARRAY_VISITOR_DEFAULT(StructArray);
-ARRAY_VISITOR_DEFAULT(UnionArray);
-ARRAY_VISITOR_DEFAULT(DictionaryArray);
-
-Status ArrayVisitor::Visit(const DecimalArray& array) {
-  return Status::NotImplemented("decimal");
-}
-
 // ----------------------------------------------------------------------
 // Instantiate templates
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index e66ac50..2a072db 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -31,6 +31,7 @@
 #include "arrow/util/bit-util.h"
 #include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
+#include "arrow/visitor.h"
 
 namespace arrow {
 
@@ -38,38 +39,6 @@ class MemoryPool;
 class MutableBuffer;
 class Status;
 
-class ARROW_EXPORT ArrayVisitor {
- public:
-  virtual ~ArrayVisitor() = default;
-
-  virtual Status Visit(const NullArray& array);
-  virtual Status Visit(const BooleanArray& array);
-  virtual Status Visit(const Int8Array& array);
-  virtual Status Visit(const Int16Array& array);
-  virtual Status Visit(const Int32Array& array);
-  virtual Status Visit(const Int64Array& array);
-  virtual Status Visit(const UInt8Array& array);
-  virtual Status Visit(const UInt16Array& array);
-  virtual Status Visit(const UInt32Array& array);
-  virtual Status Visit(const UInt64Array& array);
-  virtual Status Visit(const HalfFloatArray& array);
-  virtual Status Visit(const FloatArray& array);
-  virtual Status Visit(const DoubleArray& array);
-  virtual Status Visit(const StringArray& array);
-  virtual Status Visit(const BinaryArray& array);
-  virtual Status Visit(const FixedWidthBinaryArray& array);
-  virtual Status Visit(const Date32Array& array);
-  virtual Status Visit(const Date64Array& array);
-  virtual Status Visit(const TimeArray& array);
-  virtual Status Visit(const TimestampArray& array);
-  virtual Status Visit(const IntervalArray& array);
-  virtual Status Visit(const DecimalArray& array);
-  virtual Status Visit(const ListArray& array);
-  virtual Status Visit(const StructArray& array);
-  virtual Status Visit(const UnionArray& array);
-  virtual Status Visit(const DictionaryArray& type);
-};
-
 /// Immutable data array with some logical type and some length.
 ///
 /// Any memory is owned by the respective Buffer instance (or its parents).

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/column-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/column-test.cc b/cpp/src/arrow/column-test.cc
index 24d58c8..872fcb9 100644
--- a/cpp/src/arrow/column-test.cc
+++ b/cpp/src/arrow/column-test.cc
@@ -25,6 +25,7 @@
 #include "arrow/array.h"
 #include "arrow/column.h"
 #include "arrow/schema.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/ipc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/CMakeLists.txt b/cpp/src/arrow/ipc/CMakeLists.txt
index 629cc5b..056e7db 100644
--- a/cpp/src/arrow/ipc/CMakeLists.txt
+++ b/cpp/src/arrow/ipc/CMakeLists.txt
@@ -170,3 +170,7 @@ if (ARROW_BUILD_UTILITIES)
   add_executable(stream-to-file stream-to-file.cc)
   target_link_libraries(stream-to-file ${UTIL_LINK_LIBS})
 endif()
+
+ADD_ARROW_BENCHMARK(ipc-read-write-benchmark)
+ARROW_TEST_LINK_LIBRARIES(ipc-read-write-benchmark
+  ${ARROW_IPC_TEST_LINK_LIBS})

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
new file mode 100644
index 0000000..e27e513
--- /dev/null
+++ b/cpp/src/arrow/ipc/ipc-read-write-benchmark.cc
@@ -0,0 +1,134 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "benchmark/benchmark.h"
+
+#include <cstdint>
+#include <sstream>
+#include <string>
+
+#include "arrow/api.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/api.h"
+#include "arrow/test-util.h"
+
+namespace arrow {
+
+template <typename TYPE>
+std::shared_ptr<RecordBatch> MakeRecordBatch(int64_t total_size, int64_t num_fields) {
+  using T = typename TYPE::c_type;
+  size_t itemsize = sizeof(T);
+  int64_t length = total_size / num_fields / itemsize;
+
+  auto type = TypeTraits<TYPE>::type_singleton();
+
+  std::vector<bool> is_valid;
+  test::random_is_valid(length, 0.1, &is_valid);
+
+  std::vector<T> values;
+  test::randint<T>(length, 0, 100, &values);
+
+  MemoryPool* pool = default_memory_pool();
+  typename TypeTraits<TYPE>::BuilderType builder(pool, type);
+  for (size_t i = 0; i < values.size(); ++i) {
+    if (is_valid[i]) {
+      builder.Append(values[i]);
+    } else {
+      builder.AppendNull();
+    }
+  }
+  std::shared_ptr<Array> array;
+  builder.Finish(&array);
+
+  ArrayVector arrays;
+  std::vector<std::shared_ptr<Field>> fields;
+  for (int64_t i = 0; i < num_fields; ++i) {
+    std::stringstream ss;
+    ss << "f" << i;
+    fields.push_back(field(ss.str(), type));
+    arrays.push_back(array);
+  }
+
+  auto schema = std::make_shared<Schema>(fields);
+  return std::make_shared<RecordBatch>(schema, length, arrays);
+}
+
+static void BM_WriteRecordBatch(benchmark::State& state) {  // NOLINT non-const reference
+  // 1MB
+  constexpr int64_t kTotalSize = 1 << 20;
+
+  auto buffer = std::make_shared<PoolBuffer>(default_memory_pool());
+  buffer->Resize(kTotalSize & 2);
+  auto record_batch = MakeRecordBatch<Int64Type>(kTotalSize, state.range(0));
+
+  while (state.KeepRunning()) {
+    io::BufferOutputStream stream(buffer);
+    int32_t metadata_length;
+    int64_t body_length;
+    if (!ipc::WriteRecordBatch(*record_batch, 0, &stream, &metadata_length, &body_length,
+             default_memory_pool())
+             .ok()) {
+      state.SkipWithError("Failed to write!");
+    }
+  }
+  state.SetBytesProcessed(int64_t(state.iterations()) * kTotalSize);
+}
+
+static void BM_ReadRecordBatch(benchmark::State& state) {  // NOLINT non-const reference
+  // 1MB
+  constexpr int64_t kTotalSize = 1 << 20;
+
+  auto buffer = std::make_shared<PoolBuffer>(default_memory_pool());
+  buffer->Resize(kTotalSize & 2);
+  auto record_batch = MakeRecordBatch<Int64Type>(kTotalSize, state.range(0));
+
+  io::BufferOutputStream stream(buffer);
+
+  int32_t metadata_length;
+  int64_t body_length;
+  if (!ipc::WriteRecordBatch(*record_batch, 0, &stream, &metadata_length, &body_length,
+           default_memory_pool())
+           .ok()) {
+    state.SkipWithError("Failed to write!");
+  }
+
+  while (state.KeepRunning()) {
+    std::shared_ptr<RecordBatch> result;
+    io::BufferReader reader(buffer);
+
+    if (!ipc::ReadRecordBatch(record_batch->schema(), 0, &reader, &result).ok()) {
+      state.SkipWithError("Failed to read!");
+    }
+  }
+  state.SetBytesProcessed(int64_t(state.iterations()) * kTotalSize);
+}
+
+BENCHMARK(BM_WriteRecordBatch)
+    ->RangeMultiplier(4)
+    ->Range(1, 1 << 13)
+    ->MinTime(1.0)
+    ->Unit(benchmark::kMicrosecond)
+    ->UseRealTime();
+
+BENCHMARK(BM_ReadRecordBatch)
+    ->RangeMultiplier(4)
+    ->Range(1, 1 << 13)
+    ->MinTime(1.0)
+    ->Unit(benchmark::kMicrosecond)
+    ->UseRealTime();
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/ipc/reader.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/reader.h b/cpp/src/arrow/ipc/reader.h
index 1e8636c..ffd0a11 100644
--- a/cpp/src/arrow/ipc/reader.h
+++ b/cpp/src/arrow/ipc/reader.h
@@ -120,10 +120,9 @@ class ARROW_EXPORT FileReader {
   std::unique_ptr<FileReaderImpl> impl_;
 };
 
-
 /// Read encapsulated message and RecordBatch
-Status ARROW_EXPORT ReadRecordBatch(const std::shared_ptr<Schema>& schema,
-    int64_t offset, io::RandomAccessFile* file, std::shared_ptr<RecordBatch>* out);
+Status ARROW_EXPORT ReadRecordBatch(const std::shared_ptr<Schema>& schema, int64_t offset,
+    io::RandomAccessFile* file, std::shared_ptr<RecordBatch>* out);
 
 }  // namespace ipc
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/ipc/writer.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index 0f55f8e..dc991ab 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -47,9 +47,8 @@ namespace ipc {
 
 class RecordBatchWriter : public ArrayVisitor {
  public:
-  RecordBatchWriter(
-      MemoryPool* pool, int64_t buffer_start_offset, int max_recursion_depth,
-      bool allow_64bit)
+  RecordBatchWriter(MemoryPool* pool, int64_t buffer_start_offset,
+      int max_recursion_depth, bool allow_64bit)
       : pool_(pool),
         max_recursion_depth_(max_recursion_depth),
         buffer_start_offset_(buffer_start_offset),
@@ -501,8 +500,7 @@ class DictionaryWriter : public RecordBatchWriter {
 Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset,
     io::OutputStream* dst, int32_t* metadata_length, int64_t* body_length,
     MemoryPool* pool, int max_recursion_depth, bool allow_64bit) {
-  RecordBatchWriter writer(pool, buffer_start_offset, max_recursion_depth,
-      allow_64bit);
+  RecordBatchWriter writer(pool, buffer_start_offset, max_recursion_depth, allow_64bit);
   return writer.Write(batch, dst, metadata_length, body_length);
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/loader.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/loader.cc b/cpp/src/arrow/loader.cc
index bc506be..a67a3e9 100644
--- a/cpp/src/arrow/loader.cc
+++ b/cpp/src/arrow/loader.cc
@@ -28,6 +28,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/visibility.h"
+#include "arrow/visitor_inline.h"
 
 namespace arrow {
 
@@ -35,7 +36,7 @@ class Array;
 struct DataType;
 class Status;
 
-class ArrayLoader : public TypeVisitor {
+class ArrayLoader {
  public:
   ArrayLoader(const std::shared_ptr<DataType>& type, ArrayLoaderContext* context)
       : type_(type), context_(context) {}
@@ -45,8 +46,7 @@ class ArrayLoader : public TypeVisitor {
       return Status::Invalid("Max recursion depth reached");
     }
 
-    // Load the array
-    RETURN_NOT_OK(type_->Accept(this));
+    RETURN_NOT_OK(VisitTypeInline(*type_, this));
 
     *out = std::move(result_);
     return Status::OK();
@@ -92,8 +92,10 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-  template <typename CONTAINER>
+  template <typename TYPE>
   Status LoadBinary() {
+    using CONTAINER = typename TypeTraits<TYPE>::ArrayType;
+
     FieldMetadata field_meta;
     std::shared_ptr<Buffer> null_bitmap, offsets, values;
 
@@ -131,33 +133,24 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-#define VISIT_PRIMITIVE(TYPE) \
-  Status Visit(const TYPE& type) override { return LoadPrimitive<TYPE>(); }
-
-  VISIT_PRIMITIVE(BooleanType);
-  VISIT_PRIMITIVE(Int8Type);
-  VISIT_PRIMITIVE(Int16Type);
-  VISIT_PRIMITIVE(Int32Type);
-  VISIT_PRIMITIVE(Int64Type);
-  VISIT_PRIMITIVE(UInt8Type);
-  VISIT_PRIMITIVE(UInt16Type);
-  VISIT_PRIMITIVE(UInt32Type);
-  VISIT_PRIMITIVE(UInt64Type);
-  VISIT_PRIMITIVE(HalfFloatType);
-  VISIT_PRIMITIVE(FloatType);
-  VISIT_PRIMITIVE(DoubleType);
-  VISIT_PRIMITIVE(Date32Type);
-  VISIT_PRIMITIVE(Date64Type);
-  VISIT_PRIMITIVE(TimeType);
-  VISIT_PRIMITIVE(TimestampType);
-
-#undef VISIT_PRIMITIVE
-
-  Status Visit(const StringType& type) override { return LoadBinary<StringArray>(); }
-
-  Status Visit(const BinaryType& type) override { return LoadBinary<BinaryArray>(); }
-
-  Status Visit(const FixedWidthBinaryType& type) override {
+  Status Visit(const NullType& type) { return Status::NotImplemented("null"); }
+
+  template <typename T>
+  typename std::enable_if<std::is_base_of<FixedWidthType, T>::value &&
+                              !std::is_base_of<FixedWidthBinaryType, T>::value &&
+                              !std::is_base_of<DictionaryType, T>::value,
+      Status>::type
+  Visit(const T& type) {
+    return LoadPrimitive<T>();
+  }
+
+  template <typename T>
+  typename std::enable_if<std::is_base_of<BinaryType, T>::value, Status>::type Visit(
+      const T& type) {
+    return LoadBinary<T>();
+  }
+
+  Status Visit(const FixedWidthBinaryType& type) {
     FieldMetadata field_meta;
     std::shared_ptr<Buffer> null_bitmap, data;
 
@@ -169,7 +162,7 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-  Status Visit(const ListType& type) override {
+  Status Visit(const ListType& type) {
     FieldMetadata field_meta;
     std::shared_ptr<Buffer> null_bitmap, offsets;
 
@@ -196,7 +189,7 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-  Status Visit(const StructType& type) override {
+  Status Visit(const StructType& type) {
     FieldMetadata field_meta;
     std::shared_ptr<Buffer> null_bitmap;
     RETURN_NOT_OK(LoadCommon(&field_meta, &null_bitmap));
@@ -209,7 +202,7 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-  Status Visit(const UnionType& type) override {
+  Status Visit(const UnionType& type) {
     FieldMetadata field_meta;
     std::shared_ptr<Buffer> null_bitmap, type_ids, offsets;
 
@@ -230,12 +223,12 @@ class ArrayLoader : public TypeVisitor {
     return Status::OK();
   }
 
-  Status Visit(const DictionaryType& type) override {
+  Status Visit(const DictionaryType& type) {
     std::shared_ptr<Array> indices;
     RETURN_NOT_OK(LoadArray(type.index_type(), context_, &indices));
     result_ = std::make_shared<DictionaryArray>(type_, indices);
     return Status::OK();
-  };
+  }
 
   std::shared_ptr<Array> result() const { return result_; }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/table-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table-test.cc b/cpp/src/arrow/table-test.cc
index 3637473..6bb3163 100644
--- a/cpp/src/arrow/table-test.cc
+++ b/cpp/src/arrow/table-test.cc
@@ -26,6 +26,7 @@
 #include "arrow/schema.h"
 #include "arrow/status.h"
 #include "arrow/table.h"
+#include "arrow/test-common.h"
 #include "arrow/test-util.h"
 #include "arrow/type.h"
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/test-common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/test-common.h b/cpp/src/arrow/test-common.h
new file mode 100644
index 0000000..f704b6b
--- /dev/null
+++ b/cpp/src/arrow/test-common.h
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef ARROW_TEST_COMMON_H
+#define ARROW_TEST_COMMON_H
+
+#include <cstdint>
+#include <memory>
+#include <random>
+#include <string>
+#include <vector>
+
+#include "gtest/gtest.h"
+
+#include "arrow/array.h"
+#include "arrow/buffer.h"
+#include "arrow/builder.h"
+#include "arrow/column.h"
+#include "arrow/memory_pool.h"
+#include "arrow/table.h"
+#include "arrow/test-util.h"
+
+namespace arrow {
+
+class TestBase : public ::testing::Test {
+ public:
+  void SetUp() {
+    pool_ = default_memory_pool();
+    random_seed_ = 0;
+  }
+
+  template <typename ArrayType>
+  std::shared_ptr<Array> MakePrimitive(int64_t length, int64_t null_count = 0) {
+    auto data = std::make_shared<PoolBuffer>(pool_);
+    const int64_t data_nbytes = length * sizeof(typename ArrayType::value_type);
+    EXPECT_OK(data->Resize(data_nbytes));
+
+    // Fill with random data
+    test::random_bytes(data_nbytes, random_seed_++, data->mutable_data());
+
+    auto null_bitmap = std::make_shared<PoolBuffer>(pool_);
+    EXPECT_OK(null_bitmap->Resize(BitUtil::BytesForBits(length)));
+    return std::make_shared<ArrayType>(length, data, null_bitmap, null_count);
+  }
+
+ protected:
+  uint32_t random_seed_;
+  MemoryPool* pool_;
+};
+
+class TestBuilder : public ::testing::Test {
+ public:
+  void SetUp() {
+    pool_ = default_memory_pool();
+    type_ = TypePtr(new UInt8Type());
+    builder_.reset(new UInt8Builder(pool_));
+    builder_nn_.reset(new UInt8Builder(pool_));
+  }
+
+ protected:
+  MemoryPool* pool_;
+
+  TypePtr type_;
+  std::unique_ptr<ArrayBuilder> builder_;
+  std::unique_ptr<ArrayBuilder> builder_nn_;
+};
+
+}  // namespace arrow
+
+#endif  // ARROW_TEST_COMMON_H_

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/test-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/test-util.h b/cpp/src/arrow/test-util.h
index f05a541..bed5559 100644
--- a/cpp/src/arrow/test-util.h
+++ b/cpp/src/arrow/test-util.h
@@ -25,7 +25,7 @@
 #include <string>
 #include <vector>
 
-#include "gtest/gtest.h"
+#include <gtest/gtest.h>
 
 #include "arrow/array.h"
 #include "arrow/buffer.h"
@@ -208,32 +208,6 @@ Status MakeRandomBytePoolBuffer(int64_t length, MemoryPool* pool,
 
 }  // namespace test
 
-class TestBase : public ::testing::Test {
- public:
-  void SetUp() {
-    pool_ = default_memory_pool();
-    random_seed_ = 0;
-  }
-
-  template <typename ArrayType>
-  std::shared_ptr<Array> MakePrimitive(int64_t length, int64_t null_count = 0) {
-    auto data = std::make_shared<PoolBuffer>(pool_);
-    const int64_t data_nbytes = length * sizeof(typename ArrayType::value_type);
-    EXPECT_OK(data->Resize(data_nbytes));
-
-    // Fill with random data
-    test::random_bytes(data_nbytes, random_seed_++, data->mutable_data());
-
-    auto null_bitmap = std::make_shared<PoolBuffer>(pool_);
-    EXPECT_OK(null_bitmap->Resize(BitUtil::BytesForBits(length)));
-    return std::make_shared<ArrayType>(length, data, null_bitmap, null_count);
-  }
-
- protected:
-  uint32_t random_seed_;
-  MemoryPool* pool_;
-};
-
 template <typename TYPE, typename C_TYPE>
 void ArrayFromVector(const std::shared_ptr<DataType>& type,
     const std::vector<bool>& is_valid, const std::vector<C_TYPE>& values,
@@ -275,23 +249,6 @@ void ArrayFromVector(const std::vector<C_TYPE>& values, std::shared_ptr<Array>*
   ASSERT_OK(builder.Finish(out));
 }
 
-class TestBuilder : public ::testing::Test {
- public:
-  void SetUp() {
-    pool_ = default_memory_pool();
-    type_ = TypePtr(new UInt8Type());
-    builder_.reset(new UInt8Builder(pool_));
-    builder_nn_.reset(new UInt8Builder(pool_));
-  }
-
- protected:
-  MemoryPool* pool_;
-
-  TypePtr type_;
-  std::unique_ptr<ArrayBuilder> builder_;
-  std::unique_ptr<ArrayBuilder> builder_nn_;
-};
-
 template <class T, class Builder>
 Status MakeArray(const std::vector<uint8_t>& valid_bytes, const std::vector<T>& values,
     int64_t size, Builder* builder, std::shared_ptr<Array>* out) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 937cbc5..1c61eb6 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -24,6 +24,7 @@
 #include "arrow/compare.h"
 #include "arrow/status.h"
 #include "arrow/util/logging.h"
+#include "arrow/visitor.h"
 
 namespace arrow {
 
@@ -331,39 +332,4 @@ std::vector<BufferDescr> DecimalType::GetBufferLayout() const {
   return {};
 }
 
-// ----------------------------------------------------------------------
-// Default implementations of TypeVisitor methods
-
-#define TYPE_VISITOR_DEFAULT(TYPE_CLASS)              \
-  Status TypeVisitor::Visit(const TYPE_CLASS& type) { \
-    return Status::NotImplemented(type.ToString());   \
-  }
-
-TYPE_VISITOR_DEFAULT(NullType);
-TYPE_VISITOR_DEFAULT(BooleanType);
-TYPE_VISITOR_DEFAULT(Int8Type);
-TYPE_VISITOR_DEFAULT(Int16Type);
-TYPE_VISITOR_DEFAULT(Int32Type);
-TYPE_VISITOR_DEFAULT(Int64Type);
-TYPE_VISITOR_DEFAULT(UInt8Type);
-TYPE_VISITOR_DEFAULT(UInt16Type);
-TYPE_VISITOR_DEFAULT(UInt32Type);
-TYPE_VISITOR_DEFAULT(UInt64Type);
-TYPE_VISITOR_DEFAULT(HalfFloatType);
-TYPE_VISITOR_DEFAULT(FloatType);
-TYPE_VISITOR_DEFAULT(DoubleType);
-TYPE_VISITOR_DEFAULT(StringType);
-TYPE_VISITOR_DEFAULT(BinaryType);
-TYPE_VISITOR_DEFAULT(FixedWidthBinaryType);
-TYPE_VISITOR_DEFAULT(Date64Type);
-TYPE_VISITOR_DEFAULT(Date32Type);
-TYPE_VISITOR_DEFAULT(TimeType);
-TYPE_VISITOR_DEFAULT(TimestampType);
-TYPE_VISITOR_DEFAULT(IntervalType);
-TYPE_VISITOR_DEFAULT(DecimalType);
-TYPE_VISITOR_DEFAULT(ListType);
-TYPE_VISITOR_DEFAULT(StructType);
-TYPE_VISITOR_DEFAULT(UnionType);
-TYPE_VISITOR_DEFAULT(DictionaryType);
-
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index c179bf3..40c00a4 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -28,6 +28,7 @@
 #include "arrow/type_fwd.h"
 #include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
+#include "arrow/visitor.h"
 
 namespace arrow {
 
@@ -119,38 +120,6 @@ class BufferDescr {
   int bit_width_;
 };
 
-class ARROW_EXPORT TypeVisitor {
- public:
-  virtual ~TypeVisitor() = default;
-
-  virtual Status Visit(const NullType& type);
-  virtual Status Visit(const BooleanType& type);
-  virtual Status Visit(const Int8Type& type);
-  virtual Status Visit(const Int16Type& type);
-  virtual Status Visit(const Int32Type& type);
-  virtual Status Visit(const Int64Type& type);
-  virtual Status Visit(const UInt8Type& type);
-  virtual Status Visit(const UInt16Type& type);
-  virtual Status Visit(const UInt32Type& type);
-  virtual Status Visit(const UInt64Type& type);
-  virtual Status Visit(const HalfFloatType& type);
-  virtual Status Visit(const FloatType& type);
-  virtual Status Visit(const DoubleType& type);
-  virtual Status Visit(const StringType& type);
-  virtual Status Visit(const BinaryType& type);
-  virtual Status Visit(const FixedWidthBinaryType& type);
-  virtual Status Visit(const Date64Type& type);
-  virtual Status Visit(const Date32Type& type);
-  virtual Status Visit(const TimeType& type);
-  virtual Status Visit(const TimestampType& type);
-  virtual Status Visit(const IntervalType& type);
-  virtual Status Visit(const DecimalType& type);
-  virtual Status Visit(const ListType& type);
-  virtual Status Visit(const StructType& type);
-  virtual Status Visit(const UnionType& type);
-  virtual Status Visit(const DictionaryType& type);
-};
-
 struct ARROW_EXPORT DataType {
   Type::type type;
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/type_fwd.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_fwd.h b/cpp/src/arrow/type_fwd.h
index ae85593..f62c031 100644
--- a/cpp/src/arrow/type_fwd.h
+++ b/cpp/src/arrow/type_fwd.h
@@ -18,6 +18,8 @@
 #ifndef ARROW_TYPE_FWD_H
 #define ARROW_TYPE_FWD_H
 
+#include <memory>
+
 #include "arrow/util/visibility.h"
 
 namespace arrow {

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/visitor.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/visitor.cc b/cpp/src/arrow/visitor.cc
new file mode 100644
index 0000000..181e932
--- /dev/null
+++ b/cpp/src/arrow/visitor.cc
@@ -0,0 +1,96 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/visitor.h"
+
+#include "arrow/array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+
+namespace arrow {
+
+#define ARRAY_VISITOR_DEFAULT(ARRAY_CLASS)                   \
+  Status ArrayVisitor::Visit(const ARRAY_CLASS& array) {     \
+    return Status::NotImplemented(array.type()->ToString()); \
+  }
+
+ARRAY_VISITOR_DEFAULT(NullArray);
+ARRAY_VISITOR_DEFAULT(BooleanArray);
+ARRAY_VISITOR_DEFAULT(Int8Array);
+ARRAY_VISITOR_DEFAULT(Int16Array);
+ARRAY_VISITOR_DEFAULT(Int32Array);
+ARRAY_VISITOR_DEFAULT(Int64Array);
+ARRAY_VISITOR_DEFAULT(UInt8Array);
+ARRAY_VISITOR_DEFAULT(UInt16Array);
+ARRAY_VISITOR_DEFAULT(UInt32Array);
+ARRAY_VISITOR_DEFAULT(UInt64Array);
+ARRAY_VISITOR_DEFAULT(HalfFloatArray);
+ARRAY_VISITOR_DEFAULT(FloatArray);
+ARRAY_VISITOR_DEFAULT(DoubleArray);
+ARRAY_VISITOR_DEFAULT(BinaryArray);
+ARRAY_VISITOR_DEFAULT(StringArray);
+ARRAY_VISITOR_DEFAULT(FixedWidthBinaryArray);
+ARRAY_VISITOR_DEFAULT(Date32Array);
+ARRAY_VISITOR_DEFAULT(Date64Array);
+ARRAY_VISITOR_DEFAULT(TimeArray);
+ARRAY_VISITOR_DEFAULT(TimestampArray);
+ARRAY_VISITOR_DEFAULT(IntervalArray);
+ARRAY_VISITOR_DEFAULT(ListArray);
+ARRAY_VISITOR_DEFAULT(StructArray);
+ARRAY_VISITOR_DEFAULT(UnionArray);
+ARRAY_VISITOR_DEFAULT(DictionaryArray);
+
+Status ArrayVisitor::Visit(const DecimalArray& array) {
+  return Status::NotImplemented("decimal");
+}
+
+// ----------------------------------------------------------------------
+// Default implementations of TypeVisitor methods
+
+#define TYPE_VISITOR_DEFAULT(TYPE_CLASS)              \
+  Status TypeVisitor::Visit(const TYPE_CLASS& type) { \
+    return Status::NotImplemented(type.ToString());   \
+  }
+
+TYPE_VISITOR_DEFAULT(NullType);
+TYPE_VISITOR_DEFAULT(BooleanType);
+TYPE_VISITOR_DEFAULT(Int8Type);
+TYPE_VISITOR_DEFAULT(Int16Type);
+TYPE_VISITOR_DEFAULT(Int32Type);
+TYPE_VISITOR_DEFAULT(Int64Type);
+TYPE_VISITOR_DEFAULT(UInt8Type);
+TYPE_VISITOR_DEFAULT(UInt16Type);
+TYPE_VISITOR_DEFAULT(UInt32Type);
+TYPE_VISITOR_DEFAULT(UInt64Type);
+TYPE_VISITOR_DEFAULT(HalfFloatType);
+TYPE_VISITOR_DEFAULT(FloatType);
+TYPE_VISITOR_DEFAULT(DoubleType);
+TYPE_VISITOR_DEFAULT(StringType);
+TYPE_VISITOR_DEFAULT(BinaryType);
+TYPE_VISITOR_DEFAULT(FixedWidthBinaryType);
+TYPE_VISITOR_DEFAULT(Date64Type);
+TYPE_VISITOR_DEFAULT(Date32Type);
+TYPE_VISITOR_DEFAULT(TimeType);
+TYPE_VISITOR_DEFAULT(TimestampType);
+TYPE_VISITOR_DEFAULT(IntervalType);
+TYPE_VISITOR_DEFAULT(DecimalType);
+TYPE_VISITOR_DEFAULT(ListType);
+TYPE_VISITOR_DEFAULT(StructType);
+TYPE_VISITOR_DEFAULT(UnionType);
+TYPE_VISITOR_DEFAULT(DictionaryType);
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/visitor.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/visitor.h b/cpp/src/arrow/visitor.h
new file mode 100644
index 0000000..a9c59c8
--- /dev/null
+++ b/cpp/src/arrow/visitor.h
@@ -0,0 +1,93 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef ARROW_VISITOR_H
+#define ARROW_VISITOR_H
+
+#include "arrow/status.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class ARROW_EXPORT ArrayVisitor {
+ public:
+  virtual ~ArrayVisitor() = default;
+
+  virtual Status Visit(const NullArray& array);
+  virtual Status Visit(const BooleanArray& array);
+  virtual Status Visit(const Int8Array& array);
+  virtual Status Visit(const Int16Array& array);
+  virtual Status Visit(const Int32Array& array);
+  virtual Status Visit(const Int64Array& array);
+  virtual Status Visit(const UInt8Array& array);
+  virtual Status Visit(const UInt16Array& array);
+  virtual Status Visit(const UInt32Array& array);
+  virtual Status Visit(const UInt64Array& array);
+  virtual Status Visit(const HalfFloatArray& array);
+  virtual Status Visit(const FloatArray& array);
+  virtual Status Visit(const DoubleArray& array);
+  virtual Status Visit(const StringArray& array);
+  virtual Status Visit(const BinaryArray& array);
+  virtual Status Visit(const FixedWidthBinaryArray& array);
+  virtual Status Visit(const Date32Array& array);
+  virtual Status Visit(const Date64Array& array);
+  virtual Status Visit(const TimeArray& array);
+  virtual Status Visit(const TimestampArray& array);
+  virtual Status Visit(const IntervalArray& array);
+  virtual Status Visit(const DecimalArray& array);
+  virtual Status Visit(const ListArray& array);
+  virtual Status Visit(const StructArray& array);
+  virtual Status Visit(const UnionArray& array);
+  virtual Status Visit(const DictionaryArray& type);
+};
+
+class ARROW_EXPORT TypeVisitor {
+ public:
+  virtual ~TypeVisitor() = default;
+
+  virtual Status Visit(const NullType& type);
+  virtual Status Visit(const BooleanType& type);
+  virtual Status Visit(const Int8Type& type);
+  virtual Status Visit(const Int16Type& type);
+  virtual Status Visit(const Int32Type& type);
+  virtual Status Visit(const Int64Type& type);
+  virtual Status Visit(const UInt8Type& type);
+  virtual Status Visit(const UInt16Type& type);
+  virtual Status Visit(const UInt32Type& type);
+  virtual Status Visit(const UInt64Type& type);
+  virtual Status Visit(const HalfFloatType& type);
+  virtual Status Visit(const FloatType& type);
+  virtual Status Visit(const DoubleType& type);
+  virtual Status Visit(const StringType& type);
+  virtual Status Visit(const BinaryType& type);
+  virtual Status Visit(const FixedWidthBinaryType& type);
+  virtual Status Visit(const Date64Type& type);
+  virtual Status Visit(const Date32Type& type);
+  virtual Status Visit(const TimeType& type);
+  virtual Status Visit(const TimestampType& type);
+  virtual Status Visit(const IntervalType& type);
+  virtual Status Visit(const DecimalType& type);
+  virtual Status Visit(const ListType& type);
+  virtual Status Visit(const StructType& type);
+  virtual Status Visit(const UnionType& type);
+  virtual Status Visit(const DictionaryType& type);
+};
+
+}  // namespace arrow
+
+#endif  // ARROW_VISITOR_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/e968ca6e/cpp/src/arrow/visitor_inline.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/visitor_inline.h b/cpp/src/arrow/visitor_inline.h
new file mode 100644
index 0000000..b69468d
--- /dev/null
+++ b/cpp/src/arrow/visitor_inline.h
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// Private header, not to be exported
+
+#ifndef ARROW_VISITOR_INLINE_H
+#define ARROW_VISITOR_INLINE_H
+
+#include "arrow/array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+
+namespace arrow {
+
+#define TYPE_VISIT_INLINE(TYPE_CLASS) \
+  case TYPE_CLASS::type_id:           \
+    return visitor->Visit(static_cast<const TYPE_CLASS&>(type));
+
+template <typename VISITOR>
+inline Status VisitTypeInline(const DataType& type, VISITOR* visitor) {
+  switch (type.type) {
+    TYPE_VISIT_INLINE(NullType);
+    TYPE_VISIT_INLINE(BooleanType);
+    TYPE_VISIT_INLINE(Int8Type);
+    TYPE_VISIT_INLINE(UInt8Type);
+    TYPE_VISIT_INLINE(Int16Type);
+    TYPE_VISIT_INLINE(UInt16Type);
+    TYPE_VISIT_INLINE(Int32Type);
+    TYPE_VISIT_INLINE(UInt32Type);
+    TYPE_VISIT_INLINE(Int64Type);
+    TYPE_VISIT_INLINE(UInt64Type);
+    TYPE_VISIT_INLINE(FloatType);
+    TYPE_VISIT_INLINE(DoubleType);
+    TYPE_VISIT_INLINE(StringType);
+    TYPE_VISIT_INLINE(BinaryType);
+    TYPE_VISIT_INLINE(FixedWidthBinaryType);
+    TYPE_VISIT_INLINE(Date32Type);
+    TYPE_VISIT_INLINE(Date64Type);
+    TYPE_VISIT_INLINE(TimestampType);
+    TYPE_VISIT_INLINE(TimeType);
+    TYPE_VISIT_INLINE(ListType);
+    TYPE_VISIT_INLINE(StructType);
+    TYPE_VISIT_INLINE(UnionType);
+    TYPE_VISIT_INLINE(DictionaryType);
+    default:
+      break;
+  }
+  return Status::NotImplemented("Type not implemented");
+}
+
+}  // namespace arrow
+
+#endif  // ARROW_VISITOR_INLINE_H