You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ko...@apache.org on 2018/07/09 01:56:42 UTC

[arrow] branch master updated: ARROW-1722: [C++] Add linting script to find C++/CLI incompatibilities

This is an automated email from the ASF dual-hosted git repository.

kou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new fbfddfd  ARROW-1722: [C++] Add linting script to find C++/CLI incompatibilities
fbfddfd is described below

commit fbfddfd4b4ba869d83e0e51b17b0166e61caf285
Author: Wes McKinney <we...@apache.org>
AuthorDate: Mon Jul 9 10:56:17 2018 +0900

    ARROW-1722: [C++] Add linting script to find C++/CLI incompatibilities
    
    This also fixes some issues that the script found
    
    Author: Wes McKinney <we...@apache.org>
    
    Closes #2225 from wesm/ARROW-1722 and squashes the following commits:
    
    3b5a16a2 [Wes McKinney] Remove print statements
    824ef0b8 [Wes McKinney] Location of generated Plasma flatbuffers files changed, did not realize they are being installed
    43c2cce4 [Wes McKinney] Fix bug in lint script, fix other C++/CLI issues
    8d94d7fe [Wes McKinney] Fix cpplint error
    7adcc27e [Wes McKinney] Work around confounding toolchain variables in travis_lint.sh
    ceb2dd78 [Wes McKinney] Use common env variables in travis_lint.sh
    11635d31 [Wes McKinney] Add lint_cpp_cli.py script, fix issues found
---
 ci/travis_lint.sh                         |  18 +++--
 cpp/build-support/bootstrap_toolchain.py  | 114 ------------------------------
 cpp/build-support/lint_cpp_cli.py         |  79 +++++++++++++++++++++
 cpp/src/arrow/compute/kernel.h            |   4 +-
 cpp/src/arrow/ipc/test-common.h           |   4 +-
 cpp/src/arrow/memory_pool.cc              |  77 ++++++++++++++------
 cpp/src/arrow/memory_pool.h               |  10 ++-
 cpp/src/arrow/public-api-test.cc          |   6 ++
 cpp/src/arrow/python/common.h             |   4 +-
 cpp/src/arrow/python/helpers.h            |   6 +-
 cpp/src/arrow/python/numpy-internal.h     |   2 +-
 cpp/src/arrow/util/thread-pool.cc         |   4 ++
 cpp/src/arrow/util/thread-pool.h          |   4 --
 cpp/src/arrow/util/variant.h              |  30 ++++----
 cpp/src/arrow/util/variant/variant_cast.h |  12 ++--
 cpp/src/plasma/CMakeLists.txt             |   6 +-
 16 files changed, 194 insertions(+), 186 deletions(-)

diff --git a/ci/travis_lint.sh b/ci/travis_lint.sh
index 7cdbad4..7e2b05d 100755
--- a/ci/travis_lint.sh
+++ b/ci/travis_lint.sh
@@ -19,11 +19,15 @@
 
 set -ex
 
+# Disable toolchain variables in this script
+export ARROW_TRAVIS_USE_TOOLCHAIN=0
+source $TRAVIS_BUILD_DIR/ci/travis_env_common.sh
+
 # Fail fast for code linting issues
 
 if [ "$ARROW_CI_CPP_AFFECTED" != "0" ]; then
-  mkdir $TRAVIS_BUILD_DIR/cpp/lint
-  pushd $TRAVIS_BUILD_DIR/cpp/lint
+  mkdir $ARROW_CPP_DIR/lint
+  pushd $ARROW_CPP_DIR/lint
 
   cmake ..
   make lint
@@ -32,6 +36,8 @@ if [ "$ARROW_CI_CPP_AFFECTED" != "0" ]; then
     make check-format
   fi
 
+  python $ARROW_CPP_DIR/build-support/lint_cpp_cli.py $ARROW_CPP_DIR/src
+
   popd
 fi
 
@@ -41,11 +47,9 @@ fi
 if [ "$ARROW_CI_PYTHON_AFFECTED" != "0" ]; then
   sudo pip install -q flake8
 
-  PYTHON_DIR=$TRAVIS_BUILD_DIR/python
-
-  flake8 --count $PYTHON_DIR
+  flake8 --count $ARROW_PYTHON_DIR
 
   # Check Cython files with some checks turned off
-  flake8 --count --config=$PYTHON_DIR/.flake8.cython \
-         $PYTHON_DIR
+  flake8 --count --config=$ARROW_PYTHON_DIR/.flake8.cython \
+         $ARROW_PYTHON_DIR
 fi
diff --git a/cpp/build-support/bootstrap_toolchain.py b/cpp/build-support/bootstrap_toolchain.py
deleted file mode 100755
index 128be78..0000000
--- a/cpp/build-support/bootstrap_toolchain.py
+++ /dev/null
@@ -1,114 +0,0 @@
-#!/usr/bin/env python
-# Copyright (c) 2015, Cloudera, inc.
-# Confidential Cloudera Information: Covered by NDA.
-# 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.
-
-# Bootstrapping the native toolchain with prebuilt binaries
-#
-# The purpose of this script is to download prebuilt artifacts of the native toolchain to
-# satisfy the third-party dependencies. The script checks for the presence of
-# NATIVE_TOOLCHAIN. NATIVE_TOOLCHAIN indicates the location where the prebuilt artifacts
-# should be extracted to.
-#
-# The script is called as follows without any additional parameters:
-#
-#     python bootstrap_toolchain.py
-import sh
-import os
-import sys
-import re
-
-HOST = "https://native-toolchain.s3.amazonaws.com/build"
-
-OS_MAPPING = {
-  "centos6" : "ec2-package-centos-6",
-  "centos5" : "ec2-package-centos-5",
-  "centos7" : "ec2-package-centos-7",
-  "debian6" : "ec2-package-debian-6",
-  "debian7" : "ec2-package-debian-7",
-  "suselinux11": "ec2-package-sles-11",
-  "ubuntu12.04" : "ec2-package-ubuntu-12-04",
-  "ubuntu14.04" : "ec2-package-ubuntu-14-04"
-}
-
-def get_release_label():
-  """Gets the right package label from the OS version"""
-  release = "".join(map(lambda x: x.lower(), sh.lsb_release("-irs").split()))
-  for k, v in OS_MAPPING.iteritems():
-    if re.search(k, release):
-      return v
-
-  print("Pre-built toolchain archives not available for your platform.")
-  print("Clone and build native toolchain from source using this repository:")
-  print("    https://github.com/cloudera/native-toolchain")
-  raise Exception("Could not find package label for OS version: {0}.".format(release))
-
-def download_package(destination, product, version, compiler):
-  label = get_release_label()
-  file_name = "{0}-{1}-{2}-{3}.tar.gz".format(product, version, compiler, label)
-  url_path="/{0}/{1}-{2}/{0}-{1}-{2}-{3}.tar.gz".format(product, version, compiler, label)
-  download_path = HOST + url_path
-
-  print "URL {0}".format(download_path)
-  print "Downloading {0} to {1}".format(file_name, destination)
-  # --no-clobber avoids downloading the file if a file with the name already exists
-  sh.wget(download_path, directory_prefix=destination, no_clobber=True)
-  print "Extracting {0}".format(file_name)
-  sh.tar(z=True, x=True, f=os.path.join(destination, file_name), directory=destination)
-  sh.rm(os.path.join(destination, file_name))
-
-
-def bootstrap(packages):
-  """Validates the presence of $NATIVE_TOOLCHAIN in the environment. By checking
-  $NATIVE_TOOLCHAIN is present, we assume that {LIB}_VERSION will be present as well. Will
-  create the directory specified by $NATIVE_TOOLCHAIN if it does not yet exist. Each of
-  the packages specified in `packages` is downloaded and extracted into $NATIVE_TOOLCHAIN.
-  """
-  # Create the destination directory if necessary
-  destination = os.getenv("NATIVE_TOOLCHAIN")
-  if not destination:
-    print("Build environment not set up correctly, make sure "
-          "$NATIVE_TOOLCHAIN is present.")
-    sys.exit(1)
-
-  if not os.path.exists(destination):
-    os.makedirs(destination)
-
-  # Detect the compiler
-  if "SYSTEM_GCC" in os.environ:
-    compiler = "gcc-system"
-  else:
-    compiler = "gcc-{0}".format(os.environ["GCC_VERSION"])
-
-  for p in packages:
-    pkg_name, pkg_version = unpack_name_and_version(p)
-    download_package(destination, pkg_name, pkg_version, compiler)
-
-def unpack_name_and_version(package):
-  """A package definition is either a string where the version is fetched from the
-  environment or a tuple where the package name and the package version are fully
-  specified.
-  """
-  if isinstance(package, basestring):
-    env_var = "{0}_VERSION".format(package).replace("-", "_").upper()
-    try:
-      return package, os.environ[env_var]
-    except KeyError:
-      raise Exception("Could not find version for {0} in environment var {1}".format(
-        package, env_var))
-  return package[0], package[1]
-
-if __name__ == "__main__":
-  packages = [("gcc","4.9.2"), ("gflags", "2.0"), ("glog", "0.3.3-p1"),
-              ("gperftools", "2.3"), ("libunwind", "1.1"), ("googletest", "20151222")]
-  bootstrap(packages)
diff --git a/cpp/build-support/lint_cpp_cli.py b/cpp/build-support/lint_cpp_cli.py
new file mode 100644
index 0000000..78f3eea
--- /dev/null
+++ b/cpp/build-support/lint_cpp_cli.py
@@ -0,0 +1,79 @@
+#!/usr/bin/env python
+# 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.
+
+import argparse
+import re
+import os
+
+parser = argparse.ArgumentParser(
+    description="Check for illegal headers for C++/CLI applications")
+parser.add_argument("source_path",
+                    help="Path to source code")
+arguments = parser.parse_args()
+
+
+_STRIP_COMMENT_REGEX = re.compile('(.+)?(?=//)')
+
+
+def _strip_comments(line):
+    m = _STRIP_COMMENT_REGEX.match(line)
+    if not m:
+        return line
+    else:
+        return m.group(0)
+
+
+def lint_file(path):
+    fail_rules = [
+        (lambda x: '<mutex>' in x, 'Uses <mutex>'),
+        (lambda x: 'nullptr' in x, 'Uses nullptr')
+    ]
+
+    with open(path) as f:
+        for i, line in enumerate(f):
+            stripped_line = _strip_comments(line)
+            for rule, why in fail_rules:
+                if rule(stripped_line):
+                    raise Exception('File {0} failed C++/CLI lint check: {1}\n'
+                                    'Line {2}: {3}'
+                                    .format(path, why, i + 1, line))
+
+
+EXCLUSIONS = [
+    'arrow/util/macros.h',
+    'arrow/util/parallel.h',
+    'arrow/io/hdfs-internal.h'
+]
+
+
+for dirpath, _, filenames in os.walk(arguments.source_path):
+    for filename in filenames:
+        full_path = os.path.join(dirpath, filename)
+
+        exclude = False
+        for exclusion in EXCLUSIONS:
+            if exclusion in full_path:
+                exclude = True
+                break
+
+        if exclude:
+            continue
+
+        # Only run on header files
+        if filename.endswith('.h'):
+            lint_file(full_path)
diff --git a/cpp/src/arrow/compute/kernel.h b/cpp/src/arrow/compute/kernel.h
index 0bfa55c..3691f4a 100644
--- a/cpp/src/arrow/compute/kernel.h
+++ b/cpp/src/arrow/compute/kernel.h
@@ -58,7 +58,7 @@ struct ARROW_EXPORT Datum {
       value;
 
   /// \brief Empty datum, to be populated elsewhere
-  Datum() : value(nullptr) {}
+  Datum() : value(NULLPTR) {}
 
   explicit Datum(const std::shared_ptr<Scalar>& value) : value(value) {}
 
@@ -124,7 +124,7 @@ struct ARROW_EXPORT Datum {
     } else if (this->kind() == Datum::CHUNKED_ARRAY) {
       return util::get<std::shared_ptr<ChunkedArray>>(this->value)->type();
     }
-    return nullptr;
+    return NULLPTR;
   }
 };
 
diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h
index c914be6..87d826f 100644
--- a/cpp/src/arrow/ipc/test-common.h
+++ b/cpp/src/arrow/ipc/test-common.h
@@ -171,7 +171,7 @@ Status MakeRandomBooleanArray(const int length, bool include_nulls,
     test::random_null_bytes(length, 0.1, valid_bytes.data());
     *out = std::make_shared<BooleanArray>(length, data, null_bitmap, -1);
   } else {
-    *out = std::make_shared<BooleanArray>(length, data, nullptr, 0);
+    *out = std::make_shared<BooleanArray>(length, data, NULLPTR, 0);
   }
   return Status::OK();
 }
@@ -461,7 +461,7 @@ Status MakeUnion(std::shared_ptr<RecordBatch>* out) {
   auto sparse_no_nulls =
       std::make_shared<UnionArray>(sparse_type, length, sparse_children, type_ids_buffer);
   auto sparse = std::make_shared<UnionArray>(sparse_type, length, sparse_children,
-                                             type_ids_buffer, nullptr, null_bitmask, 1);
+                                             type_ids_buffer, NULLPTR, null_bitmask, 1);
 
   auto dense =
       std::make_shared<UnionArray>(dense_type, length, dense_children, type_ids_buffer,
diff --git a/cpp/src/arrow/memory_pool.cc b/cpp/src/arrow/memory_pool.cc
index 34bd600..a92bfbf 100644
--- a/cpp/src/arrow/memory_pool.cc
+++ b/cpp/src/arrow/memory_pool.cc
@@ -18,10 +18,12 @@
 #include "arrow/memory_pool.h"
 
 #include <algorithm>
+#include <atomic>
 #include <cerrno>
 #include <cstdlib>
 #include <cstring>
 #include <iostream>
+#include <memory>
 #include <mutex>
 #include <sstream>  // IWYU pragma: keep
 
@@ -202,39 +204,70 @@ int64_t LoggingMemoryPool::max_memory() const {
   return mem;
 }
 
-ProxyMemoryPool::ProxyMemoryPool(MemoryPool* pool) : pool_(pool) {}
+class ProxyMemoryPool::ProxyMemoryPoolImpl {
+ public:
+  explicit ProxyMemoryPoolImpl(MemoryPool* pool) : pool_(pool) {}
 
-Status ProxyMemoryPool::Allocate(int64_t size, uint8_t** out) {
-  RETURN_NOT_OK(pool_->Allocate(size, out));
-  bytes_allocated_ += size;
-  {
-    std::lock_guard<std::mutex> guard(lock_);
-    if (bytes_allocated_ > max_memory_) {
-      max_memory_ = bytes_allocated_.load();
+  Status Allocate(int64_t size, uint8_t** out) {
+    RETURN_NOT_OK(pool_->Allocate(size, out));
+    bytes_allocated_ += size;
+    {
+      std::lock_guard<std::mutex> guard(lock_);
+      if (bytes_allocated_ > max_memory_) {
+        max_memory_ = bytes_allocated_.load();
+      }
     }
+    return Status::OK();
   }
-  return Status::OK();
-}
 
-Status ProxyMemoryPool::Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) {
-  RETURN_NOT_OK(pool_->Reallocate(old_size, new_size, ptr));
-  bytes_allocated_ += new_size - old_size;
-  {
-    std::lock_guard<std::mutex> guard(lock_);
-    if (bytes_allocated_ > max_memory_) {
-      max_memory_ = bytes_allocated_.load();
+  Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) {
+    RETURN_NOT_OK(pool_->Reallocate(old_size, new_size, ptr));
+    bytes_allocated_ += new_size - old_size;
+    {
+      std::lock_guard<std::mutex> guard(lock_);
+      if (bytes_allocated_ > max_memory_) {
+        max_memory_ = bytes_allocated_.load();
+      }
     }
+    return Status::OK();
   }
-  return Status::OK();
+
+  void Free(uint8_t* buffer, int64_t size) {
+    pool_->Free(buffer, size);
+    bytes_allocated_ -= size;
+  }
+
+  int64_t bytes_allocated() const { return bytes_allocated_.load(); }
+
+  int64_t max_memory() const { return max_memory_.load(); }
+
+ private:
+  mutable std::mutex lock_;
+  MemoryPool* pool_;
+  std::atomic<int64_t> bytes_allocated_{0};
+  std::atomic<int64_t> max_memory_{0};
+};
+
+ProxyMemoryPool::ProxyMemoryPool(MemoryPool* pool) {
+  impl_.reset(new ProxyMemoryPoolImpl(pool));
+}
+
+ProxyMemoryPool::~ProxyMemoryPool() {}
+
+Status ProxyMemoryPool::Allocate(int64_t size, uint8_t** out) {
+  return impl_->Allocate(size, out);
+}
+
+Status ProxyMemoryPool::Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) {
+  return impl_->Reallocate(old_size, new_size, ptr);
 }
 
 void ProxyMemoryPool::Free(uint8_t* buffer, int64_t size) {
-  pool_->Free(buffer, size);
-  bytes_allocated_ -= size;
+  return impl_->Free(buffer, size);
 }
 
-int64_t ProxyMemoryPool::bytes_allocated() const { return bytes_allocated_.load(); }
+int64_t ProxyMemoryPool::bytes_allocated() const { return impl_->bytes_allocated(); }
 
-int64_t ProxyMemoryPool::max_memory() const { return max_memory_.load(); }
+int64_t ProxyMemoryPool::max_memory() const { return impl_->max_memory(); }
 
 }  // namespace arrow
diff --git a/cpp/src/arrow/memory_pool.h b/cpp/src/arrow/memory_pool.h
index de58896..0ee46a3 100644
--- a/cpp/src/arrow/memory_pool.h
+++ b/cpp/src/arrow/memory_pool.h
@@ -18,9 +18,8 @@
 #ifndef ARROW_MEMORY_POOL_H
 #define ARROW_MEMORY_POOL_H
 
-#include <atomic>
 #include <cstdint>
-#include <mutex>
+#include <memory>
 
 #include "arrow/util/visibility.h"
 
@@ -94,6 +93,7 @@ class ARROW_EXPORT LoggingMemoryPool : public MemoryPool {
 class ARROW_EXPORT ProxyMemoryPool : public MemoryPool {
  public:
   explicit ProxyMemoryPool(MemoryPool* pool);
+  ~ProxyMemoryPool() override;
 
   Status Allocate(int64_t size, uint8_t** out) override;
   Status Reallocate(int64_t old_size, int64_t new_size, uint8_t** ptr) override;
@@ -105,10 +105,8 @@ class ARROW_EXPORT ProxyMemoryPool : public MemoryPool {
   int64_t max_memory() const override;
 
  private:
-  mutable std::mutex lock_;
-  MemoryPool* pool_;
-  std::atomic<int64_t> bytes_allocated_{0};
-  std::atomic<int64_t> max_memory_{0};
+  class ProxyMemoryPoolImpl;
+  std::unique_ptr<ProxyMemoryPoolImpl> impl_;
 };
 
 ARROW_EXPORT MemoryPool* default_memory_pool();
diff --git a/cpp/src/arrow/public-api-test.cc b/cpp/src/arrow/public-api-test.cc
index 8298d74..088da3f 100644
--- a/cpp/src/arrow/public-api-test.cc
+++ b/cpp/src/arrow/public-api-test.cc
@@ -16,11 +16,17 @@
 // under the License.
 
 #include "arrow/api.h"
+#include "arrow/io/api.h"
+#include "arrow/ipc/api.h"
 
 #ifdef DCHECK
 #error "DCHECK should not be visible from Arrow public headers."
 #endif
 
+#ifdef ARROW_UTIL_PARALLEL_H
+#error "arrow/util/parallel.h is an internal header"
+#endif
+
 #include <gtest/gtest.h>
 
 TEST(_, _) {}
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index 6eecd59..cd54e5a 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -143,7 +143,7 @@ struct ARROW_EXPORT PyBytesView {
   const char* bytes;
   Py_ssize_t size;
 
-  PyBytesView() : bytes(nullptr), size(0), ref(nullptr) {}
+  PyBytesView() : bytes(NULLPTR), size(0), ref(NULLPTR) {}
 
   // View the given Python object as binary-like, i.e. bytes
   Status FromBinary(PyObject* obj) { return FromBinary(obj, "a bytes object"); }
@@ -180,7 +180,7 @@ struct ARROW_EXPORT PyBytesView {
   }
 
  protected:
-  PyBytesView(const char* b, Py_ssize_t s, PyObject* obj = nullptr)
+  PyBytesView(const char* b, Py_ssize_t s, PyObject* obj = NULLPTR)
       : bytes(b), size(s), ref(obj) {}
 
   Status FromBinary(PyObject* obj, const char* expected_msg) {
diff --git a/cpp/src/arrow/python/helpers.h b/cpp/src/arrow/python/helpers.h
index 195d5fb..ea3cc6a 100644
--- a/cpp/src/arrow/python/helpers.h
+++ b/cpp/src/arrow/python/helpers.h
@@ -112,10 +112,10 @@ inline Status CastSize(Py_ssize_t size, int32_t* out,
 }
 
 Status BuilderAppend(StringBuilder* builder, PyObject* obj, bool check_valid = false,
-                     bool* is_full = nullptr);
-Status BuilderAppend(BinaryBuilder* builder, PyObject* obj, bool* is_full = nullptr);
+                     bool* is_full = NULLPTR);
+Status BuilderAppend(BinaryBuilder* builder, PyObject* obj, bool* is_full = NULLPTR);
 Status BuilderAppend(FixedSizeBinaryBuilder* builder, PyObject* obj,
-                     bool* is_full = nullptr);
+                     bool* is_full = NULLPTR);
 
 }  // namespace internal
 }  // namespace py
diff --git a/cpp/src/arrow/python/numpy-internal.h b/cpp/src/arrow/python/numpy-internal.h
index 12319aa..cfaae22 100644
--- a/cpp/src/arrow/python/numpy-internal.h
+++ b/cpp/src/arrow/python/numpy-internal.h
@@ -36,7 +36,7 @@ class Ndarray1DIndexer {
  public:
   typedef int64_t size_type;
 
-  Ndarray1DIndexer() : arr_(nullptr), data_(nullptr) {}
+  Ndarray1DIndexer() : arr_(NULLPTR), data_(NULLPTR) {}
 
   explicit Ndarray1DIndexer(PyArrayObject* arr) : Ndarray1DIndexer() { Init(arr); }
 
diff --git a/cpp/src/arrow/util/thread-pool.cc b/cpp/src/arrow/util/thread-pool.cc
index 997ff5d..90fa4b7 100644
--- a/cpp/src/arrow/util/thread-pool.cc
+++ b/cpp/src/arrow/util/thread-pool.cc
@@ -20,7 +20,11 @@
 #include "arrow/util/logging.h"
 
 #include <algorithm>
+#include <condition_variable>
+#include <deque>
+#include <mutex>
 #include <string>
+#include <thread>
 
 namespace arrow {
 namespace internal {
diff --git a/cpp/src/arrow/util/thread-pool.h b/cpp/src/arrow/util/thread-pool.h
index c1b10f7..20b6b2e 100644
--- a/cpp/src/arrow/util/thread-pool.h
+++ b/cpp/src/arrow/util/thread-pool.h
@@ -18,15 +18,11 @@
 #ifndef ARROW_UTIL_THREAD_POOL_H
 #define ARROW_UTIL_THREAD_POOL_H
 
-#include <condition_variable>
-#include <deque>
 #include <exception>
 #include <functional>
 #include <future>
 #include <list>
 #include <memory>
-#include <mutex>
-#include <thread>
 #include <type_traits>
 #include <utility>
 #include <vector>
diff --git a/cpp/src/arrow/util/variant.h b/cpp/src/arrow/util/variant.h
index 3fcd18b..1aa9aa3 100644
--- a/cpp/src/arrow/util/variant.h
+++ b/cpp/src/arrow/util/variant.h
@@ -681,14 +681,14 @@ public:
     }
 
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE bool is() const
     {
         return type_index == detail::direct_type<T, Types...>::index;
     }
 
     template <typename T,typename std::enable_if<
-                         (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                         (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE bool is() const
     {
         return type_index == detail::direct_type<recursive_wrapper<T>, Types...>::index;
@@ -710,7 +710,7 @@ public:
 
     // get_unchecked<T>()
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get_unchecked()
     {
         return *reinterpret_cast<T*>(&data);
@@ -719,7 +719,7 @@ public:
 #ifdef HAS_EXCEPTIONS
     // get<T>()
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get()
     {
         if (type_index == detail::direct_type<T, Types...>::index)
@@ -734,7 +734,7 @@ public:
 #endif
 
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get_unchecked() const
     {
         return *reinterpret_cast<T const*>(&data);
@@ -742,7 +742,7 @@ public:
 
 #ifdef HAS_EXCEPTIONS
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get() const
     {
         if (type_index == detail::direct_type<T, Types...>::index)
@@ -758,7 +758,7 @@ public:
 
     // get_unchecked<T>() - T stored as recursive_wrapper<T>
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get_unchecked()
     {
         return (*reinterpret_cast<recursive_wrapper<T>*>(&data)).get();
@@ -767,7 +767,7 @@ public:
 #ifdef HAS_EXCEPTIONS
     // get<T>() - T stored as recursive_wrapper<T>
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get()
     {
         if (type_index == detail::direct_type<recursive_wrapper<T>, Types...>::index)
@@ -782,7 +782,7 @@ public:
 #endif
 
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get_unchecked() const
     {
         return (*reinterpret_cast<recursive_wrapper<T> const*>(&data)).get();
@@ -790,7 +790,7 @@ public:
 
 #ifdef HAS_EXCEPTIONS
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<recursive_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get() const
     {
         if (type_index == detail::direct_type<recursive_wrapper<T>, Types...>::index)
@@ -806,7 +806,7 @@ public:
 
     // get_unchecked<T>() - T stored as std::reference_wrapper<T>
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<std::reference_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<std::reference_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get_unchecked()
     {
         return (*reinterpret_cast<std::reference_wrapper<T>*>(&data)).get();
@@ -815,7 +815,7 @@ public:
 #ifdef HAS_EXCEPTIONS
     // get<T>() - T stored as std::reference_wrapper<T>
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<std::reference_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<std::reference_wrapper<T>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T& get()
     {
         if (type_index == detail::direct_type<std::reference_wrapper<T>, Types...>::index)
@@ -830,7 +830,7 @@ public:
 #endif
 
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<std::reference_wrapper<T const>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<std::reference_wrapper<T const>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get_unchecked() const
     {
         return (*reinterpret_cast<std::reference_wrapper<T const> const*>(&data)).get();
@@ -838,7 +838,7 @@ public:
 
 #ifdef HAS_EXCEPTIONS
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<std::reference_wrapper<T const>, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<std::reference_wrapper<T const>, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE T const& get() const
     {
         if (type_index == detail::direct_type<std::reference_wrapper<T const>, Types...>::index)
@@ -866,7 +866,7 @@ public:
     }
 
     template <typename T, typename std::enable_if<
-                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = nullptr>
+                          (detail::direct_type<T, Types...>::index != detail::invalid_value)>::type* = NULLPTR>
     VARIANT_INLINE static constexpr int which() noexcept
     {
         return static_cast<int>(sizeof...(Types)-detail::direct_type<T, Types...>::index - 1);
diff --git a/cpp/src/arrow/util/variant/variant_cast.h b/cpp/src/arrow/util/variant/variant_cast.h
index 558f1d9..71ae80b 100644
--- a/cpp/src/arrow/util/variant/variant_cast.h
+++ b/cpp/src/arrow/util/variant/variant_cast.h
@@ -29,6 +29,8 @@
 
 #include <type_traits>
 
+#include "arrow/util/macros.h"
+
 namespace arrow {
 namespace util {
 
@@ -51,12 +53,12 @@ class dynamic_caster
 public:
     using result_type = T&;
     template <class V>
-    T& operator()(V& v, typename std::enable_if<!std::is_polymorphic<V>::value>::type* = nullptr) const
+    T& operator()(V& v, typename std::enable_if<!std::is_polymorphic<V>::value>::type* = NULLPTR) const
     {
         throw std::bad_cast();
     }
     template <class V>
-    T& operator()(V& v, typename std::enable_if<std::is_polymorphic<V>::value>::type* = nullptr) const
+    T& operator()(V& v, typename std::enable_if<std::is_polymorphic<V>::value>::type* = NULLPTR) const
     {
         return dynamic_cast<T&>(v);
     }
@@ -68,12 +70,12 @@ class dynamic_caster<T*>
 public:
     using result_type = T*;
     template <class V>
-    T* operator()(V& v, typename std::enable_if<!std::is_polymorphic<V>::value>::type* = nullptr) const
+    T* operator()(V& v, typename std::enable_if<!std::is_polymorphic<V>::value>::type* = NULLPTR) const
     {
-        return nullptr;
+        return NULLPTR;
     }
     template <class V>
-    T* operator()(V& v, typename std::enable_if<std::is_polymorphic<V>::value>::type* = nullptr) const
+    T* operator()(V& v, typename std::enable_if<std::is_polymorphic<V>::value>::type* = NULLPTR) const
     {
         return dynamic_cast<T*>(&v);
     }
diff --git a/cpp/src/plasma/CMakeLists.txt b/cpp/src/plasma/CMakeLists.txt
index 744f9ad..aa4d2b0 100644
--- a/cpp/src/plasma/CMakeLists.txt
+++ b/cpp/src/plasma/CMakeLists.txt
@@ -39,7 +39,7 @@ set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-conversion")
 # Compile flatbuffers
 
 set(PLASMA_FBS_SRC "${CMAKE_CURRENT_LIST_DIR}/format/plasma.fbs" "${CMAKE_CURRENT_LIST_DIR}/format/common.fbs")
-set(OUTPUT_DIR ${CMAKE_CURRENT_LIST_DIR}/)
+set(OUTPUT_DIR ${CMAKE_BINARY_DIR}/src/plasma)
 
 set(PLASMA_FBS_OUTPUT_FILES
   "${OUTPUT_DIR}/common_generated.h"
@@ -140,12 +140,12 @@ endif()
 # Headers: top level
 install(FILES
   common.h
-  common_generated.h
+  "${OUTPUT_DIR}/common_generated.h"
   compat.h
   client.h
   events.h
   plasma.h
-  plasma_generated.h
+  "${OUTPUT_DIR}/plasma_generated.h"
   protocol.h
   DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/plasma")