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/30 18:25:57 UTC

arrow git commit: ARROW-392: [C++/Java] String IPC integration testing / fixes. Add array / record batch pretty-printing

Repository: arrow
Updated Branches:
  refs/heads/master 65b74b350 -> 859018b3c


ARROW-392: [C++/Java] String IPC integration testing / fixes. Add array / record batch pretty-printing

Was blocked by ARROW-393

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

Closes #217 from wesm/ARROW-392 and squashes the following commits:

1efeaed [Wes McKinney] Remove debug printing from Java
57e4926 [Wes McKinney] cpplint
56a1c41 [Wes McKinney] We are only padding to 8 byte boundaries
e33bed3 [Wes McKinney] clang-format, add all-OK message to integration_test.py
8e8d6d3 [Wes McKinney] Implement simple C++ pretty printer for record batches. Debugging efforts


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

Branch: refs/heads/master
Commit: 859018b3c79bfc0cb2259bdfc3d5930a9a936432
Parents: 65b74b3
Author: Wes McKinney <we...@twosigma.com>
Authored: Wed Nov 30 13:25:42 2016 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Wed Nov 30 13:25:42 2016 -0500

----------------------------------------------------------------------
 cpp/CMakeLists.txt                         |   1 +
 cpp/src/arrow/CMakeLists.txt               |   2 +
 cpp/src/arrow/ipc/ipc-json-test.cc         |  20 ---
 cpp/src/arrow/ipc/json-integration-test.cc |   7 +
 cpp/src/arrow/ipc/json-internal.cc         |   2 +-
 cpp/src/arrow/pretty_print-test.cc         |  87 +++++++++++
 cpp/src/arrow/pretty_print.cc              | 192 ++++++++++++++++++++++++
 cpp/src/arrow/pretty_print.h               |  35 +++++
 cpp/src/arrow/test-util.h                  |  22 +++
 cpp/src/arrow/type_traits.h                |   6 +
 format/IPC.md                              |   4 +-
 integration/data/simple.json               |   2 +-
 integration/integration_test.py            |   2 +-
 13 files changed, 357 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index 1a97008..798d75f 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -745,6 +745,7 @@ set(ARROW_SRCS
   src/arrow/array.cc
   src/arrow/builder.cc
   src/arrow/column.cc
+  src/arrow/pretty_print.cc
   src/arrow/schema.cc
   src/arrow/table.cc
   src/arrow/type.cc

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index 81851bc..6c0dea2 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -21,6 +21,7 @@ install(FILES
   array.h
   column.h
   builder.h
+  pretty_print.h
   schema.h
   table.h
   type.h
@@ -37,6 +38,7 @@ set(ARROW_TEST_LINK_LIBS ${ARROW_MIN_TEST_LIBS})
 
 ADD_ARROW_TEST(array-test)
 ADD_ARROW_TEST(column-test)
+ADD_ARROW_TEST(pretty_print-test)
 ADD_ARROW_TEST(schema-test)
 ADD_ARROW_TEST(table-test)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/ipc/ipc-json-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-json-test.cc b/cpp/src/arrow/ipc/ipc-json-test.cc
index e5c3a08..ba4d9ca 100644
--- a/cpp/src/arrow/ipc/ipc-json-test.cc
+++ b/cpp/src/arrow/ipc/ipc-json-test.cc
@@ -96,26 +96,6 @@ void CheckPrimitive(const std::shared_ptr<DataType>& type,
   TestArrayRoundTrip(*array.get());
 }
 
-template <typename TYPE, typename C_TYPE>
-void MakeArray(const std::shared_ptr<DataType>& type, const std::vector<bool>& is_valid,
-    const std::vector<C_TYPE>& values, std::shared_ptr<Array>* out) {
-  std::shared_ptr<Buffer> values_buffer;
-  std::shared_ptr<Buffer> values_bitmap;
-
-  ASSERT_OK(test::CopyBufferFromVector(values, &values_buffer));
-  ASSERT_OK(test::GetBitmapFromBoolVector(is_valid, &values_bitmap));
-
-  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
-
-  int32_t null_count = 0;
-  for (bool val : is_valid) {
-    if (!val) { ++null_count; }
-  }
-
-  *out = std::make_shared<ArrayType>(type, static_cast<int32_t>(values.size()),
-      values_buffer, null_count, values_bitmap);
-}
-
 TEST(TestJsonSchemaWriter, FlatTypes) {
   std::vector<std::shared_ptr<Field>> fields = {field("f0", int8()),
       field("f1", int16(), false), field("f2", int32()), field("f3", int64(), false),

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/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 7a313f7..c4e6847 100644
--- a/cpp/src/arrow/ipc/json-integration-test.cc
+++ b/cpp/src/arrow/ipc/json-integration-test.cc
@@ -31,6 +31,7 @@
 #include "arrow/io/file.h"
 #include "arrow/ipc/file.h"
 #include "arrow/ipc/json.h"
+#include "arrow/pretty_print.h"
 #include "arrow/schema.h"
 #include "arrow/table.h"
 #include "arrow/test-util.h"
@@ -171,6 +172,12 @@ static Status ValidateArrowVsJson(
     if (!json_batch->Equals(*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 << "\nArrow: \n ";
+      RETURN_NOT_OK(PrettyPrint(*arrow_batch.get(), &ss));
       return Status::Invalid(ss.str());
     }
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/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 e56bcb3..50f5b0c 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -343,7 +343,7 @@ class JsonSchemaWriter : public TypeVisitor {
 
 class JsonArrayWriter : public ArrayVisitor {
  public:
-  explicit JsonArrayWriter(const std::string& name, const Array& array, RjWriter* writer)
+  JsonArrayWriter(const std::string& name, const Array& array, RjWriter* writer)
       : name_(name), array_(array), writer_(writer) {}
 
   Status Write() { return VisitArray(name_, array_); }

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/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
new file mode 100644
index 0000000..10af41d
--- /dev/null
+++ b/cpp/src/arrow/pretty_print-test.cc
@@ -0,0 +1,87 @@
+// 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 <cstdint>
+#include <cstdio>
+#include <cstdlib>
+#include <cstring>
+#include <memory>
+#include <sstream>
+#include <vector>
+
+#include "gtest/gtest.h"
+
+#include "arrow/array.h"
+#include "arrow/pretty_print.h"
+#include "arrow/test-util.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/types/list.h"
+#include "arrow/types/primitive.h"
+#include "arrow/types/string.h"
+#include "arrow/types/struct.h"
+
+namespace arrow {
+
+class TestArrayPrinter : public ::testing::Test {
+ public:
+  void SetUp() {}
+
+  void Print(const Array& array) {}
+
+ private:
+  std::ostringstream sink_;
+};
+
+template <typename TYPE, typename C_TYPE>
+void CheckPrimitive(const std::vector<bool>& is_valid, const std::vector<C_TYPE>& values,
+    const char* expected) {
+  std::ostringstream sink;
+
+  MemoryPool* pool = default_memory_pool();
+  typename TypeTraits<TYPE>::BuilderType builder(pool, std::make_shared<TYPE>());
+
+  for (size_t i = 0; i < values.size(); ++i) {
+    if (is_valid[i]) {
+      ASSERT_OK(builder.Append(values[i]));
+    } else {
+      ASSERT_OK(builder.AppendNull());
+    }
+  }
+
+  std::shared_ptr<Array> array;
+  ASSERT_OK(builder.Finish(&array));
+
+  ASSERT_OK(PrettyPrint(*array.get(), &sink));
+
+  std::string result = sink.str();
+  ASSERT_EQ(std::string(expected, strlen(expected)), result);
+}
+
+TEST_F(TestArrayPrinter, PrimitiveType) {
+  std::vector<bool> is_valid = {true, true, false, true, false};
+
+  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);
+
+  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);
+}
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/pretty_print.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
new file mode 100644
index 0000000..c0b4b08
--- /dev/null
+++ b/cpp/src/arrow/pretty_print.cc
@@ -0,0 +1,192 @@
+// 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 <ostream>
+#include <string>
+
+#include "arrow/array.h"
+#include "arrow/pretty_print.h"
+#include "arrow/table.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/types/list.h"
+#include "arrow/types/string.h"
+#include "arrow/types/struct.h"
+#include "arrow/util/status.h"
+
+namespace arrow {
+
+class ArrayPrinter : public ArrayVisitor {
+ public:
+  ArrayPrinter(const Array& array, std::ostream* sink) : array_(array), 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(
+      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)) {
+        (*sink_) << "null";
+      } else {
+        (*sink_) << data[i];
+      }
+    }
+  }
+
+  // String (Utf8), Binary
+  template <typename T>
+  typename std::enable_if<std::is_base_of<BinaryArray, T>::value, void>::type
+  WriteDataValues(const T& array) {
+    int32_t length;
+    for (int i = 0; i < array.length(); ++i) {
+      if (i > 0) { (*sink_) << ", "; }
+      if (array.IsNull(i)) {
+        (*sink_) << "null";
+      } else {
+        const char* buf = reinterpret_cast<const char*>(array.GetValue(i, &length));
+        (*sink_) << "\"" << std::string(buf, length) << "\"";
+      }
+    }
+  }
+
+  template <typename T>
+  typename std::enable_if<std::is_base_of<BooleanArray, T>::value, void>::type
+  WriteDataValues(const T& array) {
+    for (int i = 0; i < array.length(); ++i) {
+      if (i > 0) { (*sink_) << ", "; }
+      if (array.IsNull(i)) {
+        (*sink_) << "null";
+      } else {
+        (*sink_) << (array.Value(i) ? "true" : "false");
+      }
+    }
+  }
+
+  void OpenArray() { (*sink_) << "["; }
+
+  void CloseArray() { (*sink_) << "]"; }
+
+  template <typename T>
+  Status WritePrimitive(const T& array) {
+    OpenArray();
+    WriteDataValues(array);
+    CloseArray();
+    return Status::OK();
+  }
+
+  template <typename T>
+  Status WriteVarBytes(const T& array) {
+    OpenArray();
+    WriteDataValues(array);
+    CloseArray();
+    return Status::OK();
+  }
+
+  Status Visit(const NullArray& array) override { return Status::OK(); }
+
+  Status Visit(const BooleanArray& array) override { return WritePrimitive(array); }
+
+  Status Visit(const Int8Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const Int16Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const Int32Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const Int64Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const UInt8Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const UInt16Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const UInt32Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const UInt64Array& array) override { return WritePrimitive(array); }
+
+  Status Visit(const HalfFloatArray& array) override { return WritePrimitive(array); }
+
+  Status Visit(const FloatArray& array) override { return WritePrimitive(array); }
+
+  Status Visit(const DoubleArray& array) override { return WritePrimitive(array); }
+
+  Status Visit(const StringArray& array) override { return WriteVarBytes(array); }
+
+  Status Visit(const BinaryArray& array) override { return WriteVarBytes(array); }
+
+  Status Visit(const DateArray& array) override { return Status::NotImplemented("date"); }
+
+  Status Visit(const TimeArray& array) override { return Status::NotImplemented("time"); }
+
+  Status Visit(const TimestampArray& array) override {
+    return Status::NotImplemented("timestamp");
+  }
+
+  Status Visit(const IntervalArray& array) override {
+    return Status::NotImplemented("interval");
+  }
+
+  Status Visit(const DecimalArray& array) override {
+    return Status::NotImplemented("decimal");
+  }
+
+  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()});
+    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());
+    return Status::OK();
+  }
+
+  Status Visit(const UnionArray& array) override {
+    return Status::NotImplemented("union");
+  }
+
+ private:
+  const Array& array_;
+  std::ostream* sink_;
+};
+
+Status PrettyPrint(const Array& arr, std::ostream* sink) {
+  ArrayPrinter printer(arr, sink);
+  return printer.Print();
+}
+
+Status PrettyPrint(const RecordBatch& batch, 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));
+    (*sink) << "\n";
+  }
+  return Status::OK();
+}
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/pretty_print.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/pretty_print.h b/cpp/src/arrow/pretty_print.h
new file mode 100644
index 0000000..dcb236d
--- /dev/null
+++ b/cpp/src/arrow/pretty_print.h
@@ -0,0 +1,35 @@
+// 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_PRETTY_PRINT_H
+#define ARROW_PRETTY_PRINT_H
+
+#include <ostream>
+
+#include "arrow/type_fwd.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Status;
+
+Status ARROW_EXPORT PrettyPrint(const RecordBatch& batch, std::ostream* sink);
+Status ARROW_EXPORT PrettyPrint(const Array& arr, std::ostream* sink);
+
+}  // namespace arrow
+
+#endif  // ARROW_PRETTY_PRINT_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/test-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/test-util.h b/cpp/src/arrow/test-util.h
index 63c2166..b86a180 100644
--- a/cpp/src/arrow/test-util.h
+++ b/cpp/src/arrow/test-util.h
@@ -32,6 +32,7 @@
 #include "arrow/schema.h"
 #include "arrow/table.h"
 #include "arrow/type.h"
+#include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
 #include "arrow/util/buffer.h"
 #include "arrow/util/logging.h"
@@ -250,6 +251,27 @@ Status MakeRandomBytePoolBuffer(int32_t length, MemoryPool* pool,
 }
 
 }  // namespace test
+
+template <typename TYPE, typename C_TYPE>
+void MakeArray(const std::shared_ptr<DataType>& type, const std::vector<bool>& is_valid,
+    const std::vector<C_TYPE>& values, std::shared_ptr<Array>* out) {
+  std::shared_ptr<Buffer> values_buffer;
+  std::shared_ptr<Buffer> values_bitmap;
+
+  ASSERT_OK(test::CopyBufferFromVector(values, &values_buffer));
+  ASSERT_OK(test::GetBitmapFromBoolVector(is_valid, &values_bitmap));
+
+  using ArrayType = typename TypeTraits<TYPE>::ArrayType;
+
+  int32_t null_count = 0;
+  for (bool val : is_valid) {
+    if (!val) { ++null_count; }
+  }
+
+  *out = std::make_shared<ArrayType>(type, static_cast<int32_t>(values.size()),
+      values_buffer, null_count, values_bitmap);
+}
+
 }  // namespace arrow
 
 #endif  // ARROW_TEST_UTIL_H_

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/cpp/src/arrow/type_traits.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type_traits.h b/cpp/src/arrow/type_traits.h
index bbb8074..c21c500 100644
--- a/cpp/src/arrow/type_traits.h
+++ b/cpp/src/arrow/type_traits.h
@@ -192,6 +192,12 @@ struct IsFloatingPoint {
   static constexpr bool value = std::is_floating_point<c_type>::value;
 };
 
+template <typename T>
+struct IsNumeric {
+  PRIMITIVE_TRAITS(T);
+  static constexpr bool value = std::is_arithmetic<c_type>::value;
+};
+
 }  // namespace arrow
 
 #endif  // ARROW_TYPE_TRAITS_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/format/IPC.md
----------------------------------------------------------------------
diff --git a/format/IPC.md b/format/IPC.md
index a55dcdf..d386e60 100644
--- a/format/IPC.md
+++ b/format/IPC.md
@@ -24,7 +24,7 @@ In general, the file looks like:
 
 ```
 <magic number "ARROW1">
-<empty padding bytes [to 64 byte boundary]>
+<empty padding bytes [to 8 byte boundary]>
 <DICTIONARY 0>
 ...
 <DICTIONARY k - 1>
@@ -49,7 +49,7 @@ appropriate alignment and padding):
 ```
 <int32: metadata flatbuffer size>
 <metadata: org.apache.arrow.flatbuf.RecordBatch>
-<padding bytes [to 64-byte boundary]>
+<padding bytes [to 8-byte boundary]>
 <body: buffers end to end>
 ```
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/integration/data/simple.json
----------------------------------------------------------------------
diff --git a/integration/data/simple.json b/integration/data/simple.json
index a91b405..fb903e7 100644
--- a/integration/data/simple.json
+++ b/integration/data/simple.json
@@ -31,7 +31,7 @@
           "vectors": [
             {"type": "VALIDITY", "typeBitWidth": 1},
             {"type": "OFFSET", "typeBitWidth": 32},
-            {"type": "DATA", "typeBitWidth": 64}
+            {"type": "DATA", "typeBitWidth": 8}
           ]
         }
       }

http://git-wip-us.apache.org/repos/asf/arrow/blob/859018b3/integration/integration_test.py
----------------------------------------------------------------------
diff --git a/integration/integration_test.py b/integration/integration_test.py
index 6ea634d..88dc3ad 100644
--- a/integration/integration_test.py
+++ b/integration/integration_test.py
@@ -165,7 +165,7 @@ def run_all_tests(debug=False):
 
     runner = IntegrationRunner(json_files, testers, debug=debug)
     runner.run()
-
+    print('-- All tests passed!')
 
 if __name__ == '__main__':
     parser = argparse.ArgumentParser(description='Arrow integration test CLI')