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/11/21 22:48:19 UTC

arrow git commit: ARROW-383: [C++] Integration testing CLI tool

Repository: arrow
Updated Branches:
  refs/heads/master 997f502ce -> f082b1732


ARROW-383: [C++] Integration testing CLI tool

Modeled after Java version in ARROW-367

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

Closes #209 from wesm/ARROW-383 and squashes the following commits:

7b29b24 [Wes McKinney] Use git master gflags to avoid valgrind error
f563d3a [Wes McKinney] Build integration test as a normal unit test executable, opt-in to integration testing
bdf1f7a [Wes McKinney] Call the RunCommand method instead
dbc6aab [Wes McKinney] Add unit tests for the integration validator
ca1eade [Wes McKinney] Clean up includes
1752249 [Wes McKinney] Draft integration testing CLI tool, untested
b773d0d [Wes McKinney] Add gflags external project and json-integration-test executable stub


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

Branch: refs/heads/master
Commit: f082b17323354dc2af31f39c15c58b995ba08360
Parents: 997f502
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Nov 21 17:48:09 2016 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Nov 21 17:48:09 2016 -0500

----------------------------------------------------------------------
 cpp/CMakeLists.txt                         |  44 +++
 cpp/cmake_modules/FindGFlags.cmake         |  60 ++++
 cpp/src/arrow/io/file.cc                   |   7 +-
 cpp/src/arrow/ipc/CMakeLists.txt           |  27 ++
 cpp/src/arrow/ipc/adapter.h                |   2 +-
 cpp/src/arrow/ipc/json-integration-test.cc | 381 ++++++++++++++++++++++++
 6 files changed, 517 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index 0bff752..839ea17 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -26,6 +26,7 @@ include(ExternalProject)
 set(BUILD_SUPPORT_DIR "${CMAKE_SOURCE_DIR}/build-support")
 set(THIRDPARTY_DIR "${CMAKE_SOURCE_DIR}/thirdparty")
 
+set(GFLAGS_VERSION "2.1.2")
 set(GTEST_VERSION "1.7.0")
 set(GBENCHMARK_VERSION "1.0.0")
 set(FLATBUFFERS_VERSION "1.3.0")
@@ -506,6 +507,49 @@ if(ARROW_BUILD_TESTS)
   if(GTEST_VENDORED)
     add_dependencies(gtest googletest_ep)
   endif()
+
+  # gflags (formerly Googleflags) command line parsing
+  if("$ENV{GFLAGS_HOME}" STREQUAL "")
+    if(APPLE)
+      set(GFLAGS_CMAKE_CXX_FLAGS "-fPIC -std=c++11 -stdlib=libc++")
+    else()
+      set(GFLAGS_CMAKE_CXX_FLAGS "-fPIC")
+    endif()
+
+    set(GFLAGS_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/gflags_ep-prefix/src/gflags_ep")
+    ExternalProject_Add(gflags_ep
+      GIT_REPOSITORY https://github.com/gflags/gflags.git
+      GIT_TAG cce68f0c9c5d054017425e6e6fd54f696d36e8ee
+      # URL "https://github.com/gflags/gflags/archive/v${GFLAGS_VERSION}.tar.gz"
+      BUILD_IN_SOURCE 1
+      CMAKE_ARGS -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE}
+                 -DCMAKE_INSTALL_PREFIX=${GFLAGS_PREFIX}
+                 -DBUILD_SHARED_LIBS=OFF
+                 -DBUILD_STATIC_LIBS=ON
+                 -DBUILD_PACKAGING=OFF
+                 -DBUILD_TESTING=OFF
+                 -BUILD_CONFIG_TESTS=OFF
+                 -DINSTALL_HEADERS=ON
+                 -DCMAKE_CXX_FLAGS=${GFLAGS_CMAKE_CXX_FLAGS})
+
+    set(GFLAGS_HOME "${GFLAGS_PREFIX}")
+    set(GFLAGS_INCLUDE_DIR "${GFLAGS_PREFIX}/include")
+    set(GFLAGS_STATIC_LIB "${GFLAGS_PREFIX}/lib/libgflags.a")
+    set(GFLAGS_VENDORED 1)
+  else()
+    set(GFLAGS_VENDORED 0)
+    find_package(GFlags REQUIRED)
+  endif()
+
+  message(STATUS "GFlags include dir: ${GFLAGS_INCLUDE_DIR}")
+  message(STATUS "GFlags static library: ${GFLAGS_STATIC_LIB}")
+  include_directories(SYSTEM ${GFLAGS_INCLUDE_DIR})
+  ADD_THIRDPARTY_LIB(gflags
+    STATIC_LIB ${GFLAGS_STATIC_LIB})
+
+  if(GFLAGS_VENDORED)
+    add_dependencies(gflags gflags_ep)
+  endif()
 endif()
 
 if(ARROW_BUILD_BENCHMARKS)

http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/cpp/cmake_modules/FindGFlags.cmake
----------------------------------------------------------------------
diff --git a/cpp/cmake_modules/FindGFlags.cmake b/cpp/cmake_modules/FindGFlags.cmake
new file mode 100644
index 0000000..eaea835
--- /dev/null
+++ b/cpp/cmake_modules/FindGFlags.cmake
@@ -0,0 +1,60 @@
+# 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.
+
+# - Find GFLAGS (gflags.h, libgflags.a, libgflags.so, and libgflags.so.0)
+# This module defines
+#  GFLAGS_INCLUDE_DIR, directory containing headers
+#  GFLAGS_SHARED_LIB, path to libgflags shared library
+#  GFLAGS_STATIC_LIB, path to libgflags static library
+#  GFLAGS_FOUND, whether gflags has been found
+
+if( NOT "$ENV{GFLAGS_HOME}" STREQUAL "")
+    file( TO_CMAKE_PATH "$ENV{GFLAGS_HOME}" _native_path )
+    list( APPEND _gflags_roots ${_native_path} )
+elseif ( GFlags_HOME )
+    list( APPEND _gflags_roots ${GFlags_HOME} )
+endif()
+
+if ( _gflags_roots )
+  find_path(GFLAGS_INCLUDE_DIR NAMES gflags/gflags.h
+    PATHS ${_gflags_roots}
+    NO_DEFAULT_PATH
+    PATH_SUFFIXES "include" )
+  find_library(GFLAGS_SHARED_LIB NAMES gflags
+    PATHS ${_gflags_roots}
+    NO_DEFAULT_PATH
+    PATH_SUFFIXES "lib" )
+  find_library(GFLAGS_SHARED_LIB NAMES libgflags.a
+    PATHS ${_gflags_roots}
+    NO_DEFAULT_PATH
+    PATH_SUFFIXES "lib" )
+else()
+  find_path(GFLAGS_INCLUDE_DIR gflags/gflags.h
+    # make sure we don't accidentally pick up a different version
+    NO_CMAKE_SYSTEM_PATH
+    NO_SYSTEM_ENVIRONMENT_PATH)
+  find_library(GFLAGS_SHARED_LIB gflags
+    NO_CMAKE_SYSTEM_PATH
+    NO_SYSTEM_ENVIRONMENT_PATH)
+  find_library(GFLAGS_STATIC_LIB libgflags.a
+    NO_CMAKE_SYSTEM_PATH
+    NO_SYSTEM_ENVIRONMENT_PATH)
+endif()
+
+include(FindPackageHandleStandardArgs)
+find_package_handle_standard_args(GFLAGS REQUIRED_VARS
+  GFLAGS_SHARED_LIB GFLAGS_STATIC_LIB GFLAGS_INCLUDE_DIR)

http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/cpp/src/arrow/io/file.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/io/file.cc b/cpp/src/arrow/io/file.cc
index 93f0ad9..05fa666 100644
--- a/cpp/src/arrow/io/file.cc
+++ b/cpp/src/arrow/io/file.cc
@@ -186,12 +186,13 @@ static inline Status FileOpenWriteable(const std::string& filename, int* fd) {
   memcpy(wpath.data(), filename.data(), filename.size());
   memcpy(wpath.data() + nwchars, L"\0", sizeof(wchar_t));
 
-  errno_actual = _wsopen_s(
-      fd, wpath.data(), _O_WRONLY | _O_CREAT | _O_BINARY, _SH_DENYNO, _S_IWRITE);
+  errno_actual = _wsopen_s(fd, wpath.data(), _O_WRONLY | _O_CREAT | _O_BINARY | _O_TRUNC,
+      _SH_DENYNO, _S_IWRITE);
   ret = *fd;
 
 #else
-  ret = *fd = open(filename.c_str(), O_WRONLY | O_CREAT | O_BINARY, ARROW_WRITE_SHMODE);
+  ret = *fd =
+      open(filename.c_str(), O_WRONLY | O_CREAT | O_BINARY | O_TRUNC, ARROW_WRITE_SHMODE);
 #endif
   return CheckOpenResult(ret, errno_actual, filename.c_str(), filename.size());
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/cpp/src/arrow/ipc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/CMakeLists.txt b/cpp/src/arrow/ipc/CMakeLists.txt
index 6955bcb..f9e7cf7 100644
--- a/cpp/src/arrow/ipc/CMakeLists.txt
+++ b/cpp/src/arrow/ipc/CMakeLists.txt
@@ -85,6 +85,33 @@ ADD_ARROW_TEST(ipc-json-test)
 ARROW_TEST_LINK_LIBRARIES(ipc-json-test
   ${ARROW_IPC_TEST_LINK_LIBS})
 
+ADD_ARROW_TEST(json-integration-test)
+
+if (APPLE)
+  target_link_libraries(json-integration-test
+    arrow_static
+    arrow_io
+    arrow_ipc
+    gflags
+    gtest
+    boost_filesystem_static
+    boost_system_static
+    dl)
+  set_target_properties(json-integration-test
+    PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
+else()
+  target_link_libraries(json-integration-test
+    arrow_static
+    arrow_io
+    arrow_ipc
+    gflags
+    gtest
+    pthread
+    boost_filesystem_static
+    boost_system_static
+    dl)
+endif()
+
 # make clean will delete the generated file
 set_source_files_properties(Metadata_generated.h PROPERTIES GENERATED TRUE)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/cpp/src/arrow/ipc/adapter.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/adapter.h b/cpp/src/arrow/ipc/adapter.h
index 3fde18d..b02de28 100644
--- a/cpp/src/arrow/ipc/adapter.h
+++ b/cpp/src/arrow/ipc/adapter.h
@@ -16,7 +16,7 @@
 // under the License.
 
 // Public API for writing and accessing (with zero copy, if possible) Arrow
-// data in shared memory
+// IPC binary formatted data (e.g. in shared memory, or from some other IO source)
 
 #ifndef ARROW_IPC_ADAPTER_H
 #define ARROW_IPC_ADAPTER_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/f082b173/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
new file mode 100644
index 0000000..5eff899
--- /dev/null
+++ b/cpp/src/arrow/ipc/json-integration-test.cc
@@ -0,0 +1,381 @@
+// 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 <cstdio>
+#include <cstdlib>
+#include <cstring>
+#include <fstream>
+#include <iostream>
+#include <memory>
+#include <string>
+
+#include "gflags/gflags.h"
+#include "gtest/gtest.h"
+
+#include <boost/filesystem.hpp>  // NOLINT
+
+#include "arrow/io/file.h"
+#include "arrow/ipc/file.h"
+#include "arrow/ipc/json.h"
+#include "arrow/schema.h"
+#include "arrow/table.h"
+#include "arrow/test-util.h"
+#include "arrow/util/status.h"
+
+DEFINE_string(arrow, "", "Arrow file name");
+DEFINE_string(json, "", "JSON file name");
+DEFINE_string(mode, "VALIDATE",
+    "Mode of integration testing tool (ARROW_TO_JSON, JSON_TO_ARROW, VALIDATE)");
+DEFINE_bool(integration, false, "Run in integration test mode");
+DEFINE_bool(verbose, true, "Verbose output");
+
+namespace fs = boost::filesystem;
+
+namespace arrow {
+
+bool file_exists(const char* path) {
+  std::ifstream handle(path);
+  return handle.good();
+}
+
+// Convert JSON file to IPC binary format
+static Status ConvertJsonToArrow(
+    const std::string& json_path, const std::string& arrow_path) {
+  std::shared_ptr<io::ReadableFile> in_file;
+  std::shared_ptr<io::FileOutputStream> out_file;
+
+  RETURN_NOT_OK(io::ReadableFile::Open(json_path, &in_file));
+  RETURN_NOT_OK(io::FileOutputStream::Open(arrow_path, &out_file));
+
+  int64_t file_size = 0;
+  RETURN_NOT_OK(in_file->GetSize(&file_size));
+
+  std::shared_ptr<Buffer> json_buffer;
+  RETURN_NOT_OK(in_file->Read(file_size, &json_buffer));
+
+  std::unique_ptr<ipc::JsonReader> reader;
+  RETURN_NOT_OK(ipc::JsonReader::Open(json_buffer, &reader));
+
+  if (FLAGS_verbose) {
+    std::cout << "Found schema: " << reader->schema()->ToString() << std::endl;
+  }
+
+  std::shared_ptr<ipc::FileWriter> writer;
+  RETURN_NOT_OK(ipc::FileWriter::Open(out_file.get(), reader->schema(), &writer));
+
+  for (int i = 0; i < reader->num_record_batches(); ++i) {
+    std::shared_ptr<RecordBatch> batch;
+    RETURN_NOT_OK(reader->GetRecordBatch(i, &batch));
+    RETURN_NOT_OK(writer->WriteRecordBatch(batch->columns(), batch->num_rows()));
+  }
+  return writer->Close();
+}
+
+// Convert IPC binary format to JSON
+static Status ConvertArrowToJson(
+    const std::string& arrow_path, const std::string& json_path) {
+  std::shared_ptr<io::ReadableFile> in_file;
+  std::shared_ptr<io::FileOutputStream> out_file;
+
+  RETURN_NOT_OK(io::ReadableFile::Open(arrow_path, &in_file));
+  RETURN_NOT_OK(io::FileOutputStream::Open(json_path, &out_file));
+
+  std::shared_ptr<ipc::FileReader> reader;
+  RETURN_NOT_OK(ipc::FileReader::Open(in_file, &reader));
+
+  if (FLAGS_verbose) {
+    std::cout << "Found schema: " << reader->schema()->ToString() << std::endl;
+  }
+
+  std::unique_ptr<ipc::JsonWriter> writer;
+  RETURN_NOT_OK(ipc::JsonWriter::Open(reader->schema(), &writer));
+
+  for (int i = 0; i < reader->num_record_batches(); ++i) {
+    std::shared_ptr<RecordBatch> batch;
+    RETURN_NOT_OK(reader->GetRecordBatch(i, &batch));
+    RETURN_NOT_OK(writer->WriteRecordBatch(batch->columns(), batch->num_rows()));
+  }
+
+  std::string result;
+  RETURN_NOT_OK(writer->Finish(&result));
+  return out_file->Write(reinterpret_cast<const uint8_t*>(result.c_str()),
+      static_cast<int64_t>(result.size()));
+}
+
+static Status ValidateArrowVsJson(
+    const std::string& arrow_path, const std::string& json_path) {
+  // Construct JSON reader
+  std::shared_ptr<io::ReadableFile> json_file;
+  RETURN_NOT_OK(io::ReadableFile::Open(json_path, &json_file));
+
+  int64_t file_size = 0;
+  RETURN_NOT_OK(json_file->GetSize(&file_size));
+
+  std::shared_ptr<Buffer> json_buffer;
+  RETURN_NOT_OK(json_file->Read(file_size, &json_buffer));
+
+  std::unique_ptr<ipc::JsonReader> json_reader;
+  RETURN_NOT_OK(ipc::JsonReader::Open(json_buffer, &json_reader));
+
+  // Construct Arrow reader
+  std::shared_ptr<io::ReadableFile> arrow_file;
+  RETURN_NOT_OK(io::ReadableFile::Open(arrow_path, &arrow_file));
+
+  std::shared_ptr<ipc::FileReader> arrow_reader;
+  RETURN_NOT_OK(ipc::FileReader::Open(arrow_file, &arrow_reader));
+
+  auto json_schema = json_reader->schema();
+  auto arrow_schema = arrow_reader->schema();
+
+  if (!json_schema->Equals(arrow_schema)) {
+    std::stringstream ss;
+    ss << "JSON schema: \n"
+       << json_schema->ToString() << "\n"
+       << "Arrow schema: \n"
+       << arrow_schema->ToString();
+
+    if (FLAGS_verbose) { std::cout << ss.str() << std::endl; }
+    return Status::Invalid("Schemas did not match");
+  }
+
+  const int json_nbatches = json_reader->num_record_batches();
+  const int arrow_nbatches = arrow_reader->num_record_batches();
+
+  if (json_nbatches != arrow_nbatches) {
+    std::stringstream ss;
+    ss << "Different number of record batches: " << json_nbatches << " (JSON) vs "
+       << arrow_nbatches << " (Arrow)";
+    return Status::Invalid(ss.str());
+  }
+
+  std::shared_ptr<RecordBatch> arrow_batch;
+  std::shared_ptr<RecordBatch> json_batch;
+  for (int i = 0; i < json_nbatches; ++i) {
+    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())) {
+      std::stringstream ss;
+      ss << "Record batch " << i << " did not match";
+      return Status::Invalid(ss.str());
+    }
+  }
+
+  return Status::OK();
+}
+
+Status RunCommand(const std::string& json_path, const std::string& arrow_path,
+    const std::string& command) {
+  if (json_path == "") { return Status::Invalid("Must specify json file name"); }
+
+  if (arrow_path == "") { return Status::Invalid("Must specify arrow file name"); }
+
+  if (command == "ARROW_TO_JSON") {
+    if (!file_exists(arrow_path.c_str())) {
+      return Status::Invalid("Input file does not exist");
+    }
+
+    return ConvertArrowToJson(arrow_path, json_path);
+  } else if (command == "JSON_TO_ARROW") {
+    if (!file_exists(json_path.c_str())) {
+      return Status::Invalid("Input file does not exist");
+    }
+
+    return ConvertJsonToArrow(json_path, arrow_path);
+  } else if (command == "VALIDATE") {
+    if (!file_exists(json_path.c_str())) {
+      return Status::Invalid("JSON file does not exist");
+    }
+
+    if (!file_exists(arrow_path.c_str())) {
+      return Status::Invalid("Arrow file does not exist");
+    }
+
+    return ValidateArrowVsJson(arrow_path, json_path);
+  } else {
+    std::stringstream ss;
+    ss << "Unknown command: " << command;
+    return Status::Invalid(ss.str());
+  }
+}
+
+static std::string temp_path() {
+  return (fs::temp_directory_path() / fs::unique_path()).native();
+}
+
+class TestJSONIntegration : public ::testing::Test {
+ public:
+  void SetUp() {}
+
+  std::string mkstemp() {
+    auto path = temp_path();
+    tmp_paths_.push_back(path);
+    return path;
+  }
+
+  Status WriteJson(const char* data, const std::string& path) {
+    do {
+      std::shared_ptr<io::FileOutputStream> out;
+      RETURN_NOT_OK(io::FileOutputStream::Open(path, &out));
+      RETURN_NOT_OK(out->Write(
+          reinterpret_cast<const uint8_t*>(data), static_cast<int64_t>(strlen(data))));
+    } while (0);
+    return Status::OK();
+  }
+
+  void TearDown() {
+    for (const std::string path : tmp_paths_) {
+      std::remove(path.c_str());
+    }
+  }
+
+ protected:
+  std::vector<std::string> tmp_paths_;
+};
+
+static const char* JSON_EXAMPLE = R"example(
+{
+  "schema": {
+    "fields": [
+      {
+        "name": "foo",
+        "type": {"name": "int", "isSigned": true, "bitWidth": 32},
+        "nullable": true, "children": [],
+        "typeLayout": [
+          {"type": "VALIDITY", "typeBitWidth": 1},
+          {"type": "DATA", "typeBitWidth": 32}
+        ]
+      },
+      {
+        "name": "bar",
+        "type": {"name": "floatingpoint", "precision": "DOUBLE"},
+        "nullable": true, "children": [],
+        "typeLayout": [
+          {"type": "VALIDITY", "typeBitWidth": 1},
+          {"type": "DATA", "typeBitWidth": 64}
+        ]
+      }
+    ]
+  },
+  "batches": [
+    {
+      "count": 5,
+      "columns": [
+        {
+          "name": "foo",
+          "count": 5,
+          "DATA": [1, 2, 3, 4, 5],
+          "VALIDITY": [1, 0, 1, 1, 1]
+        },
+        {
+          "name": "bar",
+          "count": 5,
+          "DATA": [1.0, 2.0, 3.0, 4.0, 5.0],
+          "VALIDITY": [1, 0, 0, 1, 1]
+        }
+      ]
+    }
+  ]
+}
+)example";
+
+static const char* JSON_EXAMPLE2 = R"example(
+{
+  "schema": {
+    "fields": [
+      {
+        "name": "foo",
+        "type": {"name": "int", "isSigned": true, "bitWidth": 32},
+        "nullable": true, "children": [],
+        "typeLayout": [
+          {"type": "VALIDITY", "typeBitWidth": 1},
+          {"type": "DATA", "typeBitWidth": 32}
+        ]
+      }
+    ]
+  },
+  "batches": [
+    {
+      "count": 5,
+      "columns": [
+        {
+          "name": "foo",
+          "count": 5,
+          "DATA": [1, 2, 3, 4, 5],
+          "VALIDITY": [1, 0, 1, 1, 1]
+        }
+      ]
+    }
+  ]
+}
+)example";
+
+TEST_F(TestJSONIntegration, ConvertAndValidate) {
+  std::string json_path = this->mkstemp();
+  std::string arrow_path = this->mkstemp();
+
+  ASSERT_OK(WriteJson(JSON_EXAMPLE, json_path));
+
+  ASSERT_OK(RunCommand(json_path, arrow_path, "JSON_TO_ARROW"));
+  ASSERT_OK(RunCommand(json_path, arrow_path, "VALIDATE"));
+
+  // Convert and overwrite
+  ASSERT_OK(RunCommand(json_path, arrow_path, "ARROW_TO_JSON"));
+
+  // Convert back to arrow, and validate
+  ASSERT_OK(RunCommand(json_path, arrow_path, "JSON_TO_ARROW"));
+  ASSERT_OK(RunCommand(json_path, arrow_path, "VALIDATE"));
+}
+
+TEST_F(TestJSONIntegration, ErrorStates) {
+  std::string json_path = this->mkstemp();
+  std::string json_path2 = this->mkstemp();
+  std::string arrow_path = this->mkstemp();
+
+  ASSERT_OK(WriteJson(JSON_EXAMPLE, json_path));
+  ASSERT_OK(WriteJson(JSON_EXAMPLE2, json_path2));
+
+  ASSERT_OK(ConvertJsonToArrow(json_path, arrow_path));
+  ASSERT_RAISES(Invalid, ValidateArrowVsJson(arrow_path, json_path2));
+
+  ASSERT_RAISES(IOError, ValidateArrowVsJson("does_not_exist-1234", json_path2));
+  ASSERT_RAISES(IOError, ValidateArrowVsJson(arrow_path, "does_not_exist-1234"));
+
+  ASSERT_RAISES(Invalid, RunCommand("", arrow_path, "VALIDATE"));
+  ASSERT_RAISES(Invalid, RunCommand(json_path, "", "VALIDATE"));
+}
+
+}  // namespace arrow
+
+int main(int argc, char** argv) {
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+
+  int ret = 0;
+
+  if (FLAGS_integration) {
+    arrow::Status result = arrow::RunCommand(FLAGS_json, FLAGS_arrow, FLAGS_mode);
+    if (!result.ok()) {
+      std::cout << "Error message: " << result.ToString() << std::endl;
+      ret = 1;
+    }
+  } else {
+    ::testing::InitGoogleTest(&argc, argv);
+    ret = RUN_ALL_TESTS();
+  }
+  gflags::ShutDownCommandLineFlags();
+  return ret;
+}