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 2016/12/10 00:50:26 UTC

arrow git commit: ARROW-394: [Integration] Generate tests cases for numeric types, strings, lists, structs

Repository: arrow
Updated Branches:
  refs/heads/master 8995c9230 -> 45ed7e7a3


ARROW-394: [Integration] Generate tests cases for numeric types, strings, lists, structs

Automatically generating testing files from Python.

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

Closes #219 from wesm/ARROW-394 and squashes the following commits:

7807f48 [Wes McKinney] OS X doesn't have std::fabs
c0c804c [Wes McKinney] abs -> fabs
8cd1902 [Wes McKinney] Fix compiler warning in OS X from incorrect type declaration
d51581a [Wes McKinney] Add missing apache license
527622d [Wes McKinney] ARROW-414: remove check for maximum buffer padding
2a7b0fc [Wes McKinney] Add JSON generation code to fuzz test numeric types, print integers more nicely. Add integration tests to Travis CI build matrix. Add ApproxEquals method for floating point comparisons. Add boolean, string, struct, list to generated json test case


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

Branch: refs/heads/master
Commit: 45ed7e7a36fb2a69de468c41132b6b3bbd270c92
Parents: 8995c92
Author: Wes McKinney <we...@twosigma.com>
Authored: Fri Dec 9 19:50:19 2016 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Fri Dec 9 19:50:19 2016 -0500

----------------------------------------------------------------------
 .travis.yml                                     |  10 +
 ci/travis_script_integration.sh                 |  49 ++
 cpp/src/arrow/array.cc                          |   4 +
 cpp/src/arrow/array.h                           |   1 +
 cpp/src/arrow/ipc/ipc-metadata-test.cc          |   4 +-
 cpp/src/arrow/ipc/json-integration-test.cc      |  27 +-
 cpp/src/arrow/ipc/json-internal.cc              |   8 +-
 cpp/src/arrow/ipc/metadata-internal.cc          |   2 +-
 cpp/src/arrow/pretty_print-test.cc              |  10 +-
 cpp/src/arrow/pretty_print.cc                   |  90 +++-
 cpp/src/arrow/pretty_print.h                    |   8 +-
 cpp/src/arrow/table.cc                          |  12 +
 cpp/src/arrow/table.h                           |   2 +
 cpp/src/arrow/type.cc                           |   2 +-
 cpp/src/arrow/type_traits.h                     |   6 +
 cpp/src/arrow/types/primitive.cc                |   1 +
 cpp/src/arrow/types/primitive.h                 |  91 +++-
 integration/README.md                           |  59 +++
 integration/integration_test.py                 | 508 ++++++++++++++++++-
 .../arrow/vector/BaseDataValueVector.java       |   6 -
 20 files changed, 844 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index bfc2f26..1634eba 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -46,6 +46,16 @@ matrix:
     jdk: oraclejdk7
     script:
     - $TRAVIS_BUILD_DIR/ci/travis_script_java.sh
+  - language: java
+    os: linux
+    env: ARROW_TEST_GROUP=integration
+    jdk: oraclejdk7
+    before_script:
+    - export CC="gcc-4.9"
+    - export CXX="g++-4.9"
+    - $TRAVIS_BUILD_DIR/ci/travis_before_script_cpp.sh
+    script:
+    - $TRAVIS_BUILD_DIR/ci/travis_script_integration.sh
 
 before_install:
 - ulimit -c unlimited -S

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/ci/travis_script_integration.sh
----------------------------------------------------------------------
diff --git a/ci/travis_script_integration.sh b/ci/travis_script_integration.sh
new file mode 100755
index 0000000..d93411b
--- /dev/null
+++ b/ci/travis_script_integration.sh
@@ -0,0 +1,49 @@
+#!/usr/bin/env bash
+
+#  Licensed 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. See accompanying LICENSE file.
+
+set -e
+
+: ${CPP_BUILD_DIR=$TRAVIS_BUILD_DIR/cpp-build}
+
+JAVA_DIR=${TRAVIS_BUILD_DIR}/java
+
+pushd $JAVA_DIR
+
+mvn package
+
+popd
+
+pushd $TRAVIS_BUILD_DIR/integration
+
+VERSION=0.1.1-SNAPSHOT
+export ARROW_JAVA_INTEGRATION_JAR=$JAVA_DIR/tools/target/arrow-tools-$VERSION-jar-with-dependencies.jar
+export ARROW_CPP_TESTER=$CPP_BUILD_DIR/debug/json-integration-test
+
+source $TRAVIS_BUILD_DIR/ci/travis_install_conda.sh
+export MINICONDA=$HOME/miniconda
+export PATH="$MINICONDA/bin:$PATH"
+
+CONDA_ENV_NAME=arrow-integration-test
+conda create -y -q -n $CONDA_ENV_NAME python=3.5
+source activate $CONDA_ENV_NAME
+
+# faster builds, please
+conda install -y nomkl
+
+# Expensive dependencies install from Continuum package repo
+conda install -y pip numpy six
+
+python integration_test.py --debug
+
+popd

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/array.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 3262425..1f0bb66 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -60,6 +60,10 @@ bool Array::EqualsExact(const Array& other) const {
   return true;
 }
 
+bool Array::ApproxEquals(const std::shared_ptr<Array>& arr) const {
+  return Equals(arr);
+}
+
 Status Array::Validate() const {
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index ff2b70e..78aa2b8 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -62,6 +62,7 @@ class ARROW_EXPORT Array {
 
   bool EqualsExact(const Array& arr) const;
   virtual bool Equals(const std::shared_ptr<Array>& arr) const = 0;
+  virtual bool ApproxEquals(const std::shared_ptr<Array>& arr) const;
 
   // Compare if the range of slots specified are equal for the given array and
   // this array.  end_idx exclusive.  This methods does not bounds check.

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/ipc/ipc-metadata-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-metadata-test.cc b/cpp/src/arrow/ipc/ipc-metadata-test.cc
index d29583f..de08e6d 100644
--- a/cpp/src/arrow/ipc/ipc-metadata-test.cc
+++ b/cpp/src/arrow/ipc/ipc-metadata-test.cc
@@ -70,7 +70,7 @@ const std::shared_ptr<DataType> INT32 = std::make_shared<Int32Type>();
 
 TEST_F(TestSchemaMetadata, PrimitiveFields) {
   auto f0 = std::make_shared<Field>("f0", std::make_shared<Int8Type>());
-  auto f1 = std::make_shared<Field>("f1", std::make_shared<Int16Type>());
+  auto f1 = std::make_shared<Field>("f1", std::make_shared<Int16Type>(), false);
   auto f2 = std::make_shared<Field>("f2", std::make_shared<Int32Type>());
   auto f3 = std::make_shared<Field>("f3", std::make_shared<Int64Type>());
   auto f4 = std::make_shared<Field>("f4", std::make_shared<UInt8Type>());
@@ -78,7 +78,7 @@ TEST_F(TestSchemaMetadata, PrimitiveFields) {
   auto f6 = std::make_shared<Field>("f6", std::make_shared<UInt32Type>());
   auto f7 = std::make_shared<Field>("f7", std::make_shared<UInt64Type>());
   auto f8 = std::make_shared<Field>("f8", std::make_shared<FloatType>());
-  auto f9 = std::make_shared<Field>("f9", std::make_shared<DoubleType>());
+  auto f9 = std::make_shared<Field>("f9", std::make_shared<DoubleType>(), false);
   auto f10 = std::make_shared<Field>("f10", std::make_shared<BooleanType>());
 
   Schema schema({f0, f1, f2, f3, f4, f5, f6, f7, f8, f9, f10});

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/ipc/json-integration-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/json-integration-test.cc b/cpp/src/arrow/ipc/json-integration-test.cc
index c4e6847..291a719 100644
--- a/cpp/src/arrow/ipc/json-integration-test.cc
+++ b/cpp/src/arrow/ipc/json-integration-test.cc
@@ -169,15 +169,15 @@ static Status ValidateArrowVsJson(
     RETURN_NOT_OK(json_reader->GetRecordBatch(i, &json_batch));
     RETURN_NOT_OK(arrow_reader->GetRecordBatch(i, &arrow_batch));
 
-    if (!json_batch->Equals(*arrow_batch.get())) {
+    if (!json_batch->ApproxEquals(*arrow_batch.get())) {
       std::stringstream ss;
       ss << "Record batch " << i << " did not match";
 
-      ss << "\nJSON: \n ";
-      RETURN_NOT_OK(PrettyPrint(*json_batch.get(), &ss));
+      ss << "\nJSON:\n";
+      RETURN_NOT_OK(PrettyPrint(*json_batch.get(), 0, &ss));
 
-      ss << "\nArrow: \n ";
-      RETURN_NOT_OK(PrettyPrint(*arrow_batch.get(), &ss));
+      ss << "\nArrow:\n";
+      RETURN_NOT_OK(PrettyPrint(*arrow_batch.get(), 0, &ss));
       return Status::Invalid(ss.str());
     }
   }
@@ -299,6 +299,23 @@ static const char* JSON_EXAMPLE = R"example(
           "VALIDITY": [1, 0, 0, 1, 1]
         }
       ]
+    },
+    {
+      "count": 4,
+      "columns": [
+        {
+          "name": "foo",
+          "count": 4,
+          "DATA": [1, 2, 3, 4],
+          "VALIDITY": [1, 0, 1, 1]
+        },
+        {
+          "name": "bar",
+          "count": 4,
+          "DATA": [1.0, 2.0, 3.0, 4.0],
+          "VALIDITY": [1, 0, 0, 1]
+        }
+      ]
     }
   ]
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/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 50f5b0c..ff9f598 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -418,7 +418,7 @@ class JsonArrayWriter : public ArrayVisitor {
 
   template <typename T>
   void WriteOffsetsField(const T* offsets, int32_t length) {
-    writer_->Key("OFFSETS");
+    writer_->Key("OFFSET");
     writer_->StartArray();
     for (int i = 0; i < length; ++i) {
       writer_->Int64(offsets[i]);
@@ -810,7 +810,7 @@ class JsonArrayReader {
         builder.Append(val.GetUint64());
       } else if (IsFloatingPoint<T>::value) {
         DCHECK(val.IsFloat());
-        builder.Append(val.GetFloat());
+        builder.Append(val.GetDouble());
       } else if (std::is_base_of<BooleanType, T>::value) {
         DCHECK(val.IsBool());
         builder.Append(val.GetBool());
@@ -853,8 +853,8 @@ class JsonArrayReader {
   typename std::enable_if<std::is_base_of<ListType, T>::value, Status>::type ReadArray(
       const RjObject& json_array, int32_t length, const std::vector<bool>& is_valid,
       const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* array) {
-    const auto& json_offsets = json_array.FindMember("OFFSETS");
-    RETURN_NOT_ARRAY("OFFSETS", json_offsets, json_array);
+    const auto& json_offsets = json_array.FindMember("OFFSET");
+    RETURN_NOT_ARRAY("OFFSET", json_offsets, json_array);
     const auto& json_offsets_arr = json_offsets->value.GetArray();
 
     int32_t null_count = 0;

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/ipc/metadata-internal.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/metadata-internal.cc b/cpp/src/arrow/ipc/metadata-internal.cc
index b995228..7a24161 100644
--- a/cpp/src/arrow/ipc/metadata-internal.cc
+++ b/cpp/src/arrow/ipc/metadata-internal.cc
@@ -264,7 +264,7 @@ Status FieldFromFlatbuffer(const flatbuf::Field* field, std::shared_ptr<Field>*
   RETURN_NOT_OK(
       TypeFromFlatbuffer(field->type_type(), field->type(), child_fields, &type));
 
-  *out = std::make_shared<Field>(field->name()->str(), type);
+  *out = std::make_shared<Field>(field->name()->str(), type, field->nullable());
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/pretty_print-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print-test.cc b/cpp/src/arrow/pretty_print-test.cc
index 10af41d..b1e6a11 100644
--- a/cpp/src/arrow/pretty_print-test.cc
+++ b/cpp/src/arrow/pretty_print-test.cc
@@ -48,8 +48,8 @@ class TestArrayPrinter : public ::testing::Test {
 };
 
 template <typename TYPE, typename C_TYPE>
-void CheckPrimitive(const std::vector<bool>& is_valid, const std::vector<C_TYPE>& values,
-    const char* expected) {
+void CheckPrimitive(int indent, const std::vector<bool>& is_valid,
+    const std::vector<C_TYPE>& values, const char* expected) {
   std::ostringstream sink;
 
   MemoryPool* pool = default_memory_pool();
@@ -66,7 +66,7 @@ void CheckPrimitive(const std::vector<bool>& is_valid, const std::vector<C_TYPE>
   std::shared_ptr<Array> array;
   ASSERT_OK(builder.Finish(&array));
 
-  ASSERT_OK(PrettyPrint(*array.get(), &sink));
+  ASSERT_OK(PrettyPrint(*array.get(), indent, &sink));
 
   std::string result = sink.str();
   ASSERT_EQ(std::string(expected, strlen(expected)), result);
@@ -77,11 +77,11 @@ TEST_F(TestArrayPrinter, PrimitiveType) {
 
   std::vector<int32_t> values = {0, 1, 2, 3, 4};
   static const char* expected = R"expected([0, 1, null, 3, null])expected";
-  CheckPrimitive<Int32Type, int32_t>(is_valid, values, expected);
+  CheckPrimitive<Int32Type, int32_t>(0, is_valid, values, expected);
 
   std::vector<std::string> values2 = {"foo", "bar", "", "baz", ""};
   static const char* ex2 = R"expected(["foo", "bar", null, "baz", null])expected";
-  CheckPrimitive<StringType, std::string>(is_valid, values2, ex2);
+  CheckPrimitive<StringType, std::string>(0, is_valid, values2, ex2);
 }
 
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/pretty_print.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
index c0b4b08..c63a9e9 100644
--- a/cpp/src/arrow/pretty_print.cc
+++ b/cpp/src/arrow/pretty_print.cc
@@ -16,7 +16,9 @@
 // under the License.
 
 #include <ostream>
+#include <sstream>
 #include <string>
+#include <vector>
 
 #include "arrow/array.h"
 #include "arrow/pretty_print.h"
@@ -32,14 +34,15 @@ namespace arrow {
 
 class ArrayPrinter : public ArrayVisitor {
  public:
-  ArrayPrinter(const Array& array, std::ostream* sink) : array_(array), sink_(sink) {}
+  ArrayPrinter(const Array& array, int indent, std::ostream* sink)
+      : array_(array), indent_(indent), sink_(sink) {}
 
   Status Print() { return VisitArray(array_); }
 
   Status VisitArray(const Array& array) { return array.Accept(this); }
 
   template <typename T>
-  typename std::enable_if<IsNumeric<T>::value, void>::type WriteDataValues(
+  typename std::enable_if<IsInteger<T>::value, void>::type WriteDataValues(
       const T& array) {
     const auto data = array.raw_data();
     for (int i = 0; i < array.length(); ++i) {
@@ -47,6 +50,20 @@ class ArrayPrinter : public ArrayVisitor {
       if (array.IsNull(i)) {
         (*sink_) << "null";
       } else {
+        (*sink_) << static_cast<int64_t>(data[i]);
+      }
+    }
+  }
+
+  template <typename T>
+  typename std::enable_if<IsFloatingPoint<T>::value, void>::type WriteDataValues(
+      const T& array) {
+    const auto data = array.raw_data();
+    for (int i = 0; i < array.length(); ++i) {
+      if (i > 0) { (*sink_) << ", "; }
+      if (array.IsNull(i)) {
+        Write("null");
+      } else {
         (*sink_) << data[i];
       }
     }
@@ -60,7 +77,7 @@ class ArrayPrinter : public ArrayVisitor {
     for (int i = 0; i < array.length(); ++i) {
       if (i > 0) { (*sink_) << ", "; }
       if (array.IsNull(i)) {
-        (*sink_) << "null";
+        Write("null");
       } else {
         const char* buf = reinterpret_cast<const char*>(array.GetValue(i, &length));
         (*sink_) << "\"" << std::string(buf, length) << "\"";
@@ -74,9 +91,9 @@ class ArrayPrinter : public ArrayVisitor {
     for (int i = 0; i < array.length(); ++i) {
       if (i > 0) { (*sink_) << ", "; }
       if (array.IsNull(i)) {
-        (*sink_) << "null";
+        Write("null");
       } else {
-        (*sink_) << (array.Value(i) ? "true" : "false");
+        Write(array.Value(i) ? "true" : "false");
       }
     }
   }
@@ -148,20 +165,38 @@ class ArrayPrinter : public ArrayVisitor {
   }
 
   Status Visit(const ListArray& array) override {
-    // auto type = static_cast<const ListType*>(array.type().get());
-    // for (size_t i = 0; i < fields.size(); ++i) {
-    //   RETURN_NOT_OK(VisitArray(fields[i]->name, *arrays[i].get()));
-    // }
-    // return WriteChildren(type->children(), {array.values()});
+    Newline();
+    Write("-- is_valid: ");
+    BooleanArray is_valid(array.length(), array.null_bitmap());
+    PrettyPrint(is_valid, indent_ + 2, sink_);
+
+    Newline();
+    Write("-- offsets: ");
+    Int32Array offsets(array.length() + 1, array.offsets());
+    PrettyPrint(offsets, indent_ + 2, sink_);
+
+    Newline();
+    Write("-- values: ");
+    PrettyPrint(*array.values().get(), indent_ + 2, sink_);
+
     return Status::OK();
   }
 
   Status Visit(const StructArray& array) override {
-    // auto type = static_cast<const StructType*>(array.type().get());
-    // for (size_t i = 0; i < fields.size(); ++i) {
-    //   RETURN_NOT_OK(VisitArray(fields[i]->name, *arrays[i].get()));
-    // }
-    // return WriteChildren(type->children(), array.fields());
+    Newline();
+    Write("-- is_valid: ");
+    BooleanArray is_valid(array.length(), array.null_bitmap());
+    PrettyPrint(is_valid, indent_ + 2, sink_);
+
+    const std::vector<std::shared_ptr<Array>>& fields = array.fields();
+    for (size_t i = 0; i < fields.size(); ++i) {
+      Newline();
+      std::stringstream ss;
+      ss << "-- child " << i << " type: " << fields[i]->type()->ToString() << " values: ";
+      Write(ss.str());
+      PrettyPrint(*fields[i].get(), indent_ + 2, sink_);
+    }
+
     return Status::OK();
   }
 
@@ -169,21 +204,38 @@ class ArrayPrinter : public ArrayVisitor {
     return Status::NotImplemented("union");
   }
 
+  void Write(const char* data) { (*sink_) << data; }
+
+  void Write(const std::string& data) { (*sink_) << data; }
+
+  void Newline() {
+    (*sink_) << "\n";
+    Indent();
+  }
+
+  void Indent() {
+    for (int i = 0; i < indent_; ++i) {
+      (*sink_) << " ";
+    }
+  }
+
  private:
   const Array& array_;
+  int indent_;
+
   std::ostream* sink_;
 };
 
-Status PrettyPrint(const Array& arr, std::ostream* sink) {
-  ArrayPrinter printer(arr, sink);
+Status PrettyPrint(const Array& arr, int indent, std::ostream* sink) {
+  ArrayPrinter printer(arr, indent, sink);
   return printer.Print();
 }
 
-Status PrettyPrint(const RecordBatch& batch, std::ostream* sink) {
+Status PrettyPrint(const RecordBatch& batch, int indent, std::ostream* sink) {
   for (int i = 0; i < batch.num_columns(); ++i) {
     const std::string& name = batch.column_name(i);
     (*sink) << name << ": ";
-    RETURN_NOT_OK(PrettyPrint(*batch.column(i).get(), sink));
+    RETURN_NOT_OK(PrettyPrint(*batch.column(i).get(), indent + 2, sink));
     (*sink) << "\n";
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/pretty_print.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.h b/cpp/src/arrow/pretty_print.h
index dcb236d..f508aa0 100644
--- a/cpp/src/arrow/pretty_print.h
+++ b/cpp/src/arrow/pretty_print.h
@@ -27,8 +27,12 @@ namespace arrow {
 
 class Status;
 
-Status ARROW_EXPORT PrettyPrint(const RecordBatch& batch, std::ostream* sink);
-Status ARROW_EXPORT PrettyPrint(const Array& arr, std::ostream* sink);
+struct PrettyPrintOptions {
+  int indent;
+};
+
+Status ARROW_EXPORT PrettyPrint(const RecordBatch& batch, int indent, std::ostream* sink);
+Status ARROW_EXPORT PrettyPrint(const Array& arr, int indent, std::ostream* sink);
 
 }  // namespace arrow
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/table.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.cc b/cpp/src/arrow/table.cc
index af84f27..eb1258a 100644
--- a/cpp/src/arrow/table.cc
+++ b/cpp/src/arrow/table.cc
@@ -48,6 +48,18 @@ bool RecordBatch::Equals(const RecordBatch& other) const {
   return true;
 }
 
+bool RecordBatch::ApproxEquals(const RecordBatch& other) const {
+  if (num_columns() != other.num_columns() || num_rows_ != other.num_rows()) {
+    return false;
+  }
+
+  for (int i = 0; i < num_columns(); ++i) {
+    if (!column(i)->ApproxEquals(other.column(i))) { return false; }
+  }
+
+  return true;
+}
+
 // ----------------------------------------------------------------------
 // Table methods
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/table.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index 1a856c8..f2c334f 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -45,6 +45,8 @@ class ARROW_EXPORT RecordBatch {
 
   bool Equals(const RecordBatch& other) const;
 
+  bool ApproxEquals(const RecordBatch& other) const;
+
   // @returns: the table's schema
   const std::shared_ptr<Schema>& schema() const { return schema_; }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 80f295c..dc955ac 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -209,7 +209,7 @@ std::vector<BufferDescr> ListType::GetBufferLayout() const {
 }
 
 std::vector<BufferDescr> StructType::GetBufferLayout() const {
-  return {kValidityBuffer, kTypeBuffer};
+  return {kValidityBuffer};
 }
 
 std::vector<BufferDescr> UnionType::GetBufferLayout() const {

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/type_traits.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_traits.h b/cpp/src/arrow/type_traits.h
index c21c500..3aaec0b 100644
--- a/cpp/src/arrow/type_traits.h
+++ b/cpp/src/arrow/type_traits.h
@@ -187,6 +187,12 @@ struct IsSignedInt {
 };
 
 template <typename T>
+struct IsInteger {
+  PRIMITIVE_TRAITS(T);
+  static constexpr bool value = std::is_integral<c_type>::value;
+};
+
+template <typename T>
 struct IsFloatingPoint {
   PRIMITIVE_TRAITS(T);
   static constexpr bool value = std::is_floating_point<c_type>::value;

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/types/primitive.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/primitive.cc b/cpp/src/arrow/types/primitive.cc
index f42a3ca..75e5a9f 100644
--- a/cpp/src/arrow/types/primitive.cc
+++ b/cpp/src/arrow/types/primitive.cc
@@ -17,6 +17,7 @@
 
 #include "arrow/types/primitive.h"
 
+#include <cmath>
 #include <memory>
 
 #include "arrow/type_traits.h"

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/cpp/src/arrow/types/primitive.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/primitive.h b/cpp/src/arrow/types/primitive.h
index a5a3704..c665218 100644
--- a/cpp/src/arrow/types/primitive.h
+++ b/cpp/src/arrow/types/primitive.h
@@ -18,8 +18,10 @@
 #ifndef ARROW_TYPES_PRIMITIVE_H
 #define ARROW_TYPES_PRIMITIVE_H
 
+#include <cmath>
 #include <cstdint>
 #include <cstring>
+#include <iostream>
 #include <memory>
 #include <vector>
 
@@ -55,7 +57,7 @@ class ARROW_EXPORT PrimitiveArray : public Array {
   const uint8_t* raw_data_;
 };
 
-template <class TYPE>
+template <typename TYPE>
 class ARROW_EXPORT NumericArray : public PrimitiveArray {
  public:
   using TypeClass = TYPE;
@@ -69,9 +71,11 @@ class ARROW_EXPORT NumericArray : public PrimitiveArray {
       : PrimitiveArray(type, length, data, null_count, null_bitmap) {}
 
   bool EqualsExact(const NumericArray<TypeClass>& other) const {
-    return PrimitiveArray::EqualsExact(*static_cast<const PrimitiveArray*>(&other));
+    return PrimitiveArray::EqualsExact(static_cast<const PrimitiveArray&>(other));
   }
 
+  bool ApproxEquals(const std::shared_ptr<Array>& arr) const { return Equals(arr); }
+
   bool RangeEquals(int32_t start_idx, int32_t end_idx, int32_t other_start_idx,
       const ArrayPtr& arr) const override {
     if (this == arr.get()) { return true; }
@@ -95,6 +99,78 @@ class ARROW_EXPORT NumericArray : public PrimitiveArray {
   value_type Value(int i) const { return raw_data()[i]; }
 };
 
+template <>
+inline bool NumericArray<FloatType>::ApproxEquals(
+    const std::shared_ptr<Array>& arr) const {
+  if (this == arr.get()) { return true; }
+  if (!arr) { return false; }
+  if (this->type_enum() != arr->type_enum()) { return false; }
+
+  const auto& other = *static_cast<NumericArray<FloatType>*>(arr.get());
+
+  if (this == &other) { return true; }
+  if (null_count_ != other.null_count_) { return false; }
+
+  auto this_data = reinterpret_cast<const float*>(raw_data_);
+  auto other_data = reinterpret_cast<const float*>(other.raw_data_);
+
+  static constexpr float EPSILON = 1E-5;
+
+  if (length_ == 0 && other.length_ == 0) { return true; }
+
+  if (null_count_ > 0) {
+    bool equal_bitmap =
+        null_bitmap_->Equals(*other.null_bitmap_, BitUtil::CeilByte(length_) / 8);
+    if (!equal_bitmap) { return false; }
+
+    for (int i = 0; i < length_; ++i) {
+      if (IsNull(i)) continue;
+      if (fabs(this_data[i] - other_data[i]) > EPSILON) { return false; }
+    }
+  } else {
+    for (int i = 0; i < length_; ++i) {
+      if (fabs(this_data[i] - other_data[i]) > EPSILON) { return false; }
+    }
+  }
+  return true;
+}
+
+template <>
+inline bool NumericArray<DoubleType>::ApproxEquals(
+    const std::shared_ptr<Array>& arr) const {
+  if (this == arr.get()) { return true; }
+  if (!arr) { return false; }
+  if (this->type_enum() != arr->type_enum()) { return false; }
+
+  const auto& other = *static_cast<NumericArray<DoubleType>*>(arr.get());
+
+  if (this == &other) { return true; }
+  if (null_count_ != other.null_count_) { return false; }
+
+  auto this_data = reinterpret_cast<const double*>(raw_data_);
+  auto other_data = reinterpret_cast<const double*>(other.raw_data_);
+
+  if (length_ == 0 && other.length_ == 0) { return true; }
+
+  static constexpr double EPSILON = 1E-5;
+
+  if (null_count_ > 0) {
+    bool equal_bitmap =
+        null_bitmap_->Equals(*other.null_bitmap_, BitUtil::CeilByte(length_) / 8);
+    if (!equal_bitmap) { return false; }
+
+    for (int i = 0; i < length_; ++i) {
+      if (IsNull(i)) continue;
+      if (fabs(this_data[i] - other_data[i]) > EPSILON) { return false; }
+    }
+  } else {
+    for (int i = 0; i < length_; ++i) {
+      if (fabs(this_data[i] - other_data[i]) > EPSILON) { return false; }
+    }
+  }
+  return true;
+}
+
 template <typename Type>
 class ARROW_EXPORT PrimitiveBuilder : public ArrayBuilder {
  public:
@@ -265,6 +341,13 @@ class ARROW_EXPORT BooleanBuilder : public ArrayBuilder {
   uint8_t* raw_data_;
 };
 
+// gcc and clang disagree about how to handle template visibility when you have
+// explicit specializations https://llvm.org/bugs/show_bug.cgi?id=24815
+#if defined(__GNUC__) && !defined(__clang__)
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wattributes"
+#endif
+
 // Only instantiate these templates once
 extern template class ARROW_EXPORT NumericArray<Int8Type>;
 extern template class ARROW_EXPORT NumericArray<UInt8Type>;
@@ -279,6 +362,10 @@ extern template class ARROW_EXPORT NumericArray<FloatType>;
 extern template class ARROW_EXPORT NumericArray<DoubleType>;
 extern template class ARROW_EXPORT NumericArray<TimestampType>;
 
+#if defined(__GNUC__) && !defined(__clang__)
+#pragma GCC diagnostic pop
+#endif
+
 }  // namespace arrow
 
 #endif  // ARROW_TYPES_PRIMITIVE_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/integration/README.md
----------------------------------------------------------------------
diff --git a/integration/README.md b/integration/README.md
new file mode 100644
index 0000000..b1e4e3a
--- /dev/null
+++ b/integration/README.md
@@ -0,0 +1,59 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# Arrow integration testing
+
+Our strategy for integration testing between Arrow implementations is as follows:
+
+* Test datasets are specified in a custom human-readable, JSON-based format
+  designed for Arrow
+
+* Each implementation provides a testing executable capable of converting
+  between the JSON and the binary Arrow file representation
+
+* The test executable is also capable of validating the contents of a binary
+  file against a corresponding JSON file
+
+## Running the existing integration tests
+
+First, build the Java and C++ projects. For Java, you must run
+
+```
+mvn package
+```
+
+Now, the integration tests rely on two environment variables which point to the
+Java `arrow-tool` JAR and the C++ `json-integration-test` executable:
+
+```bash
+JAVA_DIR=$ARROW_HOME/java
+CPP_BUILD_DIR=$ARROW_HOME/cpp/test-build
+
+VERSION=0.1.1-SNAPSHOT
+export ARROW_JAVA_INTEGRATION_JAR=$JAVA_DIR/tools/target/arrow-tools-$VERSION-jar-with-dependencies.jar
+export ARROW_CPP_TESTER=$CPP_BUILD_DIR/debug/json-integration-test
+```
+
+Here `$ARROW_HOME` is the location of your Arrow git clone. The
+`$CPP_BUILD_DIR` may be different depending on how you built with CMake
+(in-source of out-of-source).
+
+Once this is done, run the integration tests with (optionally adding `--debug`
+for additional output)
+
+```
+python integration_test.py
+
+python integration_test.py --debug  # additional output
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/integration/integration_test.py
----------------------------------------------------------------------
diff --git a/integration/integration_test.py b/integration/integration_test.py
index 88dc3ad..417354b 100644
--- a/integration/integration_test.py
+++ b/integration/integration_test.py
@@ -15,23 +15,53 @@
 # specific language governing permissions and limitations
 # under the License.
 
+from collections import OrderedDict
 import argparse
 import glob
 import itertools
+import json
 import os
 import six
+import string
 import subprocess
 import tempfile
 import uuid
 
+import numpy as np
 
 ARROW_HOME = os.path.abspath(__file__).rsplit("/", 2)[0]
 
+# Control for flakiness
+np.random.seed(12345)
+
 
 def guid():
     return uuid.uuid4().hex
 
 
+# from pandas
+RANDS_CHARS = np.array(list(string.ascii_letters + string.digits),
+                       dtype=(np.str_, 1))
+
+
+def rands(nchars):
+    """
+    Generate one random byte string.
+
+    See `rands_array` if you want to create an array of random strings.
+
+    """
+    return ''.join(np.random.choice(RANDS_CHARS, nchars))
+
+
+def str_from_bytes(x):
+    if six.PY2:
+        return x
+    else:
+        return x.decode('utf-8')
+
+
+# from the merge_arrow_pr.py script
 def run_cmd(cmd):
     if isinstance(cmd, six.string_types):
         cmd = cmd.split(' ')
@@ -43,13 +73,469 @@ def run_cmd(cmd):
         print('Command failed: %s' % ' '.join(cmd))
         print('With output:')
         print('--------------')
-        print(e.output)
+        print(str_from_bytes(e.output))
         print('--------------')
         raise e
 
-    if isinstance(output, six.binary_type):
-        output = output.decode('utf-8')
-    return output
+    return str_from_bytes(output)
+
+# ----------------------------------------------------------------------
+# Data generation
+
+
+class DataType(object):
+
+    def __init__(self, name, nullable=True):
+        self.name = name
+        self.nullable = nullable
+
+    def get_json(self):
+        return OrderedDict([
+            ('name', self.name),
+            ('type', self._get_type()),
+            ('nullable', self.nullable),
+            ('children', self._get_children()),
+            ('typeLayout', self._get_type_layout())
+        ])
+
+    def _make_is_valid(self, size):
+        if self.nullable:
+            return np.random.randint(0, 2, size=size)
+        else:
+            return np.ones(size)
+
+
+class Column(object):
+
+    def __init__(self, name, count):
+        self.name = name
+        self.count = count
+
+    def _get_children(self):
+        return []
+
+    def _get_buffers(self):
+        return []
+
+    def get_json(self):
+        entries = [
+            ('name', self.name),
+            ('count', self.count)
+        ]
+
+        buffers = self._get_buffers()
+        entries.extend(buffers)
+
+        children = self._get_children()
+        if len(children) > 0:
+            entries.append(('children', children))
+
+        return OrderedDict(entries)
+
+
+class PrimitiveType(DataType):
+
+    def _get_children(self):
+        return []
+
+    def _get_type_layout(self):
+        return OrderedDict([
+            ('vectors',
+             [OrderedDict([('type', 'VALIDITY'),
+                           ('typeBitWidth', 1)]),
+              OrderedDict([('type', 'DATA'),
+                           ('typeBitWidth', self.bit_width)])])])
+
+
+class PrimitiveColumn(Column):
+
+    def __init__(self, name, count, is_valid, values):
+        Column.__init__(self, name, count)
+        self.is_valid = is_valid
+        self.values = values
+
+    def _get_buffers(self):
+        return [
+            ('VALIDITY', [int(v) for v in self.is_valid]),
+            ('DATA', list(self.values))
+        ]
+
+
+TEST_INT_MIN = - 2**31 + 1
+TEST_INT_MAX = 2**31 - 1
+
+
+class IntegerType(PrimitiveType):
+
+    def __init__(self, name, is_signed, bit_width, nullable=True):
+        PrimitiveType.__init__(self, name, nullable=nullable)
+        self.is_signed = is_signed
+        self.bit_width = bit_width
+
+    @property
+    def numpy_type(self):
+        return ('int' if self.is_signed else 'uint') + str(self.bit_width)
+
+    def _get_type(self):
+        return OrderedDict([
+            ('name', 'int'),
+            ('isSigned', self.is_signed),
+            ('bitWidth', self.bit_width)
+        ])
+
+    def generate_column(self, size):
+        iinfo = np.iinfo(self.numpy_type)
+        values = [int(x) for x in
+                  np.random.randint(max(iinfo.min, TEST_INT_MIN),
+                                    min(iinfo.max, TEST_INT_MAX),
+                                    size=size)]
+
+        is_valid = self._make_is_valid(size)
+        return PrimitiveColumn(self.name, size, is_valid, values)
+
+
+class FloatingPointType(PrimitiveType):
+
+    def __init__(self, name, bit_width, nullable=True):
+        PrimitiveType.__init__(self, name, nullable=nullable)
+
+        self.bit_width = bit_width
+        self.precision = {
+            16: 'HALF',
+            32: 'SINGLE',
+            64: 'DOUBLE'
+        }[self.bit_width]
+
+    @property
+    def numpy_type(self):
+        return 'float' + str(self.bit_width)
+
+    def _get_type(self):
+        return OrderedDict([
+            ('name', 'floatingpoint'),
+            ('precision', self.precision)
+        ])
+
+    def generate_column(self, size):
+        values = np.random.randn(size) * 1000
+        values = np.round(values, 3)
+
+        is_valid = self._make_is_valid(size)
+        return PrimitiveColumn(self.name, size, is_valid, values)
+
+
+class BooleanType(PrimitiveType):
+
+    bit_width = 1
+
+    def _get_type(self):
+        return OrderedDict([('name', 'bool')])
+
+    @property
+    def numpy_type(self):
+        return 'bool'
+
+    def generate_column(self, size):
+        values = list(map(bool, np.random.randint(0, 2, size=size)))
+        is_valid = self._make_is_valid(size)
+        return PrimitiveColumn(self.name, size, is_valid, values)
+
+
+class StringType(PrimitiveType):
+
+    @property
+    def numpy_type(self):
+        return object
+
+    def _get_type(self):
+        return OrderedDict([('name', 'utf8')])
+
+    def _get_type_layout(self):
+        return OrderedDict([
+            ('vectors',
+             [OrderedDict([('type', 'VALIDITY'),
+                           ('typeBitWidth', 1)]),
+              OrderedDict([('type', 'OFFSET'),
+                           ('typeBitWidth', 32)]),
+              OrderedDict([('type', 'DATA'),
+                           ('typeBitWidth', 8)])])])
+
+    def generate_column(self, size):
+        K = 7
+        is_valid = self._make_is_valid(size)
+        values = []
+
+        for i in range(size):
+            if is_valid[i]:
+                values.append(rands(K))
+            else:
+                values.append("")
+
+        return StringColumn(self.name, size, is_valid, values)
+
+
+class JSONSchema(object):
+
+    def __init__(self, fields):
+        self.fields = fields
+
+    def get_json(self):
+        return OrderedDict([
+            ('fields', [field.get_json() for field in self.fields])
+        ])
+
+
+class StringColumn(PrimitiveColumn):
+
+    def _get_buffers(self):
+        offset = 0
+        offsets = [0]
+
+        data = []
+        for i, v in enumerate(self.values):
+            if self.is_valid[i]:
+                offset += len(v)
+            else:
+                v = ""
+
+            offsets.append(offset)
+            data.append(v)
+
+        return [
+            ('VALIDITY', [int(x) for x in self.is_valid]),
+            ('OFFSET', offsets),
+            ('DATA', data)
+        ]
+
+
+class ListType(DataType):
+
+    def __init__(self, name, value_type, nullable=True):
+        DataType.__init__(self, name, nullable=nullable)
+        self.value_type = value_type
+
+    def _get_type(self):
+        return OrderedDict([
+            ('name', 'list')
+        ])
+
+    def _get_children(self):
+        return [self.value_type.get_json()]
+
+    def _get_type_layout(self):
+        return OrderedDict([
+            ('vectors',
+             [OrderedDict([('type', 'VALIDITY'),
+                           ('typeBitWidth', 1)]),
+              OrderedDict([('type', 'OFFSET'),
+                           ('typeBitWidth', 32)])])])
+
+    def generate_column(self, size):
+        MAX_LIST_SIZE = 4
+
+        is_valid = self._make_is_valid(size)
+        list_sizes = np.random.randint(0, MAX_LIST_SIZE + 1, size=size)
+        offsets = [0]
+
+        offset = 0
+        for i in range(size):
+            if is_valid[i]:
+                offset += int(list_sizes[i])
+            offsets.append(offset)
+
+        # The offset now is the total number of elements in the child array
+        values = self.value_type.generate_column(offset)
+
+        return ListColumn(self.name, size, is_valid, offsets, values)
+
+
+class ListColumn(Column):
+
+    def __init__(self, name, count, is_valid, offsets, values):
+        Column.__init__(self, name, count)
+        self.is_valid = is_valid
+        self.offsets = offsets
+        self.values = values
+
+    def _get_buffers(self):
+        return [
+            ('VALIDITY', [int(v) for v in self.is_valid]),
+            ('OFFSET', list(self.offsets))
+        ]
+
+    def _get_children(self):
+        return [self.values.get_json()]
+
+
+class StructType(DataType):
+
+    def __init__(self, name, field_types, nullable=True):
+        DataType.__init__(self, name, nullable=nullable)
+        self.field_types = field_types
+
+    def _get_type(self):
+        return OrderedDict([
+            ('name', 'struct')
+        ])
+
+    def _get_children(self):
+        return [type_.get_json() for type_ in self.field_types]
+
+    def _get_type_layout(self):
+        return OrderedDict([
+            ('vectors',
+             [OrderedDict([('type', 'VALIDITY'),
+                           ('typeBitWidth', 1)])])])
+
+    def generate_column(self, size):
+        is_valid = self._make_is_valid(size)
+
+        field_values = [type_.generate_column(size)
+                        for type_ in self.field_types]
+
+        return StructColumn(self.name, size, is_valid, field_values)
+
+
+class StructColumn(Column):
+
+    def __init__(self, name, count, is_valid, field_values):
+        Column.__init__(self, name, count)
+        self.is_valid = is_valid
+        self.field_values = field_values
+
+    def _get_buffers(self):
+        return [
+            ('VALIDITY', [int(v) for v in self.is_valid])
+        ]
+
+    def _get_children(self):
+        return [field.get_json() for field in self.field_values]
+
+
+class JSONRecordBatch(object):
+
+    def __init__(self, count, columns):
+        self.count = count
+        self.columns = columns
+
+    def get_json(self):
+        return OrderedDict([
+            ('count', self.count),
+            ('columns', [col.get_json() for col in self.columns])
+        ])
+
+
+class JSONFile(object):
+
+    def __init__(self, schema, batches):
+        self.schema = schema
+        self.batches = batches
+
+    def get_json(self):
+        return OrderedDict([
+            ('schema', self.schema.get_json()),
+            ('batches', [batch.get_json() for batch in self.batches])
+        ])
+
+    def write(self, path):
+        with open(path, 'wb') as f:
+            f.write(json.dumps(self.get_json(), indent=2).encode('utf-8'))
+
+
+def get_field(name, type_, nullable=True):
+    if type_ == 'utf8':
+        return StringType(name, nullable=nullable)
+
+    dtype = np.dtype(type_)
+
+    if dtype.kind in ('i', 'u'):
+        return IntegerType(name, dtype.kind == 'i', dtype.itemsize * 8,
+                           nullable=nullable)
+    elif dtype.kind == 'f':
+        return FloatingPointType(name, dtype.itemsize * 8,
+                                 nullable=nullable)
+    elif dtype.kind == 'b':
+        return BooleanType(name, nullable=nullable)
+    else:
+        raise TypeError(dtype)
+
+
+def generate_primitive_case():
+    types = ['bool', 'int8', 'int16', 'int32', 'int64',
+             'uint8', 'uint16', 'uint32', 'uint64',
+             'float32', 'float64', 'utf8']
+
+    fields = []
+
+    for type_ in types:
+        fields.append(get_field(type_ + "_nullable", type_, True))
+        fields.append(get_field(type_ + "_nonnullable", type_, False))
+
+    schema = JSONSchema(fields)
+
+    batch_sizes = [7, 10]
+    batches = []
+    for size in batch_sizes:
+        columns = []
+        for field in fields:
+            col = field.generate_column(size)
+            columns.append(col)
+
+        batches.append(JSONRecordBatch(size, columns))
+
+    return JSONFile(schema, batches)
+
+
+def generate_nested_case():
+    fields = [
+        ListType('list_nullable', get_field('item', 'int32')),
+        StructType('struct_nullable', [get_field('f1', 'int32'),
+                                       get_field('f2', 'utf8')]),
+
+        # TODO(wesm): this causes segfault
+        # ListType('list_nonnullable', get_field('item', 'int32'), False),
+    ]
+
+    schema = JSONSchema(fields)
+
+    batch_sizes = [7, 10]
+    batches = []
+    for size in batch_sizes:
+        columns = []
+        for field in fields:
+            col = field.generate_column(size)
+            columns.append(col)
+
+        batches.append(JSONRecordBatch(size, columns))
+
+    return JSONFile(schema, batches)
+
+
+def get_generated_json_files():
+    temp_dir = tempfile.mkdtemp()
+
+    def _temp_path():
+        return
+
+    file_objs = []
+
+    K = 10
+    for i in range(K):
+        file_objs.append(generate_primitive_case())
+
+    file_objs.append(generate_nested_case())
+
+    generated_paths = []
+    for file_obj in file_objs:
+        out_path = os.path.join(temp_dir, guid() + '.json')
+        file_obj.write(out_path)
+        generated_paths.append(out_path)
+
+    return generated_paths
+
+
+# ----------------------------------------------------------------------
+# Testing harness
 
 
 class IntegrationRunner(object):
@@ -92,9 +578,11 @@ class Tester(object):
 
 class JavaTester(Tester):
 
-    ARROW_TOOLS_JAR = os.path.join(ARROW_HOME,
-                                   'java/tools/target/arrow-tools-0.1.1-'
-                                   'SNAPSHOT-jar-with-dependencies.jar')
+    ARROW_TOOLS_JAR = os.environ.get(
+        'ARROW_JAVA_INTEGRATION_JAR',
+        os.path.join(ARROW_HOME,
+                     'java/tools/target/arrow-tools-0.1.1-'
+                     'SNAPSHOT-jar-with-dependencies.jar'))
 
     name = 'Java'
 
@@ -154,14 +642,16 @@ class CPPTester(Tester):
         return self._run(arrow_path, json_path, 'JSON_TO_ARROW')
 
 
-def get_json_files():
+def get_static_json_files():
     glob_pattern = os.path.join(ARROW_HOME, 'integration', 'data', '*.json')
     return glob.glob(glob_pattern)
 
 
 def run_all_tests(debug=False):
     testers = [JavaTester(debug=debug), CPPTester(debug=debug)]
-    json_files = get_json_files()
+    static_json_files = get_static_json_files()
+    generated_json_files = get_generated_json_files()
+    json_files = static_json_files + generated_json_files
 
     runner = IntegrationRunner(json_files, testers, debug=debug)
     runner.run()

http://git-wip-us.apache.org/repos/asf/arrow/blob/45ed7e7a/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java b/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
index b7df8d1..7fe1615 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/BaseDataValueVector.java
@@ -30,9 +30,6 @@ public abstract class BaseDataValueVector extends BaseValueVector implements Buf
 
   protected final static byte[] emptyByteArray = new byte[]{}; // Nullable vectors use this
 
-  /** maximum extra size at the end of the buffer */
-  private static final int MAX_BUFFER_PADDING = 64;
-
   public static void load(ArrowFieldNode fieldNode, List<BufferBacked> vectors, List<ArrowBuf> buffers) {
     int expectedSize = vectors.size();
     if (buffers.size() != expectedSize) {
@@ -51,9 +48,6 @@ public abstract class BaseDataValueVector extends BaseValueVector implements Buf
     if (buffer.writerIndex() < byteSize) {
       throw new IllegalArgumentException("can not truncate buffer to a larger size " + byteSize + ": " + buffer.writerIndex());
     }
-    if (buffer.writerIndex() - byteSize > MAX_BUFFER_PADDING) {
-      throw new IllegalArgumentException("Buffer too large to resize to " + byteSize + ": " + buffer.writerIndex());
-    }
     buffer.writerIndex(byteSize);
   }