Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1B4D7200D27 for ; Wed, 25 Oct 2017 15:08:48 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 135C1160BDA; Wed, 25 Oct 2017 13:08:48 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 86ADE1609E5 for ; Wed, 25 Oct 2017 15:08:46 +0200 (CEST) Received: (qmail 70098 invoked by uid 500); 25 Oct 2017 13:08:45 -0000 Mailing-List: contact commits-help@arrow.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@arrow.apache.org Delivered-To: mailing list commits@arrow.apache.org Received: (qmail 70085 invoked by uid 99); 25 Oct 2017 13:08:45 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 25 Oct 2017 13:08:45 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id DEF1181B95; Wed, 25 Oct 2017 13:08:43 +0000 (UTC) Date: Wed, 25 Oct 2017 13:08:43 +0000 To: "commits@arrow.apache.org" Subject: [arrow] branch master updated: ARROW-1134: [C++] Support for C++/CLI compilation, add NULLPTR define to avoid using nullptr in public headers MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit Message-ID: <150893692332.22140.2893107456690779562@gitbox.apache.org> From: wesm@apache.org X-Git-Host: gitbox.apache.org X-Git-Repo: arrow X-Git-Refname: refs/heads/master X-Git-Reftype: branch X-Git-Oldrev: b08f7e326b8fc1a000a277eef3bc84a0106d6c3c X-Git-Newrev: ecb7605a640dc34452dd696240a74df9a9f7f39b X-Git-Rev: ecb7605a640dc34452dd696240a74df9a9f7f39b X-Git-NotificationType: ref_changed_plus_diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated archived-at: Wed, 25 Oct 2017 13:08:48 -0000 This is an automated email from the ASF dual-hosted git repository. wesm 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 ecb7605 ARROW-1134: [C++] Support for C++/CLI compilation, add NULLPTR define to avoid using nullptr in public headers ecb7605 is described below commit ecb7605a640dc34452dd696240a74df9a9f7f39b Author: Wes McKinney AuthorDate: Wed Oct 25 09:08:38 2017 -0400 ARROW-1134: [C++] Support for C++/CLI compilation, add NULLPTR define to avoid using nullptr in public headers cc @tobyshaw. Can you test this? Close #1098 Author: Wes McKinney Closes #1228 from wesm/ARROW-1134 and squashes the following commits: bf181584 [Wes McKinney] Only define NULLPTR if not already defined a51dd885 [Wes McKinney] Add NULLPTR macro to avoid using nullptr in public headers for C++/CLI users --- cpp/src/arrow/allocator.h | 5 +++-- cpp/src/arrow/array.h | 36 +++++++++++++++++------------------ cpp/src/arrow/buffer.h | 12 ++++++------ cpp/src/arrow/builder.h | 20 +++++++++---------- cpp/src/arrow/io/hdfs.h | 6 +++--- cpp/src/arrow/ipc/message.h | 6 +++--- cpp/src/arrow/python/common.h | 10 +++++----- cpp/src/arrow/python/helpers.h | 5 +++-- cpp/src/arrow/python/numpy_to_arrow.h | 2 +- cpp/src/arrow/table.h | 4 ++-- cpp/src/arrow/type.h | 22 ++++++++++----------- cpp/src/arrow/util/decimal.h | 3 ++- cpp/src/arrow/util/io-util.h | 2 +- cpp/src/arrow/util/macros.h | 15 +++++++++++++++ 14 files changed, 83 insertions(+), 65 deletions(-) diff --git a/cpp/src/arrow/allocator.h b/cpp/src/arrow/allocator.h index e00023d..c7780f1 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 stl_allocator(const stl_allocator& 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 b5d2530..afbd780 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 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& 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& type, int64_t length, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& 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& 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& data) { @@ -349,7 +349,7 @@ class ARROW_EXPORT NumericArray : public PrimitiveArray { NumericArray( typename std::enable_if::is_parameter_free, int64_t>::type length, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, int64_t null_count = 0, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0) : PrimitiveArray(TypeTraits::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& data); BooleanArray(int64_t length, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& 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& type, int64_t length, const std::shared_ptr& value_offsets, const std::shared_ptr& values, - const std::shared_ptr& null_bitmap = nullptr, int64_t null_count = 0, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); /// \brief Construct ListArray from array of offsets and child value array @@ -451,7 +451,7 @@ class ARROW_EXPORT BinaryArray : public FlatArray { BinaryArray(int64_t length, const std::shared_ptr& value_offsets, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); // Return the pointer to the given elements bytes @@ -503,7 +503,7 @@ class ARROW_EXPORT BinaryArray : public FlatArray { BinaryArray(const std::shared_ptr& type, int64_t length, const std::shared_ptr& value_offsets, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); const int32_t* raw_value_offsets_; @@ -518,7 +518,7 @@ class ARROW_EXPORT StringArray : public BinaryArray { StringArray(int64_t length, const std::shared_ptr& value_offsets, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); // Construct a std::string @@ -541,7 +541,7 @@ class ARROW_EXPORT FixedSizeBinaryArray : public PrimitiveArray { FixedSizeBinaryArray(const std::shared_ptr& type, int64_t length, const std::shared_ptr& data, - const std::shared_ptr& null_bitmap = nullptr, + const std::shared_ptr& null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); const uint8_t* GetValue(int64_t i) const; @@ -583,7 +583,7 @@ class ARROW_EXPORT StructArray : public Array { StructArray(const std::shared_ptr& type, int64_t length, const std::vector>& children, - std::shared_ptr null_bitmap = nullptr, int64_t null_count = 0, + std::shared_ptr null_bitmap = NULLPTR, int64_t null_count = 0, int64_t offset = 0); // Return a shared pointer in case the requestor desires to share ownership @@ -608,8 +608,8 @@ class ARROW_EXPORT UnionArray : public Array { UnionArray(const std::shared_ptr& type, int64_t length, const std::vector>& children, const std::shared_ptr& type_ids, - const std::shared_ptr& value_offsets = nullptr, - const std::shared_ptr& null_bitmap = nullptr, int64_t null_count = 0, + const std::shared_ptr& value_offsets = NULLPTR, + const std::shared_ptr& 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 b745812..8e98906 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 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(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 1720c00..c580eeb 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& 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& 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* 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* 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 value_builder, - const std::shared_ptr& type = nullptr); + const std::shared_ptr& 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 aaaafc8..0708b11 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 impl_; diff --git a/cpp/src/arrow/ipc/message.h b/cpp/src/arrow/ipc/message.h index 67a95c7..a1b6c07 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& 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 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 e3fe2ef..146864f 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 01ab916..719ed79 100644 --- a/cpp/src/arrow/python/helpers.h +++ b/cpp/src/arrow/python/helpers.h @@ -25,6 +25,7 @@ #include #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 4a870ff..5bcbea3 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 d40bdb8..d3145ff 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() 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* batch) = 0; }; diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h index 4438284..2030f37 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& type, bool nullable = true, - const std::shared_ptr& metadata = nullptr) + const std::shared_ptr& metadata = NULLPTR) : name_(name), type_(type), nullable_(nullable), metadata_(metadata) {} std::shared_ptr metadata() const { return metadata_; } @@ -737,10 +737,10 @@ class ARROW_EXPORT DictionaryType : public FixedWidthType { class ARROW_EXPORT Schema { public: explicit Schema(const std::vector>& fields, - const std::shared_ptr& metadata = nullptr); + const std::shared_ptr& metadata = NULLPTR); explicit Schema(std::vector>&& fields, - const std::shared_ptr& metadata = nullptr); + const std::shared_ptr& 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(int i) const { return fields_[i]; } - /// Returns nullptr if name not found + /// Returns null if name not found std::shared_ptr 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 metadata() const; /// \brief Render a string representation of the schema suitable for debugging @@ -850,30 +850,30 @@ dictionary(const std::shared_ptr& 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 ARROW_EXPORT field( const std::string& name, const std::shared_ptr& type, bool nullable = true, - const std::shared_ptr& metadata = nullptr); + const std::shared_ptr& 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( const std::vector>& fields, - const std::shared_ptr& metadata = nullptr); + const std::shared_ptr& 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( std::vector>&& fields, - const std::shared_ptr& metadata = nullptr); + const std::shared_ptr& metadata = NULLPTR); } // namespace arrow diff --git a/cpp/src/arrow/util/decimal.h b/cpp/src/arrow/util/decimal.h index 58496a8..6f8d5a4 100644 --- a/cpp/src/arrow/util/decimal.h +++ b/cpp/src/arrow/util/decimal.h @@ -24,6 +24,7 @@ #include #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 6fe3a5c..dbca0d8 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* out) override { - auto buffer = std::make_shared(nullptr); + auto buffer = std::make_shared(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 a5f6e57..8b1125d 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] -- To stop receiving notification emails like this one, please contact ['"commits@arrow.apache.org" '].