You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2019/02/07 19:07:16 UTC

[arrow] branch master updated: ARROW-4318: [C++] Add Tensor::CountNonZero

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

apitrou 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 7fd18a1  ARROW-4318: [C++] Add Tensor::CountNonZero
7fd18a1 is described below

commit 7fd18a17174c19de00297f04e134342c98874e94
Author: Kenta Murata <mr...@mrkn.jp>
AuthorDate: Thu Feb 7 20:07:08 2019 +0100

    ARROW-4318: [C++] Add Tensor::CountNonZero
    
    In this pull-request, I would like to move CountNonZero defined in SparseTensorConverter into Tensor class, and add tests for this function.
    
    Author: Kenta Murata <mr...@mrkn.jp>
    Author: Antoine Pitrou <an...@python.org>
    
    Closes #3452 from mrkn/tensor_count_non_zero and squashes the following commits:
    
    fa394d95 <Antoine Pitrou> Return a Status from Tensor::CountNonZero
    ecfbbf52 <Antoine Pitrou> Fix lint
    37f286a9 <Kenta Murata> Check !is_tensor_supported in NonZeroCounter::Visit
    e714d7d0 <Kenta Murata> Fix spelling
    c9958010 <Kenta Murata> Use VisitTypeInline in CountNonZero
    ce353288 <Kenta Murata> Change the return type of CountNonZero
    4b365a0b <Kenta Murata> Add docstring of Tensor::CountNonZero
    08c8e265 <Kenta Murata> Refactoring: extract TensorCountNonZero to make a function simple
    4072ebb1 <Kenta Murata> Use anonymous namespace
    d1828ac3 <Kenta Murata> Remove needless functions
    e9bfc024 <Kenta Murata> Use Tensor::CountNonZero in SparseTensorConverter
    45552a84 <Kenta Murata> Add Tensor::CountNonZero
---
 cpp/src/arrow/sparse_tensor.cc | 53 +++++---------------------------
 cpp/src/arrow/tensor-test.cc   | 50 +++++++++++++++++++++++++++++-
 cpp/src/arrow/tensor.cc        | 70 ++++++++++++++++++++++++++++++++++++++++++
 cpp/src/arrow/tensor.h         |  3 ++
 4 files changed, 129 insertions(+), 47 deletions(-)

diff --git a/cpp/src/arrow/sparse_tensor.cc b/cpp/src/arrow/sparse_tensor.cc
index a55f51a..205c17e 100644
--- a/cpp/src/arrow/sparse_tensor.cc
+++ b/cpp/src/arrow/sparse_tensor.cc
@@ -49,45 +49,6 @@ struct SparseTensorConverterBase {
 
   explicit SparseTensorConverterBase(const NumericTensorType& tensor) : tensor_(tensor) {}
 
-  bool TensorIsTriviallyIterable() const {
-    return tensor_.ndim() <= 1 || tensor_.is_contiguous();
-  }
-
-  size_t CountNonZero() const {
-    if (tensor_.size() == 0) {
-      return 0;
-    }
-
-    if (TensorIsTriviallyIterable()) {
-      const value_type* data = reinterpret_cast<const value_type*>(tensor_.raw_data());
-      return std::count_if(data, data + tensor_.size(),
-                           [](value_type x) { return x != 0; });
-    }
-
-    const std::vector<int64_t>& shape = tensor_.shape();
-    const int64_t ndim = tensor_.ndim();
-
-    size_t count = 0;
-    std::vector<int64_t> coord(ndim, 0);
-    for (int64_t n = tensor_.size(); n > 0; n--) {
-      if (tensor_.Value(coord) != 0) {
-        ++count;
-      }
-
-      // increment index
-      ++coord[ndim - 1];
-      if (n > 1 && coord[ndim - 1] == shape[ndim - 1]) {
-        int64_t d = ndim - 1;
-        while (d > 0 && coord[d] == shape[d]) {
-          coord[d] = 0;
-          ++coord[d - 1];
-          --d;
-        }
-      }
-    }
-    return count;
-  }
-
   const NumericTensorType& tensor_;
 };
 
@@ -96,14 +57,15 @@ class SparseTensorConverter<TYPE, SparseCOOIndex>
     : private SparseTensorConverterBase<TYPE> {
  public:
   using BaseClass = SparseTensorConverterBase<TYPE>;
-  using NumericTensorType = typename BaseClass::NumericTensorType;
-  using value_type = typename BaseClass::value_type;
+  using typename BaseClass::NumericTensorType;
+  using typename BaseClass::value_type;
 
   explicit SparseTensorConverter(const NumericTensorType& tensor) : BaseClass(tensor) {}
 
   Status Convert() {
     const int64_t ndim = tensor_.ndim();
-    const int64_t nonzero_count = static_cast<int64_t>(CountNonZero());
+    int64_t nonzero_count = -1;
+    RETURN_NOT_OK(tensor_.CountNonZero(&nonzero_count));
 
     std::shared_ptr<Buffer> indices_buffer;
     RETURN_NOT_OK(
@@ -170,8 +132,7 @@ class SparseTensorConverter<TYPE, SparseCOOIndex>
   std::shared_ptr<Buffer> data;
 
  private:
-  using SparseTensorConverterBase<TYPE>::tensor_;
-  using SparseTensorConverterBase<TYPE>::CountNonZero;
+  using BaseClass::tensor_;
 };
 
 template <typename TYPE, typename SparseIndexType>
@@ -206,7 +167,8 @@ class SparseTensorConverter<TYPE, SparseCSRIndex>
 
     const int64_t nr = tensor_.shape()[0];
     const int64_t nc = tensor_.shape()[1];
-    const int64_t nonzero_count = static_cast<int64_t>(CountNonZero());
+    int64_t nonzero_count = -1;
+    RETURN_NOT_OK(tensor_.CountNonZero(&nonzero_count));
 
     std::shared_ptr<Buffer> indptr_buffer;
     std::shared_ptr<Buffer> indices_buffer;
@@ -258,7 +220,6 @@ class SparseTensorConverter<TYPE, SparseCSRIndex>
 
  private:
   using BaseClass::tensor_;
-  using SparseTensorConverterBase<TYPE>::CountNonZero;
 };
 
 // ----------------------------------------------------------------------
diff --git a/cpp/src/arrow/tensor-test.cc b/cpp/src/arrow/tensor-test.cc
index 11ea7c2..caf0322 100644
--- a/cpp/src/arrow/tensor-test.cc
+++ b/cpp/src/arrow/tensor-test.cc
@@ -31,6 +31,12 @@
 
 namespace arrow {
 
+void AssertCountNonZero(const Tensor& t, int64_t expected) {
+  int64_t count = -1;
+  ASSERT_OK(t.CountNonZero(&count));
+  ASSERT_EQ(count, expected);
+}
+
 TEST(TestTensor, ZeroDim) {
   const int64_t values = 1;
   std::vector<int64_t> shape = {};
@@ -97,7 +103,7 @@ TEST(TestTensor, IsContiguous) {
   ASSERT_FALSE(t3.is_contiguous());
 }
 
-TEST(TestTensor, ZeroDimensionalTensor) {
+TEST(TestTensor, ZeroSizedTensor) {
   std::vector<int64_t> shape = {0};
 
   std::shared_ptr<Buffer> buffer;
@@ -107,6 +113,48 @@ TEST(TestTensor, ZeroDimensionalTensor) {
   ASSERT_EQ(t.strides().size(), 1);
 }
 
+TEST(TestTensor, CountNonZeroForZeroSizedTensor) {
+  std::vector<int64_t> shape = {0};
+
+  std::shared_ptr<Buffer> buffer;
+  ASSERT_OK(AllocateBuffer(0, &buffer));
+
+  Tensor t(int64(), buffer, shape);
+  AssertCountNonZero(t, 0);
+}
+
+TEST(TestTensor, CountNonZeroForContiguousTensor) {
+  std::vector<int64_t> shape = {4, 6};
+  std::vector<int64_t> values = {1, 0,  2, 0,  0,  3, 0,  4, 5, 0,  6, 0,
+                                 0, 11, 0, 12, 13, 0, 14, 0, 0, 15, 0, 16};
+  std::shared_ptr<Buffer> buffer = Buffer::Wrap(values);
+
+  std::vector<int64_t> c_strides = {48, 8};
+  std::vector<int64_t> f_strides = {8, 32};
+  Tensor t1(int64(), buffer, shape, c_strides);
+  Tensor t2(int64(), buffer, shape, f_strides);
+
+  ASSERT_TRUE(t1.is_contiguous());
+  ASSERT_TRUE(t2.is_contiguous());
+  AssertCountNonZero(t1, 12);
+  AssertCountNonZero(t2, 12);
+}
+
+TEST(TestTensor, CountNonZeroForNonContiguousTensor) {
+  std::vector<int64_t> shape = {4, 4};
+  std::vector<int64_t> values = {
+      1, 0,  2, 0,  0,  3, 0,  4, 5, 0,  6, 0,  7, 0,  8, 0,
+      0, 11, 0, 12, 13, 0, 14, 0, 0, 15, 0, 16, 0, 15, 0, 16,
+  };
+  std::shared_ptr<Buffer> buffer = Buffer::Wrap(values);
+
+  std::vector<int64_t> noncontig_strides = {64, 16};
+  Tensor t(int64(), buffer, shape, noncontig_strides);
+
+  ASSERT_FALSE(t.is_contiguous());
+  AssertCountNonZero(t, 8);
+}
+
 TEST(TestNumericTensor, ElementAccessWithRowMajorStrides) {
   std::vector<int64_t> shape = {3, 4};
 
diff --git a/cpp/src/arrow/tensor.cc b/cpp/src/arrow/tensor.cc
index a4db298..7cd4a32 100644
--- a/cpp/src/arrow/tensor.cc
+++ b/cpp/src/arrow/tensor.cc
@@ -23,6 +23,7 @@
 #include <memory>
 #include <numeric>
 #include <string>
+#include <type_traits>
 #include <vector>
 
 #include "arrow/compare.h"
@@ -30,6 +31,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/logging.h"
+#include "arrow/visitor_inline.h"
 
 namespace arrow {
 
@@ -123,4 +125,72 @@ Type::type Tensor::type_id() const { return type_->id(); }
 
 bool Tensor::Equals(const Tensor& other) const { return TensorEquals(*this, other); }
 
+namespace {
+
+template <typename TYPE>
+int64_t StridedTensorCountNonZero(int dim_index, int64_t offset, const Tensor& tensor) {
+  using c_type = typename TYPE::c_type;
+  c_type const zero = c_type(0);
+  int64_t nnz = 0;
+  if (dim_index == tensor.ndim() - 1) {
+    for (int64_t i = 0; i < tensor.shape()[dim_index]; ++i) {
+      auto const* ptr = tensor.raw_data() + offset + i * tensor.strides()[dim_index];
+      auto& elem = *reinterpret_cast<c_type const*>(ptr);
+      if (elem != zero) ++nnz;
+    }
+    return nnz;
+  }
+  for (int64_t i = 0; i < tensor.shape()[dim_index]; ++i) {
+    nnz += StridedTensorCountNonZero<TYPE>(dim_index + 1, offset, tensor);
+    offset += tensor.strides()[dim_index];
+  }
+  return nnz;
+}
+
+template <typename TYPE>
+int64_t ContiguousTensorCountNonZero(const Tensor& tensor) {
+  using c_type = typename TYPE::c_type;
+  auto* data = reinterpret_cast<c_type const*>(tensor.raw_data());
+  return std::count_if(data, data + tensor.size(),
+                       [](c_type const& x) { return x != 0; });
+}
+
+template <typename TYPE>
+inline int64_t TensorCountNonZero(const Tensor& tensor) {
+  if (tensor.is_contiguous()) {
+    return ContiguousTensorCountNonZero<TYPE>(tensor);
+  } else {
+    return StridedTensorCountNonZero<TYPE>(0, 0, tensor);
+  }
+}
+
+struct NonZeroCounter {
+  NonZeroCounter(const Tensor& tensor, int64_t* result)
+      : tensor_(tensor), result_(result) {}
+
+  template <typename TYPE>
+  typename std::enable_if<!std::is_base_of<Number, TYPE>::value, Status>::type Visit(
+      const TYPE& type) {
+    DCHECK(!is_tensor_supported(type.id()));
+    return Status::NotImplemented("Tensor of ", type.ToString(), " is not implemented");
+  }
+
+  template <typename TYPE>
+  typename std::enable_if<std::is_base_of<Number, TYPE>::value, Status>::type Visit(
+      const TYPE& type) {
+    *result_ = TensorCountNonZero<TYPE>(tensor_);
+    return Status::OK();
+  }
+
+  const Tensor& tensor_;
+  int64_t* result_;
+};
+
+}  // namespace
+
+Status Tensor::CountNonZero(int64_t* result) const {
+  NonZeroCounter counter(*this, result);
+  return VisitTypeInline(*type(), &counter);
+}
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/tensor.h b/cpp/src/arrow/tensor.h
index fb2093b..3171502 100644
--- a/cpp/src/arrow/tensor.h
+++ b/cpp/src/arrow/tensor.h
@@ -104,6 +104,9 @@ class ARROW_EXPORT Tensor {
 
   bool Equals(const Tensor& other) const;
 
+  /// Compute the number of non-zero values in the tensor
+  Status CountNonZero(int64_t* result) const;
+
  protected:
   Tensor() {}