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

[27/60] incubator-singa git commit: SINGA-164 - Add the base Tensor class

SINGA-164 - Add the base Tensor class

Implement Tensor member functions.
Test the constructors and member classes.

A simple Device implementation is provided to pass the compilation.
TODO implement Tensor math functions.


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

Branch: refs/heads/dev
Commit: dc5aa6e3a52c5ba5575b4f941d5b8b27081195f7
Parents: bdbffdc
Author: Wei Wang <wa...@comp.nus.edu.sg>
Authored: Sat May 7 23:23:51 2016 +0800
Committer: wangwei <wa...@gmail.com>
Committed: Mon May 9 23:14:57 2016 +0800

----------------------------------------------------------------------
 .gitignore                       |  43 -----
 CMakeLists.txt                   |  19 ++-
 cmake/ProtoBuf.cmake             | 116 +++++++++++++
 include/singa/core/common.h      |  64 +++++++
 include/singa/core/device.h      | 119 ++++++++++++-
 include/singa/core/math.h        |  61 +++++++
 include/singa/core/memory.h      |  12 +-
 include/singa/core/scheduler.h   |  13 +-
 include/singa/core/tensor.h      | 311 +++++++++++++++++++++++++++++++++-
 src/CMakeLists.txt               |  28 ++-
 src/core/device/cpp_device.cc    |  38 +++++
 src/core/device/cuda_device.cc   |  24 +++
 src/core/device/device.cc        |  60 +++++++
 src/core/device/opencl_device.cc |  24 +++
 src/core/math/cpu_math.cc        |  54 ++++++
 src/core/math/cuda_math.cc       |  48 ++++++
 src/core/math/opencl_math.cc     |  24 +++
 src/core/memory/memory.cc        |  20 +++
 src/core/scheduler/scheduler.cc  |  19 +++
 src/core/tensor.cc               |  27 ---
 src/core/tensor/sparse_tensor.cc |  19 +++
 src/core/tensor/tensor.cc        | 222 ++++++++++++++++++++++++
 src/proto/core.proto             |  18 +-
 src/utils/common.cc              |  27 ---
 test/CMakeLists.txt              |   4 +-
 test/singa/test_tensor.cc        | 139 +++++++++++++++
 26 files changed, 1424 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 035d147..eaa2d84 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,55 +1,12 @@
-*.orig
 *.swp
 *.o
-*.lo
 *.bin
 *.a
 *.so
-*.dat
-*~
 *.bak
-*.P
-*.odp
-*.project
-*.cproject
 *.log
-*.nfs*
 *.pb.h
 *.pb.cc
-*.hosts
-*.id
-*.pids
-*.tmp
-*.out
-tool/pb2/*
-tool/python/pb2/*
-src/test/data/*
-tmp
-log*
 build/
-tmp/
-.sync
-*lmdb
-*.binaryproto
-singa
-singatool
-singatest
-.libs
-*.la
-*.deps
-*.dirstamp
-libtool
-stamp-h1
-*.cache
-*.status
-config.h
-Makefile
-config/*
-config.h.in
-configure
-aclocal.m4
-Makefile.in
 thirdparty/*
 !thirdparty/install.sh
-!include/singa
-doc/

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index c1cc4f3..745d833 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1,12 +1,23 @@
 CMAKE_MINIMUM_REQUIRED(VERSION 2.6)
 PROJECT(singa)
 SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11")
-MESSAGE(STATUS "flags: " ${CMAKE_CXX_FLAGS})
-MESSAGE(STATUS "paths: " ${CMAKE_CXX_INCLUDE_PATH})
-INCLUDE_DIRECTORIES(${PROJECT_SOURCE_DIR}/include)
+
+# Flags
+IF(UNIX OR APPLE)
+  SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fPIC -Wall")
+ENDIF()
+
+# Includes
+SET(singa_include_dir ${PROJECT_SOURCE_DIR}/include)
+INCLUDE_DIRECTORIES(${singa_include_dir} ${PROJECT_BINARY_DIR})
+
 
 SET(LIBRARY_OUTPUT_PATH ${PROJECT_BINARY_DIR}/lib)
 SET(EXECUTABLE_OUTPUT_PATH ${PROJECT_BINARY_DIR}/bin)
+SET(singa_linker_lib glog)
 LINK_DIRECTORIES(${LIBRARY_OUTPUT_PATH})
-ADD_SUBDIRECTORY(test)
+
+INCLUDE(cmake/ProtoBuf.cmake)
+
 ADD_SUBDIRECTORY(src)
+ADD_SUBDIRECTORY(test)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/cmake/ProtoBuf.cmake
----------------------------------------------------------------------
diff --git a/cmake/ProtoBuf.cmake b/cmake/ProtoBuf.cmake
new file mode 100644
index 0000000..437d136
--- /dev/null
+++ b/cmake/ProtoBuf.cmake
@@ -0,0 +1,116 @@
+#/**
+# * 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.
+# */
+
+# This file is adpated from Caffe cmake/ProtoBuf.cmake.
+# We changed 'caffe' to 'singa'
+
+# Finds Google Protocol Buffers library and compilers and extends
+# the standard cmake script with version and python generation support
+
+find_package( Protobuf REQUIRED )
+include_directories(SYSTEM ${PROTOBUF_INCLUDE_DIR})
+MESSAGE(STATUS "proto libs " ${PROTOBUF_LIBRARIES})
+list(APPEND singa_linker_libs ${PROTOBUF_LIBRARIES})
+
+# As of Ubuntu 14.04 protoc is no longer a part of libprotobuf-dev package
+# and should be installed separately as in: sudo apt-get install
+# protobuf-compiler
+if(EXISTS ${PROTOBUF_PROTOC_EXECUTABLE})
+  message(STATUS "Found PROTOBUF Compiler: ${PROTOBUF_PROTOC_EXECUTABLE}")
+else()
+  message(FATAL_ERROR "Could not find PROTOBUF Compiler")
+endif()
+
+#if(PROTOBUF_FOUND)
+#  # fetches protobuf version
+#  caffe_parse_header(${PROTOBUF_INCLUDE_DIR}/google/protobuf/stubs/common.h VERION_LINE GOOGLE_PROTOBUF_VERSION)
+#  string(REGEX MATCH "([0-9])00([0-9])00([0-9])" PROTOBUF_VERSION ${GOOGLE_PROTOBUF_VERSION})
+#  set(PROTOBUF_VERSION "${CMAKE_MATCH_1}.${CMAKE_MATCH_2}.${CMAKE_MATCH_3}")
+#  unset(GOOGLE_PROTOBUF_VERSION)
+#endif()
+
+# place where to generate protobuf sources
+set(proto_gen_folder "${PROJECT_BINARY_DIR}/include/singa/proto")
+include_directories("${PROJECT_BINARY_DIR}/include")
+
+set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE)
+
+###############################################################################
+# Modification of standard 'protobuf_generate_cpp()' with output dir parameter
+# and python support
+# Usage:
+#   singa_protobuf_generate_cpp_py(<output_dir> <srcs_var> <hdrs_var>
+#                                  <python_var> <proto_files>)
+function(singa_protobuf_generate_cpp_py output_dir srcs_var hdrs_var python_var)
+  if(NOT ARGN)
+    message(SEND_ERROR
+      "Error: singa_protobuf_generate_cpp_py() called without any proto files")
+    return()
+  endif()
+
+  if(PROTOBUF_GENERATE_CPP_APPEND_PATH)
+    # Create an include path for each file specified
+    foreach(fil ${ARGN})
+      get_filename_component(abs_fil ${fil} ABSOLUTE)
+      get_filename_component(abs_path ${abs_fil} PATH)
+      list(FIND _protoc_include ${abs_path} _contains_already)
+      if(${_contains_already} EQUAL -1)
+        list(APPEND _protoc_include -I ${abs_path})
+      endif()
+    endforeach()
+  else()
+    set(_protoc_include -I ${CMAKE_CURRENT_SOURCE_DIR})
+  endif()
+
+  if(DEFINED PROTOBUF_IMPORT_DIRS)
+    foreach(dir ${PROTOBUF_IMPORT_DIRS})
+      get_filename_component(abs_path ${dir} ABSOLUTE)
+      list(FIND _protoc_include ${abs_path} _contains_already)
+      if(${_contains_already} EQUAL -1)
+        list(APPEND _protoc_include -I ${abs_path})
+      endif()
+    endforeach()
+  endif()
+
+  set(${srcs_var})
+  set(${hdrs_var})
+  set(${python_var})
+  foreach(fil ${ARGN})
+    get_filename_component(abs_fil ${fil} ABSOLUTE)
+    get_filename_component(fil_we ${fil} NAME_WE)
+
+    list(APPEND ${srcs_var} "${output_dir}/${fil_we}.pb.cc")
+    list(APPEND ${hdrs_var} "${output_dir}/${fil_we}.pb.h")
+    list(APPEND ${python_var} "${output_dir}/${fil_we}_pb2.py")
+
+    add_custom_command(
+      OUTPUT "${output_dir}/${fil_we}.pb.cc"
+             "${output_dir}/${fil_we}.pb.h"
+             "${output_dir}/${fil_we}_pb2.py"
+      COMMAND ${CMAKE_COMMAND} -E make_directory "${output_dir}"
+      COMMAND ${PROTOBUF_PROTOC_EXECUTABLE} --cpp_out    ${output_dir} ${_protoc_include} ${abs_fil}
+      COMMAND ${PROTOBUF_PROTOC_EXECUTABLE} --python_out ${output_dir} ${_protoc_include} ${abs_fil}
+      DEPENDS ${abs_fil}
+      COMMENT "Running C++/Python protocol buffer compiler on ${fil}" VERBATIM )
+  endforeach()
+
+  set_source_files_properties(${${srcs_var}} ${${hdrs_var}} ${${python_var}} PROPERTIES GENERATED TRUE)
+  set(${srcs_var} ${${srcs_var}} PARENT_SCOPE)
+  set(${hdrs_var} ${${hdrs_var}} PARENT_SCOPE)
+  set(${python_var} ${${python_var}} PARENT_SCOPE)
+endfunction()

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/common.h
----------------------------------------------------------------------
diff --git a/include/singa/core/common.h b/include/singa/core/common.h
new file mode 100644
index 0000000..4724fef
--- /dev/null
+++ b/include/singa/core/common.h
@@ -0,0 +1,64 @@
+/**
+ * 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.
+ */
+
+#ifndef SINGA_CORE_COMMON_H_
+#define SINGA_CORE_COMMON_H_
+
+#include "glog/logging.h"
+
+namespace singa {
+namespace lib {
+/// To implemente functions using cpp libraries
+typedef struct _Cpp { } Cpp;
+/// To implemente functions using cuda libraries
+typedef struct _Cuda { } Cuda;
+/// To implement function using cudnn
+typedef struct _Cudnn { } Cudnn;
+/// To implement function using opencl libraries
+typedef struct _Opencl { } Opencl;
+}  // namespace lib;
+
+typedef unsigned char Byte;
+/// Blob reprent a chunk of memory (on device or host) managed by VirtualMemory.
+class Blob {
+ public:
+  Blob(void* ptr, int size) : data_(ptr), size_(size), ref_count_(1) {}
+  void* mutable_data() const { return data_; }
+  const void* data() const { return data_; }
+  int size() const { return size_; }
+  int IncRefCount() {
+    ref_count_++;
+    return ref_count_;
+  }
+  int DecRefCount() {
+    ref_count_--;
+    CHECK_GE(ref_count_, 0);
+    return ref_count_;
+  }
+  int ref_count() const { return ref_count_; }
+
+ private:
+  void* data_ = nullptr;
+  int size_ = 0;
+  int ref_count_ = 0;
+};
+
+class Context {};
+
+}  // namespace singa
+#endif  // SINGA_CORE_COMMON_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/device.h
----------------------------------------------------------------------
diff --git a/include/singa/core/device.h b/include/singa/core/device.h
index ef7c208..fa30d6d 100644
--- a/include/singa/core/device.h
+++ b/include/singa/core/device.h
@@ -16,15 +16,130 @@
  * limitations under the License.
  */
 
+#ifndef SINGA_CORE_DEVICE_H_
+#define SINGA_CORE_DEVICE_H_
 
+#include <type_traits>
+#include <vector>
+#include <string>
+#include <functional>
+
+#include "singa/core/common.h"
+#include "singa/core/memory.h"
+#include "singa/core/scheduler.h"
+#include "singa/proto/core.pb.h"
+
+using std::vector;
+using std::string;
+using std::function;
 namespace singa {
+/// The base type of callback argument structure.
+/// The specific arg should inherit from this one.
+class CallbackArg {
+ public:
+  template <typename T>
+  T* CastTo() {
+    static_assert(std::is_base_of<CallbackArg, T>::value,
+                  "The casted type must be a sub-class of CallbackArg");
+    return static_cast<T*>(this);
+  }
+};
+/// Type of callback functions for executing tensor ops.
+typedef function<void(CallbackArg*)> CallbackFn;
 
-/// Allocate memory for Tensor objects and execute Tensor operations.
+/// Allocate memory and execute Tensor operations.
 class Device {
+ public:
+  /// Operation has a function, and read/write blobs.
+  typedef struct _Operation {
+    function<void(Context*)> fn;
+    const vector<Blob*> read_blobs;
+    const vector<Blob*> write_blobs;
+  } Operation;
+
+ public:
+  Device() = default;
+  /// Constructor with device ID, num of executors (e.g., cuda streams),
+  /// max mem size to use (in MB), identifier of scheduler type (default
+  /// scheduler run operations synchronously) and virtual memory type (default
+  /// vm only provides garbage collection).
+  Device(int id, int num_executors = 16, string scheduler = "sync",
+         string vm = "gc-only");
+
+  /// Called by Tensor.
+  Blob* NewBlob(int size);
+
+  /// Called by Tensor.
+  void FreeBlob(Blob* blob);
+
+  /// Copy data within or across devices.
+  void CopyData(Blob* dst, const Blob& src, int len, int dst_offset,
+                int src_offset);
+
+  void CopyDataFromHostPtr(Blob* dst, const void* src, size_t size);
+  /// Submit the operation to the device, which may execute it right now or
+  /// delay it depending on the scheduler.
+  void Submit(function<void(Context*)> fn, const vector<Blob*> read_blobs,
+              const vector<Blob*> write_blobs);
+
+  // Wait for one event.
+  // void WaitFor();
+
+  /// wait for all operations submitted to this device.
+  void Sync();
 
+  LibType device_lib() const { return device_lib_; }
+  LibType nn_lib() const { return nn_lib_; }
 
+  Device* host() const { return host_; }
 
+ protected:
+  /// Execute one operation on one executor.
+  virtual void Exec(int operation, int executor) = 0;
 
+  /// Allocate device memory.
+  virtual void* Malloc(int size) = 0;
+
+  /// Free device memory.
+  virtual void Free(void* ptr) = 0;
+
+ protected:
+  int id_ = 0;
+  Scheduler* scheduler_ = nullptr;
+  VirtualMemory* vm_ = nullptr;
+  /// could be kCudnn
+  LibType nn_lib_;
+  /// could be kCpp, kCuda, kOpencl
+  LibType device_lib_;
+  // SafeQueue<Operation> op_queue_;
+  // SafeQueue<Operation> op_log_;
+  /// The host device
+  Device* host_;
+};
+// Implement Device using Cpp libs.
+class CppDevice : public Device {
+ public:
+  CppDevice(int id, int num_executors);
+
+  void Exec(int operation, int executor) override;
+
+ protected:
+  /// Allocate cpu memory.
+  void* Malloc(int size) override;
+
+  /// Free cpu memory.
+  void Free(void* ptr) override;
 };
 
-}  /* singa */
+/// a singleton CppDevice as the host for all devices.
+extern CppDevice hostDeviceSingleton;
+
+// Implement Device using OpenCL libs.
+// class OpenclDevice : public Device { };
+
+// Implement Device using Cuda libs for Nvidia GPUs.
+// class CudaDevice : public Device { };
+
+}  // namespace singa
+
+#endif  // SINGA_CORE_DEVICE_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/math.h
----------------------------------------------------------------------
diff --git a/include/singa/core/math.h b/include/singa/core/math.h
new file mode 100644
index 0000000..948dc8f
--- /dev/null
+++ b/include/singa/core/math.h
@@ -0,0 +1,61 @@
+/**
+ * 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.
+ */
+#ifndef SINGA_CORE_MATH_H_
+#define SINGA_CORE_MATH_H_
+#include <glog/logging.h>
+#include <type_traits>
+#include "singa/core/common.h"
+
+namespace singa {
+
+/// \file math.h Math functions for linear algebra, neural net and random
+/// operations.
+/// All functions have a template argument, DType for DataType, Lib for the
+/// backend library, e.g., lib::Cublas, lib::Cudnn, etc.
+
+/// Some operations would have many config/hyper-parameters, e.g., Conv, and
+/// these config vary among diff implementations, e.g., cuda/cudnn/opencl.
+/// To separate the modules, we pass a OpConf pointer to the Tensor Op function.
+/// The specific fields are implemented by inheriting OpConf, and casting the
+/// pointer between the base and the sub-class.
+class OpConf {
+ public:
+  template <typename T>
+  T* CastTo() {
+    static_assert(std::is_base_of<OpConf, T>::value,
+                  "The cast type must be a sub-class of OpConf");
+    return static_cast<T*>(this);
+  }
+};
+
+template <typename DType, typename Lib>
+void Add(int count, const Blob* lhs, const Blob* rhs, Blob* ret, Context* ctx) {
+  LOG(FATAL) << "Not Implemented";
+}
+
+// ================Neural Net operations======================================
+
+class ConvConf : public OpConf {};
+template <typename DType, typename Lib>
+void Conv(const OpConf* conf, const Blob* input, const Blob* W, const Blob* b,
+          Blob* ret, Context* ctx) {
+  LOG(FATAL) << "Not Implemented";
+}
+}  // namespace singa
+
+#endif  // SINGA_CORE_MATH_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/memory.h
----------------------------------------------------------------------
diff --git a/include/singa/core/memory.h b/include/singa/core/memory.h
index 0a7aa82..db09043 100644
--- a/include/singa/core/memory.h
+++ b/include/singa/core/memory.h
@@ -16,15 +16,13 @@
  * limitations under the License.
  */
 
+#ifndef SINGA_CORE_MEMORY_H_
+#define SINGA_CORE_MEMORY_H_
 
 namespace singa {
 
 /// Manage device memory pool including garbage collection, memory opt.
-class VirtualMemory {
+class VirtualMemory {};
 
-
-
-
-};
-
-}  /* singa */
+}  // namespace singa
+#endif  // SINGA_CORE_MEMORY_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/scheduler.h
----------------------------------------------------------------------
diff --git a/include/singa/core/scheduler.h b/include/singa/core/scheduler.h
index 6071b33..3673c6b 100644
--- a/include/singa/core/scheduler.h
+++ b/include/singa/core/scheduler.h
@@ -15,16 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+#ifndef SINGA_CORE_SCHEDULER_H_
+#define SINGA_CORE_SCHEDULER_H_
 
 namespace singa {
 
 /// Scheduling Tensor operations with dependency detection.
-class Scheduler {
-
-
-
-
-};
+class Scheduler {};
 
-}  /* singa */
+}  // namespace singa
+#endif  // SINGA_CORE_SCHEDULER_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/include/singa/core/tensor.h
----------------------------------------------------------------------
diff --git a/include/singa/core/tensor.h b/include/singa/core/tensor.h
index 795891d..6401738 100644
--- a/include/singa/core/tensor.h
+++ b/include/singa/core/tensor.h
@@ -16,15 +16,322 @@
  * limitations under the License.
  */
 
+#ifndef SINGA_CORE_TENSOR_H_
+#define SINGA_CORE_TENSOR_H_
 
+#include <glog/logging.h>
+#include <vector>
+
+#include "singa/core/common.h"
+#include "singa/core/device.h"
+#include "singa/core/math.h"
+#include "singa/proto/core.pb.h"
+
+using std::vector;
 namespace singa {
 
-/// A multi-dimensional array resident on a device.
+typedef vector<int> Shape;
+inline int Product(Shape shape) {
+  if (shape.size() == 0)
+    return 0;
+  int v = 1;
+  for (auto s : shape)
+    v *= s;
+  return v;
+}
+
+/// hardcode the width of types defined in DataType
+const int kDataWidth[] = {4, 2, 4, 1};
+inline int SizeOf(DataType t) {
+  static_assert(kNumDataType == sizeof(kDataWidth) / sizeof(int),
+      "Num of data types not match num of data width");
+  CHECK_GT(kNumDataType, t);
+  return kDataWidth[t];
+}
+
+/// A Tensor instance is a multi-dimensional array resident on a Device
+/// (default device is the host CPU). The internal data is allocated in lazy
+/// manner.
+/// Linear algebra, neural net and random operations are provided against
+/// Tensor.
+/// For all operations, if the result tensor is passed as an argument,
+/// then it must be set up correctly (shape, device). Otherwise, runtime error
+/// like SegmentFault would happen. Simply type/device check would be conducted.
 class Tensor {
+ public:
+  ~Tensor();
+  Tensor() = default;
+  explicit Tensor(const Shape& shape, DataType dtype = kFloat32);
+  Tensor(const Shape& shape, Device* dev, DataType dtype = kFloat32);
+
+  /// Copy Tensor to share the internal data.  No deep copy.
+  Tensor(const Tensor& from);
+
+  /// Copy Tensor to share the internal data.  No deep copy.
+  Tensor(Tensor&& from);
+
+  /// For functions in xx_math.cc to access the blob.
+  /// Users should not operate against Blob directly.
+  /// It will malloc memory for the tensor if not allocated before.
+  Blob* blob() const {
+    return blob_;
+  }
+
+  Device* device() const {
+    return device_;
+  }
+
+  /// Return immutable Tensor values with given type.
+  template <typename T>
+  const T* data() {
+    return static_cast<const T*> (blob()->data());
+  }
+
+  /// data type, including kFloat16, kFloat32, kInt
+  const DataType data_type() const {
+    return data_type_;
+  }
+
+  const Shape& shape() const {
+    return shape_;
+  }
+
+  bool transpose() const {
+    return transpose_;
+  }
+
+  int Size() const {
+    return blob_->size() / SizeOf(data_type_);
+  }
+
+  int MemSize() const {
+    return blob_->size();
+  }
+
+  void ReShape(const Shape& shape);
+
+  void AsType(DataType type);
+
+  /// Reset the device.
+  /// If the target device is a diff device, then do deep data copy.
+  void ToDevice(Device* dev);
+
+  /// Equivalent to ToDevice(host_dev).
+  void ToHost();
+
+  /// For init the tensor values, copy 'size' bytes data.
+  void CopyDataFromHostPtr(const void* src, size_t size);
+
+  /// Copy data from another Tensor which may be on a diff device.
+  /// Meta data would not be copied!
+  void CopyData(const Tensor& other);
+
+  /// Return an exactly the same Tensor with data been deep copied.
+  Tensor Clone();
+
+  // Tensor operations
+
+  /// Matrix transpose.  Valid only if shape.size() == 2.
+  /// No data copy, just set the transpose_ filed of the returned tensor.
+  Tensor T() const;
+
+  /// Copy the meta info with data blob shared.
+  void operator=(const Tensor& t);
+
+  /// Copy the meta info with data blob shared.
+  void operator=(Tensor&& t);
 
+  void operator+=(const Tensor& t);
+  void operator+=(Tensor&& t);
+  void operator-=(const Tensor& t);
+  void operator-=(Tensor&& t);
+  void operator*=(const Tensor& t);
+  void operator*=(Tensor&& t);
+  void operator/=(const Tensor& t);
+  void operator/=(Tensor&& t);
 
+  // Scalar operations.
 
+  /// T is a scalar type
+  template <typename T>
+  void operator+=(const T x);
 
+  /*
+  /// T is a scalar type
+  template <typename T>
+  void operator-=(const T x);
+
+  /// T is a scalar type
+  template <typename T>
+  void operator*=(const T x);
+
+  /// T is a scalar type
+  template <typename T>
+  void operator/=(const T x);
+
+  void Log(int base = 2);
+  void Tanh();
+  void Sigmoid();
+  void ReLU();
+
+  // random functions.
+  void Uniform(float low, float high);
+  template <typename T>
+  void Gaussian(float mean, float std);
+
+  /// save Tensor into a proto msg
+  void ToProto(TensorProto* t);
+  /// load Tensor from proto msg
+  void FromProto(const TensorProto& t);
+  */
+ protected:
+  bool transpose_ = false;
+  DataType data_type_ = kFloat32;
+  Device* device_ = nullptr;
+  /// Note: blob_ is allocated in lazy manner to avoid frequent malloc/free.
+  /// If you want to get an allocated Blob, use blob() instead of blob_.
+  Blob* blob_ = nullptr;
+  Shape shape_;
 };
 
-}  /* singa */
+/// For tensors with sparse content, e.g., missing columns or rows.
+// class SparseTensor : public Tensor {};
+
+// ==================Simple Linear Algebra Operations=========================
+
+/*
+Tensor Tanh(const Tensor& t);
+Tensor Log(const Tensor& t);
+Tensor Sigmoid(const Tensor& t);
+Tensor ReLU(const Tensor& t);
+Tensor Softmax(const Tensor& t);
+*/
+void CopyData(Tensor* dst,
+              const Tensor& src,
+              int msize,
+              int src_offset = 0,
+              int dst_offset = 0);
+
+// element-wise ops
+
+Tensor operator+(const Tensor& lhs, const Tensor& rhs);
+void Add(const Tensor& lhs, const Tensor& rhs, Tensor* ret);
+Tensor operator-(const Tensor& lhs, const Tensor& rhs);
+void Sub(const Tensor& lhs, const Tensor& rhs, Tensor* ret);
+/*
+Tensor operator*(const Tensor& lhs, const Tensor& rhs);
+void operator*(const Tensor& lhs, const Tensor& rhs, Tensor* ret);
+Tensor operator/(const Tensor& lhs, const Tensor& rhs);
+void operator/(const Tensor& lhs, const Tensor& rhs, Tensor* ret);
+
+template <typename T>
+Tensor operator+(const T x, const Tensor& t);
+template <typename T>
+void operator+(const T x, const Tensor& t, Tensor* ret);
+
+template <typename T>
+Tensor operator-(const T x, const Tensor& t);
+template <typename T>
+void operator-(const T x, const Tensor& t, Tensor* ret);
+
+template <typename T>
+Tensor operator*(const T x, const Tensor& t);
+template <typename T>
+void operator*(const T x, const Tensor& t, Tensor* ret);
+
+template <typename T>
+Tensor operator/(const T x, const Tensor& t);
+template <typename T>
+void operator/(const T x, const Tensor& t, Tensor* ret);
+
+//================Blas operations============================================
+Tensor Mult(const Tensor& lhs, const Tensor& rhs);
+void Mult(const Tensor& lhs, const Tensor& rhs, Tensor* ret);
+
+tempalte<typename T> T Dot(const Tensor& lhs, const Tensor& rhs);
+
+//================Neural Net operations======================================
+
+/// Convolution Op. 'Conf' is ConvConf;
+void Conv(const OpConf* conf,
+          const Tensor& input,
+          const Tensor& W,
+          const Tensor &b,
+          Tensor* ret);
+
+
+//================Random operations==========================================
+Tensor Uniform(float low, float high, const Shape& shape, Device* dev);
+
+Tensor Gaussian(float mean, float std, const Shape& shape, Device* dev);
+*/
+//============================================================================
+/// typedef DType accroding to type value.
+/// DType would be used in the code block __VA_ARGS__.
+#define TYPE_SWITCH(type, DType, ...)                               \
+  do {                                                              \
+    switch (type) {                                                 \
+      case kFloat32: {                                              \
+        typedef float DType;                                        \
+        { __VA_ARGS__ }                                             \
+        break;                                                      \
+      }                                                             \
+      case kInt: {                                                  \
+        typedef int DType;                                          \
+        { __VA_ARGS__ }                                             \
+        break;                                                      \
+      }                                                             \
+      case kChar: {                                                 \
+        typedef char DType;                                         \
+        { __VA_ARGS__ }                                             \
+        break;                                                      \
+      }                                                             \
+      default:                                                      \
+        LOG(FATAL) << "Unknow data type = " << DataType_Name(type); \
+    }                                                               \
+  } while (0)
+
+/// typedef DType and Lib according to values of type and lib respectively.
+/// type is from DataType, and lib is from LibType.
+/// DType and Lib would be used in __VA_ARGS__.
+#define TYPE_LIB_SWITCH(dtype, DType, ltype, Lib, ...)                 \
+  do {                                                               \
+    const int _SwitchShift = 3;                                      \
+    int _SwitchHash = ((dtype) << _SwitchShift) + (ltype);                 \
+    switch (_SwitchHash) {                                           \
+      case ((kFloat32 << _SwitchShift) + kCuda): {                   \
+        typedef float DType;                                          \
+        typedef lib::Cuda Lib;                                            \
+        { __VA_ARGS__ }                                              \
+        break;                                                       \
+      }                                                              \
+      case ((kFloat32 << _SwitchShift) + kCudnn): {                  \
+        typedef float DType;                                          \
+        typedef lib::Cudnn Lib;                                           \
+        { __VA_ARGS__ }                                              \
+        break;                                                       \
+      }                                                              \
+      case ((kFloat32 << _SwitchShift) + kCpp): {                    \
+        typedef float DType;                                          \
+        typedef lib::Cpp Lib;                                             \
+        { __VA_ARGS__ }                                              \
+        break;                                                       \
+      }                                                              \
+      case ((kFloat32 << _SwitchShift) + kOpencl): {                \
+        typedef float DType;                                          \
+        typedef lib::Opencl Lib;                                          \
+        { __VA_ARGS__ }                                              \
+        break;                                                       \
+      }                                                              \
+      default:                                                       \
+        LOG(FATAL) << "Unknown combination of data type "            \
+                   << DataType_Name(dtype) << " and library "        \
+                   << LibType_Name(ltype);                             \
+    }                                                                \
+  } while (0)
+
+
+
+}  // namespace singa
+
+#endif  // SINGA_CORE_TENSOR_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 7e5adba..d8bec8d 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -1,14 +1,28 @@
-PROJECT(singa)
-#AUX_SOURCE_DIRECTORY(core CORE_SOURCE)
-#AUX_SOURCE_DIRECTORY(layer LAYER_SOURCE)
-#AUX_SOURCE_DIRECTORY(model MODEL_SOURCE)
-AUX_SOURCE_DIRECTORY(utils UTILS_SOURCE)
+# generate protobuf sources
+file(GLOB proto_files proto/*.proto)
+singa_protobuf_generate_cpp_py(${proto_gen_folder} proto_srcs proto_hdrs proto_python ${proto_files})
+# include python files either to force generation
+add_library(proto STATIC ${proto_hdrs} ${proto_srcs} ${proto_python})
+list(APPEND singa_linker_libs proto)
+
+FILE(GLOB_RECURSE utils_source ${CMAKE_CURRENT_SOURCE_DIR}/utils/ "*.cc")
+ADD_LIBRARY(singa_utils SHARED ${utils_source})
+TARGET_LINK_LIBRARIES(singa_utils ${singa_linker_libs})
+list(APPEND singa_linker_libs singa_utils)
+
+
+FILE(GLOB_RECURSE core_source ${CMAKE_CURRENT_SOURCE_DIR}/core/ "*.cc")
+ADD_LIBRARY(singa_core SHARED ${core_source})
+TARGET_LINK_LIBRARIES(singa_core ${singa_linker_libs})
+list(APPEND singa_linker_libs singa_core)
+MESSAGE(STATUS "link libs " ${singa_linker_libs})
 
-#ADD_LIBRARY(singa_core SHARED ${CORE_SOURCE})
 #ADD_LIBRARY(singa_layer SHARED ${LAYER_SOURCE})
 #ADD_LIBRARY(singa_model SHARED ${MODEL_SOURCE})
-ADD_LIBRARY(singa_utils SHARED ${UTILS_SOURCE})
+#ADD_LIBRARY(singa_utils SHARED ${UTILS_SOURCE})
 
 #TARGET_LINK_LIBRARIES(singa_core singa_utils)
 #TARGET_LINK_LIBRARIES(singa_layer singa_core singa_utils)
 #TARGET_LINK_LIBRARIES(singa_model singa_layer singa_core singa_utils)
+
+#ADD_LIBRARY(singa SHARED ${singa_linker_libs})

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/device/cpp_device.cc
----------------------------------------------------------------------
diff --git a/src/core/device/cpp_device.cc b/src/core/device/cpp_device.cc
new file mode 100644
index 0000000..42f95c3
--- /dev/null
+++ b/src/core/device/cpp_device.cc
@@ -0,0 +1,38 @@
+/**
+ * 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 "singa/core/device.h"
+namespace singa {
+CppDevice hostDeviceSingleton(-1, 1);
+CppDevice::CppDevice(int id, int num_executors) {
+  nn_lib_ = kCpp;
+  device_lib_ = kCpp;
+  host_ = &hostDeviceSingleton;
+}
+
+void CppDevice::Exec(int operation, int executor) {
+}
+
+void* CppDevice::Malloc(int size) {
+  return malloc(size);
+}
+
+void CppDevice::Free(void* ptr) {
+  free(ptr);
+}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/device/cuda_device.cc
----------------------------------------------------------------------
diff --git a/src/core/device/cuda_device.cc b/src/core/device/cuda_device.cc
new file mode 100644
index 0000000..76c646e
--- /dev/null
+++ b/src/core/device/cuda_device.cc
@@ -0,0 +1,24 @@
+/**
+ * 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 "singa/core/device.h"
+namespace singa {
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/device/device.cc
----------------------------------------------------------------------
diff --git a/src/core/device/device.cc b/src/core/device/device.cc
new file mode 100644
index 0000000..5bdab6f
--- /dev/null
+++ b/src/core/device/device.cc
@@ -0,0 +1,60 @@
+/**
+ * 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 "singa/core/device.h"
+
+namespace singa {
+Device::Device(int id, int num_executors, string scheduler, string vm)
+    : id_(id) {
+  scheduler_ = nullptr;
+  vm_ = nullptr;
+}
+
+void Device::Submit(function<void(Context*)> fn, const vector<Blob*> read_blobs,
+                    const vector<Blob*> write_blobs) {
+  fn(nullptr);
+}
+
+Blob* Device::NewBlob(int size) {
+  if (size > 0) {
+    void* ptr = malloc(size);
+    memset(ptr, 0, size);
+    return new Blob(ptr, size);
+  } else {
+    return nullptr;
+  }
+}
+
+void Device::FreeBlob(Blob* blob) {
+  if (blob != nullptr) {
+    free(blob->mutable_data());
+    delete blob;
+  }
+}
+
+void Device::CopyData(Blob* dst, const Blob& src, int len, int dst_offset,
+                      int src_offset) {
+  memcpy(reinterpret_cast<Byte*>(dst->mutable_data()) + dst_offset,
+         (const Byte*)src.data() + src_offset, len);
+}
+
+void Device::CopyDataFromHostPtr(Blob* dst, const void* src, size_t size) {
+  memcpy(dst->mutable_data(), src, size);
+}
+void Device::Sync() {}
+}  // namespace singa

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/device/opencl_device.cc
----------------------------------------------------------------------
diff --git a/src/core/device/opencl_device.cc b/src/core/device/opencl_device.cc
new file mode 100644
index 0000000..76c646e
--- /dev/null
+++ b/src/core/device/opencl_device.cc
@@ -0,0 +1,24 @@
+/**
+ * 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 "singa/core/device.h"
+namespace singa {
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/math/cpu_math.cc
----------------------------------------------------------------------
diff --git a/src/core/math/cpu_math.cc b/src/core/math/cpu_math.cc
new file mode 100644
index 0000000..638d693
--- /dev/null
+++ b/src/core/math/cpu_math.cc
@@ -0,0 +1,54 @@
+/**
+ * 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 "singa/core/math.h"
+#include "singa/core/common.h"
+
+#ifdef USE_CBLAS
+#include <cblas.h>
+#endif
+
+namespace singa {
+template<>
+void Add<float, lib::Cpp>(int count,
+                     const Blob* lhs,
+                     const Blob* rhs,
+                     Blob* ret,
+                     Context* ctx) {
+  // CHECK_EQ(ctx->stream, nullptr);
+  float *dptr = static_cast<float*>(ret->mutable_data());
+  const float *lptr = static_cast<const float*>(lhs->data());
+  const float *rptr = static_cast<const float*>(rhs->data());
+  for (int i = 0; i < count; i++) {
+    dptr[i] = lptr[i] + rptr[i];
+  }
+}
+
+#ifdef USE_CBLAS
+template<>
+void Dot<float, lib::Cpp>(int count,
+                     const Blob* lhs,
+                     const Blob* rhs,
+                     float* ret,
+                     Context* ctx) {
+  float dptr = ret->mutable_data(), lptr = lhs->data(), rptr = rhs->data();
+  *ret = cblas_sdot(count, lptr, 1, rptr, 1);
+}
+
+#endif
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/math/cuda_math.cc
----------------------------------------------------------------------
diff --git a/src/core/math/cuda_math.cc b/src/core/math/cuda_math.cc
new file mode 100644
index 0000000..1cff1c2
--- /dev/null
+++ b/src/core/math/cuda_math.cc
@@ -0,0 +1,48 @@
+/**
+ * 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 "singa/core/math.h"
+#include "singa/core/common.h"
+
+
+namespace singa {
+
+#ifdef USE_CUDA
+template<>
+void Add<float, lib::Cuda>(int count, const Blob* lhs, const Blob* rhs,
+                        Blob* ret, Context* ctx) {
+  cublasSetStream(ctx->handle, ctx->stream);
+  cublasScopy(ctx->handle, count, lhs->data(), 1, ret->mutable_data(), 1);
+  cublasSaxpy(ctx->handle, 1.0f, rhs->data(), 1, ret->mutable_data(), 1);
+}
+
+#ifdef USE_CUDNN
+template<>
+void Conv<float, lib::Cudnn>(const OpConf *conf,
+          const Blob* input,
+          const Blob* W,
+          const Blob* b,
+          Blob* ret,
+          Context* ctx) {
+  // auto conv_conf = conf->CastTo<ConvConf>();
+  // conv op
+}
+
+#endif
+#endif
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/math/opencl_math.cc
----------------------------------------------------------------------
diff --git a/src/core/math/opencl_math.cc b/src/core/math/opencl_math.cc
new file mode 100644
index 0000000..7012610
--- /dev/null
+++ b/src/core/math/opencl_math.cc
@@ -0,0 +1,24 @@
+/**
+ * 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 "singa/core/math.h"
+
+namespace singa {
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/memory/memory.cc
----------------------------------------------------------------------
diff --git a/src/core/memory/memory.cc b/src/core/memory/memory.cc
new file mode 100644
index 0000000..a1cf5db
--- /dev/null
+++ b/src/core/memory/memory.cc
@@ -0,0 +1,20 @@
+/**
+ * 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 "singa/core/memory.h"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/scheduler/scheduler.cc
----------------------------------------------------------------------
diff --git a/src/core/scheduler/scheduler.cc b/src/core/scheduler/scheduler.cc
new file mode 100644
index 0000000..183674f
--- /dev/null
+++ b/src/core/scheduler/scheduler.cc
@@ -0,0 +1,19 @@
+/**
+ * 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 "singa/core/scheduler.h"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/tensor.cc
----------------------------------------------------------------------
diff --git a/src/core/tensor.cc b/src/core/tensor.cc
deleted file mode 100644
index d1a7d2c..0000000
--- a/src/core/tensor.cc
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.
- */
-
-
-namespace singa {
-
-
-
-
-
-
-}  /* singa */

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/tensor/sparse_tensor.cc
----------------------------------------------------------------------
diff --git a/src/core/tensor/sparse_tensor.cc b/src/core/tensor/sparse_tensor.cc
new file mode 100644
index 0000000..a8ae973
--- /dev/null
+++ b/src/core/tensor/sparse_tensor.cc
@@ -0,0 +1,19 @@
+/**
+ * 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 "singa/core/tensor.h"
+namespace singa {}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/core/tensor/tensor.cc
----------------------------------------------------------------------
diff --git a/src/core/tensor/tensor.cc b/src/core/tensor/tensor.cc
new file mode 100644
index 0000000..e03fd22
--- /dev/null
+++ b/src/core/tensor/tensor.cc
@@ -0,0 +1,222 @@
+/**
+ * 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 "singa/core/tensor.h"
+#include "singa/core/math.h"
+
+namespace singa {
+Tensor::~Tensor() {
+  if (blob_ != nullptr && blob_->DecRefCount() == 0)
+    device_->FreeBlob(blob_);
+  blob_ = nullptr;
+}
+
+Tensor::Tensor(const Shape& shape, DataType dtype)
+    : data_type_(dtype), device_(&hostDeviceSingleton), shape_(shape) {
+  device_ = &hostDeviceSingleton;
+  blob_ = device_->NewBlob(Product(shape_) * SizeOf(data_type_));
+}
+
+Tensor::Tensor(const Shape& shape, Device* device, DataType dtype)
+    : data_type_(dtype), device_(device), shape_(shape) {
+  blob_ = device_->NewBlob(Product(shape_) * SizeOf(data_type_));
+}
+
+Tensor::Tensor(const Tensor& t)
+    : transpose_(t.transpose_),
+      data_type_(t.data_type_),
+      device_(t.device_),
+      blob_(t.blob()),
+      shape_(t.shape_) {
+  blob_->IncRefCount();
+}
+
+Tensor::Tensor(Tensor&& t)
+    : transpose_(t.transpose_),
+      data_type_(t.data_type_),
+      device_(t.device_),
+      shape_(t.shape_) {
+  blob_ = t.blob_;
+  t.blob_ = nullptr;
+}
+
+void Tensor::ReShape(const Shape& shape) {
+  if (shape_ != shape) {
+    if (blob_ != nullptr && blob_->DecRefCount() == 0)
+      device_->FreeBlob(blob_);
+    blob_ = device_->NewBlob(Product(shape) * SizeOf(data_type_));
+    shape_ = shape;
+  }
+}
+
+void Tensor::AsType(DataType type) {
+  if (data_type_ != type) {
+    if (blob_ != nullptr && blob_->DecRefCount() == 0)
+      device_->FreeBlob(blob_);
+    blob_ = device_->NewBlob(Product(shape_) * SizeOf(type));
+    data_type_ = type;
+  }
+}
+
+void Tensor::ToDevice(Device* dst) {
+  // TODO(wangwei) the comparison is very strict. May compare against device ID?
+  if (device_ != dst) {
+    Tensor tmp(shape_, dst, data_type_);
+    tmp.CopyData(*this);
+    if (blob_ != nullptr && blob_->DecRefCount() == 0)
+      device_->FreeBlob(blob_);
+    blob_ = tmp.blob_;
+    tmp.blob_ = nullptr;
+    device_ = dst;
+  }
+}
+
+void Tensor::ToHost() {
+  ToDevice(device_->host());
+}
+
+void Tensor::CopyDataFromHostPtr(const void* src, size_t size) {
+  if (src != nullptr)
+    device_->CopyDataFromHostPtr(blob(), src, size);
+  else
+    LOG(WARNING) << "Copy data from null host ptr";
+}
+
+void Tensor::CopyData(const Tensor& src) {
+  CHECK_EQ(Size(), src.Size());
+  // Do copy only if the src's blob is already initialized.
+  if (src.blob_ != nullptr)
+    singa::CopyData(this, src, Size() * SizeOf(data_type_), 0, 0);
+}
+
+Tensor Tensor::Clone() {
+  Tensor t(shape_, device_, data_type_);
+  t.transpose_ = transpose_;
+  t.CopyData(*this);
+  return t;
+}
+
+Tensor Tensor::T() const {
+  Tensor t(*this);
+  t.transpose_ = ~transpose_;
+  return t;
+}
+
+void Tensor::operator=(const Tensor& t) {
+  if (blob_ != nullptr && blob_->DecRefCount() == 0)
+    device_->FreeBlob(blob_);
+  transpose_ = t.transpose_;
+  data_type_ = t.data_type_;
+  shape_ = t.shape_;
+  device_ = t.device_;
+  blob_ = t.blob();
+  blob_->IncRefCount();
+}
+
+void Tensor::operator=(Tensor&& t) {
+  if (blob_ != nullptr && blob_->DecRefCount() == 0)
+    device_->FreeBlob(blob_);
+  transpose_ = t.transpose_;
+  shape_ = t.shape_;
+  device_ = t.device_;
+  blob_ = t.blob_;
+  t.blob_ = nullptr;
+}
+
+void Tensor::operator+=(const Tensor& t) {
+  Add(*this, t, this);
+}
+// ====================Tensor Operations=======================================
+
+void CopyData(Tensor* dst,
+              const Tensor& src,
+              int len,
+              int dst_offset,
+              int src_offset) {
+  CHECK_GE(src.MemSize(), src_offset + len);
+  CHECK_GE(dst->MemSize(), dst_offset + len);
+  Device* src_dev = src.device(), *dst_dev = dst->device();
+  Blob* src_blob = src.blob(), *dst_blob = dst->blob();
+  if (dst_dev->device_lib() != src_dev->device_lib()) {
+    // let the none cpp device conduct copy op
+    if (dst_dev->device_lib() == kCpp) {
+      src_dev->CopyData(dst_blob, *src_blob, len, dst_offset, src_offset);
+    } else if (src_dev->device_lib() == kCpp) {
+      dst_dev->CopyData(dst_blob, *src_blob, len, dst_offset, src_offset);
+    } else {
+      LOG(FATAL) << "Not support mem copy betwee Cuda and OpenCL device";
+    }
+  } else {
+    src_dev->CopyData(dst_blob, *src_blob, len, dst_offset, src_offset);
+  }
+}
+
+Tensor operator+(const Tensor& lhs, const Tensor& rhs) {
+  Tensor ret(lhs.shape(), lhs.device());
+  Add(lhs, rhs, &ret);
+  return ret;
+}
+
+void Add(const Tensor& lhs, const Tensor& rhs, Tensor* ret) {
+  TYPE_LIB_SWITCH(lhs.data_type(), DType, lhs.device()->device_lib(), Lib, {
+    ret->device()->Submit(
+        [lhs, rhs, ret](Context* ctx) {
+          Add<DType, Lib>(lhs.Size(), lhs.blob(), rhs.blob(), ret->blob(), ctx);
+        },
+        {lhs.blob(), rhs.blob()}, {ret->blob()});
+  });
+}
+/*
+Tensor operator-(const Tensor& lhs, const Tensor& rhs) {
+  Tensor ret(lhs.shape(), lhs.device());
+  Sub(lhs, rhs, &ret);
+  return ret;
+}
+
+void Sub(const Tensor& lhs, const Tensor& rhs, Tensor *ret) {
+  TYPE_LIB_SWITCH(lhs.data_type(), DType, lhs.device()->device_lib(), Lib, {
+      ret->device()->Submit(
+        [lhs, rhs, ret](Context* ctx) {
+          Sub<DType, Lib>(
+            lhs.Size(),
+            lhs.blob(),
+            rhs.blob(),
+            ret->blob(),
+            ctx);}
+        , {lhs.blob(), rhs.blob()}, {ret->blob()});
+      });
+}
+*/
+
+// ================Blas operations============================================
+
+// ================Neural Net operations======================================
+
+void Conv(const OpConf* conf, const Tensor& input, const Tensor& W,
+          const Tensor& b, Tensor* ret) {
+  TYPE_LIB_SWITCH(input.data_type(), DType, input.device()->nn_lib(), Lib, {
+    ret->device()->Submit(
+        [conf, input, W, b, ret](Context* ctx) {
+          Conv<DType, Lib>(conf, input.blob(), W.blob(), b.blob(), ret->blob(),
+                           ctx);
+        },
+        {input.blob(), W.blob(), b.blob()}, {ret->blob()});
+  });
+}
+
+}  // namespace singa

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/proto/core.proto
----------------------------------------------------------------------
diff --git a/src/proto/core.proto b/src/proto/core.proto
index 5b5fea0..5d31560 100644
--- a/src/proto/core.proto
+++ b/src/proto/core.proto
@@ -18,4 +18,20 @@
 
 package singa;
 
-syntax = "proto2";
+//syntax = "proto2";
+
+enum DataType {
+  kFloat32 = 0;
+  kFloat16 = 1;
+  kInt = 2;
+  kChar = 3;
+  kNumDataType = 4;
+}
+
+enum LibType {
+  kCpp = 0;
+  kCuda = 1;
+  kOpencl = 2;
+  kCudnn = 3;
+  kNumLibType = 4;
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/src/utils/common.cc
----------------------------------------------------------------------
diff --git a/src/utils/common.cc b/src/utils/common.cc
deleted file mode 100644
index d1a7d2c..0000000
--- a/src/utils/common.cc
+++ /dev/null
@@ -1,27 +0,0 @@
-/**
- * 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.
- */
-
-
-namespace singa {
-
-
-
-
-
-
-}  /* singa */

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/test/CMakeLists.txt b/test/CMakeLists.txt
index f3e16c0..7fcada8 100644
--- a/test/CMakeLists.txt
+++ b/test/CMakeLists.txt
@@ -4,5 +4,7 @@ ADD_LIBRARY(gtest STATIC EXCLUDE_FROM_ALL "gtest/gtest.h" "gtest/gtest-all.cc")
 AUX_SOURCE_DIRECTORY(singa singa_test_source)
 
 ADD_EXECUTABLE(test_singa "gtest/gtest_main.cc" ${singa_test_source})
-TARGET_LINK_LIBRARIES(test_singa gtest)
+ADD_DEPENDENCIES(test_singa singa_core singa_utils)
+MESSAGE(STATUS "link libs" ${singa_linker_libs})
+TARGET_LINK_LIBRARIES(test_singa gtest singa_core singa_utils proto protobuf glog)
 SET_TARGET_PROPERTIES(test_singa PROPERTIES LINK_FLAGS "${LINK_FLAGS} -pthread")

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/dc5aa6e3/test/singa/test_tensor.cc
----------------------------------------------------------------------
diff --git a/test/singa/test_tensor.cc b/test/singa/test_tensor.cc
new file mode 100644
index 0000000..04068ae
--- /dev/null
+++ b/test/singa/test_tensor.cc
@@ -0,0 +1,139 @@
+#include "gtest/gtest.h"
+#include "singa/core/tensor.h"
+using singa::Tensor;
+using singa::Shape;
+using singa::Device;
+
+TEST(TensorTest, TestConstructor) {
+  singa::Tensor float_t(singa::Shape{2,3});
+  EXPECT_EQ(6, float_t.Size());
+  EXPECT_EQ(sizeof(float) * 6, float_t.MemSize());
+  EXPECT_EQ(singa::kFloat32, float_t.data_type());
+  auto s = float_t.shape();
+  EXPECT_EQ(s[0], 2);
+  EXPECT_EQ(s[1], 3);
+
+  EXPECT_NE(float_t.device(), nullptr);
+
+  singa::Tensor float16_t(singa::Shape{2,3}, singa::kFloat16);
+  EXPECT_EQ(singa::kFloat16, float16_t.data_type());
+  EXPECT_EQ(6, float16_t.Size());
+  EXPECT_EQ(12, float16_t.blob()->size());
+
+  singa::Tensor x(float16_t);
+  EXPECT_EQ(float16_t.Size(), x.Size());
+  EXPECT_EQ(float16_t.blob(), x.blob());
+  EXPECT_EQ(float16_t.data_type(), x.data_type());
+  EXPECT_EQ(float16_t.device(), x.device());
+
+  singa::Tensor y = float16_t;
+  EXPECT_EQ(float16_t.Size(), x.Size());
+  EXPECT_EQ(float16_t.blob(), x.blob());
+  EXPECT_EQ(float16_t.data_type(), x.data_type());
+  EXPECT_EQ(float16_t.device(), x.device());
+}
+
+TEST(TensorClass, Reshape) {
+  Tensor t;
+  t.ReShape(Shape{2,3});
+  EXPECT_TRUE((Shape{2,3} == t.shape()));
+
+  t.ReShape(Shape{3,3, 4});
+  EXPECT_TRUE((Shape{3,3, 4} == t.shape()));
+
+  t.ReShape(Shape{12});
+  EXPECT_TRUE((Shape{12} == t.shape()));
+
+  Tensor o;
+  EXPECT_TRUE(o.shape() != t.shape());
+  o.ReShape(Shape{3, 3});
+  EXPECT_TRUE(o.shape() != t.shape());
+}
+
+TEST(TensorClass, AsType) {
+  Tensor t;
+  EXPECT_EQ(singa::kFloat32, t.data_type());
+  t.AsType(singa::kFloat16);
+  EXPECT_EQ(singa::kFloat16, t.data_type());
+}
+
+TEST(TensorClass, ToDevice) {
+  Tensor t(Shape{2,3});
+  EXPECT_EQ(static_cast<Device*>(&singa::hostDeviceSingleton), t.device());
+  singa::CppDevice *dev = new singa::CppDevice(0, 1);
+  t.ToDevice(dev);
+  EXPECT_NE(static_cast<Device*>(&singa::hostDeviceSingleton), t.device());
+}
+
+TEST(TensorClass, CopyDataFromHostPtr) {
+  float data[] = {1.0f, 2.0f, 3.0f};
+  Tensor t(Shape{3});
+  t.CopyDataFromHostPtr(data, sizeof(float) * 3);
+  const float* dptr = static_cast<const float*>(t.blob()->data());
+  EXPECT_FLOAT_EQ(1.0f, dptr[0]);
+  EXPECT_FLOAT_EQ(2.0f, dptr[1]);
+  EXPECT_FLOAT_EQ(3.0f, dptr[2]);
+}
+
+TEST(TensorClass, CopyData) {
+  float data[] = {1.0f, 2.0f, 3.0f};
+  Tensor t(Shape{3});
+  t.CopyDataFromHostPtr(data, sizeof(float) * 3);
+
+  Tensor o(Shape{3});
+  o.CopyData(t);
+  const float* dptr = static_cast<const float*>(o.blob()->data());
+  EXPECT_FLOAT_EQ(1.0f, dptr[0]);
+  EXPECT_FLOAT_EQ(2.0f, dptr[1]);
+  EXPECT_FLOAT_EQ(3.0f, dptr[2]);
+}
+
+TEST(TensorClass, Clone) {
+  float data[] = {1.0f, 2.0f, 3.0f};
+  Tensor t(Shape{3});
+  t.CopyDataFromHostPtr(data, sizeof(float) * 3);
+
+  Tensor o = t.Clone();
+  const float* dptr = static_cast<const float*>(o.blob()->data());
+  EXPECT_FLOAT_EQ(1.0f, dptr[0]);
+  EXPECT_FLOAT_EQ(2.0f, dptr[1]);
+  EXPECT_FLOAT_EQ(3.0f, dptr[2]);
+}
+
+TEST(TensorClass, T) {
+  Tensor t(Shape{2,3});
+  EXPECT_FALSE(t.transpose());
+  Tensor o = t.T();
+  EXPECT_EQ(true, o.transpose());
+  EXPECT_EQ(t.blob(), o.blob());
+  EXPECT_EQ(t.data_type(), o.data_type());
+  EXPECT_TRUE((t.shape() ==  o.shape()));
+}
+
+TEST(TensorClass, Add) {
+  const float data[] = {1.0f, 2.0f, 3.0f, 1.1f, 2.1f, 3.1f};
+  Tensor t(Shape{3});
+  t.CopyDataFromHostPtr(data, sizeof(float) * 3);
+
+  Tensor o = t.Clone();
+  o += t;
+  const float* dptr = o.data<float>();
+  EXPECT_FLOAT_EQ(2.0f, dptr[0]);
+  EXPECT_FLOAT_EQ(4.0f, dptr[1]);
+  EXPECT_FLOAT_EQ(6.0f, dptr[2]);
+
+  Tensor p(Shape{3});
+  o += p;
+  const float* dptr1 = o.data<float>();
+  EXPECT_FLOAT_EQ(2.0f, dptr1[0]);
+  EXPECT_FLOAT_EQ(4.0f, dptr1[1]);
+  EXPECT_FLOAT_EQ(6.0f, dptr1[2]);
+
+  Tensor q(Shape{3});
+  q.CopyDataFromHostPtr(data + 3, sizeof(float) * 3);
+  t += q;
+  const float* dptr2 = t.data<float>();
+  EXPECT_FLOAT_EQ(2.1f, dptr2[0]);
+  EXPECT_FLOAT_EQ(4.1f, dptr2[1]);
+  EXPECT_FLOAT_EQ(6.1f, dptr2[2]);
+}