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/08/11 15:03:33 UTC

[1/2] incubator-singa git commit: SINGA-217 build python package with setup.py - Singa python binding, package with pip or create wheel - In the cmake file, specify USE_PYTHON ON, then cmake and make. After that run " pip install ." under "

Repository: incubator-singa
Updated Branches:
  refs/heads/dev 05720c216 -> 6b2ff3c98


SINGA-217 build python package with setup.py
  - Singa python binding, package with pip or create wheel
  - In the cmake file, specify USE_PYTHON ON, then cmake and make.
    After that run "<sudo> pip install ." under "build/python" path
    User can run command 'singa' in a model workspace to serve and train
  - dependent on numpy>=1.11.0; protobuf>=2.5.0,<3; flask>=0.10.1
    Users should also install openblas
  - Models are sharing on github gists.
  - Todo, optimize cmake compile method to avoid compile source file twice
    refine singa command


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

Branch: refs/heads/dev
Commit: bf81f252647f60d525ffc9ddfa7031183afbc6e5
Parents: 05720c2
Author: aaronwwf <dc...@gmail.com>
Authored: Wed Jul 20 17:23:19 2016 +0800
Committer: aaronwwf <dc...@gmail.com>
Committed: Thu Aug 11 16:55:43 2016 +0800

----------------------------------------------------------------------
 CMakeLists.txt                |   5 +-
 cmake/Dependencies.cmake      |   2 +-
 include/singa/core/device.h   |   3 +
 src/CMakeLists.txt            |  39 +++---
 src/python/setup.py.in        |   7 +-
 src/python/singa/__init__.py  | 244 ++++++++++++++++++++++++++++++++++++-
 src/python/singa/net.py       |   2 +-
 src/python/swig/config.i.in   |   5 +
 src/python/swig/core_device.i |   2 +
 src/python/swig/model_layer.i |   6 +
 src/python/swig/singa.i       |   1 +
 test/python/test_layer.py     |   2 +-
 12 files changed, 285 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9efadc0..9c4f326 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -1,6 +1,7 @@
 CMAKE_MINIMUM_REQUIRED(VERSION 2.6)
 
 PROJECT(singa)
+SET(PACKAGE_VERSION "1.0.0")
 SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++11 -g -O2 ")
 
 LIST(APPEND CMAKE_MODULE_PATH ${PROJECT_SOURCE_DIR}/cmake/Thirdparty)
@@ -19,8 +20,8 @@ SET(SINGA_INCLUDE_DIR
 INCLUDE_DIRECTORIES(${SINGA_INCLUDE_DIR})
 
 OPTION(USE_CBLAS "Use CBlas libs" ON)
-OPTION(USE_CUDA "Use Cuda libs" ON)
-OPTION(USE_CUDNN "Use Cudnn libs" ON)
+OPTION(USE_CUDA "Use Cuda libs" OFF)
+OPTION(USE_CUDNN "Use Cudnn libs" OFF)
 OPTION(USE_OPENCV "Use opencv" OFF)
 OPTION(USE_LMDB "Use LMDB libs" OFF)
 OPTION(USE_PYTHON "Generate py wrappers" ON)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/cmake/Dependencies.cmake
----------------------------------------------------------------------
diff --git a/cmake/Dependencies.cmake b/cmake/Dependencies.cmake
index 68d0bfc..ceef429 100644
--- a/cmake/Dependencies.cmake
+++ b/cmake/Dependencies.cmake
@@ -5,7 +5,7 @@ SET(SINGA_LINKER_LIBS "")
 FIND_PACKAGE( Protobuf REQUIRED )
 INCLUDE_DIRECTORIES(SYSTEM ${PROTOBUF_INCLUDE_DIR})
 MESSAGE(STATUS "proto libs " ${PROTOBUF_LIBRARIES})
-LIST(APPEND singa_linker_libs ${PROTOBUF_LIBRARIES})
+LIST(APPEND SINGA_LINKER_LIBS ${PROTOBUF_LIBRARIES})
 INCLUDE("cmake/Protobuf.cmake")
 
 IF(USE_LMDB)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/include/singa/core/device.h
----------------------------------------------------------------------
diff --git a/include/singa/core/device.h b/include/singa/core/device.h
index 4c46114..a564524 100644
--- a/include/singa/core/device.h
+++ b/include/singa/core/device.h
@@ -303,6 +303,7 @@ private:
 /// If CUDA or OPENCL are not enabled, then the respective related methods should
 /// return something that indicates their absence (for example, 0 devices);
 /// however they should always be available regardless of compile-time switches.
+#ifdef USE_CUDA
 class Platform {
 public:
 
@@ -372,6 +373,8 @@ private:
 #endif  // USE_OPENCL
 };
 
+#endif // USE_CUDA
+
 }  // namespace singa
 
 #endif  // SINGA_CORE_DEVICE_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index 06f177d..66d89dc 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -6,6 +6,7 @@ IF (USE_PYTHON)
     protobuf_generate_python(proto_pys ${proto_files})
 ENDIF()
 INCLUDE_DIRECTORIES("${CMAKE_BINARY_DIR}/include")
+
 #message(STATUS "include: ${CMAKE_BINARY_DIR} ")
 #message(STATUS "srcs: ${proto_srcs}")
 #message(STATUS "hdrs: ${proto_hdrs}")
@@ -21,6 +22,8 @@ FOREACH(fil ${proto_hdrs})
 ENDFOREACH()
 LIST(APPEND SINGA_LINKER_LIBS proto)
 
+SET(PREVIOUS_LINKER_LIBS ${SINGA_LINKER_LIBS})
+
 #FILE(GLOB_RECURSE utils_source ${CMAKE_CURRENT_SOURCE_DIR}/utils/ "*.cc")
 AUX_SOURCE_DIRECTORY(utils utils_source)
 #message(STATUS "UTILS ${utils_source}")
@@ -28,7 +31,6 @@ 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")
 AUX_SOURCE_DIRECTORY(core/device core_source)
 AUX_SOURCE_DIRECTORY(core/memory core_source)
@@ -74,18 +76,11 @@ ADD_LIBRARY(singa_io SHARED ${io_source})
 TARGET_LINK_LIBRARIES(singa_io ${SINGA_LINKER_LIBS})
 LIST(APPEND SINGA_LINKER_LIBS singa_io)
 
-#ADD_LIBRARY(singa_layer SHARED ${LAYER_SOURCE})
-#ADD_LIBRARY(singa_model SHARED ${MODEL_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})
-
 IF(USE_PYTHON)
 
+    FILE(REMOVE "${CMAKE_CURRENT_SOURCE_DIR}/python/swig/config.i")
+    CONFIGURE_FILE("${CMAKE_CURRENT_SOURCE_DIR}/python/swig/config.i.in" "${CMAKE_CURRENT_SOURCE_DIR}/python/swig/config.i")
+
     FILE(GLOB python_files python/swig/singa.i)
     # delete old .cxx file
     FILE(REMOVE "${CMAKE_CURRENT_SOURCE_DIR}/python/swig/singa_wrap.cxx")
@@ -98,17 +93,13 @@ IF(USE_PYTHON)
     file(GLOB_RECURSE python_source_files RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} *.py)
 
     create_symlinks(${python_source_files})
-
-    FILE(GLOB_RECURSE layer_source_files ${CMAKE_SOURCE_DIR}/src/model/layer/*.cc)
-    # message(STATUS "layer_source_files ${layer_source_files}")
-    FILE(GLOB_RECURSE metric_source_files ${CMAKE_SOURCE_DIR}/src/model/metric/*.cc)
-    FILE(GLOB_RECURSE loss_source_files ${CMAKE_SOURCE_DIR}/src/model/loss/*.cc)
-    SET(python_cxxs "${CMAKE_SOURCE_DIR}/src/core/tensor/tensor.cc;${CMAKE_SOURCE_DIR}/src/core/device/device.cc;${layer_source_files};${metric_source_files};${loss_source_files}")
-    ADD_LIBRARY(_singa_wrap SHARED ${python_srcs} ${python_cxxs})
-    SET(WRAPPER_LINKER_LIBS "${SINGA_LINKER_LIBS};protobuf")
+    
+    SET(python_cxxs "${core_source};${io_source};${model_source};${utils_source}")
+    ADD_LIBRARY(_singa_wrap SHARED ${python_srcs} ${python_cxxs} ${cuda_objs})
+    SET(WRAPPER_LINKER_LIBS "${PREVIOUS_LINKER_LIBS}")
     TARGET_LINK_LIBRARIES(_singa_wrap ${WRAPPER_LINKER_LIBS})
     TARGET_INCLUDE_DIRECTORIES(_singa_wrap PRIVATE ${PYTHON_INCLUDE_DIRS})
-    #message(STATUS "PYTHON_INCLUDE_DIRS ${PYTHON_INCLUDE_DIRS}")
+    #message(STATUS "PREVIOUS_LINKER_LIBS ${PREVIOUS_LINKER_LIBS}")
 
     SET_TARGET_PROPERTIES(_singa_wrap
         PROPERTIES PREFIX ""
@@ -117,10 +108,10 @@ IF(USE_PYTHON)
 
     #SETUP
     SET(SETUP_PY_IN "python/setup.py.in")
-    set(SETUP_PY    "${CMAKE_BINARY_DIR}/python/setup.py")
-    configure_file(${SETUP_PY_IN} ${SETUP_PY})
+    SET(SETUP_PY    "${CMAKE_BINARY_DIR}/python/setup.py")
+    CONFIGURE_FILE(${SETUP_PY_IN} ${SETUP_PY})
 
-    #COPY protobuf files to python/singa/proto
-    FILE(COPY ${CMAKE_BINARY_DIR}/python/singa/__init__.py DESTINATION ${CMAKE_BINARY_DIR}/python/singa/proto )
+    #create python/singa/proto/__init__.py
+    FILE(WRITE ${CMAKE_BINARY_DIR}/python/singa/proto/__init__.py "")
 
 ENDIF(USE_PYTHON)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/setup.py.in
----------------------------------------------------------------------
diff --git a/src/python/setup.py.in b/src/python/setup.py.in
index b53e54b..d1ac3c9 100644
--- a/src/python/setup.py.in
+++ b/src/python/setup.py.in
@@ -40,7 +40,12 @@ setup(
 
     #py_modules=["singa"],
 
-    #install_requires=['peppercorn'],
+    install_requires=[
+        'numpy>=1.11.0',
+        'protobuf>=2.5.0,<3',
+        'flask>=0.10.1'
+        ],
+
     #List additional groups of dependencies here (e.g. development
     #dependencies). You can install these using the following syntax,
     #for example:

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/singa/__init__.py
----------------------------------------------------------------------
diff --git a/src/python/singa/__init__.py b/src/python/singa/__init__.py
index d5e48bb..ec26906 100644
--- a/src/python/singa/__init__.py
+++ b/src/python/singa/__init__.py
@@ -1,3 +1,241 @@
-def main():
-    """Entry point for the application script"""
-    print("Welcome to SINGA!")
+# 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 script is the main entrance for user to run singa inside a model workspace
+
+To use this script, user sudo install these dependencies: flask pillow and protobuf 
+'''
+
+import sys, glob, os, random, shutil, time
+from flask import Flask, request, redirect, url_for
+from PIL import Image
+import numpy as np
+import ConfigParser 
+import urllib, traceback
+
+
+from argparse import ArgumentParser
+from argparse import RawDescriptionHelpFormatter
+sys.path.append(os.getcwd())
+
+__all__ = []
+__version__ = 0.1
+__date__ = '2016-07-20'
+__updated__ = '2016-07-20'
+__shortdesc__ = '''
+welcome to singa
+'''
+
+app = Flask(__name__)
+config = ConfigParser.RawConfigParser()    
+service = {}
+data_path = "data_"
+parameter_path = "parameter_"
+
+debug = False
+
+class CLIError(Exception):
+    '''Generic exception to raise and log different fatal errors.'''
+    def __init__(self, msg):
+        super(CLIError).__init__(type(self))
+        self.msg = "E: %s" % msg
+    def __str__(self):
+        return self.msg
+    def __unicode__(self):
+        return self.msg
+
+def main(argv=None): # IGNORE:C0111
+    '''Command line options.'''
+
+    from . import device
+
+    if argv is None:
+        argv = sys.argv
+    else:
+        sys.argv.extend(argv)
+
+    program_name = os.path.basename(sys.argv[0])
+    program_version = "v%s" % __version__
+    program_build_date = str(__updated__)
+    program_version_message = '%%(prog)s %s (%s)' % (program_version, program_build_date)
+    program_shortdesc = __shortdesc__
+    program_license = '''%s
+
+  Created by dbsystem group on %s.
+  Copyright 2016 NUS School of Computing. All rights reserved.
+
+  Licensed under the Apache License 2.0
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Distributed on an "AS IS" basis without warranties
+  or conditions of any kind, either express or implied.
+
+USAGE
+''' % (program_shortdesc, str(__date__))
+
+    global debug
+
+    try:
+        # Setup argument parser
+        parser = ArgumentParser(description=program_license, formatter_class=RawDescriptionHelpFormatter)
+        parser.add_argument("-p", "--port", dest="port", default=5000, help="the port to listen to, default is 5000")
+        parser.add_argument("-param", "--parameter", dest="parameter",  help="the parameter file path to be loaded")
+        parser.add_argument("-D", "--debug", dest="debug", action="store_true", help="whether need to debug")
+        parser.add_argument("-R", "--reload", dest="reload_data", action="store_true", help="whether need to reload data")
+        parser.add_argument("-C", "--cpu", dest="use_cpu", action="store_true", help="Using cpu or not, default is using gpu")
+        parser.add_argument("-m", "--mode", dest="mode", choices=['train','test','serve'], default='serve', help="On Which mode (train,test,serve) to run singa")
+        parser.add_argument('-V', '--version', action='version', version=program_version_message)
+
+        # Process arguments
+        args = parser.parse_args()
+
+        port = args.port
+        parameter_file = args.parameter
+        mode = args.mode
+        need_reload = args.reload_data
+        use_cpu = args.use_cpu
+        debug = args.debug
+
+        #prepare data files
+        config.read('file.cfg')
+        file_prepare(need_reload)
+
+
+        import network as net
+        model = net.create()
+
+        #load parameter
+        parameter_file=get_parameter(parameter_file)
+
+        if parameter_file:
+            print "load parameter file: %s" % parameter_file
+            model.load(parameter_file)
+        
+        if use_cpu:
+            raise CLIError("Currently cpu is not support!")
+        else:
+            print "runing with gpu"
+            d = device.create_cuda_gpu()
+
+        model.to_device(d)
+
+        if mode == "serve":
+            print "runing singa in serve mode, listen to  port: %s " % port
+            global service
+            from serve import Service
+            service =Service(model,d)
+
+            app.debug = debug 
+            app.run(host='0.0.0.0', port= port)
+        elif mode == "train":
+            print "runing singa in train mode"
+            global trainer 
+            from train import Trainer 
+            trainer= Trainer(model,d)
+            if not parameter_file:
+                trainer.initialize()
+            trainer.train()
+        else:
+            raise CLIError("Currently only serve mode is surpported!")
+        return 0
+    except KeyboardInterrupt:
+        ### handle keyboard interrupt ###
+        return 0
+    except Exception, e:
+        if debug:
+            traceback.print_exc()
+            raise(e)
+        indent = len(program_name) * " "
+        sys.stderr.write(program_name + ": " + str(e) + "\n")
+        sys.stderr.write(indent + "  for help use --help \n\n")
+        return 2
+
+def file_prepare(reload_data=False):
+    '''
+        download all files and generate data.py
+    '''
+    if not reload_data and os.path.exists("data_.py"):
+        return
+
+    print "download file"
+    #clean data
+    shutil.rmtree("data_.py",ignore_errors=True)
+    shutil.rmtree("data_",ignore_errors=True)
+
+    data_py=open("data_.py",'w') 
+    data_py.write("#%s" % "This file is Generated by SINGA, please don't edit\n\n")
+    if config.has_section("data"):
+        file_list = config.items("data")
+        #download files
+        for f in file_list:
+            name,path=download_file(f[0],f[1],data_path)
+            data_py.write("%s=\"%s\"\n" % (name,path)) 
+
+    data_py.flush()
+    data_py.close()
+    
+    if config.has_section("parameter"):
+        parameter_list = config.items("parameter")
+        for p in parameter_list:
+            download_file(p[0],p[1],parameter_path)
+
+def download_file(name,path,dest):
+    '''
+    download one file to dest
+    '''
+    if not os.path.exists(dest):
+        os.makedirs(dest)
+    if (path.startswith('http')):
+        file_name = path.split('/')[-1]
+        target = os.path.join(dest,file_name)
+        urllib.urlretrieve(path,target)
+    return name,target
+
+
+def get_parameter(file_name=None):
+    '''
+    get the paticular file name or get the last parameter file
+    '''
+    if not os.path.exists(parameter_path):
+        os.makedirs(parameter_path)
+        return 
+
+    if file_name:
+	return os.path.join(parameter_path,file_name)
+
+    parameter_list = [ os.path.join(parameter_path,f) for f in os.listdir(parameter_path)]
+    if len(parameter_list)==0:
+        return
+    parameter_list.sort()
+
+    return parameter_list[-1]
+	
+@app.route("/")
+def index():
+    return "Hello SINGA User!"
+
+@app.route('/predict', methods=['POST'])
+def predict():
+    if request.method == 'POST':
+        try:
+            response=service.serve(request)
+        except Exception as e:
+            return e
+        return response
+    return "error, should be post request"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/singa/net.py
----------------------------------------------------------------------
diff --git a/src/python/singa/net.py b/src/python/singa/net.py
index 1617717..f040378 100644
--- a/src/python/singa/net.py
+++ b/src/python/singa/net.py
@@ -48,7 +48,7 @@ class FeedForwardNet(object):
         """
         if len(self.layers) > 0 and lyr.has_setup is False:
             shape = self.layers[-1].get_output_sample_shape()
-            print shape
+            #print shape
             lyr.setup(shape)
         self.layers.append(lyr)
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/swig/config.i.in
----------------------------------------------------------------------
diff --git a/src/python/swig/config.i.in b/src/python/swig/config.i.in
new file mode 100644
index 0000000..ed386b2
--- /dev/null
+++ b/src/python/swig/config.i.in
@@ -0,0 +1,5 @@
+// Pass in cmake configurations to swig
+#cmakedefine01 USE_CUDA 
+#cmakedefine01 USE_CUDNN 
+#cmakedefine CUDNN_VERSION_MAJOR ${CUDNN_VERSION_MAJOR}
+#cmakedefine CUDNN_VERSION_PATCH ${CUDNN_VERSION_PATCH} 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/swig/core_device.i
----------------------------------------------------------------------
diff --git a/src/python/swig/core_device.i b/src/python/swig/core_device.i
index a5d0731..21b97b4 100644
--- a/src/python/swig/core_device.i
+++ b/src/python/swig/core_device.i
@@ -49,6 +49,7 @@ class Device {
   int id() const;
 };
 
+#if USE_CUDA
 class Platform {
  public:
   static int GetNumGPUs();
@@ -63,5 +64,6 @@ class Platform {
   static std::shared_ptr<Device> GetDefaultDevice();
 };
 
+#endif // USE_CUDA
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/swig/model_layer.i
----------------------------------------------------------------------
diff --git a/src/python/swig/model_layer.i b/src/python/swig/model_layer.i
index a6cdad1..f82b0f1 100644
--- a/src/python/swig/model_layer.i
+++ b/src/python/swig/model_layer.i
@@ -44,7 +44,9 @@ using singa::LayerConf;
 
 %shared_ptr(singa::Layer)
 %shared_ptr(singa::RNN)
+#if USE_CUDNN
 %shared_ptr(singa::CudnnRNN)
+#endif
 
 namespace std {
   %template(strVector) vector<string>;
@@ -81,6 +83,8 @@ const std::vector<std::string> GetRegisteredLayers();
 class RNN : public Layer {
 };
 
+#if USE_CUDA && USE_CUDNN
+#if CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
 class CudnnRNN : public RNN {
  public:
  // note: Must use std::vector instead of vector.
@@ -92,5 +96,7 @@ class CudnnRNN : public RNN {
     const std::vector<size_t> GetOutputSampleShape() const override;
 };
 
+#endif  // CUDNN_VERSION_MINOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#endif  // USE_CUDA && USE_CUDNN
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/src/python/swig/singa.i
----------------------------------------------------------------------
diff --git a/src/python/swig/singa.i b/src/python/swig/singa.i
index 3f12569..12f46f3 100644
--- a/src/python/swig/singa.i
+++ b/src/python/swig/singa.i
@@ -22,6 +22,7 @@
 /*interface file for swig */
 
 %module singa_wrap
+%include "config.i"
 %include "core_tensor.i"
 %include "core_device.i"
 %include "model_layer.i"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/bf81f252/test/python/test_layer.py
----------------------------------------------------------------------
diff --git a/test/python/test_layer.py b/test/python/test_layer.py
index 7e1059e..7078240 100644
--- a/test/python/test_layer.py
+++ b/test/python/test_layer.py
@@ -3,7 +3,7 @@ import os
 import unittest
 import numpy as np
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+#sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 
 from singa import layer
 from singa import device


[2/2] incubator-singa git commit: SINGA-217 build python package with setup.py

Posted by wa...@apache.org.
SINGA-217 build python package with setup.py

CudnnRNN requires cudnn version later than 5.05, the previous check
based on CUDNN_VERSION_MAJOR and CUDNN_VERSION_PATCH does not work for
version 5.13. Therefore, we replaced the check to CUDNN_VERSION >= 5005.
CUDNN_VERSION is defined in cudnn.h.

cmake defines CUDNN_VERSION_SWIG which is used by model_layer.i for cudnn
rnn.

Remvoe the hardcoded sys imports for python files. Now users need to
install the python module to run python examples and tests.

Tested installation in python virtual enviornment and conda
virtural enviorment:

Inside build/python
```
pip install -e .
```


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

Branch: refs/heads/dev
Commit: 6b2ff3c98c2a82aa7bc3db245393c3a55f0449cd
Parents: bf81f25
Author: Wei Wang <wa...@gmail.com>
Authored: Thu Aug 11 22:20:45 2016 +0800
Committer: Wei Wang <wa...@gmail.com>
Committed: Thu Aug 11 23:00:48 2016 +0800

----------------------------------------------------------------------
 CMakeLists.txt                     |   2 +-
 cmake/Templates/singa_config.h.in  |   1 +
 cmake/Thirdparty/FindCUDNN.cmake   |   2 +-
 examples/char-rnn/sample.py        |   2 +-
 examples/char-rnn/train.py         |   2 +-
 examples/cifar10/alexnet.py        |   2 +-
 examples/cifar10/predict.py        |   2 +-
 examples/cifar10/train.py          |   2 +-
 examples/cifar10/vgg.py            |   2 +-
 src/io/csv_decoder.cc              |   4 +-
 src/model/layer/cudnn_dropout.cc   |   6 +-
 src/model/layer/cudnn_dropout.h    |   5 +-
 src/model/layer/cudnn_rnn.cc       |   4 +-
 src/model/layer/cudnn_rnn.h        |   5 +-
 src/python/singa/__init__.py       |  25 +++--
 src/python/swig/config.i.in        |   7 +-
 src/python/swig/model_layer.i      |   4 +-
 test/python/example_test_device.py |  36 -------
 test/python/example_test_tensor.py | 179 --------------------------------
 test/python/test_tensor.py         |   2 +
 test/singa/test_cudnn_dropout.cc   |   4 +-
 test/singa/test_cudnn_rnn.cc       |   4 +-
 22 files changed, 45 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 9c4f326..257fb14 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -21,7 +21,7 @@ INCLUDE_DIRECTORIES(${SINGA_INCLUDE_DIR})
 
 OPTION(USE_CBLAS "Use CBlas libs" ON)
 OPTION(USE_CUDA "Use Cuda libs" OFF)
-OPTION(USE_CUDNN "Use Cudnn libs" OFF)
+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" ON)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/cmake/Templates/singa_config.h.in
----------------------------------------------------------------------
diff --git a/cmake/Templates/singa_config.h.in b/cmake/Templates/singa_config.h.in
index d03d58b..0211f09 100644
--- a/cmake/Templates/singa_config.h.in
+++ b/cmake/Templates/singa_config.h.in
@@ -16,6 +16,7 @@
 #cmakedefine CUDNN_VERSION_MAJOR @CUDNN_VERSION_MAJOR@
 #cmakedefine CUDNN_VERSION_MINOR @CUDNN_VERSION_MINOR@
 #cmakedefine CUDNN_VERSION_PATCH @CUDNN_VERSION_PATCH@
+#cmakedefine CUDNN_VERSION_SWIG @CUDNN_VERSION_SWIG@
 
 #cmakedefine USE_OPENCL
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/cmake/Thirdparty/FindCUDNN.cmake
----------------------------------------------------------------------
diff --git a/cmake/Thirdparty/FindCUDNN.cmake b/cmake/Thirdparty/FindCUDNN.cmake
index cefc4fe..fbc103c 100644
--- a/cmake/Thirdparty/FindCUDNN.cmake
+++ b/cmake/Thirdparty/FindCUDNN.cmake
@@ -25,7 +25,7 @@ IF(CUDNN_FOUND)
     IF(NOT CUDNN_VERSION_MAJOR)
         SET(CUDNN_VERSION "???")
     ELSE()
-        SET(CUDNN_VERSION "${CUDNN_VERSION_MAJOR}.${CUDNN_VERSION_MINOR}.${CUDNN_VERSION_PATCH}")
+      MATH(EXPR CUDNN_VERSION_SWIG "${CUDNN_VERSION_MAJOR} * 1000 + ${CUDNN_VERSION_MINOR} * 100 + ${CUDNN_VERSION_PATCH}")
     ENDIF()
     MESSAGE(STATUS "Found Cudnn_v${CUDNN_VERSION} at ${CUDNN_INCLUDE_DIR} ${CUDNN_LIBRARIES}")
     MARK_AS_ADVANCED(CUDNN_INCLUDE_DIR CUDNN_LIBRARIES)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/char-rnn/sample.py
----------------------------------------------------------------------
diff --git a/examples/char-rnn/sample.py b/examples/char-rnn/sample.py
index a8fcb73..8147732 100644
--- a/examples/char-rnn/sample.py
+++ b/examples/char-rnn/sample.py
@@ -21,7 +21,7 @@ import cPickle as pickle
 import numpy as np
 import argparse
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+#sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 from singa import layer
 from singa import tensor
 from singa import device

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/char-rnn/train.py
----------------------------------------------------------------------
diff --git a/examples/char-rnn/train.py b/examples/char-rnn/train.py
index 3dfa0d9..fb5e71f 100644
--- a/examples/char-rnn/train.py
+++ b/examples/char-rnn/train.py
@@ -25,7 +25,7 @@ import cPickle as pickle
 import numpy as np
 import argparse
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+# sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 from singa import layer
 from singa import loss
 from singa import device

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/cifar10/alexnet.py
----------------------------------------------------------------------
diff --git a/examples/cifar10/alexnet.py b/examples/cifar10/alexnet.py
index 9ed5599..ddad1d5 100644
--- a/examples/cifar10/alexnet.py
+++ b/examples/cifar10/alexnet.py
@@ -23,7 +23,7 @@ validation accuracy would be about 82%.
 import sys
 import os
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+# sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 from singa import layer
 from singa import initializer
 from singa import metric

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/cifar10/predict.py
----------------------------------------------------------------------
diff --git a/examples/cifar10/predict.py b/examples/cifar10/predict.py
index 07b1145..8a9ea4e 100644
--- a/examples/cifar10/predict.py
+++ b/examples/cifar10/predict.py
@@ -19,7 +19,7 @@ import numpy as np
 import sys
 import os
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+#sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 
 from singa import device
 from singa import tensor

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/cifar10/train.py
----------------------------------------------------------------------
diff --git a/examples/cifar10/train.py b/examples/cifar10/train.py
index 3285651..20ce5a7 100644
--- a/examples/cifar10/train.py
+++ b/examples/cifar10/train.py
@@ -25,7 +25,7 @@ import os
 import sys
 import argparse
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+# sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 from singa import utils
 from singa import optimizer
 from singa import device

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/examples/cifar10/vgg.py
----------------------------------------------------------------------
diff --git a/examples/cifar10/vgg.py b/examples/cifar10/vgg.py
index 97e690c..327592f 100644
--- a/examples/cifar10/vgg.py
+++ b/examples/cifar10/vgg.py
@@ -24,7 +24,7 @@ import sys
 import os
 import math
 
-sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
+#sys.path.append(os.path.join(os.path.dirname(__file__), '../../build/python'))
 
 from singa import layer
 from singa import initializer

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/io/csv_decoder.cc
----------------------------------------------------------------------
diff --git a/src/io/csv_decoder.cc b/src/io/csv_decoder.cc
index 0c11028..72b4e10 100644
--- a/src/io/csv_decoder.cc
+++ b/src/io/csv_decoder.cc
@@ -33,7 +33,7 @@ std::vector<Tensor> CSVDecoder::Decode(std::string value) {
     ss >> l;
   std::string str;
   float d[kMaxCSVBufSize];
-  size_t size = 0;
+  int size = 0;
   while (std::getline(ss, str, ',')) {
     float temp;
     if (std::stringstream(str) >> temp) {
@@ -42,7 +42,7 @@ std::vector<Tensor> CSVDecoder::Decode(std::string value) {
     }
   }
 
-  Tensor data(Shape {size}, kFloat32);
+  Tensor data(Shape {static_cast<size_t>(size)}, kFloat32);
   data.CopyDataFromHostPtr(d, size);
   output.push_back(data);
   if (has_label_ == true) {

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/model/layer/cudnn_dropout.cc
----------------------------------------------------------------------
diff --git a/src/model/layer/cudnn_dropout.cc b/src/model/layer/cudnn_dropout.cc
index c5b62cf..e05a425 100644
--- a/src/model/layer/cudnn_dropout.cc
+++ b/src/model/layer/cudnn_dropout.cc
@@ -17,10 +17,10 @@
  */
 #include "./cudnn_dropout.h"
 #ifdef USE_CUDNN
+#include <cudnn.h>
 // cudnn dropout is added in cudnn 5
-#if CUDNN_VERSION_MAJOR >= 5
+#if CUDNN_MAJOR >= 5
 
-#include <cudnn.h>
 #include <chrono>
 
 #include "./cudnn_utils.h"
@@ -112,5 +112,5 @@ void CudnnDropout::ToDevice(std::shared_ptr<Device> device) {
   state_.ToDevice(device);
 }
 }  // namespace singa
-#endif  // CUDNN_VERSION_MAJOR>=5
+#endif  // CUDNN_MAJOR>=5
 #endif  // USE_CUDNN

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/model/layer/cudnn_dropout.h
----------------------------------------------------------------------
diff --git a/src/model/layer/cudnn_dropout.h b/src/model/layer/cudnn_dropout.h
index 1241911..c6fc1c9 100644
--- a/src/model/layer/cudnn_dropout.h
+++ b/src/model/layer/cudnn_dropout.h
@@ -20,8 +20,9 @@
 #define SRC_MODEL_LAYER_CUDNN_DROPOUT_H_
 #include "singa/singa_config.h"
 #ifdef USE_CUDNN
+#include <cudnn.h>
 // cudnn dropout is added in cudnn 5
-#if CUDNN_VERSION_MAJOR >= 5
+#if CUDNN_MAJOR >= 5
 #include "./dropout.h"
 
 #include <cudnn.h>
@@ -56,6 +57,6 @@ class CudnnDropout : public Dropout {
   Tensor state_;
 };
 }  // namespace
-#endif  // CUDNN_VERSION_MAJOR>=5
+#endif  // CUDNN_MAJOR>=5
 #endif  // USE_CUDNN
 #endif  // SRC_MODEL_LAYER_CUDNN_DROPOUT_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/model/layer/cudnn_rnn.cc
----------------------------------------------------------------------
diff --git a/src/model/layer/cudnn_rnn.cc b/src/model/layer/cudnn_rnn.cc
index 9961df2..0788801 100644
--- a/src/model/layer/cudnn_rnn.cc
+++ b/src/model/layer/cudnn_rnn.cc
@@ -17,8 +17,8 @@
  */
 #include "./cudnn_rnn.h"
 #ifdef USE_CUDNN
-#if CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
 #include <cudnn.h>
+#if CUDNN_VERSION >= 5005
 #include <chrono>
 #include "./cudnn_utils.h"
 #include "singa/utils/logging.h"
@@ -423,5 +423,5 @@ const std::pair<vector<Tensor>, vector<Tensor>> CudnnRNN::Backward(
 }
 
 }  // namespace singa
-#endif  // CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#endif  // CUDNN_VERSION >= 5005
 #endif  // USE_CUDNN

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/model/layer/cudnn_rnn.h
----------------------------------------------------------------------
diff --git a/src/model/layer/cudnn_rnn.h b/src/model/layer/cudnn_rnn.h
index 82c68b0..5e642e0 100644
--- a/src/model/layer/cudnn_rnn.h
+++ b/src/model/layer/cudnn_rnn.h
@@ -20,7 +20,8 @@
 #define SRC_MODEL_LAYER_CUDNN_RNN_H_
 #include "singa/singa_config.h"
 #ifdef USE_CUDNN
-#if CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#include <cudnn.h>
+#if CUDNN_VERSION >= 5005
 #include <string>
 #include <utility>
 #include <vector>
@@ -82,6 +83,6 @@ class CudnnRNN : public RNN {
 
 }  // namespace singa
 
-#endif  // CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#endif  // CUDNN_VERSION >= 5005
 #endif  // USE_CUDNN
 #endif  // SRC_MODEL_LAYER_CUDNN_RNN_H_

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/python/singa/__init__.py
----------------------------------------------------------------------
diff --git a/src/python/singa/__init__.py b/src/python/singa/__init__.py
index ec26906..f14c8c5 100644
--- a/src/python/singa/__init__.py
+++ b/src/python/singa/__init__.py
@@ -19,14 +19,13 @@
 '''
 This script is the main entrance for user to run singa inside a model workspace
 
-To use this script, user sudo install these dependencies: flask pillow and protobuf 
+To use this script, user sudo install these dependencies: flask pillow and protobuf
 '''
 
 import sys, glob, os, random, shutil, time
 from flask import Flask, request, redirect, url_for
-from PIL import Image
 import numpy as np
-import ConfigParser 
+import ConfigParser
 import urllib, traceback
 
 
@@ -43,7 +42,7 @@ welcome to singa
 '''
 
 app = Flask(__name__)
-config = ConfigParser.RawConfigParser()    
+config = ConfigParser.RawConfigParser()
 service = {}
 data_path = "data_"
 parameter_path = "parameter_"
@@ -126,7 +125,7 @@ USAGE
         if parameter_file:
             print "load parameter file: %s" % parameter_file
             model.load(parameter_file)
-        
+
         if use_cpu:
             raise CLIError("Currently cpu is not support!")
         else:
@@ -141,12 +140,12 @@ USAGE
             from serve import Service
             service =Service(model,d)
 
-            app.debug = debug 
+            app.debug = debug
             app.run(host='0.0.0.0', port= port)
         elif mode == "train":
             print "runing singa in train mode"
-            global trainer 
-            from train import Trainer 
+            global trainer
+            from train import Trainer
             trainer= Trainer(model,d)
             if not parameter_file:
                 trainer.initialize()
@@ -178,18 +177,18 @@ def file_prepare(reload_data=False):
     shutil.rmtree("data_.py",ignore_errors=True)
     shutil.rmtree("data_",ignore_errors=True)
 
-    data_py=open("data_.py",'w') 
+    data_py=open("data_.py",'w')
     data_py.write("#%s" % "This file is Generated by SINGA, please don't edit\n\n")
     if config.has_section("data"):
         file_list = config.items("data")
         #download files
         for f in file_list:
             name,path=download_file(f[0],f[1],data_path)
-            data_py.write("%s=\"%s\"\n" % (name,path)) 
+            data_py.write("%s=\"%s\"\n" % (name,path))
 
     data_py.flush()
     data_py.close()
-    
+
     if config.has_section("parameter"):
         parameter_list = config.items("parameter")
         for p in parameter_list:
@@ -214,7 +213,7 @@ def get_parameter(file_name=None):
     '''
     if not os.path.exists(parameter_path):
         os.makedirs(parameter_path)
-        return 
+        return
 
     if file_name:
 	return os.path.join(parameter_path,file_name)
@@ -225,7 +224,7 @@ def get_parameter(file_name=None):
     parameter_list.sort()
 
     return parameter_list[-1]
-	
+
 @app.route("/")
 def index():
     return "Hello SINGA User!"

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/python/swig/config.i.in
----------------------------------------------------------------------
diff --git a/src/python/swig/config.i.in b/src/python/swig/config.i.in
index ed386b2..5743ba3 100644
--- a/src/python/swig/config.i.in
+++ b/src/python/swig/config.i.in
@@ -1,5 +1,4 @@
 // Pass in cmake configurations to swig
-#cmakedefine01 USE_CUDA 
-#cmakedefine01 USE_CUDNN 
-#cmakedefine CUDNN_VERSION_MAJOR ${CUDNN_VERSION_MAJOR}
-#cmakedefine CUDNN_VERSION_PATCH ${CUDNN_VERSION_PATCH} 
+#cmakedefine01 USE_CUDA
+#cmakedefine01 USE_CUDNN
+#cmakedefine CUDNN_VERSION_SWIG ${CUDNN_VERSION_SWIG}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/src/python/swig/model_layer.i
----------------------------------------------------------------------
diff --git a/src/python/swig/model_layer.i b/src/python/swig/model_layer.i
index f82b0f1..ae651d5 100644
--- a/src/python/swig/model_layer.i
+++ b/src/python/swig/model_layer.i
@@ -84,7 +84,7 @@ class RNN : public Layer {
 };
 
 #if USE_CUDA && USE_CUDNN
-#if CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#if CUDNN_VERSION_SWIG >= 5005
 class CudnnRNN : public RNN {
  public:
  // note: Must use std::vector instead of vector.
@@ -96,7 +96,7 @@ class CudnnRNN : public RNN {
     const std::vector<size_t> GetOutputSampleShape() const override;
 };
 
-#endif  // CUDNN_VERSION_MINOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#endif  // CUDNN_VERSION_SWIG >= 5005
 #endif  // USE_CUDA && USE_CUDNN
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/test/python/example_test_device.py
----------------------------------------------------------------------
diff --git a/test/python/example_test_device.py b/test/python/example_test_device.py
deleted file mode 100644
index c545ad1..0000000
--- a/test/python/example_test_device.py
+++ /dev/null
@@ -1,36 +0,0 @@
-import sys, os
-
-sys.path.append(os.path.join(os.path.dirname(__file__),
-                             '../../src/python'))
-from device import *
-from tensor import *
-
-sys.path.append(os.path.join(os.path.dirname(__file__),
-                             '../../build/src'))
-from core_pb2 import *
-
-#---------------------------------------------------------
-# example usage
-#---------------------------------------------------------
-
-d1 = CudaGPU(123)
-print d1.singa_device
-print d1.get_host()
-print d1.get_id()
-print
-
-d2 = CppCPU(345)
-print d2.singa_device
-print d2.get_host()
-print d2.get_id()
-print
-
-s = (2, 3)
-t = Tensor(s, d2.get_host())
-print t.singa_tensor
-print t.device
-print
-
-d = Device(0)
-print d.singa_device
-print

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/test/python/example_test_tensor.py
----------------------------------------------------------------------
diff --git a/test/python/example_test_tensor.py b/test/python/example_test_tensor.py
deleted file mode 100644
index c5ce5f1..0000000
--- a/test/python/example_test_tensor.py
+++ /dev/null
@@ -1,179 +0,0 @@
-import sys, os
-import numpy as np
-
-sys.path.append(os.path.join(os.path.dirname(__file__),
-                             '../../build/python'))
-sys.path.append(os.path.join(os.path.dirname(__file__),
-                             '../../build/lib'))
-sys.path.append(os.path.join(os.path.dirname(__file__),'../../build/src'))
-from core_pb2 import *
-from singa.tensor import *
-
-
-#---------------------------------------------------------
-# example usage
-#---------------------------------------------------------
-
-print '----------------------------'
-print 'global SizeOf kFloat32:', sizeof(kFloat32)
-print 'global SizeOf kFloat16:', sizeof(kFloat16)
-print 'global SizeOf kInt:', sizeof(kInt)
-print 'global SizeOf kDouble:', sizeof(kDouble)
-print
-
-a = Tensor()
-print 'a = Tensor()'
-print 'only defaultdevice is assigned \n'
-
-shape = (1, 6)
-t = Tensor(shape)
-print 'shape = (1, 6):', t.shape()
-print 'shape(0), shape(1):', t.shape(0), t.shape(1)
-print 'global Product:', product(shape)
-print 't = Tensor(shape)'
-#t.singa_tensor.AsType(kInt)
-print 'data_type():', t.data_type()
-print 'transpose', t.is_transpose()
-print 'nDim:', t.ndim()
-print 'size:', t.size()
-print 'memsize:', t.memsize()
-print 'data():', t.to_numpy()
-print
-
-print '----------------------------'
-print 't.data\n', t.to_numpy()
-q = t.copy()
-w = t.deepcopy()
-print 'q.data\n', q.to_numpy()
-print 'w.data\n', w.to_numpy()
-t += 1.23
-print 'q.data\n', q.to_numpy()
-print 'w.data\n', w.to_numpy()
-
-print '----------------------------'
-shape = (2, 3)
-t.reshape(shape)
-print 'shape = (3, 2)'
-print 'after reshape, t.shape():', t.shape()
-print 't.data(): \n', t.to_numpy()
-shape = (3, 2)
-t0 = reshape(t, shape)
-print 'shape = (2, 3)'
-print 'after t0 = reshape(t, shape) \n'
-print 't.shape():', t.shape()
-print 't0.shape():', t0.shape()
-print
-
-print '----------------------------'
-t += 1.2345
-print 't += 1.234, i.e., t.__iadd__(1.2345): \n', t.to_numpy()
-print
-
-t1 = t
-print 'copy\nt1 = t'
-print 't1.shape():', t1.shape()
-print 't1.data(): \n', t1.to_numpy()
-print
-
-r = t1.transpose()
-print 'r = t1.transpose()\nr.data() \n', r.to_numpy()
-
-
-print '----------------------------'
-t2 = log(t1)
-print 't2 = log(t1): \n', t2.to_numpy()
-print
-
-t1 += t2
-print 't1 += t2, i.e., t1.__iadd__(t2): \n', t1.to_numpy()
-print
-
-t1 *= 2
-print 't1 *= 2, i.e., t1.__imul__(2): \n', t1.to_numpy()
-print
-
-print '----------------------------'
-tc = t2.clone()
-print 'clone\ntc = t2.clone()\ntc.data(): \n', tc.to_numpy()
-print
-
-print 'sum(tc) \n', sum(tc)
-print
-t3 = sum(tc,0)
-print 'sum(tc,0) \n', t3.to_numpy()
-t3 = sum(tc,1)
-print 'sum(tc,1) \n', t3.to_numpy()
-print
-
-t3 = average(tc,0)
-print 'average(tc,0) \n', t3.to_numpy()
-t3 = average(tc,1)
-print 'average(tc,1) \n', t3.to_numpy()
-print
-
-t3 = softmax(tc,0)
-print 'softmax(tc,0)\n', t3.to_numpy()
-t3 = softmax(tc,1)
-print 'softmax(tc,1)\n', t3.to_numpy()
-
-print '----------------------------'
-print 't1 \n', t1.to_numpy()
-print
-
-n = t1 + t2
-print 't1 + t2: \n', n.to_numpy()
-print
-
-n = t1 * t2
-print 't1*t2: \n', n.to_numpy()
-print
-
-n = t1 - 1.2
-print 't1 - 1.2 \n', n.to_numpy()
-print
-
-n = add(t1, t1)
-print 'add(t1, t1) \n', n.to_numpy()
-print
-
-n = add(t1, 3.4)
-print 'add(t1, 3.4) \n', n.to_numpy()
-print
-
-n = div(t1, 2.0)
-print 'div(t1, 2.0) \n', n.to_numpy()
-print
-
-print '----------------------------'
-shape = (2, 2)
-t4 = Tensor(shape)
-t4 += 3.45
-print 't4 += 3.45 \n', t4.to_numpy()
-print
-
-n = t4 < 3.45
-print 't4 < 3.45 \n', n.to_numpy()
-print
-
-n = lt(t4, 3.45)
-print 'lt(t4, 3.45) \n', n.to_numpy()
-print
-
-n = ge(t4, 3.45)
-print 'ge(t4, 3.45) \n', n.to_numpy()
-print
-
-print '----------------------------'
-print t1.to_numpy()
-print tc.to_numpy()
-print
-copy_data_to_from(t1, tc, 2)
-print t1.to_numpy()
-print tc.to_numpy()
-
-#ttt = t1.singa_tensor < 5.2
-#ttt = lessthan(t1, 5.2)
-#print ttt.data()
-
-#devCPU = singa.CppCPU(1)
-#devGPU = singa.CudaGPU(2)

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/test/python/test_tensor.py
----------------------------------------------------------------------
diff --git a/test/python/test_tensor.py b/test/python/test_tensor.py
index 4d8b940..2374adc 100644
--- a/test/python/test_tensor.py
+++ b/test/python/test_tensor.py
@@ -34,6 +34,8 @@ class TestTensorMethods(unittest.TestCase):
         self.shape = (2, 3)
         self.t = tensor.Tensor(self.shape)
         self.s = tensor.Tensor(self.shape)
+        self.t.set_value(0)
+        self.s.set_value(0)
 
     def test_tensor_fields(self):
         t = self.t

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/test/singa/test_cudnn_dropout.cc
----------------------------------------------------------------------
diff --git a/test/singa/test_cudnn_dropout.cc b/test/singa/test_cudnn_dropout.cc
index 4a89235..f1b8437 100644
--- a/test/singa/test_cudnn_dropout.cc
+++ b/test/singa/test_cudnn_dropout.cc
@@ -21,7 +21,7 @@
 #include "../src/model/layer/cudnn_dropout.h"
 #ifdef USE_CUDNN
 // cudnn dropout is added in cudnn 5
-#if CUDNN_VERSION_MAJOR >= 5
+#if CUDNN_MAJOR >= 5
 
 #include "gtest/gtest.h"
 
@@ -122,5 +122,5 @@ TEST(CudnnDropout, Backward) {
   EXPECT_FLOAT_EQ(dx[1], dy[1] * GetBitValue(mptr, 1) * scale);
   EXPECT_FLOAT_EQ(dx[7], dy[7] * GetBitValue(mptr, 7) * scale);
 }
-#endif  // CUDNN_VERSION_MAJOR>=5
+#endif  // CUDNN_MAJOR>=5
 #endif  // USE_CUDNN

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/6b2ff3c9/test/singa/test_cudnn_rnn.cc
----------------------------------------------------------------------
diff --git a/test/singa/test_cudnn_rnn.cc b/test/singa/test_cudnn_rnn.cc
index e293cf7..07336a2 100644
--- a/test/singa/test_cudnn_rnn.cc
+++ b/test/singa/test_cudnn_rnn.cc
@@ -21,7 +21,7 @@
 
 #include "../src/model/layer/cudnn_rnn.h"
 #ifdef USE_CUDNN
-#if CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#if CUDNN_VERSION >= 5005
 
 #include "gtest/gtest.h"
 
@@ -177,5 +177,5 @@ TEST_F(TestCudnnRNN, Backward) {
     std::copy(tmp.begin(), tmp.end(), dhyptr.begin());
   }
 }
-#endif  // CUDNN_VERSION_MAJOR >= 5 && CUDNN_VERSION_PATCH >= 5
+#endif  // CUDNN_VERSION >= 5005
 #endif  // USE_CUDNN