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

arrow git commit: ARROW-1151: [C++] Add branch prediction to RETURN_NOT_OK

Repository: arrow
Updated Branches:
  refs/heads/master 5de6eb5ff -> ec6e183ee


ARROW-1151: [C++] Add branch prediction to RETURN_NOT_OK

Also added some missing status checks to builder-benchmark

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

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

9b488a0e [Wes McKinney] Try to fix snappy warning
06276119 [Wes McKinney] Restore check macros used in libplasma
83b3f36d [Wes McKinney] Add branch prediction to RETURN_NOT_OK


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

Branch: refs/heads/master
Commit: ec6e183ee31411a18c531507216c21cc80060c8b
Parents: 5de6eb5
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Jun 26 14:01:59 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Jun 26 14:01:59 2017 -0400

----------------------------------------------------------------------
 LICENSE.txt                        | 34 ++++++++++++++++++++++++++++
 cpp/src/arrow/builder-benchmark.cc | 40 +++++++++++++++++----------------
 cpp/src/arrow/status.h             | 36 +++++++++--------------------
 cpp/src/arrow/util/compression.cc  |  5 +++++
 cpp/src/arrow/util/macros.h        | 14 ++++++++++++
 5 files changed, 84 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/ec6e183e/LICENSE.txt
----------------------------------------------------------------------
diff --git a/LICENSE.txt b/LICENSE.txt
index 7000733..95c506f 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -296,3 +296,37 @@ You can contact the author at :
 - xxHash source repository : https://github.com/Cyan4973/xxHash
 
 --------------------------------------------------------------------------------
+
+src/arrow/util (some portions): Apache 2.0, and 3-clause BSD
+
+Some portions of this module are derived from code in the Chromium project,
+copyright (c) Google inc and (c) The Chromium Authors and licensed under the
+Apache 2.0 License or the under the 3-clause BSD license:
+
+  Copyright (c) 2013 The Chromium Authors. All rights reserved.
+
+  Redistribution and use in source and binary forms, with or without
+  modification, are permitted provided that the following conditions are
+  met:
+
+     * Redistributions of source code must retain the above copyright
+  notice, this list of conditions and the following disclaimer.
+     * Redistributions in binary form must reproduce the above
+  copyright notice, this list of conditions and the following disclaimer
+  in the documentation and/or other materials provided with the
+  distribution.
+     * Neither the name of Google Inc. nor the names of its
+  contributors may be used to endorse or promote products derived from
+  this software without specific prior written permission.
+
+  THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+  "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+  LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+  A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+  OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+  SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+  LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+  DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+  THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+  (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+  OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

http://git-wip-us.apache.org/repos/asf/arrow/blob/ec6e183e/cpp/src/arrow/builder-benchmark.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder-benchmark.cc b/cpp/src/arrow/builder-benchmark.cc
index 62f2fd6..3c49c63 100644
--- a/cpp/src/arrow/builder-benchmark.cc
+++ b/cpp/src/arrow/builder-benchmark.cc
@@ -25,6 +25,12 @@ namespace arrow {
 
 constexpr int64_t kFinalSize = 256;
 
+#define ABORT_NOT_OK(s)                                 \
+  do {                                                  \
+    ::arrow::Status _s = (s);                           \
+    if (ARROW_PREDICT_FALSE(!_s.ok())) { exit(-1); }    \
+  } while (0);
+
 static void BM_BuildPrimitiveArrayNoNulls(
     benchmark::State& state) {  // NOLINT non-const reference
   // 2 MiB block
@@ -33,17 +39,15 @@ static void BM_BuildPrimitiveArrayNoNulls(
     Int64Builder builder(default_memory_pool());
     for (int i = 0; i < kFinalSize; i++) {
       // Build up an array of 512 MiB in size
-      builder.Append(data.data(), data.size(), nullptr);
+      ABORT_NOT_OK(builder.Append(data.data(), data.size(), nullptr));
     }
     std::shared_ptr<Array> out;
-    builder.Finish(&out);
+    ABORT_NOT_OK(builder.Finish(&out));
   }
   state.SetBytesProcessed(
       state.iterations() * data.size() * sizeof(int64_t) * kFinalSize);
 }
 
-BENCHMARK(BM_BuildPrimitiveArrayNoNulls)->Repetitions(3)->Unit(benchmark::kMillisecond);
-
 static void BM_BuildVectorNoNulls(
     benchmark::State& state) {  // NOLINT non-const reference
   // 2 MiB block
@@ -59,8 +63,6 @@ static void BM_BuildVectorNoNulls(
       state.iterations() * data.size() * sizeof(int64_t) * kFinalSize);
 }
 
-BENCHMARK(BM_BuildVectorNoNulls)->Repetitions(3)->Unit(benchmark::kMillisecond);
-
 static void BM_BuildAdaptiveIntNoNulls(
     benchmark::State& state) {  // NOLINT non-const reference
   int64_t size = static_cast<int64_t>(std::numeric_limits<int16_t>::max()) * 256;
@@ -73,16 +75,14 @@ static void BM_BuildAdaptiveIntNoNulls(
     AdaptiveIntBuilder builder(default_memory_pool());
     for (int64_t i = 0; i < size; i += chunk_size) {
       // Build up an array of 512 MiB in size
-      builder.Append(data.data() + i, chunk_size, nullptr);
+      ABORT_NOT_OK(builder.Append(data.data() + i, chunk_size, nullptr));
     }
     std::shared_ptr<Array> out;
-    builder.Finish(&out);
+    ABORT_NOT_OK(builder.Finish(&out));
   }
   state.SetBytesProcessed(state.iterations() * data.size() * sizeof(int64_t));
 }
 
-BENCHMARK(BM_BuildAdaptiveIntNoNulls)->Repetitions(3)->Unit(benchmark::kMillisecond);
-
 static void BM_BuildAdaptiveIntNoNullsScalarAppend(
     benchmark::State& state) {  // NOLINT non-const reference
   int64_t size = static_cast<int64_t>(std::numeric_limits<int16_t>::max()) * 256;
@@ -93,18 +93,14 @@ static void BM_BuildAdaptiveIntNoNullsScalarAppend(
   while (state.KeepRunning()) {
     AdaptiveIntBuilder builder(default_memory_pool());
     for (int64_t i = 0; i < size; i++) {
-      builder.Append(data[i]);
+      ABORT_NOT_OK(builder.Append(data[i]));
     }
     std::shared_ptr<Array> out;
-    builder.Finish(&out);
+    ABORT_NOT_OK(builder.Finish(&out));
   }
   state.SetBytesProcessed(state.iterations() * data.size() * sizeof(int64_t));
 }
 
-BENCHMARK(BM_BuildAdaptiveIntNoNullsScalarAppend)
-    ->Repetitions(3)
-    ->Unit(benchmark::kMillisecond);
-
 static void BM_BuildAdaptiveUIntNoNulls(
     benchmark::State& state) {  // NOLINT non-const reference
   int64_t size = static_cast<int64_t>(std::numeric_limits<uint16_t>::max()) * 256;
@@ -117,14 +113,20 @@ static void BM_BuildAdaptiveUIntNoNulls(
     AdaptiveUIntBuilder builder(default_memory_pool());
     for (int64_t i = 0; i < size; i += chunk_size) {
       // Build up an array of 512 MiB in size
-      builder.Append(data.data() + i, chunk_size, nullptr);
+      ABORT_NOT_OK(builder.Append(data.data() + i, chunk_size, nullptr));
     }
     std::shared_ptr<Array> out;
-    builder.Finish(&out);
+    ABORT_NOT_OK(builder.Finish(&out));
   }
   state.SetBytesProcessed(state.iterations() * data.size() * sizeof(int64_t));
 }
 
-BENCHMARK(BM_BuildAdaptiveUIntNoNulls)->Repetitions(3)->Unit(benchmark::kMillisecond);
+BENCHMARK(BM_BuildPrimitiveArrayNoNulls)->Repetitions(3)->Unit(benchmark::kMicrosecond);
+BENCHMARK(BM_BuildVectorNoNulls)->Repetitions(3)->Unit(benchmark::kMicrosecond);
+BENCHMARK(BM_BuildAdaptiveIntNoNulls)->Repetitions(3)->Unit(benchmark::kMicrosecond);
+BENCHMARK(BM_BuildAdaptiveIntNoNullsScalarAppend)
+    ->Repetitions(3)
+    ->Unit(benchmark::kMicrosecond);
+BENCHMARK(BM_BuildAdaptiveUIntNoNulls)->Repetitions(3)->Unit(benchmark::kMicrosecond);
 
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/ec6e183e/cpp/src/arrow/status.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/status.h b/cpp/src/arrow/status.h
index 7e7f67c..9a75a58 100644
--- a/cpp/src/arrow/status.h
+++ b/cpp/src/arrow/status.h
@@ -10,7 +10,7 @@
 // non-const method, all threads accessing the same Status must use
 // external synchronization.
 
-// Adapted from Kudu github.com/cloudera/kudu
+// Adapted from Kudu github.com/apache/kudu
 
 #ifndef ARROW_STATUS_H_
 #define ARROW_STATUS_H_
@@ -19,30 +19,14 @@
 #include <cstring>
 #include <string>
 
+#include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
 
 // Return the given status if it is not OK.
-#define ARROW_RETURN_NOT_OK(s)   \
-  do {                           \
-    ::arrow::Status _s = (s);    \
-    if (!_s.ok()) { return _s; } \
-  } while (0);
-
-// Return the given status if it is not OK, but first clone it and
-// prepend the given message.
-#define ARROW_RETURN_NOT_OK_PREPEND(s, msg)                               \
-  do {                                                                    \
-    ::arrow::Status _s = (s);                                             \
-    if (::gutil::PREDICT_FALSE(!_s.ok())) return _s.CloneAndPrepend(msg); \
-  } while (0);
-
-// Return 'to_return' if 'to_call' returns a bad status.
-// The substitution for 'to_return' may reference the variable
-// 's' for the bad status.
-#define ARROW_RETURN_NOT_OK_RET(to_call, to_return)          \
-  do {                                                       \
-    ::arrow::Status s = (to_call);                           \
-    if (::gutil::PREDICT_FALSE(!s.ok())) return (to_return); \
+#define ARROW_RETURN_NOT_OK(s)                          \
+  do {                                                  \
+    ::arrow::Status _s = (s);                           \
+    if (ARROW_PREDICT_FALSE(!_s.ok())) { return _s; }   \
   } while (0);
 
 // If 'to_call' returns a bad status, CHECK immediately with a logged message
@@ -59,10 +43,10 @@
 
 namespace arrow {
 
-#define RETURN_NOT_OK(s)         \
-  do {                           \
-    Status _s = (s);             \
-    if (!_s.ok()) { return _s; } \
+#define RETURN_NOT_OK(s)                                \
+  do {                                                  \
+    Status _s = (s);                                    \
+    if (ARROW_PREDICT_FALSE(!_s.ok())) { return _s; }   \
   } while (0);
 
 #define RETURN_NOT_OK_ELSE(s, else_) \

http://git-wip-us.apache.org/repos/asf/arrow/blob/ec6e183e/cpp/src/arrow/util/compression.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/compression.cc b/cpp/src/arrow/util/compression.cc
index f82ae5c..070f857 100644
--- a/cpp/src/arrow/util/compression.cc
+++ b/cpp/src/arrow/util/compression.cc
@@ -17,6 +17,11 @@
 
 #include "arrow/util/compression.h"
 
+// Work around warning caused by Snappy include
+#ifdef DISALLOW_COPY_AND_ASSIGN
+#undef DISALLOW_COPY_AND_ASSIGN
+#endif
+
 #include <cstdint>
 #include <memory>
 #include <sstream>

http://git-wip-us.apache.org/repos/asf/arrow/blob/ec6e183e/cpp/src/arrow/util/macros.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/macros.h b/cpp/src/arrow/util/macros.h
index c4a62a4..affe8bc 100644
--- a/cpp/src/arrow/util/macros.h
+++ b/cpp/src/arrow/util/macros.h
@@ -27,4 +27,18 @@
 
 #define UNUSED(x) (void)x
 
+//
+// GCC can be told that a certain branch is not likely to be taken (for
+// instance, a CHECK failure), and use that information in static analysis.
+// Giving it this information can help it optimize for the common case in
+// the absence of better information (ie. -fprofile-arcs).
+//
+#if defined(__GNUC__)
+#define ARROW_PREDICT_FALSE(x) (__builtin_expect(x, 0))
+#define ARROW_PREDICT_TRUE(x) (__builtin_expect(!!(x), 1))
+#else
+#define ARROW_PREDICT_FALSE(x) x
+#define ARROW_PREDICT_TRUE(x) x
+#endif
+
 #endif  // ARROW_UTIL_MACROS_H