You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@singa.apache.org by zh...@apache.org on 2016/06/24 05:03:41 UTC

incubator-singa git commit: SINGA-205 - Enable slice and concatenate operations for Tensor objects

Repository: incubator-singa
Updated Branches:
  refs/heads/dev 14d31a44f -> 9abd7910d


SINGA-205 - Enable slice and concatenate operations for Tensor objects

Update the Block class to add the `offset_` field and change the type of
ref_count_ to shared_ptr<atomic<int>>.
Now, we can share the ref_count_ and data_ between different Blocks.
It is useful if Block A is a sub-block of Blokc B, i.e., Block A's data() is Block B's data() + offset.
This feature is not used currently.

Implement CopyRows, CopyColumns, ConcatenateRows and ConcatenateColumns.
SliceRows() is not implmented yet, which may affect the block management
(some blocks sharing internal data).


Project: http://git-wip-us.apache.org/repos/asf/incubator-singa/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-singa/commit/9abd7910
Tree: http://git-wip-us.apache.org/repos/asf/incubator-singa/tree/9abd7910
Diff: http://git-wip-us.apache.org/repos/asf/incubator-singa/diff/9abd7910

Branch: refs/heads/dev
Commit: 9abd7910d5f4d19202813150339f51cf0500eab1
Parents: 14d31a4
Author: Wei Wang <wa...@comp.nus.edu.sg>
Authored: Thu Jun 23 12:24:01 2016 +0800
Committer: Wei Wang <wa...@comp.nus.edu.sg>
Committed: Fri Jun 24 12:50:29 2016 +0800

----------------------------------------------------------------------
 include/singa/core/common.h    |  28 +++++----
 include/singa/core/tensor.h    |  19 +++++-
 src/core/tensor/tensor.cc      |  97 ++++++++++++++++++++++++++++++
 test/singa/test_tensor_math.cc | 114 ++++++++++++++++++++++++++++++++++++
 4 files changed, 246 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/9abd7910/include/singa/core/common.h
----------------------------------------------------------------------
diff --git a/include/singa/core/common.h b/include/singa/core/common.h
index b556750..22a2b49 100644
--- a/include/singa/core/common.h
+++ b/include/singa/core/common.h
@@ -20,6 +20,8 @@
 #define SINGA_CORE_COMMON_H_
 #include <random>
 #include <chrono>
+#include <atomic>
+#include <memory>
 #include "./singa_config.h"
 #include "singa/utils/logging.h"
 
@@ -31,7 +33,7 @@
 #include <cudnn.h>
 #endif
 #endif
-
+using std::atomic;
 namespace singa {
 namespace lang {
 /// To implemente functions using cpp libraries
@@ -45,25 +47,29 @@ typedef struct _Opencl { } Opencl;
 /// Block represent a chunk of memory (on device or host).
 class Block {
  public:
-  Block(void* ptr, size_t size) : data_(ptr), size_(size), ref_count_(1) {}
-  void* mutable_data() const { return data_; }
-  const void* data() const { return data_; }
+  Block(void* ptr, size_t size, size_t offset = 0)
+      : data_(ptr), size_(size), offset_(offset) {
+    ref_count_ = std::make_shared<std::atomic<int>>(1);
+  }
+  Block(void* ptr, size_t size, size_t offset, std::shared_ptr<atomic<int>> ref)
+      : data_(ptr), size_(size), offset_(offset), ref_count_(ref) {}
+  void* mutable_data() const { return static_cast<char*>(data_) + offset_; }
+  const void* data() const { return static_cast<char*>(data_) + offset_; }
   size_t size() const { return size_; }
+  size_t offset() const { return offset_; }
   int IncRefCount() {
-    ref_count_++;
-    return ref_count_;
+    return (*ref_count_)++;
   }
   int DecRefCount() {
-    ref_count_--;
-    CHECK_GE(ref_count_, 0);
-    return ref_count_;
+    return  (*ref_count_)--;
   }
-  int ref_count() const { return ref_count_; }
+  int ref_count() const { return ref_count_->load(); }
 
  private:
   void* data_ = nullptr;
   size_t size_ = 0;
-  int ref_count_ = 0;
+  size_t offset_ = 0;
+  std::shared_ptr<std::atomic<int>> ref_count_ = nullptr;
 };
 
 typedef struct _Context {

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/9abd7910/include/singa/core/tensor.h
----------------------------------------------------------------------
diff --git a/include/singa/core/tensor.h b/include/singa/core/tensor.h
index 7e51f97..eb72bd3 100644
--- a/include/singa/core/tensor.h
+++ b/include/singa/core/tensor.h
@@ -68,6 +68,7 @@ class Tensor {
   /// Users should not operate against Block directly.
   /// block_ is allocated in constructors.
   Block *block() const { return block_; }
+  void SetBlock(Block* block);
 
   Device *device() const { return device_; }
 
@@ -214,7 +215,7 @@ Tensor Reshape(const Tensor &in, Shape &&s);
 /// Copy 'num' elements of src to dst.
 /// The first 'src_offset' ('dst_offset') elements will be skipped.
 void CopyDataToFrom(Tensor *dst, const Tensor &src, const size_t num,
-                    const size_t src_offset = 0, const size_t dst_offset = 0);
+                    const size_t dst_offset = 0, const size_t src_offset = 0);
 
 // =============Element-wise operations====================================
 Tensor Abs(const Tensor &in);
@@ -398,6 +399,22 @@ void ComputeCrossEntropy(const Tensor& p, const Tensor& t, Tensor* loss);
 /// the target (ground truth) labels 't'. 'p' and 't' are either 1-d vector
 /// or 2-d matrix. 'grad' has the same shape as 'p'. dx is computed into p.
 void SoftmaxCrossEntropyBwd(const Tensor& t, Tensor* p);
+
+/// Return a tensor consisting of rows ([start, end)) from 'in'. It shares the
+/// memory with 'in'. 'in' is a 1D or 2D Tensor.
+Tensor SliceRows(const Tensor& in, const size_t start, const size_t end);
+/// Return a tensor consisting of rows ([start, end)) from 'in'. It copies the
+/// values from 'in'. 'in' ia a 2D Tensor.
+Tensor CopyRows(const Tensor& in, const size_t start, const size_t end);
+/// Return a tensor consisting of columns ([start, end)) from 'in'. It copies
+/// the values from 'in'. 'in' is a  2D Tensor.
+Tensor CopyColumns(const Tensor& in, const size_t start, const size_t end);
+/// Return a tensor which is vertically stacked from tensors in 'in'. Each
+/// tensor in 'in' is a 2D tensor. Values are copied, no memory sharing.
+Tensor ConcatenateRows(const vector<Tensor>& in);
+/// Return a tensor which is horizontally stacked from tensors in 'in'. Each
+/// tensor in 'in' is a 2D tensor. Values are copied, no memory sharing.
+Tensor ConcatenateColumns(const vector<Tensor>& in);
 }  // namespace singa
 
 #endif  // SINGA_CORE_TENSOR_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/9abd7910/src/core/tensor/tensor.cc
----------------------------------------------------------------------
diff --git a/src/core/tensor/tensor.cc b/src/core/tensor/tensor.cc
index 3aa5c0a..9b3eeff 100644
--- a/src/core/tensor/tensor.cc
+++ b/src/core/tensor/tensor.cc
@@ -69,6 +69,14 @@ Tensor::Tensor(Tensor &&in)
   in.block_ = nullptr;
 }
 
+void Tensor::SetBlock(Block* block) {
+  LOG(WARNING) << "Pls avoid using this function, which may have side-effect.";
+  if (block_ != nullptr)
+    if (block_->DecRefCount())
+      device_->FreeBlock(block_);
+  block_ = block;
+}
+
 void Tensor::ResetLike(const Tensor &in) {
   if (block_ == nullptr || device_ != in.device_ || MemSize() != in.MemSize()) {
     if (block_ != nullptr && block_->DecRefCount() == 0)
@@ -577,6 +585,76 @@ void DivColumn(const Tensor &v, Tensor *M) {
   MultColumn(inv, M);
 }
 
+Tensor ConcatenateRows(const vector<Tensor> &in) {
+  size_t nrow = 0, ncol = 0;
+  CHECK(in.size());
+  for (const auto &x : in) {
+    CHECK(!x.transpose());
+    CHECK_EQ(x.nDim(), 2u);
+    nrow += x.shape(0);
+    if (ncol == 0)
+      ncol = x.shape(1);
+    else
+      CHECK_EQ(ncol, x.shape(1));
+  }
+  Tensor out(Shape{nrow, ncol}, in.at(0).device(), in.at(0).data_type());
+  size_t dst_offset = 0;
+  for (const auto &x : in) {
+    CopyDataToFrom(&out, x, x.Size(), dst_offset, 0);
+    dst_offset += x.Size();
+  }
+  return out;
+}
+
+// TODO(wangwei) add a copypatch function for improve the efficiency on GPU.
+Tensor ConcatenateColumns(const vector<Tensor> &in) {
+  size_t nrow = 0, ncol = 0;
+  CHECK(in.size());
+  for (const auto &x : in) {
+    CHECK(!x.transpose());
+    CHECK_EQ(x.nDim(), 2u);
+    ncol += x.shape(1);
+    if (nrow == 0)
+      nrow = x.shape(0);
+    else
+      CHECK_EQ(nrow, x.shape(0));
+  }
+  Tensor out(Shape{nrow, ncol}, in.at(0).device(), in.at(0).data_type());
+  for (size_t row = 0; row < nrow; row++) {
+    size_t dst_offset = row * ncol;
+    for (const auto &x : in) {
+      size_t src_offset = row * x.shape(1);
+      CopyDataToFrom(&out, x, x.shape(1), dst_offset, src_offset);
+      dst_offset += x.shape(1);
+    }
+    CHECK_EQ(dst_offset, row * ncol + ncol);
+  }
+  return out;
+}
+Tensor CopyRows(const Tensor &in, const size_t start, const size_t end) {
+  CHECK_EQ(in.nDim(), 2u);
+  CHECK_LT(start, end);
+  CHECK_GE(in.shape(0), end);
+  Shape s;
+  s = Shape{end - start, in.shape(1)};
+  Tensor out(s, in.device(), in.data_type());
+  CopyDataToFrom(&out, in, out.Size(), 0, start * out.shape(1));
+  return out;
+}
+Tensor CopyColumns(const Tensor &in, const size_t start, const size_t end) {
+  CHECK_EQ(in.nDim(), 2u);
+  CHECK_LT(start, end);
+  CHECK_GE(in.shape(1), end);
+  Shape s{in.shape(0), end - start};
+  Tensor out(s, in.device(), in.data_type());
+  for (size_t row = 0; row < out.shape(0); row++) {
+    size_t src_offset = row * in.shape(1) + start;
+    size_t dst_offset = row * out.shape(1);
+    CopyDataToFrom(&out, in, end - start, dst_offset, src_offset);
+  }
+  return out;
+}
+
 /// Divide row 'v' by each row of matrix M; write results into 'out'
 void DivRow(const Tensor &v, Tensor *M) {
   Tensor inv;
@@ -614,6 +692,23 @@ void MultRow(const Tensor &v, Tensor *M) {
   });
 }
 
+Tensor SliceRows(const Tensor& in, const size_t start, const size_t end) {
+  LOG(FATAL) << "Tensor::SliceRows is not implemented";
+  Tensor ret;
+  /*
+  CHECK_LE(in.nDim(), 2);
+  CHECK_LT(start, end);
+  CHECK_LE(in.shape(0), end);
+  Shape s;
+  if (in.nDim() == 2)
+    s = Shape{end - start, in.shape(1)};
+  else
+    s = Shape{end - start};
+  Tensor out(s, in.device(), in.data_type());
+  Block *b = out.block();
+  */
+  return ret;
+}
 void SubColumn(const Tensor &v, Tensor *M) { AddColumn(-1, 1, v, M); }
 
 void SubRow(const Tensor &v, Tensor *M) { AddRow(-1, 1, v, M); }
@@ -766,4 +861,6 @@ void SoftmaxCrossEntropyBwd(const Tensor &t, Tensor *p) {
     }, {p->block(), t.block()}, {p->block()});
   });
 }
+
+
 }  // namespace singa

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/9abd7910/test/singa/test_tensor_math.cc
----------------------------------------------------------------------
diff --git a/test/singa/test_tensor_math.cc b/test/singa/test_tensor_math.cc
index 7deb339..a40a848 100644
--- a/test/singa/test_tensor_math.cc
+++ b/test/singa/test_tensor_math.cc
@@ -760,3 +760,117 @@ TEST_F(TestTensorMath, SumColumnCuda) {
 }
 
 #endif
+
+TEST_F(TestTensorMath, ConcatenateRowsCpp) {
+  d.CopyDataFromHostPtr<float>(dat1, 6);
+  e.CopyDataFromHostPtr<float>(dat2, 6);
+  const auto ret = singa::ConcatenateRows(vector<Tensor>{d, e});
+  EXPECT_EQ(ret.shape(0), d.shape(0) + e.shape(0));
+  EXPECT_EQ(ret.shape(1), d.shape(1));
+  const float *retPtr = ret.data<float>();
+  for (int i = 0; i < 6; i++) EXPECT_FLOAT_EQ(retPtr[i], dat1[i]);
+  for (int i = 0; i < 6; i++) EXPECT_FLOAT_EQ(retPtr[i + 6], dat2[i]);
+}
+
+TEST_F(TestTensorMath, ConcatenateColumnsCpp) {
+  d.CopyDataFromHostPtr<float>(dat1, 6);
+  e.CopyDataFromHostPtr<float>(dat2, 6);
+  const auto ret = singa::ConcatenateColumns(vector<Tensor>{d, e});
+  EXPECT_EQ(ret.shape(0), d.shape(0));
+  EXPECT_EQ(ret.shape(1), d.shape(1) + e.shape(1));
+
+  const float *retPtr = ret.data<float>();
+  for (int i = 0; i < 3; i++) {
+    for (int j = 0; j < 2; j++)
+      EXPECT_FLOAT_EQ(retPtr[i * 4 + j], dat1[i * 2 + j]);
+    for (int j = 0; j < 2; j++)
+      EXPECT_FLOAT_EQ(retPtr[i * 4 + 2 + j], dat2[i * 2 + j]);
+  }
+}
+
+TEST_F(TestTensorMath, CopyRowsCpp) {
+  const auto ret = singa::CopyRows(e, 1, 2);
+  EXPECT_EQ(ret.shape(0), 1u);
+  EXPECT_EQ(ret.shape(1), e.shape(1));
+  const float *retPtr = ret.data<float>();
+  for (size_t i = 0; i < ret.Size(); i++)
+    EXPECT_FLOAT_EQ(retPtr[i], dat1[1 * 2 + i]);
+}
+
+TEST_F(TestTensorMath, CopyColumnsCpp) {
+  a.Reshape(Shape{2, 3});
+  const auto ret = singa::CopyColumns(a, 1, 3);
+  EXPECT_EQ(ret.shape(0), a.shape(0));
+  EXPECT_EQ(ret.shape(1), 2u);
+  const float *retPtr = ret.data<float>();
+  for (size_t i = 0; i < ret.shape(0); i++)
+    for (size_t j = 0; j < ret.shape(1); j++)
+      EXPECT_FLOAT_EQ(retPtr[i * ret.shape(1) + j],
+                      dat1[i * a.shape(1) + j + 1]);
+}
+
+#ifdef USE_CUDA
+
+TEST_F(TestTensorMath, ConcatenateRowsCuda) {
+  singa::CudaGPU dev;
+  d.ToDevice(&dev);
+  e.ToDevice(&dev);
+  d.CopyDataFromHostPtr<float>(dat1, 6);
+  e.CopyDataFromHostPtr<float>(dat2, 6);
+  auto ret = singa::ConcatenateRows(vector<Tensor>{d, e});
+  EXPECT_EQ(ret.shape(0), d.shape(0) + e.shape(0));
+  EXPECT_EQ(ret.shape(1), d.shape(1));
+  ret.ToHost();
+  const float *retPtr = ret.data<float>();
+  for (int i = 0; i < 6; i++) EXPECT_FLOAT_EQ(retPtr[i], dat1[i]);
+  for (int i = 0; i < 6; i++) EXPECT_FLOAT_EQ(retPtr[i + 6], dat2[i]);
+}
+
+TEST_F(TestTensorMath, ConcatenateColumnsCuda) {
+  singa::CudaGPU dev;
+  d.ToDevice(&dev);
+  e.ToDevice(&dev);
+  d.CopyDataFromHostPtr<float>(dat1, 6);
+  e.CopyDataFromHostPtr<float>(dat2, 6);
+  auto ret = singa::ConcatenateColumns(vector<Tensor>{d, e});
+  ret.ToHost();
+  EXPECT_EQ(ret.shape(0), d.shape(0));
+  EXPECT_EQ(ret.shape(1), d.shape(1) + e.shape(1));
+
+  const float *retPtr = ret.data<float>();
+  for (int i = 0; i < 3; i++) {
+    for (int j = 0; j < 2; j++)
+      EXPECT_FLOAT_EQ(retPtr[i * 4 + j], dat1[i * 2 + j]);
+    for (int j = 0; j < 2; j++)
+      EXPECT_FLOAT_EQ(retPtr[i * 4 + 2 + j], dat2[i * 2 + j]);
+  }
+}
+
+TEST_F(TestTensorMath, CopyRowsCuda) {
+  singa::CudaGPU dev;
+  e.ToDevice(&dev);
+  auto ret = singa::CopyRows(e, 1, 2);
+  ret.ToHost();
+  EXPECT_EQ(ret.shape(0), 1u);
+  EXPECT_EQ(ret.shape(1), e.shape(1));
+  const float *retPtr = ret.data<float>();
+  for (size_t i = 0; i < ret.Size(); i++)
+    EXPECT_FLOAT_EQ(retPtr[i], dat1[1 * 2 + i]);
+}
+
+TEST_F(TestTensorMath, CopyColumnsCuda) {
+  singa::CudaGPU dev;
+  a.Reshape(Shape{2, 3});
+  a.ToDevice(&dev);
+  auto ret = singa::CopyColumns(a, 1, 3);
+  EXPECT_EQ(ret.shape(0), a.shape(0));
+  EXPECT_EQ(ret.shape(1), 2u);
+  ret.ToHost();
+  const float *retPtr = ret.data<float>();
+  for (size_t i = 0; i < ret.shape(0); i++)
+    for (size_t j = 0; j < ret.shape(1); j++)
+      EXPECT_FLOAT_EQ(retPtr[i * ret.shape(1) + j],
+                      dat1[i * a.shape(1) + j + 1]);
+}
+
+#endif