[ 
https://issues.apache.org/jira/browse/ARROW-1134?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16218588#comment-16218588
 ] 

ASF GitHub Bot commented on ARROW-1134:
---------------------------------------

wesm closed pull request #1228: ARROW-1134: [C++] Support for C++/CLI 
compilation, add NULLPTR define to avoid using nullptr in public headers
URL: https://github.com/apache/arrow/pull/1228
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/cpp/src/arrow/allocator.h b/cpp/src/arrow/allocator.h
index e00023dc4..c7780f19e 100644
--- a/cpp/src/arrow/allocator.h
+++ b/cpp/src/arrow/allocator.h
@@ -24,6 +24,7 @@
 
 #include "arrow/memory_pool.h"
 #include "arrow/status.h"
+#include "arrow/util/macros.h"
 
 namespace arrow {
 
@@ -49,13 +50,13 @@ class stl_allocator {
   template <class U>
   stl_allocator(const stl_allocator<U>& rhs) noexcept : pool_(rhs.pool_) {}
 
-  ~stl_allocator() { pool_ = nullptr; }
+  ~stl_allocator() { pool_ = NULLPTR; }
 
   pointer address(reference r) const noexcept { return std::addressof(r); }
 
   const_pointer address(const_reference r) const noexcept { return 
std::addressof(r); }
 
-  pointer allocate(size_type n, const void* /*hint*/ = nullptr) {
+  pointer allocate(size_type n, const void* /*hint*/ = NULLPTR) {
     uint8_t* data;
     Status s = pool_->Allocate(n * sizeof(T), &data);
     if (!s.ok()) throw std::bad_alloc();
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 75dda4a75..a45563aa9 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -183,14 +183,14 @@ class ARROW_EXPORT Array {
 
   /// \brief Return true if value at index is null. Does not boundscheck
   bool IsNull(int64_t i) const {
-    return null_bitmap_data_ != nullptr &&
+    return null_bitmap_data_ != NULLPTR &&
            BitUtil::BitNotSet(null_bitmap_data_, i + data_->offset);
   }
 
   /// \brief Return true if value at index is valid (not null). Does not
   /// boundscheck
   bool IsValid(int64_t i) const {
-    return null_bitmap_data_ != nullptr &&
+    return null_bitmap_data_ != NULLPTR &&
            BitUtil::GetBit(null_bitmap_data_, i + data_->offset);
   }
 
@@ -212,13 +212,13 @@ class ARROW_EXPORT Array {
 
   /// Buffer for the null bitmap.
   ///
-  /// Note that for `null_count == 0`, this can be a `nullptr`.
+  /// Note that for `null_count == 0`, this can be null.
   /// This buffer does not account for any slice offset
   std::shared_ptr<Buffer> null_bitmap() const { return data_->buffers[0]; }
 
   /// Raw pointer to the null bitmap.
   ///
-  /// Note that for `null_count == 0`, this can be a `nullptr`.
+  /// Note that for `null_count == 0`, this can be null.
   /// This buffer does not account for any slice offset
   const uint8_t* null_bitmap_data() const { return null_bitmap_data_; }
 
@@ -270,7 +270,7 @@ class ARROW_EXPORT Array {
     if (data->buffers.size() > 0 && data->buffers[0]) {
       null_bitmap_data_ = data->buffers[0]->data();
     } else {
-      null_bitmap_data_ = nullptr;
+      null_bitmap_data_ = NULLPTR;
     }
     data_ = data;
   }
@@ -299,7 +299,7 @@ class ARROW_EXPORT NullArray : public FlatArray {
 
  private:
   inline void SetData(const std::shared_ptr<ArrayData>& data) {
-    null_bitmap_data_ = nullptr;
+    null_bitmap_data_ = NULLPTR;
     data->null_count = data->length;
     data_ = data;
   }
@@ -310,7 +310,7 @@ class ARROW_EXPORT PrimitiveArray : public FlatArray {
  public:
   PrimitiveArray(const std::shared_ptr<DataType>& type, int64_t length,
                  const std::shared_ptr<Buffer>& data,
-                 const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+                 const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
                  int64_t null_count = 0, int64_t offset = 0);
 
   /// Does not account for any slice offset
@@ -325,7 +325,7 @@ class ARROW_EXPORT PrimitiveArray : public FlatArray {
   inline void SetData(const std::shared_ptr<ArrayData>& data) {
     auto values = data->buffers[1];
     this->Array::SetData(data);
-    raw_values_ = values == nullptr ? nullptr : values->data();
+    raw_values_ = values == NULLPTR ? NULLPTR : values->data();
   }
 
   explicit inline PrimitiveArray(const std::shared_ptr<ArrayData>& data) {
@@ -349,7 +349,7 @@ class ARROW_EXPORT NumericArray : public PrimitiveArray {
   NumericArray(
       typename std::enable_if<TypeTraits<T1>::is_parameter_free, 
int64_t>::type length,
       const std::shared_ptr<Buffer>& data,
-      const std::shared_ptr<Buffer>& null_bitmap = nullptr, int64_t null_count 
= 0,
+      const std::shared_ptr<Buffer>& null_bitmap = NULLPTR, int64_t null_count 
= 0,
       int64_t offset = 0)
       : PrimitiveArray(TypeTraits<T1>::type_singleton(), length, data, 
null_bitmap,
                        null_count, offset) {}
@@ -371,7 +371,7 @@ class ARROW_EXPORT BooleanArray : public PrimitiveArray {
   explicit BooleanArray(const std::shared_ptr<ArrayData>& data);
 
   BooleanArray(int64_t length, const std::shared_ptr<Buffer>& data,
-               const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+               const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
                int64_t null_count = 0, int64_t offset = 0);
 
   bool Value(int64_t i) const {
@@ -395,7 +395,7 @@ class ARROW_EXPORT ListArray : public Array {
   ListArray(const std::shared_ptr<DataType>& type, int64_t length,
             const std::shared_ptr<Buffer>& value_offsets,
             const std::shared_ptr<Array>& values,
-            const std::shared_ptr<Buffer>& null_bitmap = nullptr, int64_t 
null_count = 0,
+            const std::shared_ptr<Buffer>& null_bitmap = NULLPTR, int64_t 
null_count = 0,
             int64_t offset = 0);
 
   /// \brief Construct ListArray from array of offsets and child value array
@@ -448,7 +448,7 @@ class ARROW_EXPORT BinaryArray : public FlatArray {
 
   BinaryArray(int64_t length, const std::shared_ptr<Buffer>& value_offsets,
               const std::shared_ptr<Buffer>& data,
-              const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+              const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
               int64_t null_count = 0, int64_t offset = 0);
 
   // Return the pointer to the given elements bytes
@@ -500,7 +500,7 @@ class ARROW_EXPORT BinaryArray : public FlatArray {
   BinaryArray(const std::shared_ptr<DataType>& type, int64_t length,
               const std::shared_ptr<Buffer>& value_offsets,
               const std::shared_ptr<Buffer>& data,
-              const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+              const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
               int64_t null_count = 0, int64_t offset = 0);
 
   const int32_t* raw_value_offsets_;
@@ -515,7 +515,7 @@ class ARROW_EXPORT StringArray : public BinaryArray {
 
   StringArray(int64_t length, const std::shared_ptr<Buffer>& value_offsets,
               const std::shared_ptr<Buffer>& data,
-              const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+              const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
               int64_t null_count = 0, int64_t offset = 0);
 
   // Construct a std::string
@@ -538,7 +538,7 @@ class ARROW_EXPORT FixedSizeBinaryArray : public 
PrimitiveArray {
 
   FixedSizeBinaryArray(const std::shared_ptr<DataType>& type, int64_t length,
                        const std::shared_ptr<Buffer>& data,
-                       const std::shared_ptr<Buffer>& null_bitmap = nullptr,
+                       const std::shared_ptr<Buffer>& null_bitmap = NULLPTR,
                        int64_t null_count = 0, int64_t offset = 0);
 
   const uint8_t* GetValue(int64_t i) const;
@@ -580,7 +580,7 @@ class ARROW_EXPORT StructArray : public Array {
 
   StructArray(const std::shared_ptr<DataType>& type, int64_t length,
               const std::vector<std::shared_ptr<Array>>& children,
-              std::shared_ptr<Buffer> null_bitmap = nullptr, int64_t 
null_count = 0,
+              std::shared_ptr<Buffer> null_bitmap = NULLPTR, int64_t 
null_count = 0,
               int64_t offset = 0);
 
   // Return a shared pointer in case the requestor desires to share ownership
@@ -605,8 +605,8 @@ class ARROW_EXPORT UnionArray : public Array {
   UnionArray(const std::shared_ptr<DataType>& type, int64_t length,
              const std::vector<std::shared_ptr<Array>>& children,
              const std::shared_ptr<Buffer>& type_ids,
-             const std::shared_ptr<Buffer>& value_offsets = nullptr,
-             const std::shared_ptr<Buffer>& null_bitmap = nullptr, int64_t 
null_count = 0,
+             const std::shared_ptr<Buffer>& value_offsets = NULLPTR,
+             const std::shared_ptr<Buffer>& null_bitmap = NULLPTR, int64_t 
null_count = 0,
              int64_t offset = 0);
 
   /// Note that this buffer does not account for any slice offset
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index b745812b1..8e989064b 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -112,7 +112,7 @@ class ARROW_EXPORT Buffer {
   int64_t size_;
   int64_t capacity_;
 
-  // nullptr by default, but may be set
+  // null by default, but may be set
   std::shared_ptr<Buffer> parent_;
 
  private:
@@ -145,7 +145,7 @@ class ARROW_EXPORT MutableBuffer : public Buffer {
                 const int64_t size);
 
  protected:
-  MutableBuffer() : Buffer(nullptr, 0) {}
+  MutableBuffer() : Buffer(NULLPTR, 0) {}
 };
 
 class ARROW_EXPORT ResizableBuffer : public MutableBuffer {
@@ -180,7 +180,7 @@ class ARROW_EXPORT ResizableBuffer : public MutableBuffer {
 /// A Buffer whose lifetime is tied to a particular MemoryPool
 class ARROW_EXPORT PoolBuffer : public ResizableBuffer {
  public:
-  explicit PoolBuffer(MemoryPool* pool = nullptr);
+  explicit PoolBuffer(MemoryPool* pool = NULLPTR);
   virtual ~PoolBuffer();
 
   Status Resize(const int64_t new_size, bool shrink_to_fit = true) override;
@@ -193,7 +193,7 @@ class ARROW_EXPORT PoolBuffer : public ResizableBuffer {
 class ARROW_EXPORT BufferBuilder {
  public:
   explicit BufferBuilder(MemoryPool* pool)
-      : pool_(pool), data_(nullptr), capacity_(0), size_(0) {}
+      : pool_(pool), data_(NULLPTR), capacity_(0), size_(0) {}
 
   /// Resizes the buffer to the nearest multiple of 64 bytes per Layout.md
   Status Resize(const int64_t elements) {
@@ -201,7 +201,7 @@ class ARROW_EXPORT BufferBuilder {
     if (elements == 0) {
       return Status::OK();
     }
-    if (buffer_ == nullptr) {
+    if (buffer_ == NULLPTR) {
       buffer_ = std::make_shared<PoolBuffer>(pool_);
     }
     int64_t old_capacity = capacity_;
@@ -264,7 +264,7 @@ class ARROW_EXPORT BufferBuilder {
   }
 
   void Reset() {
-    buffer_ = nullptr;
+    buffer_ = NULLPTR;
     capacity_ = size_ = 0;
   }
 
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index 1720c0014..c580eeb3b 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -59,9 +59,9 @@ class ARROW_EXPORT ArrayBuilder {
   explicit ArrayBuilder(const std::shared_ptr<DataType>& type, MemoryPool* 
pool)
       : type_(type),
         pool_(pool),
-        null_bitmap_(nullptr),
+        null_bitmap_(NULLPTR),
         null_count_(0),
-        null_bitmap_data_(nullptr),
+        null_bitmap_data_(NULLPTR),
         length_(0),
         capacity_(0) {}
 
@@ -188,7 +188,7 @@ class ARROW_EXPORT PrimitiveBuilder : public ArrayBuilder {
   using value_type = typename Type::c_type;
 
   explicit PrimitiveBuilder(const std::shared_ptr<DataType>& type, MemoryPool* 
pool)
-      : ArrayBuilder(type, pool), data_(nullptr), raw_data_(nullptr) {}
+      : ArrayBuilder(type, pool), data_(NULLPTR), raw_data_(NULLPTR) {}
 
   using ArrayBuilder::Advance;
 
@@ -214,7 +214,7 @@ class ARROW_EXPORT PrimitiveBuilder : public ArrayBuilder {
   /// indicates a valid (non-null) value
   /// \return Status
   Status Append(const value_type* values, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
 
   /// \brief Append a sequence of elements in one shot
   /// \param[in] values a contiguous C array of values
@@ -430,7 +430,7 @@ class ARROW_EXPORT AdaptiveUIntBuilder : public 
internal::AdaptiveIntBuilderBase
   /// indicates a valid (non-null) value
   /// \return Status
   Status Append(const uint64_t* values, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
 
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
 
@@ -492,7 +492,7 @@ class ARROW_EXPORT AdaptiveIntBuilder : public 
internal::AdaptiveIntBuilderBase
   /// indicates a valid (non-null) value
   /// \return Status
   Status Append(const int64_t* values, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
 
   Status FinishInternal(std::shared_ptr<ArrayData>* out) override;
 
@@ -557,7 +557,7 @@ class ARROW_EXPORT BooleanBuilder : public ArrayBuilder {
   /// indicates a valid (non-null) value
   /// \return Status
   Status Append(const uint8_t* values, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
 
   /// \brief Append a sequence of elements in one shot
   /// \param[in] values a contiguous C array of values
@@ -624,7 +624,7 @@ class ARROW_EXPORT ListBuilder : public ArrayBuilder {
   /// Use this constructor to incrementally build the value array along with 
offsets and
   /// null bitmap.
   ListBuilder(MemoryPool* pool, std::unique_ptr<ArrayBuilder> value_builder,
-              const std::shared_ptr<DataType>& type = nullptr);
+              const std::shared_ptr<DataType>& type = NULLPTR);
 
   Status Init(int64_t elements) override;
   Status Resize(int64_t capacity) override;
@@ -635,7 +635,7 @@ class ARROW_EXPORT ListBuilder : public ArrayBuilder {
   /// If passed, valid_bytes is of equal length to values, and any zero byte
   /// will be considered as a null for that slot
   Status Append(const int32_t* offsets, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
 
   /// \brief Start a new variable-length list slot
   ///
@@ -732,7 +732,7 @@ class ARROW_EXPORT FixedSizeBinaryBuilder : public 
ArrayBuilder {
   }
 
   Status Append(const uint8_t* data, int64_t length,
-                const uint8_t* valid_bytes = nullptr);
+                const uint8_t* valid_bytes = NULLPTR);
   Status Append(const std::string& value);
   Status AppendNull();
 
diff --git a/cpp/src/arrow/io/hdfs.h b/cpp/src/arrow/io/hdfs.h
index aaaafc819..0708b11cc 100644
--- a/cpp/src/arrow/io/hdfs.h
+++ b/cpp/src/arrow/io/hdfs.h
@@ -121,8 +121,8 @@ class ARROW_EXPORT HadoopFileSystem : public FileSystem {
   /// Change
   ///
   /// @param path file path to change
-  /// @param owner pass nullptr for no change
-  /// @param group pass nullptr for no change
+  /// @param owner pass null for no change
+  /// @param group pass null for no change
   Status Chown(const std::string& path, const char* owner, const char* group);
 
   /// Change path permissions
@@ -199,7 +199,7 @@ class ARROW_EXPORT HdfsReadableFile : public 
RandomAccessFile {
   void set_memory_pool(MemoryPool* pool);
 
  private:
-  explicit HdfsReadableFile(MemoryPool* pool = nullptr);
+  explicit HdfsReadableFile(MemoryPool* pool = NULLPTR);
 
   class ARROW_NO_EXPORT HdfsReadableFileImpl;
   std::unique_ptr<HdfsReadableFileImpl> impl_;
diff --git a/cpp/src/arrow/ipc/message.h b/cpp/src/arrow/ipc/message.h
index 67a95c7d2..a1b6c07a4 100644
--- a/cpp/src/arrow/ipc/message.h
+++ b/cpp/src/arrow/ipc/message.h
@@ -69,7 +69,7 @@ class ARROW_EXPORT Message {
   /// \brief Create and validate a Message instance from two buffers
   ///
   /// \param[in] metadata a buffer containing the Flatbuffer metadata
-  /// \param[in] body a buffer containing the message body, which may be 
nullptr
+  /// \param[in] body a buffer containing the message body, which may be null
   /// \param[out] out the created message
   /// \return Status
   static Status Open(const std::shared_ptr<Buffer>& metadata,
@@ -98,7 +98,7 @@ class ARROW_EXPORT Message {
 
   /// \brief the Message body, if any
   ///
-  /// \return buffer is nullptr if no body
+  /// \return buffer is null if no body
   std::shared_ptr<Buffer> body() const;
 
   /// \brief The Message type
@@ -179,7 +179,7 @@ Status ReadMessage(const int64_t offset, const int32_t 
metadata_length,
 
 /// \brief Read encapulated RPC message (metadata and body) from InputStream
 ///
-/// Read length-prefixed message with as-yet unknown length. Returns nullptr if
+/// Read length-prefixed message with as-yet unknown length. Returns null if
 /// there are not enough bytes available or the message length is 0 (e.g. EOS
 /// in a stream)
 ARROW_EXPORT
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index e3fe2ef42..146864ffd 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -63,7 +63,7 @@ class ARROW_EXPORT PyAcquireGIL {
 
 class ARROW_EXPORT OwnedRef {
  public:
-  OwnedRef() : obj_(nullptr) {}
+  OwnedRef() : obj_(NULLPTR) {}
 
   explicit OwnedRef(PyObject* obj) : obj_(obj) {}
 
@@ -82,7 +82,7 @@ class ARROW_EXPORT OwnedRef {
 
   void release() {
     Py_XDECREF(obj_);
-    obj_ = nullptr;
+    obj_ = NULLPTR;
   }
 
   PyObject* obj() const { return obj_; }
@@ -96,7 +96,7 @@ class ARROW_EXPORT OwnedRef {
 // reference count when release is called.
 class ARROW_EXPORT ScopedRef {
  public:
-  ScopedRef() : obj_(nullptr) {}
+  ScopedRef() : obj_(NULLPTR) {}
 
   explicit ScopedRef(PyObject* obj) : obj_(obj) {}
 
@@ -109,7 +109,7 @@ class ARROW_EXPORT ScopedRef {
 
   PyObject* release() {
     PyObject* result = obj_;
-    obj_ = nullptr;
+    obj_ = NULLPTR;
     return result;
   }
 
@@ -137,7 +137,7 @@ struct ARROW_EXPORT PyObjectStringify {
       bytes = PyBytes_AsString(obj);
       size = PyBytes_GET_SIZE(obj);
     } else {
-      bytes = nullptr;
+      bytes = NULLPTR;
       size = -1;
     }
   }
diff --git a/cpp/src/arrow/python/helpers.h b/cpp/src/arrow/python/helpers.h
index 01ab91657..719ed796e 100644
--- a/cpp/src/arrow/python/helpers.h
+++ b/cpp/src/arrow/python/helpers.h
@@ -25,6 +25,7 @@
 #include <utility>
 
 #include "arrow/type.h"
+#include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
 
 namespace arrow {
@@ -43,8 +44,8 @@ Status ImportFromModule(const OwnedRef& module, const 
std::string& module_name,
 
 Status PythonDecimalToString(PyObject* python_decimal, std::string* out);
 
-Status InferDecimalPrecisionAndScale(PyObject* python_decimal, int* precision 
= nullptr,
-                                     int* scale = nullptr);
+Status InferDecimalPrecisionAndScale(PyObject* python_decimal, int* precision 
= NULLPTR,
+                                     int* scale = NULLPTR);
 
 PyObject* DecimalFromString(PyObject* decimal_constructor,
                             const std::string& decimal_string);
diff --git a/cpp/src/arrow/python/numpy_to_arrow.h 
b/cpp/src/arrow/python/numpy_to_arrow.h
index 4a870fff9..5bcbea325 100644
--- a/cpp/src/arrow/python/numpy_to_arrow.h
+++ b/cpp/src/arrow/python/numpy_to_arrow.h
@@ -37,7 +37,7 @@ class Status;
 namespace py {
 
 /// Convert NumPy arrays to Arrow. If target data type is not known, pass a
-/// type with nullptr
+/// type with null
 ///
 /// \param[in] pool Memory pool for any memory allocations
 /// \param[in] ao an ndarray with the array data
diff --git a/cpp/src/arrow/table.h b/cpp/src/arrow/table.h
index d40bdb856..d3145ff10 100644
--- a/cpp/src/arrow/table.h
+++ b/cpp/src/arrow/table.h
@@ -288,10 +288,10 @@ class ARROW_EXPORT RecordBatchReader {
   /// \return the shared schema of the record batches in the stream
   virtual std::shared_ptr<Schema> schema() const = 0;
 
-  /// Read the next record batch in the stream. Return nullptr for batch when
+  /// Read the next record batch in the stream. Return null for batch when
   /// reaching end of stream
   ///
-  /// \param[out] batch the next loaded batch, nullptr at end of stream
+  /// \param[out] batch the next loaded batch, null at end of stream
   /// \return Status
   virtual Status ReadNext(std::shared_ptr<RecordBatch>* batch) = 0;
 };
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 443828423..2030f371d 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -241,7 +241,7 @@ class ARROW_EXPORT Field {
  public:
   Field(const std::string& name, const std::shared_ptr<DataType>& type,
         bool nullable = true,
-        const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr)
+        const std::shared_ptr<const KeyValueMetadata>& metadata = NULLPTR)
       : name_(name), type_(type), nullable_(nullable), metadata_(metadata) {}
 
   std::shared_ptr<const KeyValueMetadata> metadata() const { return metadata_; 
}
@@ -737,10 +737,10 @@ class ARROW_EXPORT DictionaryType : public FixedWidthType 
{
 class ARROW_EXPORT Schema {
  public:
   explicit Schema(const std::vector<std::shared_ptr<Field>>& fields,
-                  const std::shared_ptr<const KeyValueMetadata>& metadata = 
nullptr);
+                  const std::shared_ptr<const KeyValueMetadata>& metadata = 
NULLPTR);
 
   explicit Schema(std::vector<std::shared_ptr<Field>>&& fields,
-                  const std::shared_ptr<const KeyValueMetadata>& metadata = 
nullptr);
+                  const std::shared_ptr<const KeyValueMetadata>& metadata = 
NULLPTR);
 
   virtual ~Schema() = default;
 
@@ -750,7 +750,7 @@ class ARROW_EXPORT Schema {
   /// Return the ith schema element. Does not boundscheck
   std::shared_ptr<Field> field(int i) const { return fields_[i]; }
 
-  /// Returns nullptr if name not found
+  /// Returns null if name not found
   std::shared_ptr<Field> GetFieldByName(const std::string& name) const;
 
   /// Returns -1 if name not found
@@ -760,7 +760,7 @@ class ARROW_EXPORT Schema {
 
   /// \brief The custom key-value metadata, if any
   ///
-  /// \return metadata may be nullptr
+  /// \return metadata may be null
   std::shared_ptr<const KeyValueMetadata> metadata() const;
 
   /// \brief Render a string representation of the schema suitable for 
debugging
@@ -850,30 +850,30 @@ dictionary(const std::shared_ptr<DataType>& index_type,
 /// \param name the field name
 /// \param type the field value type
 /// \param nullable whether the values are nullable, default true
-/// \param metadata any custom key-value metadata, default nullptr
+/// \param metadata any custom key-value metadata, default null
 std::shared_ptr<Field> ARROW_EXPORT field(
     const std::string& name, const std::shared_ptr<DataType>& type, bool 
nullable = true,
-    const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
+    const std::shared_ptr<const KeyValueMetadata>& metadata = NULLPTR);
 
 /// \brief Create a Schema instance
 ///
 /// \param fields the schema's fields
-/// \param metadata any custom key-value metadata, default nullptr
+/// \param metadata any custom key-value metadata, default null
 /// \return schema shared_ptr to Schema
 ARROW_EXPORT
 std::shared_ptr<Schema> schema(
     const std::vector<std::shared_ptr<Field>>& fields,
-    const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
+    const std::shared_ptr<const KeyValueMetadata>& metadata = NULLPTR);
 
 /// \brief Create a Schema instance
 ///
 /// \param fields the schema's fields (rvalue reference)
-/// \param metadata any custom key-value metadata, default nullptr
+/// \param metadata any custom key-value metadata, default null
 /// \return schema shared_ptr to Schema
 ARROW_EXPORT
 std::shared_ptr<Schema> schema(
     std::vector<std::shared_ptr<Field>>&& fields,
-    const std::shared_ptr<const KeyValueMetadata>& metadata = nullptr);
+    const std::shared_ptr<const KeyValueMetadata>& metadata = NULLPTR);
 
 }  // namespace arrow
 
diff --git a/cpp/src/arrow/util/decimal.h b/cpp/src/arrow/util/decimal.h
index 58496a874..6f8d5a46c 100644
--- a/cpp/src/arrow/util/decimal.h
+++ b/cpp/src/arrow/util/decimal.h
@@ -24,6 +24,7 @@
 #include <type_traits>
 
 #include "arrow/status.h"
+#include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
 
 namespace arrow {
@@ -114,7 +115,7 @@ class ARROW_EXPORT Decimal128 {
   /// \brief Convert a decimal string to an Decimal128 value, optionally 
including
   /// precision and scale if they're passed in and not null.
   static Status FromString(const std::string& s, Decimal128* out,
-                           int* precision = nullptr, int* scale = nullptr);
+                           int* precision = NULLPTR, int* scale = NULLPTR);
 
  private:
   int64_t high_bits_;
diff --git a/cpp/src/arrow/util/io-util.h b/cpp/src/arrow/util/io-util.h
index 6fe3a5c17..dbca0d8be 100644
--- a/cpp/src/arrow/util/io-util.h
+++ b/cpp/src/arrow/util/io-util.h
@@ -75,7 +75,7 @@ class StdinStream : public InputStream {
   }
 
   Status Read(int64_t nbytes, std::shared_ptr<Buffer>* out) override {
-    auto buffer = std::make_shared<PoolBuffer>(nullptr);
+    auto buffer = std::make_shared<PoolBuffer>(NULLPTR);
     RETURN_NOT_OK(buffer->Resize(nbytes));
     int64_t bytes_read;
     RETURN_NOT_OK(Read(nbytes, &bytes_read, buffer->mutable_data()));
diff --git a/cpp/src/arrow/util/macros.h b/cpp/src/arrow/util/macros.h
index a5f6e5707..8b1125d02 100644
--- a/cpp/src/arrow/util/macros.h
+++ b/cpp/src/arrow/util/macros.h
@@ -58,6 +58,21 @@
 #define ARROW_MUST_USE_RESULT
 #endif
 
+// ----------------------------------------------------------------------
+// C++/CLI support macros (see ARROW-1134)
+
+#ifndef NULLPTR
+
+#ifdef __cplusplus_cli
+#define NULLPTR __nullptr
+#else
+#define NULLPTR nullptr
+#endif
+
+#endif  // ifndef NULLPTR
+
+// ----------------------------------------------------------------------
+
 // macros to disable padding
 // these macros are portable across different compilers and platforms
 
//[https://github.com/google/flatbuffers/blob/master/include/flatbuffers/flatbuffers.h#L1355]


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> [C++] Allow C++/CLI projects to build with Arrow​
> -------------------------------------------------
>
>                 Key: ARROW-1134
>                 URL: https://issues.apache.org/jira/browse/ARROW-1134
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: C++
>            Reporter: Toby Shaw
>            Assignee: Wes McKinney
>            Priority: Minor
>              Labels: pull-request-available
>             Fix For: 0.8.0
>
>
> Currently, the inclusion of <mutex> in some of Arrow's C++ headers prevents 
> C++/CLI code from building against it.
> From a C++/CLI project:
> #include <arrow/io/file.h>
> ...
> "#error directive: <mutex> is not supported when compiling with /clr or 
> /clr:pure."
> This could be patched by optionally relying on Boost's mutex/lock_guard 
> instead of std, or not exposing the #include <mutex> publically.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to