You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ja...@apache.org on 2016/02/17 13:39:48 UTC

[13/17] arrow git commit: ARROW-4: This provides an partial C++11 implementation of the Apache Arrow data structures along with a cmake-based build system. The codebase generally follows Google C++ style guide, but more cleaning to be more conforming is

http://git-wip-us.apache.org/repos/asf/arrow/blob/23c4b08d/cpp/src/arrow/util/status.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/status.h b/cpp/src/arrow/util/status.h
new file mode 100644
index 0000000..47fda40
--- /dev/null
+++ b/cpp/src/arrow/util/status.h
@@ -0,0 +1,152 @@
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// A Status encapsulates the result of an operation.  It may indicate success,
+// or it may indicate an error with an associated error message.
+//
+// Multiple threads can invoke const methods on a Status without
+// external synchronization, but if any of the threads may call a
+// non-const method, all threads accessing the same Status must use
+// external synchronization.
+
+// Adapted from Kudu github.com/cloudera/kudu
+
+#ifndef ARROW_STATUS_H_
+#define ARROW_STATUS_H_
+
+#include <cstdint>
+#include <cstring>
+#include <string>
+
+// 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);    \
+  } while (0);
+
+// If 'to_call' returns a bad status, CHECK immediately with a logged message
+// of 'msg' followed by the status.
+#define ARROW_CHECK_OK_PREPEND(to_call, msg) do {         \
+::arrow::Status _s = (to_call);                           \
+ARROW_CHECK(_s.ok()) << (msg) << ": " << _s.ToString();   \
+} while (0);
+
+// If the status is bad, CHECK immediately, appending the status to the
+// logged message.
+#define ARROW_CHECK_OK(s) ARROW_CHECK_OK_PREPEND(s, "Bad status")
+
+namespace arrow {
+
+#define RETURN_NOT_OK(s) do {                   \
+    Status _s = (s);                            \
+    if (!_s.ok()) return _s;                    \
+  } while (0);
+
+enum class StatusCode: char {
+  OK = 0,
+  OutOfMemory = 1,
+  KeyError = 2,
+  Invalid = 3,
+
+  NotImplemented = 10,
+};
+
+class Status {
+ public:
+  // Create a success status.
+  Status() : state_(NULL) { }
+  ~Status() { delete[] state_; }
+
+  // Copy the specified status.
+  Status(const Status& s);
+  void operator=(const Status& s);
+
+  // Return a success status.
+  static Status OK() { return Status(); }
+
+  // Return error status of an appropriate type.
+  static Status OutOfMemory(const std::string& msg, int16_t posix_code = -1) {
+    return Status(StatusCode::OutOfMemory, msg, posix_code);
+  }
+
+  static Status KeyError(const std::string& msg) {
+    return Status(StatusCode::KeyError, msg, -1);
+  }
+
+  static Status NotImplemented(const std::string& msg) {
+    return Status(StatusCode::NotImplemented, msg, -1);
+  }
+
+  static Status Invalid(const std::string& msg) {
+    return Status(StatusCode::Invalid, msg, -1);
+  }
+
+  // Returns true iff the status indicates success.
+  bool ok() const { return (state_ == NULL); }
+
+  bool IsOutOfMemory() const { return code() == StatusCode::OutOfMemory; }
+  bool IsKeyError() const { return code() == StatusCode::KeyError; }
+  bool IsInvalid() const { return code() == StatusCode::Invalid; }
+
+  // Return a string representation of this status suitable for printing.
+  // Returns the string "OK" for success.
+  std::string ToString() const;
+
+  // Return a string representation of the status code, without the message
+  // text or posix code information.
+  std::string CodeAsString() const;
+
+  // Get the POSIX code associated with this Status, or -1 if there is none.
+  int16_t posix_code() const;
+
+ private:
+  // OK status has a NULL state_.  Otherwise, state_ is a new[] array
+  // of the following form:
+  //    state_[0..3] == length of message
+  //    state_[4]    == code
+  //    state_[5..6] == posix_code
+  //    state_[7..]  == message
+  const char* state_;
+
+  StatusCode code() const {
+    return ((state_ == NULL) ?
+        StatusCode::OK : static_cast<StatusCode>(state_[4]));
+  }
+
+  Status(StatusCode code, const std::string& msg, int16_t posix_code);
+  static const char* CopyState(const char* s);
+};
+
+inline Status::Status(const Status& s) {
+  state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+}
+
+inline void Status::operator=(const Status& s) {
+  // The following condition catches both aliasing (when this == &s),
+  // and the common case where both s and *this are ok.
+  if (state_ != s.state_) {
+    delete[] state_;
+    state_ = (s.state_ == NULL) ? NULL : CopyState(s.state_);
+  }
+}
+
+}  // namespace arrow
+
+
+#endif // ARROW_STATUS_H_

http://git-wip-us.apache.org/repos/asf/arrow/blob/23c4b08d/cpp/src/arrow/util/test_main.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/test_main.cc b/cpp/src/arrow/util/test_main.cc
new file mode 100644
index 0000000..00139f3
--- /dev/null
+++ b/cpp/src/arrow/util/test_main.cc
@@ -0,0 +1,26 @@
+// 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 <gtest/gtest.h>
+
+int main(int argc, char **argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+
+  int ret = RUN_ALL_TESTS();
+
+  return ret;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/23c4b08d/cpp/thirdparty/build_thirdparty.sh
----------------------------------------------------------------------
diff --git a/cpp/thirdparty/build_thirdparty.sh b/cpp/thirdparty/build_thirdparty.sh
new file mode 100755
index 0000000..46794de
--- /dev/null
+++ b/cpp/thirdparty/build_thirdparty.sh
@@ -0,0 +1,62 @@
+#!/bin/bash
+
+set -x
+set -e
+TP_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd)
+
+source $TP_DIR/versions.sh
+PREFIX=$TP_DIR/installed
+
+################################################################################
+
+if [ "$#" = "0" ]; then
+  F_ALL=1
+else
+  # Allow passing specific libs to build on the command line
+  for arg in "$*"; do
+    case $arg in
+      "gtest")      F_GTEST=1 ;;
+      *)            echo "Unknown module: $arg"; exit 1 ;;
+    esac
+  done
+fi
+
+################################################################################
+
+# Determine how many parallel jobs to use for make based on the number of cores
+if [[ "$OSTYPE" =~ ^linux ]]; then
+  PARALLEL=$(grep -c processor /proc/cpuinfo)
+elif [[ "$OSTYPE" == "darwin"* ]]; then
+  PARALLEL=$(sysctl -n hw.ncpu)
+else
+  echo Unsupported platform $OSTYPE
+  exit 1
+fi
+
+mkdir -p "$PREFIX/include"
+mkdir -p "$PREFIX/lib"
+
+# On some systems, autotools installs libraries to lib64 rather than lib.  Fix
+# this by setting up lib64 as a symlink to lib.  We have to do this step first
+# to handle cases where one third-party library depends on another.
+ln -sf lib "$PREFIX/lib64"
+
+# use the compiled tools
+export PATH=$PREFIX/bin:$PATH
+
+
+# build googletest
+if [ -n "$F_ALL" -o -n "$F_GTEST" ]; then
+  cd $TP_DIR/$GTEST_BASEDIR
+
+  if [[ "$OSTYPE" == "darwin"* ]]; then
+    CXXFLAGS=-fPIC cmake -DCMAKE_CXX_FLAGS="-std=c++11 -stdlib=libc++ -DGTEST_USE_OWN_TR1_TUPLE=1 -Wno-unused-value -Wno-ignored-attributes"
+  else
+    CXXFLAGS=-fPIC cmake .
+  fi
+
+  make VERBOSE=1
+fi
+
+echo "---------------------"
+echo "Thirdparty dependencies built and installed into $PREFIX successfully"

http://git-wip-us.apache.org/repos/asf/arrow/blob/23c4b08d/cpp/thirdparty/download_thirdparty.sh
----------------------------------------------------------------------
diff --git a/cpp/thirdparty/download_thirdparty.sh b/cpp/thirdparty/download_thirdparty.sh
new file mode 100755
index 0000000..8ffb22a
--- /dev/null
+++ b/cpp/thirdparty/download_thirdparty.sh
@@ -0,0 +1,20 @@
+#!/bin/bash
+
+set -x
+set -e
+
+TP_DIR=$(cd "$(dirname "$BASH_SOURCE")"; pwd)
+
+source $TP_DIR/versions.sh
+
+download_extract_and_cleanup() {
+	filename=$TP_DIR/$(basename "$1")
+	curl -#LC - "$1" -o $filename
+	tar xzf $filename -C $TP_DIR
+	rm $filename
+}
+
+if [ ! -d ${GTEST_BASEDIR} ]; then
+  echo "Fetching gtest"
+  download_extract_and_cleanup $GTEST_URL
+fi

http://git-wip-us.apache.org/repos/asf/arrow/blob/23c4b08d/cpp/thirdparty/versions.sh
----------------------------------------------------------------------
diff --git a/cpp/thirdparty/versions.sh b/cpp/thirdparty/versions.sh
new file mode 100755
index 0000000..12ad56e
--- /dev/null
+++ b/cpp/thirdparty/versions.sh
@@ -0,0 +1,3 @@
+GTEST_VERSION=1.7.0
+GTEST_URL="https://github.com/google/googletest/archive/release-${GTEST_VERSION}.tar.gz"
+GTEST_BASEDIR=googletest-release-$GTEST_VERSION