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/17 14:28:35 UTC

[3/4] incubator-singa git commit: SINGA-186 Create Python Tensor class

SINGA-186 Create Python Tensor class

Add cmake supports for python files.
Usage:
mkdir build && cd build;
cmake -DUSE_PYTHON=ON ..;
make;

Tested SWIG version is 3.0 and python version is 2.7.

Create swig folder.
Remove useless files.


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

Branch: refs/heads/dev
Commit: 254343cb314945b43276e2020efe56538a4c3bc0
Parents: 24833fa
Author: xiezl <xi...@comp.nus.edu.sg>
Authored: Fri Jun 17 14:19:56 2016 +0800
Committer: Wei Wang <wa...@comp.nus.edu.sg>
Committed: Fri Jun 17 22:27:01 2016 +0800

----------------------------------------------------------------------
 CMakeLists.txt                       |   2 +
 cmake/Dependencies.cmake             |   5 +
 cmake/Utils.cmake                    |  15 ++
 src/CMakeLists.txt                   |  12 ++
 src/python/core_device.i             |  60 -------
 src/python/core_tensor.i             | 264 ------------------------------
 src/python/generate_singa_wrapper.sh |  39 -----
 src/python/model_layer.i             |  83 ----------
 src/python/singa.i                   |  27 ---
 src/python/swig/core_device.i        |  60 +++++++
 src/python/swig/core_tensor.i        | 263 +++++++++++++++++++++++++++++
 src/python/swig/model_layer.i        |  83 ++++++++++
 src/python/swig/singa.i              |  27 +++
 13 files changed, 467 insertions(+), 473 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index a9d9b17..46b6bfb 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -24,8 +24,10 @@ OPTION(USE_CUDA "Use Cuda libs" ON)
 OPTION(USE_CUDNN "Use Cudnn libs" ON)
 OPTION(USE_OPENCV "Use opencv" OFF)
 OPTION(USE_LMDB "Use LMDB libs" OFF)
+OPTION(USE_PYTHON "Generate py wrappers" OFF)
 
 INCLUDE("cmake/Dependencies.cmake")
+INCLUDE("cmake/Utils.cmake")
 ADD_DEFINITIONS(-DUSE_CMAKE)
 
 CONFIGURE_FILE (

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/cmake/Dependencies.cmake
----------------------------------------------------------------------
diff --git a/cmake/Dependencies.cmake b/cmake/Dependencies.cmake
index d3f0b00..11823b7 100644
--- a/cmake/Dependencies.cmake
+++ b/cmake/Dependencies.cmake
@@ -35,3 +35,8 @@ LIST(APPEND SINGA_LINKER_LIBS ${GLOG_LIBRARIES})
 
 #LIST(APPEND SINGA_LINKER_LIBS "/home/wangwei/local/lib/libopenblas.so")
 #MESSAGE(STATUS "link lib : " ${SINGA_LINKER_LIBS})
+
+IF(USE_PYTHON)
+    FIND_PACKAGE(PythonLibs REQUIRED)
+    FIND_PACKAGE(SWIG 3.0 REQUIRED)
+ENDIF()

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/cmake/Utils.cmake
----------------------------------------------------------------------
diff --git a/cmake/Utils.cmake b/cmake/Utils.cmake
new file mode 100644
index 0000000..7f9452e
--- /dev/null
+++ b/cmake/Utils.cmake
@@ -0,0 +1,15 @@
+
+macro(swig_generate_cxx pylist_variable)
+    if(NOT EXISTS "${CMKAE_BINARY_DIR}/python")
+        execute_process(
+            COMMAND mkdir ${CMAKE_BINARY_DIR}/python
+            ERROR_QUIET)
+    endif()
+    execute_process(
+        COMMAND swig -c++ -python -I${CMAKE_SOURCE_DIR}/include
+        -outdir ${CMAKE_BINARY_DIR}/python/
+        ${ARGN})
+
+    set(${pylist_variable} "${CMAKE_SOURCE_DIR}/src/python/swig/singa_wrap.cxx")
+endmacro()
+

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 823b26a..bc50a6b 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -64,3 +64,15 @@ LIST(APPEND SINGA_LINKER_LIBS singa_model)
 #TARGET_LINK_LIBRARIES(singa_model singa_layer singa_core singa_utils)
 
 #ADD_LIBRARY(singa SHARED ${SINGA_LINKER_LIBS})
+
+IF(USE_PYTHON)
+    FILE(GLOB python_files python/swig/singa.i)
+    swig_generate_cxx(python_srcs ${python_files})
+    SET(python_cxxs "${CMAKE_SOURCE_DIR}/src/core/tensor/tensor.cc;${CMAKE_SOURCE_DIR}/src/core/device/device.cc")
+    ADD_LIBRARY(_singa SHARED ${python_srcs} ${python_cxxs})
+    SET(WRAPPER_LINKER_LIBS ${SINGA_LINKER_LIBS})
+    TARGET_LINK_LIBRARIES(_singa ${SINGA_LINKER_LIBS})
+    TARGET_INCLUDE_DIRECTORIES(_singa PRIVATE ${PYTHON_INCLUDE_DIRS}) 
+    SET_TARGET_PROPERTIES(_singa 
+        PROPERTIES PREFIX "")
+ENDIF(USE_PYTHON)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/core_device.i
----------------------------------------------------------------------
diff --git a/src/python/core_device.i b/src/python/core_device.i
deleted file mode 100644
index ab9abd8..0000000
--- a/src/python/core_device.i
+++ /dev/null
@@ -1,60 +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.
-*
-*************************************************************/
-
-/*interface file for swig */
-
-%module core_device
-%include "std_vector.i"
-%include "std_string.i"
-
-%{
-#include "singa/core/device.h"
-%}
-
-namespace singa{
-
-  %nodefault Device;
-  class Device {
-   public:
-    virtual void SetRandSeed(unsigned seed) = 0;
-    Device* host();
-    int id() const;
-  };
-
-  class CppCPU : public Device {
-   public:
-    CppCPU(int id = -1, int num_executors = 1,
-           std::string scheduler = "sync", std::string vm = "gc-only");
-    void SetRandSeed(unsigned seed) override;
-    /* (TODO) add necessary functions of CppCPU class
-    */
-  };
-
-  class CudaGPU : public Device {
-   public:
-    CudaGPU(int id = 0, int num_executors = 1,
-            std::string scheduler = "sync", std::string vm = "gc-only");
-    void SetRandSeed(unsigned seed) override;
-    /* (TODO) add necessary functions of CudaGPU class
-    */
-  };
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/core_tensor.i
----------------------------------------------------------------------
diff --git a/src/python/core_tensor.i b/src/python/core_tensor.i
deleted file mode 100644
index a700602..0000000
--- a/src/python/core_tensor.i
+++ /dev/null
@@ -1,264 +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.
-*
-*************************************************************/
-
-/*interface file for swig */
-
-%module core_tensor
-%include "std_vector.i"
-%include "std_string.i"
-
-%include "carrays.i"
-%array_class(float, floatArray);
-%array_class(int, intArray);
-%array_class(char, charArray);
-%array_class(double, doubleArray);
-
-%{
-#include "core/tensor/tensor_math.h"
-#include "singa/core/tensor.h"
-#include "singa/core/device.h"
-#include "singa/proto/core.pb.h"
-#include "singa/proto/model.pb.h"
-using singa::DataType;
-%}
-
-%template(Shape) std::vector<size_t>;
-
-namespace singa{
-
-  enum DataType {
-    kFloat32, kFloat16, kInt, kChar, kDouble
-  };
-
-  inline size_t Product(const std::vector<size_t> &shape,
-                        int start = 0, size_t len = 0);
-  inline size_t SizeOf(DataType t);
-
-  class Tensor {
-
-   public:
-    Tensor();
-    explicit Tensor(const std::vector<size_t> &shape,
-                    DataType dtype = kFloat32);
-    Tensor(const std::vector<size_t> &shape,
-           singa::Device *dev, DataType dtype = kFloat32);
-    Tensor(const Tensor &from);
-
-    //Blob *blob() const;
-    singa::Device *device() const;
-
-    template <typename DType> DType data() const;
-    %template(floatData) data<const float*>;
-    %template(intData) data<const int*>;
-    %template(charData) data<const char*>;
-    %template(doubleData) data<const double*>;
-
-    const DataType data_type() const;
-    const std::vector<size_t> &shape() const;
-    const size_t shape(size_t idx) const;
-    size_t nDim() const;
-    bool transpose() const;
-    size_t Size() const;
-    size_t MemSize() const;
-    void Reshape(const std::vector<size_t> &shape);
-    void ResetLike(const Tensor &t);
-    void AsType(DataType type);
-    void ToDevice(singa::Device *dev);
-    void ToHost();
-
-    template <typename SType> void SetValue(const SType x);
-    %template(floatSetValue) SetValue<float>;
-    // ...
-
-    /* no need to expose this function
-    template <typename DType> void CopyDataFromHostPtr(const DType *src,
-                                                       size_t num);
-    */
-
-    void CopyData(const Tensor &other);
-    Tensor Clone() const;
-    Tensor T() const;
-
-    /* python has no assignment operator as c++
-    Tensor &operator=(const Tensor &t); */
-    Tensor &operator+=(const Tensor &t);
-    Tensor &operator-=(const Tensor &t);
-    Tensor &operator*=(const Tensor &t);
-    Tensor &operator/=(const Tensor &t);
-
-
-    template <typename DType> Tensor &operator+=(const DType x);
-    %template(iAdd_f) operator+=<float>;
-    /* TODO(chonho-01) for other types */
-    // ...
-
-    template <typename DType> Tensor &operator-=(DType x);
-    %template(iSub_f) operator-=<float>;
-    /* TODO(chonho-01) for other types */
-    // ...
-
-    template <typename DType> Tensor &operator*=(DType x);
-    %template(iMul_f) operator*=<float>;
-    /* TODO(chonho-01) for other types */
-    // ...
-
-    template <typename DType> Tensor &operator/=(DType x);
-    %template(iDiv_f) operator/=<float>;
-    /* TODO(chonho-01) for other types */
-    // ...
-
-  };
-
-  /* TODO
-  inline void CheckDataTypeAndLang(const Tensor &in1, const Tensor &in2);
-  */
-  void CopyDataToFrom(Tensor *dst, const Tensor &src, size_t num,
-                      size_t src_offset = 0, size_t dst_offset = 0);
-
-  Tensor Reshape(const Tensor &in, const std::vector<size_t> &s);
-
-  Tensor Abs(const Tensor &t);
-  Tensor Exp(const Tensor &t);
-  Tensor Log(const Tensor &t);
-  Tensor ReLU(const Tensor &t);
-  Tensor Sigmoid(const Tensor &t);
-  Tensor Sign(const Tensor &t);
-  Tensor Sqrt(const Tensor &t);
-  Tensor Square(const Tensor &t);
-  Tensor Tanh(const Tensor &t);
-
-  Tensor Sum(const Tensor &t, int axis);
-  template <typename SType> SType Sum(const Tensor &t);
-  %template(floatSum) Sum<float>;
-  /* TODO(chonho-03) not implemented
-  %template(intSum) Sum<int>;
-  %template(charSum) Sum<char>;
-  %template(doubleSum) Sum<double>;
-  */
-
-  /* TODO(chonho-04) not implemented
-     need average of all elements ??? */
-  Tensor Average(const Tensor &t, int axis);
-  Tensor SoftMax(const Tensor &t, int axis = 0);
-
-  /* TODO(chonho-05) not implemented ???
-  Tensor Pow(const Tensor &base, Tensor exp);
-  template <typename DType>
-  Tensor Pow(const Tensor &t, DType x);
-  */
-
-
-  /* rename comparison operators */
-  %rename(LT_Tf) operator<(const Tensor &t, const float x);
-  %rename(LE_Tf) operator<=(const Tensor &t, const float x);
-  %rename(GT_Tf) operator>(const Tensor &t, const float x);
-  %rename(GE_Tf) operator>=(const Tensor &t, const float x);
-
-  template <typename DType>
-  Tensor operator<(const Tensor &t, const DType x);
-  %template(op) operator< <float>;
-  // --- other types
-
-  template <typename DType>
-  Tensor operator<=(const Tensor &t, const DType x);
-  %template(op) operator<= <float>;
-  // --- other types
-
-  template <typename DType>
-  Tensor operator>(const Tensor &t, const DType x);
-  %template(op) operator> <float>;
-  // --- other types
-
-  template <typename DType>
-  Tensor operator>=(const Tensor &t, const DType x);
-  %template(op) operator>= <float>;
-  // --- other types
-
-  /* TODO(chonho-06)
-  no need to include theses
-  in python, these can be replaced with comparison operators
-
-  template <typename DType>
-  void LT(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void LE(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void GT(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void GE(const Tensor &t, DType x, Tensor *ret);
-  */
-
-
-  /* rename operators */
-  %rename(Add_TT) operator+(const Tensor &lhs, const Tensor &rhs);
-  %rename(Sub_TT) operator-(const Tensor &lhs, const Tensor &rhs);
-  %rename(Mul_TT) operator*(const Tensor &lhs, const Tensor &rhs);
-  %rename(Div_TT) operator/(const Tensor &lhs, const Tensor &rhs);
-  Tensor operator+(const Tensor &lhs, const Tensor &rhs);
-  Tensor operator-(const Tensor &lhs, const Tensor &rhs);
-  Tensor operator*(const Tensor &lhs, const Tensor &rhs);
-  Tensor operator/(const Tensor &lhs, const Tensor &rhs);
-
-  %rename(Add_Tf) operator+(const Tensor &t, float x);
-  template <typename DType>
-  Tensor operator+(const Tensor &t, DType x);
-  %template(op) operator+<float>;
-  // --- other types
-
-  %rename(Sub_Tf) operator-(const Tensor &t, float x);
-  template <typename DType>
-  Tensor operator-(const Tensor &t, DType x);
-  %template(op) operator-<float>;
-  // --- other types
-
-  %rename(Mul_Tf) operator*(const Tensor &t, float x);
-  template <typename DType>
-  Tensor operator*(const Tensor &t, DType x);
-  %template(op) operator*<float>;
-  // --- other types
-
-  %rename(Div_Tf) operator/(const Tensor &t, float x);
-  template <typename DType>
-  Tensor operator/(const Tensor &t, DType x);
-  %template(op) operator/<float>;
-  // --- other types
-
-  /* TODO(chonho-07)
-  no need to include theses
-  in python, these can be replaced with operators
-
-  void Add(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
-  void Sub(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
-  void EltwiseMult(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
-  void Div(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
-
-  template <typename DType>
-  void Add(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void Sub(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void EltwiseMult(const Tensor &t, DType x, Tensor *ret);
-  template <typename DType>
-  void Div(const Tensor &t, DType x, Tensor *ret);
-  */
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/generate_singa_wrapper.sh
----------------------------------------------------------------------
diff --git a/src/python/generate_singa_wrapper.sh b/src/python/generate_singa_wrapper.sh
deleted file mode 100755
index b98bf91..0000000
--- a/src/python/generate_singa_wrapper.sh
+++ /dev/null
@@ -1,39 +0,0 @@
-#!/usr/bin/env bash
-#/**
-# * 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.
-# */
-
-SINGA_ROOT=/home/chonho/incubator-singa
-SINGA_SRC=${SINGA_ROOT}/src
-SRC_CC=(${SINGA_SRC}/core/tensor/tensor.cc \
-        ${SINGA_SRC}/core/device/device.cc
-       )
-USR_LOCAL=/home/chonho/local
-
-swig -c++ -python -I../../include singa.i
-
-g++ -fPIC ${SRC_CC[@]} singa_wrap.cxx -shared -o _singa.so \
-    -L${USR_LOCAL}/lib -lprotobuf -Wl,-rpath=${USR_LOCAL}/lib \
-    -L../../lib -lsinga_core -lsinga_model -lsinga_utils -Wl,-rpath=../../lib \
-    -std=c++11 \
-    -I../.. \
-    -I../../include \
-    -I${SINGA_SRC} \
-    -I${USR_LOCAL}/include \
-    -I${USR_LOCAL}/cudnn/include \
-    -I/usr/include/python2.7 \
-    -I/usr/local/cuda-7.0/include

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/model_layer.i
----------------------------------------------------------------------
diff --git a/src/python/model_layer.i b/src/python/model_layer.i
deleted file mode 100644
index 3fb4917..0000000
--- a/src/python/model_layer.i
+++ /dev/null
@@ -1,83 +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.
-*
-*************************************************************/
-
-/*interface file for swig */
-
-%module singa_layer
-%include "std_vector.i"
-%include "std_string.i"
-%include "std_pair.i"
-
-%{
-#include "singa/model/layer.h"
-#include "singa/core/tensor.h"
-#include "singa/proto/model.pb.h"
-using singa::Tensor;
-using singa::ParamSpec;
-using singa::DataType;
-using singa::Device;
-using singa::LayerConf;
-%}
-
-namespace std {
-  %template(strVector) vector<string>;
-  %template(paramVector) vector<ParamSpec>;
-  %template(tensorVector) vector<Tensor>;
-  %template(tensorPtrVector) vector<Tensor*>;
-  %template(ttvecPair) pair<Tensor, vector<Tensor>>;
-  %template(tvectvecPair) pair<vector<Tensor>, vector<Tensor>>;
-}
-
-namespace singa {
-
-  class Layer {
-    public:
-      Layer();
-      void Setup(const std::string& proto_str);
-
-      std::string ToProtoStr() const;
-      const std::vector<ParamSpec> param_specs();
-      const ParamSpec& param_specs(size_t i);
-      const std::vector<Tensor*> param_values();
-      Tensor* param_value(size_t i);
-      const std::vector<std::string> param_names();
-      const std::string& param_name(size_t i);
-      const std::string name() const;
-
-      /* virtual functions */
-      virtual const std::string layer_type() const;
-      virtual void Setup(const LayerConf& conf);
-      virtual void ToDevice(Device* device);
-      virtual void AsType(DataType dtype);
-      virtual void ToProto(LayerConf* conf) const;
-
-      virtual const Tensor
-      Forward(int flag, const Tensor& input);
-      virtual const std::vector<Tensor>
-      Forward(int flag, const std::vector<Tensor>& inputs);
-      virtual const std::pair<Tensor, std::vector<Tensor>>
-      Backward(int flag, const Tensor& grad);
-      virtual const std::pair<std::vector<Tensor>, std::vector<Tensor>>
-      Backward(int flag, const vector<Tensor>& grads);
-  };
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/singa.i
----------------------------------------------------------------------
diff --git a/src/python/singa.i b/src/python/singa.i
deleted file mode 100644
index 8b5e2dc..0000000
--- a/src/python/singa.i
+++ /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.
-*
-*************************************************************/
-
-/*interface file for swig */
-
-%module singa
-%include "core_tensor.i"
-%include "core_device.i"
-%include "model_layer.i"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/swig/core_device.i
----------------------------------------------------------------------
diff --git a/src/python/swig/core_device.i b/src/python/swig/core_device.i
new file mode 100644
index 0000000..ab9abd8
--- /dev/null
+++ b/src/python/swig/core_device.i
@@ -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.
+*
+*************************************************************/
+
+/*interface file for swig */
+
+%module core_device
+%include "std_vector.i"
+%include "std_string.i"
+
+%{
+#include "singa/core/device.h"
+%}
+
+namespace singa{
+
+  %nodefault Device;
+  class Device {
+   public:
+    virtual void SetRandSeed(unsigned seed) = 0;
+    Device* host();
+    int id() const;
+  };
+
+  class CppCPU : public Device {
+   public:
+    CppCPU(int id = -1, int num_executors = 1,
+           std::string scheduler = "sync", std::string vm = "gc-only");
+    void SetRandSeed(unsigned seed) override;
+    /* (TODO) add necessary functions of CppCPU class
+    */
+  };
+
+  class CudaGPU : public Device {
+   public:
+    CudaGPU(int id = 0, int num_executors = 1,
+            std::string scheduler = "sync", std::string vm = "gc-only");
+    void SetRandSeed(unsigned seed) override;
+    /* (TODO) add necessary functions of CudaGPU class
+    */
+  };
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/swig/core_tensor.i
----------------------------------------------------------------------
diff --git a/src/python/swig/core_tensor.i b/src/python/swig/core_tensor.i
new file mode 100644
index 0000000..409ab0c
--- /dev/null
+++ b/src/python/swig/core_tensor.i
@@ -0,0 +1,263 @@
+/************************************************************
+*
+* 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.
+*
+*************************************************************/
+
+/*interface file for swig */
+
+%module core_tensor
+%include "std_vector.i"
+%include "std_string.i"
+
+%include "carrays.i"
+%array_class(float, floatArray);
+%array_class(int, intArray);
+%array_class(char, charArray);
+%array_class(double, doubleArray);
+
+%{
+#include "singa/core/tensor.h"
+#include "singa/core/device.h"
+#include "singa/proto/core.pb.h"
+#include "singa/proto/model.pb.h"
+using singa::DataType;
+%}
+
+%template(Shape) std::vector<size_t>;
+
+namespace singa{
+
+  enum DataType {
+    kFloat32, kFloat16, kInt, kChar, kDouble
+  };
+
+  inline size_t Product(const std::vector<size_t> &shape,
+                        int start = 0, size_t len = 0);
+  inline size_t SizeOf(DataType t);
+
+  class Tensor {
+
+   public:
+    Tensor();
+    explicit Tensor(const std::vector<size_t> &shape,
+                    DataType dtype = kFloat32);
+    Tensor(const std::vector<size_t> &shape,
+           singa::Device *dev, DataType dtype = kFloat32);
+    Tensor(const Tensor &from);
+
+    //Blob *blob() const;
+    singa::Device *device() const;
+
+    template <typename DType> DType data() const;
+    %template(floatData) data<const float*>;
+    %template(intData) data<const int*>;
+    %template(charData) data<const char*>;
+    %template(doubleData) data<const double*>;
+
+    const DataType data_type() const;
+    const std::vector<size_t> &shape() const;
+    const size_t shape(size_t idx) const;
+    size_t nDim() const;
+    bool transpose() const;
+    size_t Size() const;
+    size_t MemSize() const;
+    void Reshape(const std::vector<size_t> &shape);
+    void ResetLike(const Tensor &t);
+    void AsType(DataType type);
+    void ToDevice(singa::Device *dev);
+    void ToHost();
+
+    template <typename SType> void SetValue(const SType x);
+    %template(floatSetValue) SetValue<float>;
+    // ...
+
+    /* no need to expose this function
+    template <typename DType> void CopyDataFromHostPtr(const DType *src,
+                                                       size_t num);
+    */
+
+    void CopyData(const Tensor &other);
+    Tensor Clone() const;
+    Tensor T() const;
+
+    /* python has no assignment operator as c++
+    Tensor &operator=(const Tensor &t); */
+    Tensor &operator+=(const Tensor &t);
+    Tensor &operator-=(const Tensor &t);
+    Tensor &operator*=(const Tensor &t);
+    Tensor &operator/=(const Tensor &t);
+
+
+    template <typename DType> Tensor &operator+=(const DType x);
+    %template(iAdd_f) operator+=<float>;
+    /* TODO(chonho-01) for other types */
+    // ...
+
+    template <typename DType> Tensor &operator-=(DType x);
+    %template(iSub_f) operator-=<float>;
+    /* TODO(chonho-01) for other types */
+    // ...
+
+    template <typename DType> Tensor &operator*=(DType x);
+    %template(iMul_f) operator*=<float>;
+    /* TODO(chonho-01) for other types */
+    // ...
+
+    template <typename DType> Tensor &operator/=(DType x);
+    %template(iDiv_f) operator/=<float>;
+    /* TODO(chonho-01) for other types */
+    // ...
+
+  };
+
+  /* TODO
+  inline void CheckDataTypeAndLang(const Tensor &in1, const Tensor &in2);
+  */
+  void CopyDataToFrom(Tensor *dst, const Tensor &src, size_t num,
+                      size_t src_offset = 0, size_t dst_offset = 0);
+
+  Tensor Reshape(const Tensor &in, const std::vector<size_t> &s);
+
+  Tensor Abs(const Tensor &t);
+  Tensor Exp(const Tensor &t);
+  Tensor Log(const Tensor &t);
+  Tensor ReLU(const Tensor &t);
+  Tensor Sigmoid(const Tensor &t);
+  Tensor Sign(const Tensor &t);
+  Tensor Sqrt(const Tensor &t);
+  Tensor Square(const Tensor &t);
+  Tensor Tanh(const Tensor &t);
+
+  Tensor Sum(const Tensor &t, int axis);
+  template <typename SType> SType Sum(const Tensor &t);
+  %template(floatSum) Sum<float>;
+  /* TODO(chonho-03) not implemented
+  %template(intSum) Sum<int>;
+  %template(charSum) Sum<char>;
+  %template(doubleSum) Sum<double>;
+  */
+
+  /* TODO(chonho-04) not implemented
+     need average of all elements ??? */
+  Tensor Average(const Tensor &t, int axis);
+  Tensor SoftMax(const Tensor &t, int axis = 0);
+
+  /* TODO(chonho-05) not implemented ???
+  Tensor Pow(const Tensor &base, Tensor exp);
+  template <typename DType>
+  Tensor Pow(const Tensor &t, DType x);
+  */
+
+
+  /* rename comparison operators */
+  %rename(LT_Tf) operator<(const Tensor &t, const float x);
+  %rename(LE_Tf) operator<=(const Tensor &t, const float x);
+  %rename(GT_Tf) operator>(const Tensor &t, const float x);
+  %rename(GE_Tf) operator>=(const Tensor &t, const float x);
+
+  template <typename DType>
+  Tensor operator<(const Tensor &t, const DType x);
+  %template(op) operator< <float>;
+  // --- other types
+
+  template <typename DType>
+  Tensor operator<=(const Tensor &t, const DType x);
+  %template(op) operator<= <float>;
+  // --- other types
+
+  template <typename DType>
+  Tensor operator>(const Tensor &t, const DType x);
+  %template(op) operator> <float>;
+  // --- other types
+
+  template <typename DType>
+  Tensor operator>=(const Tensor &t, const DType x);
+  %template(op) operator>= <float>;
+  // --- other types
+
+  /* TODO(chonho-06)
+  no need to include theses
+  in python, these can be replaced with comparison operators
+
+  template <typename DType>
+  void LT(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void LE(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void GT(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void GE(const Tensor &t, DType x, Tensor *ret);
+  */
+
+
+  /* rename operators */
+  %rename(Add_TT) operator+(const Tensor &lhs, const Tensor &rhs);
+  %rename(Sub_TT) operator-(const Tensor &lhs, const Tensor &rhs);
+  %rename(Mul_TT) operator*(const Tensor &lhs, const Tensor &rhs);
+  %rename(Div_TT) operator/(const Tensor &lhs, const Tensor &rhs);
+  Tensor operator+(const Tensor &lhs, const Tensor &rhs);
+  Tensor operator-(const Tensor &lhs, const Tensor &rhs);
+  Tensor operator*(const Tensor &lhs, const Tensor &rhs);
+  Tensor operator/(const Tensor &lhs, const Tensor &rhs);
+
+  %rename(Add_Tf) operator+(const Tensor &t, float x);
+  template <typename DType>
+  Tensor operator+(const Tensor &t, DType x);
+  %template(op) operator+<float>;
+  // --- other types
+
+  %rename(Sub_Tf) operator-(const Tensor &t, float x);
+  template <typename DType>
+  Tensor operator-(const Tensor &t, DType x);
+  %template(op) operator-<float>;
+  // --- other types
+
+  %rename(Mul_Tf) operator*(const Tensor &t, float x);
+  template <typename DType>
+  Tensor operator*(const Tensor &t, DType x);
+  %template(op) operator*<float>;
+  // --- other types
+
+  %rename(Div_Tf) operator/(const Tensor &t, float x);
+  template <typename DType>
+  Tensor operator/(const Tensor &t, DType x);
+  %template(op) operator/<float>;
+  // --- other types
+
+  /* TODO(chonho-07)
+  no need to include theses
+  in python, these can be replaced with operators
+
+  void Add(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
+  void Sub(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
+  void EltwiseMult(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
+  void Div(const Tensor &lhs, const Tensor &rhs, Tensor *ret);
+
+  template <typename DType>
+  void Add(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void Sub(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void EltwiseMult(const Tensor &t, DType x, Tensor *ret);
+  template <typename DType>
+  void Div(const Tensor &t, DType x, Tensor *ret);
+  */
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/swig/model_layer.i
----------------------------------------------------------------------
diff --git a/src/python/swig/model_layer.i b/src/python/swig/model_layer.i
new file mode 100644
index 0000000..3fb4917
--- /dev/null
+++ b/src/python/swig/model_layer.i
@@ -0,0 +1,83 @@
+/************************************************************
+*
+* 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.
+*
+*************************************************************/
+
+/*interface file for swig */
+
+%module singa_layer
+%include "std_vector.i"
+%include "std_string.i"
+%include "std_pair.i"
+
+%{
+#include "singa/model/layer.h"
+#include "singa/core/tensor.h"
+#include "singa/proto/model.pb.h"
+using singa::Tensor;
+using singa::ParamSpec;
+using singa::DataType;
+using singa::Device;
+using singa::LayerConf;
+%}
+
+namespace std {
+  %template(strVector) vector<string>;
+  %template(paramVector) vector<ParamSpec>;
+  %template(tensorVector) vector<Tensor>;
+  %template(tensorPtrVector) vector<Tensor*>;
+  %template(ttvecPair) pair<Tensor, vector<Tensor>>;
+  %template(tvectvecPair) pair<vector<Tensor>, vector<Tensor>>;
+}
+
+namespace singa {
+
+  class Layer {
+    public:
+      Layer();
+      void Setup(const std::string& proto_str);
+
+      std::string ToProtoStr() const;
+      const std::vector<ParamSpec> param_specs();
+      const ParamSpec& param_specs(size_t i);
+      const std::vector<Tensor*> param_values();
+      Tensor* param_value(size_t i);
+      const std::vector<std::string> param_names();
+      const std::string& param_name(size_t i);
+      const std::string name() const;
+
+      /* virtual functions */
+      virtual const std::string layer_type() const;
+      virtual void Setup(const LayerConf& conf);
+      virtual void ToDevice(Device* device);
+      virtual void AsType(DataType dtype);
+      virtual void ToProto(LayerConf* conf) const;
+
+      virtual const Tensor
+      Forward(int flag, const Tensor& input);
+      virtual const std::vector<Tensor>
+      Forward(int flag, const std::vector<Tensor>& inputs);
+      virtual const std::pair<Tensor, std::vector<Tensor>>
+      Backward(int flag, const Tensor& grad);
+      virtual const std::pair<std::vector<Tensor>, std::vector<Tensor>>
+      Backward(int flag, const vector<Tensor>& grads);
+  };
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/254343cb/src/python/swig/singa.i
----------------------------------------------------------------------
diff --git a/src/python/swig/singa.i b/src/python/swig/singa.i
new file mode 100644
index 0000000..8b5e2dc
--- /dev/null
+++ b/src/python/swig/singa.i
@@ -0,0 +1,27 @@
+/************************************************************
+*
+* 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.
+*
+*************************************************************/
+
+/*interface file for swig */
+
+%module singa
+%include "core_tensor.i"
+%include "core_device.i"
+%include "model_layer.i"