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


##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,40 @@ 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, containing a DLPackManagedTensor.
+        """
+        if stream is None:
+            return to_dlpack(self)
+        else:
+            raise NotImplementedError(
+                "Only stream=None is supported."
+            )
+
+    def __dlpack_device__(self):
+        """
+        Performs the operation __dlpack_device__.

Review Comment:
   Where is this specified? I see nothing in 
https://dmlc.github.io/dlpack/latest/python_spec.html



##########
cpp/src/arrow/c/dlpack.h:
##########
@@ -0,0 +1,58 @@
+// 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.
+
+#pragma once
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+
+namespace arrow {
+
+namespace dlpack {

Review Comment:
   Nit: can use C++17 nested namespace declarations
   ```suggestion
   namespace arrow::dlpack {
   ```
   
   (you'll have to change the closing braces as well)



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {

Review Comment:
   1. naming style, 2. Result instead of out-ptr, 3. no need for shared_ptr 
here :-)
   ```suggestion
   Result<DLDataType> GetDLDataType(const DataType& type) {
   ```



##########
cpp/src/arrow/c/dlpack.h:
##########
@@ -0,0 +1,58 @@
+// 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.
+
+#pragma once
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+
+namespace arrow {
+
+namespace dlpack {
+
+/// \brief DLPack protocol for producing DLManagedTensor
+///
+/// DLMangedTensor is produced from an array as defined by
+/// the DLPack protocol, see https://dmlc.github.io/dlpack/latest/.
+///
+/// Data types for which the protocol is supported are
+/// primitive data types without NullType, BooleanType and
+/// Decimal types.
+///
+/// DLPack protocol only supports arrays with one contiguous
+/// memory region which means Arrow Arrays with validity buffers
+/// are not supported.
+///
+/// \param[in] arr Arrow array
+/// \param[out] out DLManagedTensor struct
+/// \return Status
+ARROW_EXPORT
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out);
+
+/// \brief Get DLDevice with enumerator specifying the
+/// type of the device data is stored on and index of the
+/// device which is 0 by default for CPU.
+///
+/// \param[in] arr Arrow array
+/// \param[out] out DLDevice struct
+/// \return DLDevice
+ARROW_EXPORT
+Status ExportDevice(const std::shared_ptr<Array>& arr, DLDevice* out);

Review Comment:
   ```suggestion
   Result<DLDevice> ExportDevice(const std::shared_ptr<Array>& arr);
   ```



##########
cpp/src/arrow/c/dlpack_test.cc:
##########
@@ -0,0 +1,122 @@
+// 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 {
+
+namespace dlpack {

Review Comment:
   Nit
   ```suggestion
   namespace arrow::dlpack {
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}

Review Comment:
   Do we plan to later support `fixed_shape_tensor` extension arrays?



##########
cpp/src/arrow/c/dlpack.h:
##########
@@ -0,0 +1,58 @@
+// 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.
+
+#pragma once
+
+#include "arrow/array/array_base.h"
+#include "arrow/c/dlpack_abi.h"
+
+namespace arrow {
+
+namespace dlpack {
+
+/// \brief DLPack protocol for producing DLManagedTensor
+///
+/// DLMangedTensor is produced from an array as defined by
+/// the DLPack protocol, see https://dmlc.github.io/dlpack/latest/.
+///
+/// Data types for which the protocol is supported are
+/// primitive data types without NullType, BooleanType and
+/// Decimal types.
+///
+/// DLPack protocol only supports arrays with one contiguous
+/// memory region which means Arrow Arrays with validity buffers
+/// are not supported.
+///
+/// \param[in] arr Arrow array
+/// \param[out] out DLManagedTensor struct
+/// \return Status
+ARROW_EXPORT
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out);

Review Comment:
   Instead of returning a Status and writing into a separate out pointer, we 
should simply return a Result:
   ```suggestion
   Result<DLManagedTensor*> ExportArray(const std::shared_ptr<Array>& arr);
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");
+  }
+
+  // Define the DLDataType struct
+  // Supported data types: int, uint, float
+  DLDataType arr_type;
+  RETURN_NOT_OK(getDLDataType(arr->type(), &arr_type));
+
+  // Create DLMTensorCtx struct with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  DLMTensorCtx* DLMTensor = new DLMTensorCtx;
+  DLMTensor->ref = array_ref;
+
+  // Define DLManagedTensor struct defined by
+  // DLPack (dlpack_structure.h)
+  DLManagedTensor* dlm_tensor = &DLMTensor->tensor;
+  dlm_tensor->manager_ctx = DLMTensor;
+  dlm_tensor->deleter = &deleter;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL
+  if (arr->length() == 0) {
+    dlm_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));
+    dlm_tensor->dl_tensor.data =
+        const_cast<void*>(reinterpret_cast<const 
void*>(sliced_buffer->address()));
+  } else {
+    dlm_tensor->dl_tensor.data = const_cast<void*>(
+        reinterpret_cast<const void*>(array_ref->buffers[1]->address()));
+  }
+
+  // Define DLDevice struct
+  DLDevice ctx;
+  if (array_ref->buffers[1]->device_type() == DeviceAllocationType::kCPU) {
+    ctx.device_id = 0;
+    ctx.device_type = DLDeviceType::kDLCPU;
+  } else {
+    return Status::NotImplemented(
+        "DLPack support is implemented only for buffers on CPU device.");
+  }
+  dlm_tensor->dl_tensor.device = ctx;
+
+  dlm_tensor->dl_tensor.ndim = 1;
+  dlm_tensor->dl_tensor.dtype = arr_type;
+  std::vector<int64_t>* shape_arr = &DLMTensor->shape;
+  shape_arr->resize(1);
+  (*shape_arr)[0] = arr->length();
+  dlm_tensor->dl_tensor.shape = shape_arr->data();
+  dlm_tensor->dl_tensor.strides = NULL;
+  dlm_tensor->dl_tensor.byte_offset = 0;
+
+  *out = dlm_tensor;
+  return Status::OK();
+}
+
+Status ExportDevice(const std::shared_ptr<Array>& arr, DLDevice* out) {
+  DLDevice device;
+  if (arr->data()->buffers[1]->device_type() == DeviceAllocationType::kCPU) {

Review Comment:
   We shouldn't blindly access buffer #1.
   Instead we should first check that the array type is supported by DLPack, 
and raise a `TypeError` if not.
   



##########
python/pyarrow/_dlpack.pxi:
##########
@@ -0,0 +1,62 @@
+# 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
+
+
+cdef void pycapsule_deleter(object dltensor) noexcept:
+    cdef DLManagedTensor* dlm_tensor
+    cdef PyObject* err_type
+    cdef PyObject* err_value
+    cdef PyObject* err_traceback
+
+    # Do nothing if the capsule has been consumed
+    if cpython.PyCapsule_IsValid(dltensor, "used_dltensor"):
+        return
+
+    # An exception may be in-flight, we must save it in case
+    # we create another one
+    cpython.PyErr_Fetch(&err_type, &err_value, &err_traceback)
+
+    if cpython.PyCapsule_IsValid(dltensor, 'dltensor'):
+        dlm_tensor = <DLManagedTensor*>cpython.PyCapsule_GetPointer(
+            dltensor, 'dltensor')
+        dlm_tensor.deleter(dlm_tensor)
+    else:
+        cpython.PyErr_WriteUnraisable(dltensor)
+
+    # Set the error indicator from err_type, err_value, err_traceback
+    cpython.PyErr_Restore(err_type, err_value, err_traceback)
+
+
+cpdef object to_dlpack(Array arr) except *:
+
+    cdef DLManagedTensor* dlm_tensor
+    check_status(ExportToDLPack(pyarrow_unwrap_array(arr), &dlm_tensor))
+
+    return PyCapsule_New(dlm_tensor, 'dltensor', pycapsule_deleter)
+
+cpdef object dlpack_device(Array arr) except *:
+
+    cdef DLDevice device
+    check_status(ExportDevice(pyarrow_unwrap_array(arr), &device))
+
+    return (device.device_type, device.device_id)

Review Comment:
   This code could go directly in the Array method implementations, I don't 
think it's useful to have them here.



##########
python/pyarrow/tests/test_array.py:
##########
@@ -3546,3 +3548,123 @@ def test_run_end_encoded_from_buffers():
     with pytest.raises(ValueError):
         pa.RunEndEncodedArray.from_buffers(ree_type, length, buffers,
                                            1, offset, children)
+
+
+def PyCapsule_IsValid(capsule, name):
+    return ctypes.pythonapi.PyCapsule_IsValid(ctypes.py_object(capsule), name) 
== 1
+
+
[email protected](
+    ('value_type', 'np_type'),
+    [
+        (pa.uint8(), np.uint8),
+        (pa.uint16(), np.uint16),
+        (pa.uint32(), np.uint32),
+        (pa.uint64(), np.uint64),
+        (pa.int8(), np.int8),
+        (pa.int16(), np.int16),
+        (pa.int32(), np.int32),
+        (pa.int64(), np.int64),
+        (pa.float16(), np.float16),
+        (pa.float32(), np.float32),
+        (pa.float64(), np.float64),
+    ]
+)
+def test_dlpack(value_type, np_type):
+    if Version(np.__version__) < Version("1.24.0"):
+        pytest.skip("No dlpack support in numpy versions older than 1.22.0, "
+                    "strict keyward in assert_array_equal added in numpy 
version "

Review Comment:
   ```suggestion
                       "strict keyword in assert_array_equal added in numpy 
version "
   ```



##########
python/pyarrow/array.pxi:
##########
@@ -1778,6 +1778,40 @@ 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, containing a DLPackManagedTensor.

Review Comment:
   ```suggestion
               A DLPack capsule for the array, pointing to a DLManagedTensor.
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");

Review Comment:
   DLPack also doesn't support temporals, strings, etc.
   ```suggestion
         return Status::TypeError(
             "DataType is not compatible with DLPack spec: ", type.ToString());
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");
+  }
+
+  // Define the DLDataType struct
+  // Supported data types: int, uint, float
+  DLDataType arr_type;
+  RETURN_NOT_OK(getDLDataType(arr->type(), &arr_type));
+
+  // Create DLMTensorCtx struct with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  DLMTensorCtx* DLMTensor = new DLMTensorCtx;
+  DLMTensor->ref = array_ref;
+
+  // Define DLManagedTensor struct defined by
+  // DLPack (dlpack_structure.h)
+  DLManagedTensor* dlm_tensor = &DLMTensor->tensor;
+  dlm_tensor->manager_ctx = DLMTensor;
+  dlm_tensor->deleter = &deleter;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL
+  if (arr->length() == 0) {
+    dlm_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));
+    dlm_tensor->dl_tensor.data =
+        const_cast<void*>(reinterpret_cast<const 
void*>(sliced_buffer->address()));
+  } else {
+    dlm_tensor->dl_tensor.data = const_cast<void*>(
+        reinterpret_cast<const void*>(array_ref->buffers[1]->address()));
+  }
+
+  // Define DLDevice struct
+  DLDevice ctx;
+  if (array_ref->buffers[1]->device_type() == DeviceAllocationType::kCPU) {
+    ctx.device_id = 0;
+    ctx.device_type = DLDeviceType::kDLCPU;
+  } else {
+    return Status::NotImplemented(
+        "DLPack support is implemented only for buffers on CPU device.");
+  }
+  dlm_tensor->dl_tensor.device = ctx;
+
+  dlm_tensor->dl_tensor.ndim = 1;
+  dlm_tensor->dl_tensor.dtype = arr_type;
+  std::vector<int64_t>* shape_arr = &DLMTensor->shape;
+  shape_arr->resize(1);
+  (*shape_arr)[0] = arr->length();
+  dlm_tensor->dl_tensor.shape = shape_arr->data();

Review Comment:
   This is not necessary, we could simply point to the existing length and 
remove `DLMTensorCtx::shape`:
   ```suggestion
     dlm_tensor->dl_tensor.shape = const_cast<int64_t*>(&array_ref->length);
   ```



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");

Review Comment:
   Let's not use the Python spelling as this could be used directly from 
third-party C++ code:
   ```suggestion
       return Status::TypeError("Can only use DLPack on arrays with no nulls.");
   ```



##########
python/pyarrow/_dlpack.pxi:
##########
@@ -0,0 +1,62 @@
+# 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
+
+
+cdef void pycapsule_deleter(object dltensor) noexcept:
+    cdef DLManagedTensor* dlm_tensor
+    cdef PyObject* err_type
+    cdef PyObject* err_value
+    cdef PyObject* err_traceback
+
+    # Do nothing if the capsule has been consumed
+    if cpython.PyCapsule_IsValid(dltensor, "used_dltensor"):
+        return
+
+    # An exception may be in-flight, we must save it in case
+    # we create another one
+    cpython.PyErr_Fetch(&err_type, &err_value, &err_traceback)
+
+    if cpython.PyCapsule_IsValid(dltensor, 'dltensor'):
+        dlm_tensor = <DLManagedTensor*>cpython.PyCapsule_GetPointer(
+            dltensor, 'dltensor')
+        dlm_tensor.deleter(dlm_tensor)
+    else:
+        cpython.PyErr_WriteUnraisable(dltensor)

Review Comment:
   This writes the raised exception but nothing is raised by 
`PyCapsule_IsValid`. Instead, you can probably call `PyCapsule_GetPointer` 
directly and check the result...



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");
+  }
+
+  // Define the DLDataType struct
+  // Supported data types: int, uint, float
+  DLDataType arr_type;
+  RETURN_NOT_OK(getDLDataType(arr->type(), &arr_type));
+
+  // Create DLMTensorCtx struct with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  DLMTensorCtx* DLMTensor = new DLMTensorCtx;
+  DLMTensor->ref = array_ref;
+
+  // Define DLManagedTensor struct defined by
+  // DLPack (dlpack_structure.h)
+  DLManagedTensor* dlm_tensor = &DLMTensor->tensor;
+  dlm_tensor->manager_ctx = DLMTensor;
+  dlm_tensor->deleter = &deleter;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL
+  if (arr->length() == 0) {
+    dlm_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));
+    dlm_tensor->dl_tensor.data =
+        const_cast<void*>(reinterpret_cast<const 
void*>(sliced_buffer->address()));
+  } else {
+    dlm_tensor->dl_tensor.data = const_cast<void*>(
+        reinterpret_cast<const void*>(array_ref->buffers[1]->address()));
+  }
+
+  // Define DLDevice struct
+  DLDevice ctx;
+  if (array_ref->buffers[1]->device_type() == DeviceAllocationType::kCPU) {
+    ctx.device_id = 0;
+    ctx.device_type = DLDeviceType::kDLCPU;
+  } else {
+    return Status::NotImplemented(
+        "DLPack support is implemented only for buffers on CPU device.");
+  }

Review Comment:
   We should not duplicate this code since this is already implemented in 
`ExportDevice` below...



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");
+  }
+
+  // Define the DLDataType struct
+  // Supported data types: int, uint, float
+  DLDataType arr_type;
+  RETURN_NOT_OK(getDLDataType(arr->type(), &arr_type));
+
+  // Create DLMTensorCtx struct with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  DLMTensorCtx* DLMTensor = new DLMTensorCtx;
+  DLMTensor->ref = array_ref;
+
+  // Define DLManagedTensor struct defined by
+  // DLPack (dlpack_structure.h)
+  DLManagedTensor* dlm_tensor = &DLMTensor->tensor;
+  dlm_tensor->manager_ctx = DLMTensor;
+  dlm_tensor->deleter = &deleter;
+
+  // Define the data pointer to the DLTensor
+  // If array is of length 0, data pointer should be NULL

Review Comment:
   Is this specified somewhere?



##########
python/pyarrow/tests/test_array.py:
##########
@@ -3546,3 +3548,123 @@ def test_run_end_encoded_from_buffers():
     with pytest.raises(ValueError):
         pa.RunEndEncodedArray.from_buffers(ree_type, length, buffers,
                                            1, offset, children)
+
+
+def PyCapsule_IsValid(capsule, name):
+    return ctypes.pythonapi.PyCapsule_IsValid(ctypes.py_object(capsule), name) 
== 1
+
+
[email protected](
+    ('value_type', 'np_type'),
+    [
+        (pa.uint8(), np.uint8),
+        (pa.uint16(), np.uint16),
+        (pa.uint32(), np.uint32),
+        (pa.uint64(), np.uint64),
+        (pa.int8(), np.int8),
+        (pa.int16(), np.int16),
+        (pa.int32(), np.int32),
+        (pa.int64(), np.int64),
+        (pa.float16(), np.float16),
+        (pa.float32(), np.float32),
+        (pa.float64(), np.float64),
+    ]
+)
+def test_dlpack(value_type, np_type):
+    if Version(np.__version__) < Version("1.24.0"):
+        pytest.skip("No dlpack support in numpy versions older than 1.22.0, "
+                    "strict keyward in assert_array_equal added in numpy 
version "
+                    "1.24.0")
+
+    expected = np.array([1, 2, 3], dtype=np_type)
+    arr = pa.array(expected, type=value_type)
+    DLTensor = arr.__dlpack__()
+    assert PyCapsule_IsValid(DLTensor, b"dltensor") is True
+    result = np.from_dlpack(arr)
+    np.testing.assert_array_equal(result, expected, strict=True)
+
+    assert arr.__dlpack_device__() == (1, 0)
+
+    arr_sliced = arr.slice(1, 1)
+    DLTensor = arr_sliced.__dlpack__()
+    assert PyCapsule_IsValid(DLTensor, b"dltensor") is True
+    expected = np.array([2], dtype=np_type)
+    result = np.from_dlpack(arr_sliced)
+    np.testing.assert_array_equal(result, expected, strict=True)

Review Comment:
   Can we avoid such copy-pasting in tests? There could be some test helper(s) 
such that you would write, e.g.:
   ```suggestion
       check_dlpack_export(arr_sliced, expected=np.array([2], dtype=np_type))
   ```



##########
python/pyarrow/tests/test_array.py:
##########
@@ -3546,3 +3548,123 @@ def test_run_end_encoded_from_buffers():
     with pytest.raises(ValueError):
         pa.RunEndEncodedArray.from_buffers(ree_type, length, buffers,
                                            1, offset, children)
+

Review Comment:
   Hmm, can we move all these tests into a separate `test_dlpack.py`?



##########
cpp/src/arrow/c/dlpack.cc:
##########
@@ -0,0 +1,151 @@
+// 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"
+
+namespace arrow {
+
+namespace dlpack {
+
+Status getDLDataType(const std::shared_ptr<DataType>& type, DLDataType* out) {
+  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;
+      *out = dtype;
+      return Status::OK();
+    case Type::UINT8:
+    case Type::UINT16:
+    case Type::UINT32:
+    case Type::UINT64:
+      dtype.code = DLDataTypeCode::kDLUInt;
+      *out = dtype;
+      return Status::OK();
+    case Type::HALF_FLOAT:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+      dtype.code = DLDataTypeCode::kDLFloat;
+      *out = dtype;
+      return Status::OK();
+    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(
+          "Can only use __dlpack__ on primitive arrays without NullType and 
Decimal "
+          "types.");
+  }
+}
+
+struct DLMTensorCtx {
+  std::shared_ptr<ArrayData> ref;
+  std::vector<int64_t> shape;
+  DLManagedTensor tensor;
+};
+
+static void deleter(DLManagedTensor* arg) {
+  delete static_cast<DLMTensorCtx*>(arg->manager_ctx);
+}
+
+Status ExportArray(const std::shared_ptr<Array>& arr, DLManagedTensor** out) {
+  if (arr->null_count() > 0) {
+    return Status::TypeError("Can only use __dlpack__ on arrays with no 
nulls.");
+  }
+
+  // Define the DLDataType struct
+  // Supported data types: int, uint, float
+  DLDataType arr_type;
+  RETURN_NOT_OK(getDLDataType(arr->type(), &arr_type));
+
+  // Create DLMTensorCtx struct with the reference to
+  // the data of the array
+  std::shared_ptr<ArrayData> array_ref = arr->data();
+  DLMTensorCtx* DLMTensor = new DLMTensorCtx;

Review Comment:
   Indeed, something like this:
   ```c++
   
   struct ManagerCtx {
     std::shared_ptr<ArrayData> ref;
     DLManagedTensor tensor;
   };
   
   std::unique_ptr<ManagerCtx> ctx(new ManagerCtx);
   ctx->ref = array_ref;
   // fill struct fields
   ...
   // end of function
   ctx->tensor->manager_ctx = ctx.release();
   ctx->tensor->deleter = [](struct DLManagedTensor *self) {
     delete reinterpret_cast<ManagerCtx*>(self->manager_ctx);
   };
   return &ctx->tensor;
   ```



-- 
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