pitrou commented on code in PR #38472:
URL: https://github.com/apache/arrow/pull/38472#discussion_r1422637533


##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,49 @@ cdef class Array(_PandasConvertible):
 
         return pyarrow_wrap_array(array)
 
+    def __dlpack__(self, stream=None):
+        """Export a primitive array as a DLPack capsule.
+
+        Parameters
+        ----------
+        stream : int, optional
+            A Python integer representing a pointer to a stream. Currently not 
supported.
+            Stream is provided by the consumer to the producer to instruct the 
producer
+            to ensure that operations can safely be performed on the array.
+
+        Returns
+        -------
+        capsule : PyCapsule
+            A DLPack capsule for the array, pointing to a DLManagedTensor.
+        """
+        cdef CResult[DLManagedTensor*] c_dlm_tensor
+
+        if stream is None:
+            c_dlm_tensor = ExportToDLPack(pyarrow_unwrap_array(self))
+            dlm_tensor = GetResultValue(c_dlm_tensor)

Review Comment:
   You can probably simplify this
   ```suggestion
           if stream is None:
               dlm_tensor = GetResultValue(ExportToDLPack(self.sp_array))
   ```



##########
python/pyarrow/_dlpack.pxi:
##########
@@ -0,0 +1,49 @@
+# 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.
+
+from libc.stdlib cimport malloc, free

Review Comment:
   This is not used anymore, is it?



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,138 @@
+// 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 "arrow/c/dlpack.h"
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/device.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+
+namespace arrow::dlpack {
+
+Result<DLDataType> GetDLDataType(const DataType& type) {
+  DLDataType dtype;
+  dtype.lanes = 1;
+  dtype.bits = type.bit_width();
+  switch (type.id()) {
+    case Type::INT8:
+    case Type::INT16:
+    case Type::INT32:
+    case Type::INT64:
+      dtype.code = DLDataTypeCode::kDLInt;
+      return dtype;
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      return dtype;
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      return dtype;
+    case Type::BOOL:
+      // DLPack supports byte-packed boolean values
+      return Status::TypeError("Bit-packed boolean data type not supported by 
DLPack.");
+    default:
+      return Status::TypeError("DataType is not compatible with DLPack spec: ",
+                               type.ToString());
+  }
+}
+
+struct ManagerCtx {
+  std::shared_ptr<ArrayData> ref;
+  DLManagedTensor tensor;
+};
+
+Result<DLManagedTensor*> ExportArray(const std::shared_ptr<Array>& arr) {
+  // Define DLDevice struct nad check if array type is supported
+  // by the DLPack protocol at the same time. Raise TypeError if not.
+  // Supported data types: int, uint, float with no validity buffer.
+  ARROW_ASSIGN_OR_RAISE(auto device, ExportDevice(arr))
+
+  // Define the DLDataType struct
+  const DataType* arrow_type = arr->type().get();
+  ARROW_ASSIGN_OR_RAISE(auto dlpack_type, GetDLDataType(*arrow_type));
+
+  // Create ManagerCtx with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  std::unique_ptr<ManagerCtx> ctx(new ManagerCtx);
+  ctx->ref = array_ref;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL
+  if (arr->length() == 0) {
+    ctx->tensor.dl_tensor.data = NULL;
+  } else if (arr->offset() > 0) {
+    const auto byte_width = arr->type()->byte_width();
+    const auto start = arr->offset() * byte_width;
+    ARROW_ASSIGN_OR_RAISE(auto sliced_buffer,
+                          SliceBufferSafe(array_ref->buffers[1], start));
+    ctx->tensor.dl_tensor.data =
+        const_cast<void*>(reinterpret_cast<const 
void*>(sliced_buffer->address()));
+  } else {
+    ctx->tensor.dl_tensor.data = const_cast<void*>(
+        reinterpret_cast<const void*>(array_ref->buffers[1]->address()));
+  }
+
+  ctx->tensor.dl_tensor.device = device;
+  ctx->tensor.dl_tensor.ndim = 1;
+  ctx->tensor.dl_tensor.dtype = dlpack_type;
+  ctx->tensor.dl_tensor.shape = const_cast<int64_t*>(&array_ref->length);
+  ctx->tensor.dl_tensor.strides = NULL;
+  ctx->tensor.dl_tensor.byte_offset = 0;
+
+  // return dlm_tensor;

Review Comment:
   Does this line need to be removed now?



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,138 @@
+// 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 "arrow/c/dlpack.h"
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/device.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+
+namespace arrow::dlpack {
+
+Result<DLDataType> GetDLDataType(const DataType& type) {
+  DLDataType dtype;
+  dtype.lanes = 1;
+  dtype.bits = type.bit_width();
+  switch (type.id()) {
+    case Type::INT8:
+    case Type::INT16:
+    case Type::INT32:
+    case Type::INT64:
+      dtype.code = DLDataTypeCode::kDLInt;
+      return dtype;
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      return dtype;
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      return dtype;
+    case Type::BOOL:
+      // DLPack supports byte-packed boolean values
+      return Status::TypeError("Bit-packed boolean data type not supported by 
DLPack.");
+    default:
+      return Status::TypeError("DataType is not compatible with DLPack spec: ",
+                               type.ToString());
+  }
+}
+
+struct ManagerCtx {
+  std::shared_ptr<ArrayData> ref;
+  DLManagedTensor tensor;
+};
+
+Result<DLManagedTensor*> ExportArray(const std::shared_ptr<Array>& arr) {
+  // Define DLDevice struct nad check if array type is supported
+  // by the DLPack protocol at the same time. Raise TypeError if not.
+  // Supported data types: int, uint, float with no validity buffer.
+  ARROW_ASSIGN_OR_RAISE(auto device, ExportDevice(arr))
+
+  // Define the DLDataType struct
+  const DataType* arrow_type = arr->type().get();
+  ARROW_ASSIGN_OR_RAISE(auto dlpack_type, GetDLDataType(*arrow_type));
+
+  // Create ManagerCtx with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  std::unique_ptr<ManagerCtx> ctx(new ManagerCtx);
+  ctx->ref = array_ref;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL
+  if (arr->length() == 0) {
+    ctx->tensor.dl_tensor.data = NULL;
+  } else if (arr->offset() > 0) {
+    const auto byte_width = arr->type()->byte_width();
+    const auto start = arr->offset() * byte_width;
+    ARROW_ASSIGN_OR_RAISE(auto sliced_buffer,
+                          SliceBufferSafe(array_ref->buffers[1], start));
+    ctx->tensor.dl_tensor.data =
+        const_cast<void*>(reinterpret_cast<const 
void*>(sliced_buffer->address()));
+  } else {
+    ctx->tensor.dl_tensor.data = const_cast<void*>(
+        reinterpret_cast<const void*>(array_ref->buffers[1]->address()));
+  }
+
+  ctx->tensor.dl_tensor.device = device;
+  ctx->tensor.dl_tensor.ndim = 1;
+  ctx->tensor.dl_tensor.dtype = dlpack_type;
+  ctx->tensor.dl_tensor.shape = const_cast<int64_t*>(&array_ref->length);
+  ctx->tensor.dl_tensor.strides = NULL;
+  ctx->tensor.dl_tensor.byte_offset = 0;
+
+  // return dlm_tensor;
+  ctx->tensor.manager_ctx = ctx.get();
+  ctx->tensor.deleter = [](struct DLManagedTensor* self) {
+    delete reinterpret_cast<ManagerCtx*>(self->manager_ctx);
+  };
+  return &ctx.release()->tensor;
+}
+
+Result<DLDevice> ExportDevice(const std::shared_ptr<Array>& arr) {
+  // Check if array is supported by the DLPack protocol.
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use DLPack on arrays with no nulls.");
+  }
+  const DataType* arrow_type = arr->type().get();
+  if (arrow_type->id() == Type::BOOL) {
+    return Status::TypeError("Bit-packed boolean data type not supported by 
DLPack.");
+  }
+  if (!is_integer(arrow_type->id()) && !is_unsigned_integer(arrow_type->id()) 
&&
+      !is_floating(arrow_type->id())) {

Review Comment:
   I think this can be simplified to:
   ```suggestion
     if (!is_integer(arrow_type->id()) && !is_floating(arrow_type->id())) {
   ```



##########
cpp/src/arrow/c/dlpack_test.cc:
##########
@@ -0,0 +1,115 @@
+// 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>
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+
+namespace arrow::dlpack {
+
+class TestExportArray : public ::testing::Test {
+ public:
+  void SetUp() {}
+};
+
+auto check_dlptensor = [](const std::shared_ptr<Array>& arr,
+                          std::shared_ptr<DataType> arrow_type,
+                          DLDataTypeCode dlpack_type, int64_t length) {
+  ASSERT_OK_AND_ASSIGN(auto dlmtensor, arrow::dlpack::ExportArray(arr));
+  auto dltensor = dlmtensor->dl_tensor;
+
+  const auto byte_width = arr->type()->byte_width();
+  const auto start = arr->offset() * byte_width;
+  ASSERT_OK_AND_ASSIGN(auto sliced_buffer,
+                       SliceBufferSafe(arr->data()->buffers[1], start));
+  ASSERT_EQ(sliced_buffer->data(), dltensor.data);
+
+  ASSERT_EQ(0, dltensor.byte_offset);
+  ASSERT_EQ(NULL, dltensor.strides);
+  ASSERT_EQ(length, dltensor.shape[0]);
+  ASSERT_EQ(1, dltensor.ndim);
+
+  ASSERT_EQ(dlpack_type, dltensor.dtype.code);
+
+  ASSERT_EQ(arrow_type->bit_width(), dltensor.dtype.bits);
+  ASSERT_EQ(1, dltensor.dtype.lanes);
+  ASSERT_EQ(DLDeviceType::kDLCPU, dltensor.device.device_type);
+  ASSERT_EQ(0, dltensor.device.device_id);
+
+  ASSERT_OK_AND_ASSIGN(auto device, arrow::dlpack::ExportDevice(arr));
+  ASSERT_EQ(DLDeviceType::kDLCPU, device.device_type);
+  ASSERT_EQ(0, device.device_id);
+
+  dlmtensor->deleter(dlmtensor);
+};
+
+TEST_F(TestExportArray, TestSupportedArray) {
+  random::RandomArrayGenerator gen(0);
+
+  std::vector<std::shared_ptr<DataType>> arrow_types = {
+      int8(),  uint8(),  int16(),   uint16(),  int32(),   uint32(),
+      int64(), uint64(), float16(), float32(), float64(),
+  };
+
+  std::vector<DLDataTypeCode> dlpack_types = {
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLInt,
+      DLDataTypeCode::kDLUInt,  DLDataTypeCode::kDLInt,   
DLDataTypeCode::kDLUInt,
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLFloat,
+      DLDataTypeCode::kDLFloat, DLDataTypeCode::kDLFloat,
+  };
+
+  for (int64_t i = 0; i < 11; ++i) {

Review Comment:
   Hard-coding array lengths like this is fragile.
   How about something like:
   ```c++
     std::vector<std::pair<std::shared_ptr<DataType>, DLDataTypeCode>> cases = {
       {int8(), DLDataTypeCode::kDLInt}, ...
     };
   
     for (auto [arrow_type, dlpack_type] : cases) {
       ...
   ```



##########
cpp/src/arrow/c/dlpack_test.cc:
##########
@@ -0,0 +1,115 @@
+// 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>
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+
+namespace arrow::dlpack {
+
+class TestExportArray : public ::testing::Test {
+ public:
+  void SetUp() {}
+};
+
+auto check_dlptensor = [](const std::shared_ptr<Array>& arr,
+                          std::shared_ptr<DataType> arrow_type,
+                          DLDataTypeCode dlpack_type, int64_t length) {
+  ASSERT_OK_AND_ASSIGN(auto dlmtensor, arrow::dlpack::ExportArray(arr));
+  auto dltensor = dlmtensor->dl_tensor;
+
+  const auto byte_width = arr->type()->byte_width();
+  const auto start = arr->offset() * byte_width;
+  ASSERT_OK_AND_ASSIGN(auto sliced_buffer,
+                       SliceBufferSafe(arr->data()->buffers[1], start));
+  ASSERT_EQ(sliced_buffer->data(), dltensor.data);
+
+  ASSERT_EQ(0, dltensor.byte_offset);
+  ASSERT_EQ(NULL, dltensor.strides);
+  ASSERT_EQ(length, dltensor.shape[0]);
+  ASSERT_EQ(1, dltensor.ndim);
+
+  ASSERT_EQ(dlpack_type, dltensor.dtype.code);
+
+  ASSERT_EQ(arrow_type->bit_width(), dltensor.dtype.bits);
+  ASSERT_EQ(1, dltensor.dtype.lanes);
+  ASSERT_EQ(DLDeviceType::kDLCPU, dltensor.device.device_type);
+  ASSERT_EQ(0, dltensor.device.device_id);
+
+  ASSERT_OK_AND_ASSIGN(auto device, arrow::dlpack::ExportDevice(arr));
+  ASSERT_EQ(DLDeviceType::kDLCPU, device.device_type);
+  ASSERT_EQ(0, device.device_id);
+
+  dlmtensor->deleter(dlmtensor);
+};
+
+TEST_F(TestExportArray, TestSupportedArray) {
+  random::RandomArrayGenerator gen(0);
+
+  std::vector<std::shared_ptr<DataType>> arrow_types = {
+      int8(),  uint8(),  int16(),   uint16(),  int32(),   uint32(),
+      int64(), uint64(), float16(), float32(), float64(),
+  };
+
+  std::vector<DLDataTypeCode> dlpack_types = {
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLInt,
+      DLDataTypeCode::kDLUInt,  DLDataTypeCode::kDLInt,   
DLDataTypeCode::kDLUInt,
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLFloat,
+      DLDataTypeCode::kDLFloat, DLDataTypeCode::kDLFloat,
+  };
+
+  for (int64_t i = 0; i < 11; ++i) {
+    const std::shared_ptr<Array> array = gen.ArrayOf(arrow_types[i], 10, 0);

Review Comment:
   You also don't need a random generator here...



##########
cpp/src/arrow/c/dlpack_test.cc:
##########
@@ -0,0 +1,115 @@
+// 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>
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+
+namespace arrow::dlpack {
+
+class TestExportArray : public ::testing::Test {
+ public:
+  void SetUp() {}
+};
+
+auto check_dlptensor = [](const std::shared_ptr<Array>& arr,
+                          std::shared_ptr<DataType> arrow_type,
+                          DLDataTypeCode dlpack_type, int64_t length) {
+  ASSERT_OK_AND_ASSIGN(auto dlmtensor, arrow::dlpack::ExportArray(arr));
+  auto dltensor = dlmtensor->dl_tensor;
+
+  const auto byte_width = arr->type()->byte_width();
+  const auto start = arr->offset() * byte_width;
+  ASSERT_OK_AND_ASSIGN(auto sliced_buffer,
+                       SliceBufferSafe(arr->data()->buffers[1], start));
+  ASSERT_EQ(sliced_buffer->data(), dltensor.data);
+
+  ASSERT_EQ(0, dltensor.byte_offset);
+  ASSERT_EQ(NULL, dltensor.strides);
+  ASSERT_EQ(length, dltensor.shape[0]);
+  ASSERT_EQ(1, dltensor.ndim);
+
+  ASSERT_EQ(dlpack_type, dltensor.dtype.code);
+
+  ASSERT_EQ(arrow_type->bit_width(), dltensor.dtype.bits);
+  ASSERT_EQ(1, dltensor.dtype.lanes);
+  ASSERT_EQ(DLDeviceType::kDLCPU, dltensor.device.device_type);
+  ASSERT_EQ(0, dltensor.device.device_id);
+
+  ASSERT_OK_AND_ASSIGN(auto device, arrow::dlpack::ExportDevice(arr));
+  ASSERT_EQ(DLDeviceType::kDLCPU, device.device_type);
+  ASSERT_EQ(0, device.device_id);
+
+  dlmtensor->deleter(dlmtensor);
+};
+
+TEST_F(TestExportArray, TestSupportedArray) {
+  random::RandomArrayGenerator gen(0);
+
+  std::vector<std::shared_ptr<DataType>> arrow_types = {
+      int8(),  uint8(),  int16(),   uint16(),  int32(),   uint32(),
+      int64(), uint64(), float16(), float32(), float64(),
+  };
+
+  std::vector<DLDataTypeCode> dlpack_types = {
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLInt,
+      DLDataTypeCode::kDLUInt,  DLDataTypeCode::kDLInt,   
DLDataTypeCode::kDLUInt,
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLFloat,
+      DLDataTypeCode::kDLFloat, DLDataTypeCode::kDLFloat,
+  };
+
+  for (int64_t i = 0; i < 11; ++i) {
+    const std::shared_ptr<Array> array = gen.ArrayOf(arrow_types[i], 10, 0);
+    check_dlptensor(array, arrow_types[i], dlpack_types[i], 10);
+    ASSERT_OK_AND_ASSIGN(auto sliced_1, array->SliceSafe(1, 5));
+    check_dlptensor(sliced_1, arrow_types[i], dlpack_types[i], 5);
+    ASSERT_OK_AND_ASSIGN(auto sliced_2, array->SliceSafe(0, 5));
+    check_dlptensor(sliced_2, arrow_types[i], dlpack_types[i], 5);
+    ASSERT_OK_AND_ASSIGN(auto sliced_3, array->SliceSafe(3));
+    check_dlptensor(sliced_3, arrow_types[i], dlpack_types[i], 7);
+  }
+}
+
+TEST_F(TestExportArray, TestUnSupportedArray) {
+  random::RandomArrayGenerator gen(0);

Review Comment:
   You don't really need a random generator for this :-)
   ```c++
     const std::shared_ptr<Array> array_with_null = ArrayFromJSON(int8(), "[1, 
null]");
     const std::shared_ptr<Array> array_string = ArrayFromJSON(utf8(), "[]");
     const std::shared_ptr<Array> array_boolean = ArrayFromJSON(boolean(), 
"[true]");
   ```
   



##########
cpp/src/arrow/c/dlpack_test.cc:
##########
@@ -0,0 +1,115 @@
+// 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>
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+
+namespace arrow::dlpack {
+
+class TestExportArray : public ::testing::Test {
+ public:
+  void SetUp() {}
+};
+
+auto check_dlptensor = [](const std::shared_ptr<Array>& arr,
+                          std::shared_ptr<DataType> arrow_type,
+                          DLDataTypeCode dlpack_type, int64_t length) {
+  ASSERT_OK_AND_ASSIGN(auto dlmtensor, arrow::dlpack::ExportArray(arr));
+  auto dltensor = dlmtensor->dl_tensor;
+
+  const auto byte_width = arr->type()->byte_width();
+  const auto start = arr->offset() * byte_width;
+  ASSERT_OK_AND_ASSIGN(auto sliced_buffer,
+                       SliceBufferSafe(arr->data()->buffers[1], start));
+  ASSERT_EQ(sliced_buffer->data(), dltensor.data);
+
+  ASSERT_EQ(0, dltensor.byte_offset);
+  ASSERT_EQ(NULL, dltensor.strides);
+  ASSERT_EQ(length, dltensor.shape[0]);
+  ASSERT_EQ(1, dltensor.ndim);
+
+  ASSERT_EQ(dlpack_type, dltensor.dtype.code);
+
+  ASSERT_EQ(arrow_type->bit_width(), dltensor.dtype.bits);
+  ASSERT_EQ(1, dltensor.dtype.lanes);
+  ASSERT_EQ(DLDeviceType::kDLCPU, dltensor.device.device_type);
+  ASSERT_EQ(0, dltensor.device.device_id);
+
+  ASSERT_OK_AND_ASSIGN(auto device, arrow::dlpack::ExportDevice(arr));
+  ASSERT_EQ(DLDeviceType::kDLCPU, device.device_type);
+  ASSERT_EQ(0, device.device_id);
+
+  dlmtensor->deleter(dlmtensor);
+};
+
+TEST_F(TestExportArray, TestSupportedArray) {
+  random::RandomArrayGenerator gen(0);
+
+  std::vector<std::shared_ptr<DataType>> arrow_types = {
+      int8(),  uint8(),  int16(),   uint16(),  int32(),   uint32(),
+      int64(), uint64(), float16(), float32(), float64(),
+  };
+
+  std::vector<DLDataTypeCode> dlpack_types = {
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLInt,
+      DLDataTypeCode::kDLUInt,  DLDataTypeCode::kDLInt,   
DLDataTypeCode::kDLUInt,
+      DLDataTypeCode::kDLInt,   DLDataTypeCode::kDLUInt,  
DLDataTypeCode::kDLFloat,
+      DLDataTypeCode::kDLFloat, DLDataTypeCode::kDLFloat,
+  };
+
+  for (int64_t i = 0; i < 11; ++i) {
+    const std::shared_ptr<Array> array = gen.ArrayOf(arrow_types[i], 10, 0);
+    check_dlptensor(array, arrow_types[i], dlpack_types[i], 10);
+    ASSERT_OK_AND_ASSIGN(auto sliced_1, array->SliceSafe(1, 5));
+    check_dlptensor(sliced_1, arrow_types[i], dlpack_types[i], 5);
+    ASSERT_OK_AND_ASSIGN(auto sliced_2, array->SliceSafe(0, 5));
+    check_dlptensor(sliced_2, arrow_types[i], dlpack_types[i], 5);
+    ASSERT_OK_AND_ASSIGN(auto sliced_3, array->SliceSafe(3));
+    check_dlptensor(sliced_3, arrow_types[i], dlpack_types[i], 7);
+  }

Review Comment:
   Can we also try to test that memory is properly deallocated?
   For example:
   ```c++
   TEST_F(TestExportArray, TestSupportedArray) {
     const auto allocated_bytes = default_memory_pool()->bytes_allocated();
   
     ... // function body
   
     ASSERT_EQ(allocated_bytes, default_memory_pool()->bytes_allocated());
   ```



##########
python/pyarrow/tests/test_dlpack.py:
##########
@@ -0,0 +1,129 @@
+# 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 ctypes
+import pytest
+
+import numpy as np
+
+import pyarrow as pa
+from pyarrow.vendored.version import Version
+
+
+def PyCapsule_IsValid(capsule, name):
+    return ctypes.pythonapi.PyCapsule_IsValid(ctypes.py_object(capsule), name) 
== 1
+
+
+def check_dlpack_export(arr, expected_arr):
+    DLTensor = arr.__dlpack__()
+    assert PyCapsule_IsValid(DLTensor, b"dltensor") is True
+
+    result = np.from_dlpack(arr)
+    np.testing.assert_array_equal(result, expected_arr, strict=True)
+
+    assert arr.__dlpack_device__() == (1, 0)
+
+

Review Comment:
   Can we also test that memory gets properly deallocated during these tests?
   Something like:
   ```python
   
   def check_bytes_allocated(f):
       @functools.wraps(f)
       def wrapper(*args, **kwargs):
              allocated_bytes = pa.bytes_allocated()
              try:
                  return f(*args, **kwargs)
              finally:
                  assert pa.bytes_allocated() == allocated_bytes
       return wrapper
   
   @check_bytes_allocated
   def test_dlpack(value_type, np_type):
        ....
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,138 @@
+// 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 "arrow/c/dlpack.h"
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+#include "arrow/device.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+
+namespace arrow::dlpack {
+
+Result<DLDataType> GetDLDataType(const DataType& type) {

Review Comment:
   If this function is not exposed publicly, then it should be enclosed in the 
anonymous namespace.



##########
python/pyarrow/_dlpack.pxi:
##########
@@ -0,0 +1,49 @@
+# 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.
+
+from libc.stdlib cimport malloc, free
+
+cimport cpython
+from cpython.pycapsule cimport PyCapsule_New
+from cython import sizeof

Review Comment:
   These are unused as well?



##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,49 @@ cdef class Array(_PandasConvertible):
 
         return pyarrow_wrap_array(array)
 
+    def __dlpack__(self, stream=None):
+        """Export a primitive array as a DLPack capsule.
+
+        Parameters
+        ----------
+        stream : int, optional
+            A Python integer representing a pointer to a stream. Currently not 
supported.
+            Stream is provided by the consumer to the producer to instruct the 
producer
+            to ensure that operations can safely be performed on the array.
+
+        Returns
+        -------
+        capsule : PyCapsule
+            A DLPack capsule for the array, pointing to a DLManagedTensor.
+        """
+        cdef CResult[DLManagedTensor*] c_dlm_tensor
+
+        if stream is None:
+            c_dlm_tensor = ExportToDLPack(pyarrow_unwrap_array(self))
+            dlm_tensor = GetResultValue(c_dlm_tensor)
+
+            return PyCapsule_New(dlm_tensor, 'dltensor', 
dlpack_pycapsule_deleter)
+        else:
+            raise NotImplementedError(
+                "Only stream=None is supported."
+            )
+
+    def __dlpack_device__(self):
+        """
+        Performs the operation __dlpack_device__.
+
+        Returns
+        -------
+        tuple : Tuple[DLDeviceType, int]
+            Tuple with enumerator specifying the type of the device
+            and index of the device which is 0 by default for CPU.
+        """
+        cdef CResult[DLDevice] c_device
+        c_device = ExportDevice(pyarrow_unwrap_array(self))
+        device = GetResultValue(c_device)

Review Comment:
   ```suggestion
           device = GetResultValue(ExportDevice(self.sp_array))
   ```



##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,49 @@ cdef class Array(_PandasConvertible):
 
         return pyarrow_wrap_array(array)
 
+    def __dlpack__(self, stream=None):
+        """Export a primitive array as a DLPack capsule.
+
+        Parameters
+        ----------
+        stream : int, optional
+            A Python integer representing a pointer to a stream. Currently not 
supported.
+            Stream is provided by the consumer to the producer to instruct the 
producer
+            to ensure that operations can safely be performed on the array.
+
+        Returns
+        -------
+        capsule : PyCapsule
+            A DLPack capsule for the array, pointing to a DLManagedTensor.
+        """
+        cdef CResult[DLManagedTensor*] c_dlm_tensor
+
+        if stream is None:
+            c_dlm_tensor = ExportToDLPack(pyarrow_unwrap_array(self))
+            dlm_tensor = GetResultValue(c_dlm_tensor)
+
+            return PyCapsule_New(dlm_tensor, 'dltensor', 
dlpack_pycapsule_deleter)
+        else:
+            raise NotImplementedError(
+                "Only stream=None is supported."
+            )
+
+    def __dlpack_device__(self):
+        """
+        Performs the operation __dlpack_device__.
+
+        Returns
+        -------
+        tuple : Tuple[DLDeviceType, int]

Review Comment:
   Does it really return a "DLDeviceType"? In the tests, the result is compared 
to an int.



##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,49 @@ cdef class Array(_PandasConvertible):
 
         return pyarrow_wrap_array(array)
 
+    def __dlpack__(self, stream=None):
+        """Export a primitive array as a DLPack capsule.
+
+        Parameters
+        ----------
+        stream : int, optional
+            A Python integer representing a pointer to a stream. Currently not 
supported.
+            Stream is provided by the consumer to the producer to instruct the 
producer
+            to ensure that operations can safely be performed on the array.
+
+        Returns
+        -------
+        capsule : PyCapsule
+            A DLPack capsule for the array, pointing to a DLManagedTensor.
+        """
+        cdef CResult[DLManagedTensor*] c_dlm_tensor
+
+        if stream is None:
+            c_dlm_tensor = ExportToDLPack(pyarrow_unwrap_array(self))
+            dlm_tensor = GetResultValue(c_dlm_tensor)
+
+            return PyCapsule_New(dlm_tensor, 'dltensor', 
dlpack_pycapsule_deleter)
+        else:
+            raise NotImplementedError(
+                "Only stream=None is supported."
+            )
+
+    def __dlpack_device__(self):
+        """
+        Performs the operation __dlpack_device__.

Review Comment:
   Hmm, can the docstring actually describe the "operation"?
   Something like:
   ```suggestion
           Return the DLPack device this arrays resides on.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to