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 2015/11/26 12:32:01 UTC

[4/6] incubator-singa git commit: SINGA-104 Add Context Class

SINGA-104 Add Context Class

Update Context class:
1. SetupDevice can setup the GPU device (device id>=0) random seed and
CPU thread random seed; random generator and handlers are created when
accessed. only GPU device has handler and curand_generator.
2. pass test_context.cc


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

Branch: refs/heads/master
Commit: 3841bc5484fc47eae3f5877d2264c5188675818c
Parents: 9aff30a
Author: Wei Wang <wa...@comp.nus.edu.sg>
Authored: Thu Nov 26 16:06:08 2015 +0800
Committer: Wei Wang <wa...@comp.nus.edu.sg>
Committed: Thu Nov 26 16:06:08 2015 +0800

----------------------------------------------------------------------
 include/singa/utils/context.h | 136 +++++++++++++++++++++++++------------
 src/test/test_context.cc      |  55 +++++++++++++++
 src/test/test_context.cu      |  55 ---------------
 src/test/test_math.cc         |   2 +
 src/utils/context.cc          |  87 ------------------------
 5 files changed, 151 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/3841bc54/include/singa/utils/context.h
----------------------------------------------------------------------
diff --git a/include/singa/utils/context.h b/include/singa/utils/context.h
index 7a41dac..5223012 100644
--- a/include/singa/utils/context.h
+++ b/include/singa/utils/context.h
@@ -37,14 +37,12 @@
 
 namespace singa {
 
-// max num of threads per process
-const int kNumMaxThreads = 1024;
 
 /**
  * Context is used as a global singleton, which stores the mapping from CPU
  * thread id to GPU device id. It manages the handlers for GPU
  * devices. It also manages the GPU and CPU random generators, which are created
- * when accessed. One CPU thread has a CPU random generator. A CPU device
+ * when accessed. One CPU thread has a CPU random generator. A GPU device
  * has a GPU random generator.
  */
 class Context {
@@ -52,20 +50,49 @@ class Context {
    /**
     * Destructor, release random generators and handlers.
     */
-	~Context();
+   ~Context() {
+#ifdef USE_GPU
+     for (auto& entry : device_id_) {
+       if (entry.second != -1) {
+         cudaSetDevice(entry.second);
+         if (cublas_handle_[entry.second] != nullptr) {
+           cublasDestroy(cublas_handle_[entry.second]);
+           cublas_handle_[entry.second] = nullptr;
+         }
+         if(curand_generator_[entry.second] != nullptr) {
+           curandDestroyGenerator(curand_generator_[entry.second]);
+           curand_generator_[entry.second] = nullptr;
+         }
+       }
+     }
+#endif
+     for (auto& entry : rand_generator_) {
+       if (entry.second != nullptr) {
+         delete entry.second;
+         entry.second = nullptr;
+       }
+     }
+
+   }
   /**
-   * Constructor, init arrays for random generators and handlers.
+   * Constructor.
    */
-  Context();
+  Context() {
+    for (int i = 0; i < kMaxNumGPU; i++) {
+      cublas_handle_.push_back(nullptr);
+      curand_generator_.push_back(nullptr);
+    }
+  }
 
   /**
-   * @return the ID of the device attached to a given CPU thread:
-   * if the device is a GPU card, then returns the GPU device ID;
-   * Else return -1.
+   * @return the ID of the device attached to a given CPU thread, or -1 if this
+   * thread has not attached GPU device.
    */
-	int device_id(const std::thread::id tid) {
-    CHECK(device_id_.find(tid) != device_id_.end());
-	  return device_id_[tid];
+	int device_id(const std::thread::id& tid) {
+    if (device_id_.find(tid) != device_id_.end())
+      return device_id_[tid];
+    else
+      return -1;
 	}
 
   /**
@@ -75,13 +102,24 @@ class Context {
    * @param[in] thread::id CPU thread ID
    * @param[in] device_id GPU device ID
    */
-	void SetupDevice(const std::thread::id tid, const int did);
-
+  void SetupDevice(const std::thread::id& tid, const int did) {
+    SetupDevice(tid, did, -1);
+  }
   /**
    * @copy SetupDevice(const int, const int);
    * @param[in] seed random seed
    */
-  void SetupDevice(const std::thread::id tid, const int did, long long seed);
+  void SetupDevice(const std::thread::id& tid, const int did, long long seed) {
+    device_id_[tid] = did;
+    seed_[tid] = seed;
+  }
+
+  void ActivateDevice(const int device_id) {
+    CHECK_GE(device_id, 0);
+#ifdef USE_GPU
+    cudaSetDevice(device_id);
+#endif
+  }
 
   /**
    * Get the CPU random generator.
@@ -90,7 +128,7 @@ class Context {
    * @param[in] thread::id CPU thread ID
    * @return the CPU random generator
    */
-  std::mt19937* rand_generator(const std::thread::id tid) {
+  std::mt19937* rand_generator(const std::thread::id& tid) {
     if (rand_generator_.find(tid) == rand_generator_.end()) {
       CHECK(seed_.find(tid) != seed_.end());
       auto seed = static_cast<unsigned>(seed_[tid]);
@@ -101,50 +139,64 @@ class Context {
     return rand_generator_[tid];
   }
 #ifdef USE_GPU
+  cublasHandle_t cublas_handle(const std::thread::id thread_id) {
+    return cublas_handle(device_id(thread_id));
+  }
   /**
-   * Get the handler of the GPU device attached to a CPU thread.
-   * @param[in] thread::id
-   * @return the GPU handler, or nullptr if this thread does not have any GPU.
+   * Get the handler of the GPU device given its device ID. The device
+   * must be set up via SetupDevice(const std::thread::id, const int) before
+   * calling this function.
+   * @param[in] device_id GPU device ID
+   * @return the GPU handler
    */
-	cublasHandle_t cublas_handle(const std::thread::id tid) {
-    CHECK(cublas_handle_.find(tid) != cublas_handle_.end());
-	  return cublas_handle_[tid];
-	}
+  cublasHandle_t cublas_handle(const int device_id) {
+    CHECK_GE(device_id, 0);
+    if (cublas_handle_.at(device_id) == nullptr) {
+      cudaSetDevice(device_id);
+      cublasCreate(&cublas_handle_[device_id]);
+    }
+    return cublas_handle_[device_id];
+  }
+	curandGenerator_t curand_generator(const std::thread::id thread_id) {
+    return curand_generator(device_id(thread_id));
+  }
   /**
-   * Get the random generator of the GPU device assigned to the given thread.
-   * @param[in] thread::id
+   * Get the random generator of the GPU device given the device id.
+   * @param[in] device_id GPU device ID
    * @return random generator. If it does not exist, then create one.
    * The random seed will be set to CURAND_RNG_PSEUDO_DEFAULT if it is not set.
    */
-	curandGenerator_t curand_generator(const std::thread::id tid) {
-    if (curand_generator_.find(tid) == curand_generator_.end()) {
+	curandGenerator_t curand_generator(const int device_id) {
+    CHECK_GE(device_id, 0);
+    if (curand_generator_.at(device_id) == nullptr) {
+      // TODO handle user set seed
+      /*
       CHECK(seed_.find(tid) != seed_.end());
       auto seed = seed_[tid];
-      // TODO handle user set seed
-      cudaSetDevice(device_id_[tid]);
-      curandCreateGenerator(&curand_generator_[tid], CURAND_RNG_PSEUDO_DEFAULT);
+      */
+      ActivateDevice(device_id);
+      curandCreateGenerator(&curand_generator_[device_id],
+          CURAND_RNG_PSEUDO_DEFAULT);
     }
-	  return curand_generator_[tid];
+	  return curand_generator_[device_id];
 	}
 
-  /*
- protected:
-	void CreateHandle(const int thread::id);
-	void DestoryHandle(const int thread::id);
-	void CreateGpuRandGenerator(const int thread::id);
-	void DestoryGpuRandGenerator(const int thread::id);
-  */
-
 #endif
 
  protected:
-
+  //!< max num of GPUs per process
+  const int kMaxNumGPU = 64;
+  //!< map from thread id to device id
 	std::unordered_map<std::thread::id, int> device_id_;
+  //!< map from thread id to cpu rand generator
   std::unordered_map<std::thread::id, std::mt19937 *> rand_generator_;
+  //!< map from thread id to cpu rand generator seed
   std::unordered_map<std::thread::id, int> seed_;
 #ifdef USE_GPU
-	std::unordered_map<std::thread::id, cublasHandle_t> cublas_handle_;
-	std::unordered_map<std::thread::id, curandGenerator_t> curand_generator_;
+  //!< cublas handler indexed by GPU device ID
+	std::vector<cublasHandle_t> cublas_handle_;
+  //!< cublas rand generator indexed by GPU device ID
+	std::vector<curandGenerator_t> curand_generator_;
 #endif
 };
 

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/3841bc54/src/test/test_context.cc
----------------------------------------------------------------------
diff --git a/src/test/test_context.cc b/src/test/test_context.cc
new file mode 100644
index 0000000..5e501b9
--- /dev/null
+++ b/src/test/test_context.cc
@@ -0,0 +1,55 @@
+#include <thread>
+#include "gtest/gtest.h"
+#include "singa/utils/singleton.h"
+#include "singa/utils/context.h"
+#include "singa/utils/cuda_utils.h"
+
+using namespace singa;
+using namespace std;
+
+TEST(ContextTest, TestDevice) {
+  auto context = Singleton<Context>::Instance();
+
+  auto id = std::this_thread::get_id();
+  context->SetupDevice(id, 0);
+  auto device_id = context->device_id(id);
+  ASSERT_EQ(0, device_id);
+}
+
+TEST(ContextTest, TestHandle) {
+  auto context = Singleton<Context>::Instance();
+
+  float cpu_ret = 0.0f;
+  float gpu_ret = 0.0f;
+
+  float A[12];
+  float B[12];
+
+  for(int i = 0; i < 12; i++) {
+    A[i]=i-1;
+    B[i]=i+1;
+  }
+
+  float* A_gpu = NULL;
+  float* B_gpu = NULL;
+  context->SetupDevice(std::this_thread::get_id(), 0);
+
+  cudaMalloc((void**)&A_gpu, 12 * sizeof(float));
+  cudaMalloc((void**)&B_gpu, 12 * sizeof(float));
+
+  cudaMemcpy(A_gpu, A, 12 * sizeof(float), cudaMemcpyHostToDevice);
+  cudaMemcpy(B_gpu, B, 12 * sizeof(float), cudaMemcpyHostToDevice);
+
+  cublasHandle_t handle = context->cublas_handle(std::this_thread::get_id());
+
+  cublasSdot(handle, 12, A_gpu, 1, B_gpu, 1, &gpu_ret);
+
+  for(int i = 0; i < 12;++i) {
+    cpu_ret += A[i] * B[i];
+  }
+
+  ASSERT_EQ(gpu_ret,cpu_ret);
+
+  cudaFree(A_gpu);
+  cudaFree(B_gpu);
+}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/3841bc54/src/test/test_context.cu
----------------------------------------------------------------------
diff --git a/src/test/test_context.cu b/src/test/test_context.cu
deleted file mode 100644
index 88ab06b..0000000
--- a/src/test/test_context.cu
+++ /dev/null
@@ -1,55 +0,0 @@
-#include <thread>
-#include "gtest/gtest.h"
-#include "singa/utils/singleton.h"
-#include "singa/utils/context.h"
-#include "singa/utils/cuda_utils.h"
-
-using namespace singa;
-using namespace std;
-
-TEST(ContextTest, TestDevice) {
-  auto context = Singleton<Context>::Instance();
-
-  auto id = std::this_thread::get_id();
-  context->SetupDevice(id, 0);
-  auto device_id = context->device_id(id);
-  ASSERT_EQ(1,device_id);
-}
-
-TEST(ContextTest, TestHandle) {
-  auto context = Singleton<Context>::Instance();
-
-  float cpu_ret = 0.0f;
-  float gpu_ret = 0.0f;
-
-  float A[12];
-  float B[12];
-
-  for(int i = 0; i < 12; i++) {
-    A[i]=i-1;
-    B[i]=i+1;
-  }
-
-  float* A_gpu = NULL;
-  float* B_gpu = NULL;
-  context->SetupDevice(std::this_thread::get_id(), 0);
-
-  cudaMalloc((void**)&A_gpu, 12 * sizeof(float));
-  cudaMalloc((void**)&B_gpu, 12 * sizeof(float));
-
-  cudaMemcpy(A_gpu, A, 12 * sizeof(float), cudaMemcpyHostToDevice);
-  cudaMemcpy(B_gpu, B, 12 * sizeof(float), cudaMemcpyHostToDevice);
-
-  cublasHandle_t handle = context->cublas_handle(std::this_thread::get_id());
-
-  cublasSdot(handle, 12, A_gpu, 1, B_gpu, 1, &gpu_ret);
-
-  for(int i = 0; i < 12;++i) {
-    cpu_ret += A[i] * B[i];
-  }
-
-  ASSERT_EQ(gpu_ret,cpu_ret);
-
-  cudaFree(A_gpu);
-  cudaFree(B_gpu);
-}

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/3841bc54/src/test/test_math.cc
----------------------------------------------------------------------
diff --git a/src/test/test_math.cc b/src/test/test_math.cc
index a5bf180..8f8c633 100644
--- a/src/test/test_math.cc
+++ b/src/test/test_math.cc
@@ -236,6 +236,7 @@ TEST(MathTest, TestGemvGPU) {
 }
 
 
+/*
 TEST(MathTest, TestAxpyGPU) {
 	float A[4][3] = {};
 	float C[4][3] = {};
@@ -281,6 +282,7 @@ TEST(MathTest, TestAxpyGPU) {
 	cudaFree(A_gpu);
 	cudaFree(B_gpu);
 }
+*/
 
 
 TEST(MathTest, TestDotGPU) {

http://git-wip-us.apache.org/repos/asf/incubator-singa/blob/3841bc54/src/utils/context.cc
----------------------------------------------------------------------
diff --git a/src/utils/context.cc b/src/utils/context.cc
deleted file mode 100644
index 37c8f39..0000000
--- a/src/utils/context.cc
+++ /dev/null
@@ -1,87 +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.
-*
-*************************************************************/
-#include "singa/utils/context.h"
-#include "singa/utils/factory.h"
-#include "singa/utils/singleton.h"
-
-namespace singa {
-
-Context::~Context() {
-#ifdef USE_GPU
-  for (auto& entry : device_id_) {
-    if (entry.second != -1) {
-      cudaSetDevice(entry.second);
-      if (cublas_handle_[entry.first] != nullptr) {
-        cublasDestroy(cublas_handle_[entry.first]);
-        cublas_handle_[entry.first] = nullptr;
-      }
-      if(curand_generator_[entry.first] != nullptr) {
-        curandDestroyGenerator(curand_generator_[entry.first]);
-        curand_generator_[entry.first] = nullptr;
-      }
-    }
-  }
-#endif
-  for (auto& entry : rand_generator_) {
-    if (entry.second != nullptr) {
-      delete entry.second;
-      entry.second = nullptr;
-    }
-  }
-}
-
-Context::Context() { }
-
-void Context::SetupDevice(const std::thread::id thread, const int did) {
-  SetupDevice(thread, did, -1);
-}
-
-void Context::SetupDevice(const std::thread::id thread, const int did,
-    long long seed) {
-  device_id_[thread] = did;
-#ifdef USE_GPU
-  if (did > -1) {
-    cudaSetDevice(did);
-    cublasCreate(&handle_[thread]);
-  }
-#endif
-  seed_[thread] = seed;
-}
-
-/*
-#ifdef USE_GPU
-void Context::DestoryHandle(const int thread::id) {
-  cudaSetDevice(device_id_[thread::id]);
-  cublasDestroy(handle_[thread::id]);
-  handle_[thread::id] = nullptr;
-}
-
-void Context::DestoryGpuRandGenerator(const int thread::id) {
-  cudaSetDevice(device_id_[thread::id]);
-  curandDestroyGenerator(curand_generator_[thread::id]);
-  curand_generator_[thread::id] = nullptr;
-}
-#endif
-*/
-
-
-}  // namespace singa
-