Repository: incubator-systemml
Updated Branches:
  refs/heads/master 9137f7b02 -> 80225f014


[SYSTEMML-1396] cudaFree is called lazily instead of asynchronously

- If a block is available in a free list of the exact matching size, it
  is memset to 0 and allocated, otherwise, the LRU blocks are cudaFree-ed
  until enough memory is available on the GPU
- Fixed timers
- bug fix for SYSTEMML-1340,1341 related to redundantly releasing an
  input block on the GPU

Closes #426


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

Branch: refs/heads/master
Commit: 80225f014338d9671aaf187186f117feb5d0c093
Parents: 9137f7b
Author: Nakul Jindal <naku...@gmail.com>
Authored: Mon Mar 13 13:55:39 2017 -0700
Committer: Nakul Jindal <naku...@gmail.com>
Committed: Mon Mar 13 13:55:39 2017 -0700

----------------------------------------------------------------------
 .../instructions/gpu/GPUInstruction.java        |   2 +
 .../instructions/gpu/context/GPUContext.java    |  19 +-
 .../instructions/gpu/context/GPUObject.java     |  76 ++--
 .../instructions/gpu/context/JCudaContext.java  |  52 ++-
 .../instructions/gpu/context/JCudaObject.java   | 353 +++++++++++--------
 .../runtime/matrix/data/LibMatrixCUDA.java      | 186 +++++-----
 .../org/apache/sysml/utils/GPUStatistics.java   |  10 +-
 .../org/apache/sysml/utils/LRUCacheMap.java     |  71 ++++
 .../sysml/test/utils/LRUCacheMapTest.java       | 102 ++++++
 9 files changed, 537 insertions(+), 334 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
index dcb2edc..04a2f1a 100644
--- 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
+++ 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/GPUInstruction.java
@@ -44,6 +44,8 @@ public abstract class GPUInstruction extends Instruction
        public final static String MISC_TIMER_CUDA_FREE =                       
                        "f";            // time spent in calling cudaFree
        public final static String MISC_TIMER_ALLOCATE =                        
                                "a";            // time spent to allocate 
memory on gpu
        public final static String MISC_TIMER_ALLOCATE_DENSE_OUTPUT = "ao";     
        // time spent to allocate dense output (recorded differently than 
MISC_TIMER_ALLOCATE)
+       public final static String MISC_TIMER_SET_ZERO =                        
                                "az";           // time spent to allocate
+       public final static String MISC_TIMER_REUSE =                           
                                "r";            // time spent in reusing 
already allocated memory on GPU (mainly for the count)
 
        // Matmult instructions
        public final static String MISC_TIMER_SPARSE_ALLOCATE_LIB =             
                                "Msao";         // time spend in allocating for 
sparse matrix output

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
index b792882..f076350 100644
--- 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
+++ 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUContext.java
@@ -18,11 +18,6 @@
  */
 package org.apache.sysml.runtime.instructions.gpu.context;
 
-import java.util.ArrayList;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -33,18 +28,6 @@ import 
org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 @SuppressWarnings("rawtypes")
 public abstract class GPUContext {
 
-       public static ArrayList<GPUObject> allocatedPointers = new 
ArrayList<GPUObject>();
-
-       /** cudaFree calls are done asynchronously on a separate thread,
-        *  this list preserve the list of currently happening cudaFree calls */
-       public static ConcurrentLinkedQueue<Future> pendingDeallocates = new 
ConcurrentLinkedQueue<Future>();
-
-       /** All asynchronous cudaFree calls will be done on this executor 
service */
-       public static ExecutorService deallocExecutorService;
-
-       /** Synchronization object to make sure no allocations happen when 
something is being evicted from memory */
-       public static final Object syncObj = new Object();
-
        protected static GPUContext currContext;
        public static volatile Boolean isGPUContextCreated = false;
 
@@ -76,7 +59,7 @@ public abstract class GPUContext {
                        synchronized(isGPUContextCreated) {
                                currContext = new JCudaContext();
                                currContext.ensureComputeCapability();
-                               OptimizerUtils.GPU_MEMORY_BUDGET = 
((JCudaContext)currContext).getAvailableMemory();
+                               OptimizerUtils.GPU_MEMORY_BUDGET = 
currContext.getAvailableMemory();
                                isGPUContextCreated = true;
                        }
                }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
index 366713f..e3c2df4 100644
--- 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
+++ 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUObject.java
@@ -18,16 +18,18 @@
  */
 package org.apache.sysml.runtime.instructions.gpu.context;
 
+import jcuda.Pointer;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.CacheException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.utils.GPUStatistics;
+import org.apache.sysml.utils.LRUCacheMap;
 
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
+import java.util.LinkedList;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -95,7 +97,7 @@ public abstract class GPUObject
 
        abstract void allocateDenseMatrixOnDevice() throws DMLRuntimeException;
        abstract void allocateSparseMatrixOnDevice() throws DMLRuntimeException;
-       abstract void deallocateMemoryOnDevice(boolean synchronous) throws 
DMLRuntimeException;
+       abstract void deallocateMemoryOnDevice(boolean eager) throws 
DMLRuntimeException;
        abstract long getSizeOnDevice() throws DMLRuntimeException;
        
        abstract void copyFromHostToDevice() throws DMLRuntimeException;
@@ -109,47 +111,55 @@ public abstract class GPUObject
         * @throws DMLRuntimeException if DMLRuntimeException occurs
         */
        abstract void copyFromDeviceToHost() throws DMLRuntimeException; // 
Called by export()
-       
+
+
+       /**
+        * Convenience wrapper over {@link GPUObject#evict(String, long)}
+        * @param GPUSize Desired size to be freed up on the GPU
+        * @throws DMLRuntimeException If no blocks to free up or if not enough 
blocks with zero locks on them.
+        */
+       protected static void evict(final long GPUSize) throws 
DMLRuntimeException {
+               evict(null, GPUSize);
+       }
+
        /**
         * Cycles through the sorted list of allocated {@link GPUObject} 
instances. Sorting is based on
         * number of (read) locks that have been obtained on it (reverse 
order). It repeatedly frees up 
         * blocks on which there are zero locks until the required size has 
been freed up.
         * // TODO: update it with hybrid policy
+        * @param instructionName name of the instruction for which performance 
measurements are made
         * @param GPUSize Desired size to be freed up on the GPU
         * @throws DMLRuntimeException If no blocks to free up or if not enough 
blocks with zero locks on them.  
         */
        @SuppressWarnings("rawtypes")
-       protected static void evict(final long GPUSize) throws 
DMLRuntimeException {
-               synchronized (GPUContext.syncObj) {
-                       // Check for the completion of asynchronous cudaFree 
calls
-                       try {
-                               while (GPUSize > getAvailableMemory()) {
-                                       Future f = 
GPUContext.pendingDeallocates.poll();
-                                       if (f == null) {
-                                               break;
-                                       } else if (f.isDone()) {
-                                               continue;
-                                       } else {
-                                               f.get();
-                                       }
-                               }
-                       } catch (InterruptedException e) {
-                               throw new DMLRuntimeException("There was an 
error with pending deallocates", e);
-                       } catch (ExecutionException e) {
-                               throw new DMLRuntimeException("There was an 
error with pending deallocates", e);
+       protected static void evict(String instructionName, final long GPUSize) 
throws DMLRuntimeException {
+               synchronized (JCudaContext.syncObj) {
+
+                       GPUStatistics.cudaEvictionCount.addAndGet(1);
+                       // Release the set of free blocks maintained in a 
JCudaObject.freeCUDASpaceMap
+                       // to free up space
+                       LRUCacheMap<Long, LinkedList<Pointer>> lruCacheMap = 
JCudaObject.freeCUDASpaceMap;
+                       while (lruCacheMap.size() > 0) {
+                               if (GPUSize <= getAvailableMemory())
+                                       break;
+                               Map.Entry<Long, LinkedList<Pointer>> 
toFreeListPair = lruCacheMap.removeAndGetLRUEntry();
+                               LinkedList<Pointer> toFreeList = 
toFreeListPair.getValue();
+                               Long size = toFreeListPair.getKey();
+                               Pointer toFree = toFreeList.pop();
+                               if (toFreeList.isEmpty())
+                                       lruCacheMap.remove(size);
+                               JCudaObject.cudaFreeHelper(instructionName, 
toFree, true);
                        }
 
                        if (GPUSize <= getAvailableMemory())
                                return;
 
-                       if (GPUContext.allocatedPointers.size() == 0) {
+                       if (JCudaContext.allocatedPointers.size() == 0) {
                                throw new DMLRuntimeException("There is not 
enough memory on device for this matrix!");
                        }
 
-                       GPUStatistics.cudaEvictionCount.addAndGet(1);
-
                        synchronized (evictionLock) {
-                               Collections.sort(GPUContext.allocatedPointers, 
new Comparator<GPUObject>() {
+                               
Collections.sort(JCudaContext.allocatedPointers, new Comparator<GPUObject>() {
 
                                        @Override
                                        public int compare(GPUObject p1, 
GPUObject p2) {
@@ -189,8 +199,8 @@ public abstract class GPUObject
                                        }
                                });
 
-                               while (GPUSize > getAvailableMemory() && 
GPUContext.allocatedPointers.size() > 0) {
-                                       GPUObject toBeRemoved = 
GPUContext.allocatedPointers.get(GPUContext.allocatedPointers.size() - 1);
+                               while (GPUSize > getAvailableMemory() && 
JCudaContext.allocatedPointers.size() > 0) {
+                                       GPUObject toBeRemoved = 
JCudaContext.allocatedPointers.get(JCudaContext.allocatedPointers.size() - 1);
                                        if (toBeRemoved.numLocks.get() > 0) {
                                                throw new 
DMLRuntimeException("There is not enough memory on device for this matrix!");
                                        }
@@ -205,7 +215,7 @@ public abstract class GPUObject
        }
 
        /**
-        * Asynchronously clears the data associated with this {@link 
GPUObject} instance
+        * lazily clears the data associated with this {@link GPUObject} 
instance
         * @throws CacheException ?
         */
        public void clearData() throws CacheException {
@@ -214,15 +224,15 @@ public abstract class GPUObject
 
        /**
         * Clears the data associated with this {@link GPUObject} instance
-        * @param synchronous whether to be done synchronously or asynchronously
+        * @param eager whether to be done synchronously or asynchronously
         * @throws CacheException ?
         */
-       public void clearData(boolean synchronous) throws CacheException {
+       public void clearData(boolean eager) throws CacheException {
                synchronized(evictionLock) {
-                       GPUContext.allocatedPointers.remove(this);
+                       JCudaContext.allocatedPointers.remove(this);
                }
                try {
-                       deallocateMemoryOnDevice(synchronous);
+                       deallocateMemoryOnDevice(eager);
                } catch (DMLRuntimeException e) {
                        throw new CacheException(e);
                }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaContext.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaContext.java
 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaContext.java
index b743a3d..bb73f4b 100644
--- 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaContext.java
+++ 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaContext.java
@@ -18,48 +18,47 @@
  */
 package org.apache.sysml.runtime.instructions.gpu.context;
 
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.conf.DMLConfig;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
-import org.apache.sysml.utils.GPUStatistics;
-
 import jcuda.driver.JCudaDriver;
 import jcuda.jcublas.JCublas2;
 import jcuda.jcublas.cublasHandle;
 import jcuda.jcudnn.JCudnn;
-import jcuda.runtime.JCuda;
 import jcuda.jcudnn.cudnnHandle;
 import jcuda.jcusparse.JCusparse;
 import jcuda.jcusparse.cusparseHandle;
+import jcuda.runtime.JCuda;
 import jcuda.runtime.cudaDeviceProp;
-import static jcuda.jcudnn.JCudnn.cudnnCreate;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
+import org.apache.sysml.utils.GPUStatistics;
+
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static jcuda.driver.JCudaDriver.cuDeviceGetCount;
+import static jcuda.driver.JCudaDriver.cuInit;
 import static jcuda.jcublas.JCublas2.cublasCreate;
 import static jcuda.jcublas.JCublas2.cublasDestroy;
+import static jcuda.jcudnn.JCudnn.cudnnCreate;
 import static jcuda.jcudnn.JCudnn.cudnnDestroy;
-import static jcuda.jcusparse.JCusparse.cusparseDestroy;
 import static jcuda.jcusparse.JCusparse.cusparseCreate;
-import static jcuda.driver.JCudaDriver.cuInit;
-import static jcuda.driver.JCudaDriver.cuDeviceGetCount;
+import static jcuda.jcusparse.JCusparse.cusparseDestroy;
 import static jcuda.runtime.JCuda.*;
 import static jcuda.runtime.cudaError.cudaSuccess;
 
-/**
- * Setup:
- * 1. Install CUDA 7.5
- * 2. Install CuDNN v4 from 
http://developer.download.nvidia.com/compute/redist/cudnn/v4/cudnn-7.0-win-x64-v4.0-prod.zip
- * 3. Download JCuda binaries version 0.7.5b and JCudnn version 0.7.5. Copy 
the DLLs into C:\lib (or /lib) directory. Link: 
http://www.jcuda.org/downloads/downloads.html
- *
- */
+
 public class JCudaContext extends GPUContext {
 
+       /** Synchronization object to make sure no allocations happen when 
something is being evicted from memory */
+       public static final Object syncObj = new Object();
        private static final Log LOG = 
LogFactory.getLog(JCudaContext.class.getName());
 
+       /** Global list of allocated {@link GPUObject} instances. This list 
must be accessed in a synchronized way */
+       public static ArrayList<GPUObject> allocatedPointers = new 
ArrayList<GPUObject>();
+
        // The minimum CUDA Compute capability needed for SystemML.
        // After compute capability 3.0, 2^31 - 1 blocks and 1024 threads per 
block are supported.
        // If SystemML needs to run on an older card, this logic can be 
revisited.
@@ -73,7 +72,7 @@ public class JCudaContext extends GPUContext {
        public static boolean DEBUG = false;
 
        /** total bytes available on currently active cude device, please be 
careful with its bookkeeping */
-       private AtomicLong deviceMemBytes = new AtomicLong(0);
+       AtomicLong deviceMemBytes = new AtomicLong(0);
 
        /** Stores the cached deviceProperties */
        private static cudaDeviceProp[] deviceProperties;
@@ -105,7 +104,6 @@ public class JCudaContext extends GPUContext {
                LOG.info("Active CUDA device number : " + device[0]);
                LOG.info("Max Blocks/Threads/SharedMem : " + maxBlocks + "/" + 
maxThreadsPerBlock + "/" + sharedMemPerBlock);
 
-
                GPUStatistics.cudaInitTime = System.nanoTime() - start;
        }
 
@@ -267,9 +265,6 @@ public class JCudaContext extends GPUContext {
                        LibMatrixCUDA.kernels = null;
                }
                GPUStatistics.cudaLibrariesInitTime = System.nanoTime() - start;
-
-               GPUContext.deallocExecutorService = 
Executors.newSingleThreadExecutor();
-
        }
 
        @Override
@@ -279,7 +274,6 @@ public class JCudaContext extends GPUContext {
                                cudnnDestroy(LibMatrixCUDA.cudnnHandle);
                                cublasDestroy(LibMatrixCUDA.cublasHandle);
                                cusparseDestroy(LibMatrixCUDA.cusparseHandle);
-                               GPUContext.deallocExecutorService.shutdown();
                                currContext = null;
                                isGPUContextCreated = false;
                        }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaObject.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaObject.java
 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaObject.java
index c256be3..31664f6 100644
--- 
a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaObject.java
+++ 
b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaObject.java
@@ -18,24 +18,17 @@
  */
 package org.apache.sysml.runtime.instructions.gpu.context;
 
-import static jcuda.jcublas.cublasOperation.CUBLAS_OP_T;
-import static jcuda.jcusparse.JCusparse.cusparseCreateMatDescr;
-import static jcuda.jcusparse.JCusparse.cusparseDcsr2dense;
-import static jcuda.jcusparse.JCusparse.cusparseDdense2csr;
-import static jcuda.jcusparse.JCusparse.cusparseDnnz;
-import static jcuda.jcusparse.JCusparse.cusparseSetMatIndexBase;
-import static jcuda.jcusparse.JCusparse.cusparseSetMatType;
-import static jcuda.jcusparse.JCusparse.cusparseSetPointerMode;
-import static jcuda.jcusparse.JCusparse.cusparseXcsrgemmNnz;
-import static jcuda.jcusparse.JCusparse.cusparseXcsrgeamNnz;
-import static jcuda.jcusparse.cusparseIndexBase.CUSPARSE_INDEX_BASE_ZERO;
-import static jcuda.jcusparse.cusparseMatrixType.CUSPARSE_MATRIX_TYPE_GENERAL;
-import static jcuda.runtime.JCuda.*;
-import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
-import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
-
+import jcuda.Pointer;
+import jcuda.jcublas.JCublas2;
+import jcuda.jcublas.cublasHandle;
+import jcuda.jcusparse.JCusparse;
+import jcuda.jcusparse.cusparseDirection;
+import jcuda.jcusparse.cusparseHandle;
+import jcuda.jcusparse.cusparseMatDescr;
+import jcuda.jcusparse.cusparsePointerMode;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.CacheException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
@@ -47,17 +40,30 @@ import org.apache.sysml.runtime.matrix.data.SparseBlockCOO;
 import org.apache.sysml.runtime.matrix.data.SparseBlockCSR;
 import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
 import org.apache.sysml.utils.GPUStatistics;
+import org.apache.sysml.utils.LRUCacheMap;
 
-import jcuda.Pointer;
-import jcuda.jcublas.JCublas2;
-import jcuda.jcublas.cublasHandle;
-import jcuda.jcusparse.JCusparse;
-import jcuda.jcusparse.cusparseDirection;
-import jcuda.jcusparse.cusparseHandle;
-import jcuda.jcusparse.cusparseMatDescr;
-import jcuda.jcusparse.cusparsePointerMode;
+import java.util.HashMap;
+import java.util.LinkedList;
 
-import java.util.concurrent.Future;
+import static jcuda.jcublas.cublasOperation.CUBLAS_OP_T;
+import static jcuda.jcusparse.JCusparse.cusparseCreateMatDescr;
+import static jcuda.jcusparse.JCusparse.cusparseDcsr2dense;
+import static jcuda.jcusparse.JCusparse.cusparseDdense2csr;
+import static jcuda.jcusparse.JCusparse.cusparseDnnz;
+import static jcuda.jcusparse.JCusparse.cusparseSetMatIndexBase;
+import static jcuda.jcusparse.JCusparse.cusparseSetMatType;
+import static jcuda.jcusparse.JCusparse.cusparseSetPointerMode;
+import static jcuda.jcusparse.JCusparse.cusparseXcsrgeamNnz;
+import static jcuda.jcusparse.JCusparse.cusparseXcsrgemmNnz;
+import static jcuda.jcusparse.cusparseIndexBase.CUSPARSE_INDEX_BASE_ZERO;
+import static jcuda.jcusparse.cusparseMatrixType.CUSPARSE_MATRIX_TYPE_GENERAL;
+import static jcuda.runtime.JCuda.cudaDeviceSynchronize;
+import static jcuda.runtime.JCuda.cudaFree;
+import static jcuda.runtime.JCuda.cudaMalloc;
+import static jcuda.runtime.JCuda.cudaMemcpy;
+import static jcuda.runtime.JCuda.cudaMemset;
+import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
+import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
 
 /**
  * Handle to a matrix block on the GPU
@@ -190,13 +196,14 @@ public class JCudaObject extends GPUObject {
                 */
                public static void copyToDevice(CSRPointer dest, int rows, long 
nnz, int[] rowPtr, int[] colInd, double[] values) {
                        CSRPointer r = dest;
-                       long t0 = System.nanoTime();
+                       long t0=0;
+                       if (DMLScript.STATISTICS) t0 = System.nanoTime();
                        r.nnz = nnz;
                        cudaMemcpy(r.rowPtr, Pointer.to(rowPtr), 
getIntSizeOf(rows + 1), cudaMemcpyHostToDevice);
                        cudaMemcpy(r.colInd, Pointer.to(colInd), 
getIntSizeOf(nnz), cudaMemcpyHostToDevice);
                        cudaMemcpy(r.val, Pointer.to(values), 
getDoubleSizeOf(nnz), cudaMemcpyHostToDevice);
-                       
GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime()-t0);
-                       GPUStatistics.cudaToDevCount.addAndGet(3);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime()-t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaToDevCount.addAndGet(3);
                }
                
                /**
@@ -210,12 +217,13 @@ public class JCudaObject extends GPUObject {
                 */
                public static void copyToHost(CSRPointer src, int rows, long 
nnz, int[] rowPtr, int[] colInd, double[] values){
                        CSRPointer r = src;
-                       long t0 = System.nanoTime();
+                       long t0=0;
+                       if (DMLScript.STATISTICS) t0 = System.nanoTime();
                        cudaMemcpy(Pointer.to(rowPtr), r.rowPtr, 
getIntSizeOf(rows + 1), cudaMemcpyDeviceToHost);
                        cudaMemcpy(Pointer.to(colInd), r.colInd, 
getIntSizeOf(nnz), cudaMemcpyDeviceToHost);
                        cudaMemcpy(Pointer.to(values), r.val, 
getDoubleSizeOf(nnz), cudaMemcpyDeviceToHost);
-                       
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime()-t0);
-                       GPUStatistics.cudaFromDevCount.addAndGet(3);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime()-t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevCount.addAndGet(3);
                }
                
                // 
==============================================================================================
@@ -408,20 +416,22 @@ public class JCudaObject extends GPUObject {
                }
                
                /**
-                * Calls cudaFree asynchronously on the allocated {@link 
Pointer} instances
+                * Calls cudaFree lazily on the allocated {@link Pointer} 
instances
                 */
                public void deallocate() {
                        deallocate(false);
                }
 
                /**
-                * Calls cudaFree asynchronously or synchronously on the 
allocated {@link Pointer} instances
-                * @param synchronous whether to do synchronous or async 
cudaFrees
+                * Calls cudaFree lazily or eagerly on the allocated {@link 
Pointer} instances
+                * @param eager whether to do eager or lazy cudaFrees
                 */
-               public void deallocate(boolean synchronous){
-                       cudaFreeHelper(val);
-                       cudaFreeHelper(rowPtr);
-                       cudaFreeHelper(colInd);
+               public void deallocate(boolean eager){
+                       if (nnz > 0) {
+                               cudaFreeHelper(val, eager);
+                               cudaFreeHelper(rowPtr, eager);
+                               cudaFreeHelper(colInd, eager);
+                       }
                }
        };
        
@@ -451,54 +461,6 @@ public class JCudaObject extends GPUObject {
        JCudaObject(MatrixObject m) {
                super(m);
        }
-       
-       /**
-        * Allocates temporary space on the device.
-        * Does not update bookkeeping.
-        * The caller is responsible for freeing up after usage.
-        * @param instructionName name of instruction for which to record per 
instruction performance statistics, null if don't want to record
-        * @param size                          Size of data (in bytes) to 
allocate
-        * @param statsCount    amount to increment the cudaAllocCount by
-        * @return jcuda Pointer
-        * @throws DMLRuntimeException if DMLRuntimeException occurs
-        */
-       public static Pointer allocate(String instructionName, long size, int 
statsCount) throws DMLRuntimeException{
-               synchronized (GPUContext.syncObj) {
-                       Pointer A = new Pointer();
-                       ensureFreeSpace(size);
-                       long t0 = System.nanoTime();
-                       cudaMalloc(A, size);
-                       // Set all elements to 0 since newly allocated space 
will contain garbage
-                       cudaMemset(A, 0, size);
-                       GPUStatistics.cudaAllocTime.getAndAdd(System.nanoTime() 
- t0);
-                       GPUStatistics.cudaAllocCount.getAndAdd(statsCount);
-                       if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_ALLOCATE, System.nanoTime() - t0);
-
-                       return A;
-               }
-       }
-
-       /**
-        * Convenience method for {@link #allocate(String, long, int)}, 
defaults statsCount to 1.
-        * @param size size of data (in bytes) to allocate
-        * @return jcuda pointer
-        * @throws DMLRuntimeException if DMLRuntimeException occurs
-        */
-       public static Pointer allocate(long size) throws DMLRuntimeException {
-               return allocate(null, size, 1);
-       }
-
-       /**
-        * Convenience method for {@link #allocate(String, long, int)}, 
defaults statsCount to 1.
-        * @param instructionName name of instruction for which to record per 
instruction performance statistics, null if don't want to record
-        * @param size size of data (in bytes) to allocate
-        * @return jcuda pointer
-        * @throws DMLRuntimeException if DMLRuntimeException occurs
-        */
-       public static Pointer allocate(String instructionName, long size) 
throws DMLRuntimeException {
-               return allocate(instructionName, size, 1);
-       }
-
 
        /**
         * Allocates a sparse and empty {@link JCudaObject}
@@ -565,7 +527,7 @@ public class JCudaObject extends GPUObject {
                        allocateDenseMatrixOnDevice();
                        allocated = true;
                        synchronized(evictionLock) {
-                               GPUContext.allocatedPointers.add(this);
+                               JCudaContext.allocatedPointers.add(this);
                        }
                }
                isDeviceCopyModified = true;
@@ -583,7 +545,7 @@ public class JCudaObject extends GPUObject {
                        allocateSparseMatrixOnDevice();
                        allocated = true;
                        synchronized(evictionLock) {
-                               GPUContext.allocatedPointers.add(this);
+                               JCudaContext.allocatedPointers.add(this);
                        }
                }
                isDeviceCopyModified = true;
@@ -724,20 +686,14 @@ public class JCudaObject extends GPUObject {
        }
 
        @Override
-       void deallocateMemoryOnDevice(boolean synchronous) {
+       void deallocateMemoryOnDevice(boolean eager) {
                if(jcudaDenseMatrixPtr != null) {
-                       long start = System.nanoTime();
-                       cudaFreeHelper(null, jcudaDenseMatrixPtr, synchronous);
+                       cudaFreeHelper(null, jcudaDenseMatrixPtr, eager);
                        
((JCudaContext)GPUContext.currContext).getAndAddAvailableMemory(numBytes);
-                       
GPUStatistics.cudaDeAllocTime.addAndGet(System.nanoTime()-start);
-                       GPUStatistics.cudaDeAllocCount.addAndGet(1);
                }
                if (jcudaSparseMatrixPtr != null) {
-                       long start = System.nanoTime();
-                       jcudaSparseMatrixPtr.deallocate(synchronous);
+                       jcudaSparseMatrixPtr.deallocate(eager);
                        
((JCudaContext)GPUContext.currContext).getAndAddAvailableMemory(numBytes);
-                       
GPUStatistics.cudaDeAllocTime.addAndGet(System.nanoTime()-start);
-                       GPUStatistics.cudaDeAllocCount.addAndGet(1);
                }
                jcudaDenseMatrixPtr = null;
                jcudaSparseMatrixPtr = null;
@@ -750,8 +706,18 @@ public class JCudaObject extends GPUObject {
         * @throws DMLRuntimeException if DMLRuntimeException occurs
         */
        static void ensureFreeSpace(long size) throws DMLRuntimeException {
+               ensureFreeSpace(null, size);
+       }
+
+       /**
+        * Thin wrapper over {@link #evict(long)}
+        * @param instructionName instructionName name of the instruction for 
which performance measurements are made
+        * @param size size to check
+        * @throws DMLRuntimeException if DMLRuntimeException occurs
+        */
+       static void ensureFreeSpace(String instructionName, long size) throws 
DMLRuntimeException {
                if(size >= getAvailableMemory()) {
-                       evict(size);
+                       evict(instructionName, size);
                }
        }
        
@@ -760,7 +726,8 @@ public class JCudaObject extends GPUObject {
                throws DMLRuntimeException 
        {
                printCaller();
-               long start = System.nanoTime();
+               long start=0;
+               if (DMLScript.STATISTICS) start = System.nanoTime();
                
                MatrixBlock tmp = mat.acquireRead();
                if(tmp.isInSparseFormat()) {
@@ -790,21 +757,22 @@ public class JCudaObject extends GPUObject {
                                // CSR is the preferred format for cuSparse GEMM
                                // Converts MCSR and COO to CSR
                                SparseBlockCSR csrBlock = null;
+                               long t0=0;
                                if (block instanceof SparseBlockCSR){ 
                                        csrBlock = (SparseBlockCSR)block;
                                } else if (block instanceof SparseBlockCOO) {
                                        // TODO - should we do this on the GPU 
using cusparse<t>coo2csr() ?
-                                       long t0 = System.nanoTime();
+                                       if (DMLScript.STATISTICS) t0 = 
System.nanoTime();
                                        SparseBlockCOO cooBlock = 
(SparseBlockCOO)block;
                                        csrBlock = new 
SparseBlockCSR(toIntExact(mat.getNumRows()), cooBlock.rowIndexes(), 
cooBlock.indexes(), cooBlock.values());
-                                       
GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
-                                       
GPUStatistics.cudaSparseConversionCount.incrementAndGet();
+                                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
+                                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseConversionCount.incrementAndGet();
                                } else if (block instanceof SparseBlockMCSR) {
-                                       long t0 = System.nanoTime();
+                                       if (DMLScript.STATISTICS) t0 = 
System.nanoTime();
                                        SparseBlockMCSR mcsrBlock = 
(SparseBlockMCSR)block;
                                        csrBlock = new 
SparseBlockCSR(mcsrBlock.getRows(), toIntExact(mcsrBlock.size()));
-                                       
GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
-                                       
GPUStatistics.cudaSparseConversionCount.incrementAndGet();
+                                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseConversionTime.addAndGet(System.nanoTime() - t0);
+                                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseConversionCount.incrementAndGet();
                                } else {
                                        throw new 
DMLRuntimeException("Unsupported sparse matrix format for CUDA operations");
                                }
@@ -814,7 +782,7 @@ public class JCudaObject extends GPUObject {
                        }
                        allocateSparseMatrixOnDevice();
                        synchronized(evictionLock) {
-                               GPUContext.allocatedPointers.add(this);
+                               JCudaContext.allocatedPointers.add(this);
                        }
                        if(copyToDevice) {
                                CSRPointer.copyToDevice(jcudaSparseMatrixPtr, 
tmp.getNumRows(), tmp.getNonZeros(), rowPtr, colInd, values);
@@ -835,15 +803,15 @@ public class JCudaObject extends GPUObject {
                        // Copy dense block
                        allocateDenseMatrixOnDevice();
                        synchronized(evictionLock) {
-                               GPUContext.allocatedPointers.add(this);
+                               JCudaContext.allocatedPointers.add(this);
                        }
                        cudaMemcpy(jcudaDenseMatrixPtr, Pointer.to(data), 
getDoubleSizeOf(mat.getNumRows()*mat.getNumColumns()), cudaMemcpyHostToDevice);
                }
                
                mat.release();
-               
-               GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime()-start);
-               GPUStatistics.cudaToDevCount.addAndGet(1);
+
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaToDevTime.addAndGet(System.nanoTime()-start);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaToDevCount.addAndGet(1);
        }
        
        public static int toIntExact(long l) throws DMLRuntimeException {
@@ -861,7 +829,8 @@ public class JCudaObject extends GPUObject {
 
                if(jcudaDenseMatrixPtr != null) {
                        printCaller();
-                       long start = System.nanoTime();
+                       long start=0;
+                       if (DMLScript.STATISTICS) start = System.nanoTime();
                        MatrixBlock tmp = new 
MatrixBlock(toIntExact(mat.getNumRows()), toIntExact(mat.getNumColumns()), 
false);
                        tmp.allocateDenseBlock();
                        double [] data = tmp.getDenseBlock();
@@ -871,9 +840,9 @@ public class JCudaObject extends GPUObject {
                        tmp.recomputeNonZeros();
                        mat.acquireModify(tmp);
                        mat.release();
-                       
-                       
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime()-start);
-                       GPUStatistics.cudaFromDevCount.addAndGet(1);
+
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime()-start);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevCount.addAndGet(1);
                }
                else if (jcudaSparseMatrixPtr != null){
                        printCaller();
@@ -885,7 +854,8 @@ public class JCudaObject extends GPUObject {
                                mat.acquireModify(tmp);
                                mat.release();
                        } else {
-                               long start = System.nanoTime();
+                               long start=0;
+                               if (DMLScript.STATISTICS) start = 
System.nanoTime();
 
                                int rows = toIntExact(mat.getNumRows());
                                int cols = toIntExact(mat.getNumColumns());
@@ -899,8 +869,8 @@ public class JCudaObject extends GPUObject {
                                MatrixBlock tmp = new MatrixBlock(rows, cols, 
nnz, sparseBlock);
                                mat.acquireModify(tmp);
                                mat.release();
-                               
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - start);
-                               GPUStatistics.cudaFromDevCount.addAndGet(1);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - start);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevCount.addAndGet(1);
                        }
                }
                else {
@@ -990,7 +960,8 @@ public class JCudaObject extends GPUObject {
         * @throws DMLRuntimeException if DMLRuntimeException occurs
         */
        public void denseToSparse() throws DMLRuntimeException {
-               long t0 = System.nanoTime();
+               long t0=0;
+               if (DMLScript.STATISTICS) t0 = System.nanoTime();
                cusparseHandle cusparseHandle = LibMatrixCUDA.cusparseHandle;
                if(cusparseHandle == null)
                        throw new DMLRuntimeException("Expected cusparse to be 
initialized");
@@ -1004,8 +975,8 @@ public class JCudaObject extends GPUObject {
                
setSparseMatrixCudaPointer(columnMajorDenseToRowMajorSparse(cusparseHandle, 
rows, cols, jcudaDenseMatrixPtr));
                // TODO: What if mat.getNnz() is -1 ?
                numBytes = CSRPointer.estimateSize(mat.getNnz(), rows);
-               GPUStatistics.cudaDenseToSparseTime.addAndGet(System.nanoTime() 
- t0);
-               GPUStatistics.cudaDenseToSparseCount.addAndGet(1);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseTime.addAndGet(System.nanoTime() - t0);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseCount.addAndGet(1);
        }
 
        /**
@@ -1077,16 +1048,17 @@ public class JCudaObject extends GPUObject {
         * @throws DMLRuntimeException ?
         */
        public void sparseToDense(String instructionName) throws 
DMLRuntimeException {
-               long start = System.nanoTime();
+               long start=0, end=0;
+               if (DMLScript.STATISTICS) start = System.nanoTime();
                if(jcudaSparseMatrixPtr == null || !isAllocated())
                        throw new DMLRuntimeException("Expected allocated 
sparse matrix before sparseToDense() call");
 
                sparseToColumnMajorDense();
                convertDensePtrFromColMajorToRowMajor();
-               long end = System.nanoTime();
+               if (DMLScript.STATISTICS) end = System.nanoTime();
                if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, end - start);
-               GPUStatistics.cudaSparseToDenseTime.addAndGet(end - start);
-               GPUStatistics.cudaSparseToDenseCount.addAndGet(1);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseTime.addAndGet(end - start);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseCount.addAndGet(1);
        }
        
        /**
@@ -1122,26 +1094,23 @@ public class JCudaObject extends GPUObject {
         */
        public static CSRPointer 
columnMajorDenseToRowMajorSparse(cusparseHandle cusparseHandle, int rows, int 
cols, Pointer densePtr) throws DMLRuntimeException {
                cusparseMatDescr matDescr = 
CSRPointer.getDefaultCuSparseMatrixDescriptor();
-               Pointer nnzPerRowPtr = new Pointer();
-               Pointer nnzTotalDevHostPtr = new Pointer();
+               Pointer nnzPerRowPtr = null;
+               Pointer nnzTotalDevHostPtr = null;
                
                ensureFreeSpace(getIntSizeOf(rows + 1));
-               
-               long t1 = System.nanoTime();
                nnzPerRowPtr = allocate(getIntSizeOf(rows));
                nnzTotalDevHostPtr = allocate(getIntSizeOf(1));
-               GPUStatistics.cudaAllocTime.addAndGet(System.nanoTime() - t1);
-               GPUStatistics.cudaAllocCount.addAndGet(2);
                
                // Output is in dense vector format, convert it to CSR
                cusparseDnnz(cusparseHandle, 
cusparseDirection.CUSPARSE_DIRECTION_ROW, rows, cols, matDescr, densePtr, rows, 
nnzPerRowPtr, nnzTotalDevHostPtr);
                cudaDeviceSynchronize();
                int[] nnzC = {-1};
-               
-               long t2 = System.nanoTime();
+
+               long t2=0;
+               if (DMLScript.STATISTICS) t2 = System.nanoTime();
                cudaMemcpy(Pointer.to(nnzC), nnzTotalDevHostPtr, 
getIntSizeOf(1), cudaMemcpyDeviceToHost);
-               GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - t2);
-               GPUStatistics.cudaFromDevCount.addAndGet(2);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevTime.addAndGet(System.nanoTime() - t2);
+               if (DMLScript.STATISTICS) 
GPUStatistics.cudaFromDevCount.addAndGet(1);
                
                if (nnzC[0] == -1){
                        throw new DMLRuntimeException("cusparseDnnz did not 
calculate the correct number of nnz from the sparse-matrix vector mulitply on 
the GPU");
@@ -1157,8 +1126,79 @@ public class JCudaObject extends GPUObject {
                return C;
        }
 
+
+       /** Map of free blocks allocate on GPU. maps size_of_block -> pointer 
on GPU */
+       static LRUCacheMap<Long, LinkedList<Pointer>> freeCUDASpaceMap = new 
LRUCacheMap<Long, LinkedList<Pointer>>();
+       /** To record size of allocated blocks */
+       static HashMap<Pointer, Long> cudaBlockSizeMap = new HashMap<Pointer, 
Long>();
+
+
+       /**
+        * Convenience method for {@link #allocate(String, long, int)}, 
defaults statsCount to 1.
+        * @param size size of data (in bytes) to allocate
+        * @return jcuda pointer
+        * @throws DMLRuntimeException if DMLRuntimeException occurs
+        */
+       public static Pointer allocate(long size) throws DMLRuntimeException {
+               return allocate(null, size, 1);
+       }
+
+       /**
+        * Convenience method for {@link #allocate(String, long, int)}, 
defaults statsCount to 1.
+        * @param instructionName name of instruction for which to record per 
instruction performance statistics, null if don't want to record
+        * @param size size of data (in bytes) to allocate
+        * @return jcuda pointer
+        * @throws DMLRuntimeException if DMLRuntimeException occurs
+        */
+       public static Pointer allocate(String instructionName, long size) 
throws DMLRuntimeException {
+               return allocate(instructionName, size, 1);
+       }
+
+       /**
+        * Allocates temporary space on the device.
+        * Does not update bookkeeping.
+        * The caller is responsible for freeing up after usage.
+        * @param instructionName name of instruction for which to record per 
instruction performance statistics, null if don't want to record
+        * @param size                          Size of data (in bytes) to 
allocate
+        * @param statsCount    amount to increment the cudaAllocCount by
+        * @return jcuda Pointer
+        * @throws DMLRuntimeException if DMLRuntimeException occurs
+        */
+       public static Pointer allocate(String instructionName, long size, int 
statsCount) throws DMLRuntimeException{
+               long t0=0, t1=0, end=0;
+               synchronized (JCudaContext.syncObj) {
+                       Pointer A;
+                       if (freeCUDASpaceMap.containsKey(size)) {
+                               if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+                               LinkedList<Pointer> freeList = 
freeCUDASpaceMap.get(size);
+                               A = freeList.pop();
+                               if (freeList.isEmpty())
+                                       freeCUDASpaceMap.remove(size);
+                               if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_REUSE, System.nanoTime() - t0);
+                       } else {
+                               if (DMLScript.STATISTICS) t0 = 
System.nanoTime();
+                               ensureFreeSpace(instructionName, size);
+                               A = new Pointer();
+                               cudaMalloc(A, size);
+                               
((JCudaContext)(JCudaContext.currContext)).deviceMemBytes.addAndGet(size);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaAllocTime.getAndAdd(System.nanoTime() - t0);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaAllocCount.getAndAdd(statsCount);
+                               if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_ALLOCATE, System.nanoTime() - t0);
+                       }
+                       // Set all elements to 0 since newly allocated space 
will contain garbage
+                       if (DMLScript.STATISTICS) t1 = System.nanoTime();
+                       cudaMemset(A, 0, size);
+                       if (DMLScript.STATISTICS) end = System.nanoTime() - t1;
+                       if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_SET_ZERO, end - t1);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaMemSet0Time.getAndAdd(end - t1);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaMemSet0Count.getAndAdd(1);
+                       cudaBlockSizeMap.put(A, size);
+                       return A;
+               }
+       }
+
        /**
-        * Does asynchronous cudaFree calls
+        * Does lazy cudaFree calls
         * @param toFree {@link Pointer} instance to be freed
         */
        public static void cudaFreeHelper(final Pointer toFree) {
@@ -1166,7 +1206,17 @@ public class JCudaObject extends GPUObject {
        }
 
        /**
-        * Does asynchronous cudaFree calls
+        * does lazy/eager cudaFree calls
+        * @param toFree {@link Pointer} instance to be freed
+        * @param eager true if to be done eagerly
+        * @throws DMLRuntimeException
+        */
+       public static void cudaFreeHelper(final Pointer toFree, boolean eager) {
+               cudaFreeHelper(null, toFree, eager);
+       }
+
+       /**
+        * Does lazy cudaFree calls
         * @param instructionName name of the instruction for which to record 
per instruction free time, null if do not want to record
         * @param toFree {@link Pointer} instance to be freed
         */
@@ -1175,27 +1225,32 @@ public class JCudaObject extends GPUObject {
        }
 
        /**
-        * Does cudaFree calls, either synchronously or asynchronously
+        * Does cudaFree calls, lazily
         * @param instructionName name of the instruction for which to record 
per instruction free time, null if do not want to record
         * @param toFree {@link Pointer} instance to be freed
-        * @param synchronous true if to be done synchronously
+        * @param eager true if to be done eagerly
         */
        @SuppressWarnings("rawtypes")
-       public static void cudaFreeHelper(String instructionName, final Pointer 
toFree, boolean synchronous) {
+       public static void cudaFreeHelper(String instructionName, final Pointer 
toFree, boolean eager){
                long t0 = 0;
-               if (instructionName != null) t0 = System.nanoTime();
-               if (synchronous) {
+               assert cudaBlockSizeMap.containsKey(toFree) : "ERROR : Internal 
state corrupted, cache block size map is not aware of a block it trying to free 
up";
+               long size = cudaBlockSizeMap.get(toFree);
+               if (eager) {
+                       if (DMLScript.STATISTICS) t0 = System.nanoTime();
+                       
((JCudaContext)(JCudaContext.currContext)).deviceMemBytes.addAndGet(-size);
                        cudaFree(toFree);
+                       cudaBlockSizeMap.remove(toFree);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaDeAllocTime.addAndGet(System.nanoTime() - t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaDeAllocCount.addAndGet(1);
+                       if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_CUDA_FREE, System.nanoTime() - t0);
                } else {
-                       Future submitted = 
GPUContext.deallocExecutorService.submit(new Runnable() {
-                               @Override
-                               public void run() {
-                                       cudaFree(toFree);
-                               }
-                       });
-                       GPUContext.pendingDeallocates.offer(submitted);
+                       LinkedList<Pointer> freeList = 
freeCUDASpaceMap.get(size);
+                       if (freeList == null) {
+                               freeList = new LinkedList<Pointer>();
+                               freeCUDASpaceMap.put(size, freeList);
+                       }
+                       freeList.add(toFree);
                }
-               if (instructionName != null && 
GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instructionName, 
GPUInstruction.MISC_TIMER_CUDA_FREE, System.nanoTime() - t0);
        }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
----------------------------------------------------------------------
diff --git 
a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java 
b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
index 4ff9849..a3b9168 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCUDA.java
@@ -19,51 +19,24 @@
 
 package org.apache.sysml.runtime.matrix.data;
 
-import static jcuda.jcublas.cublasOperation.CUBLAS_OP_N;
-import static jcuda.jcublas.cublasOperation.CUBLAS_OP_T;
-import static jcuda.jcudnn.JCudnn.cudnnActivationForward;
-import static jcuda.jcudnn.JCudnn.cudnnConvolutionBackwardData;
-import static jcuda.jcudnn.JCudnn.cudnnConvolutionBackwardFilter;
-import static jcuda.jcudnn.JCudnn.cudnnConvolutionForward;
-import static jcuda.jcudnn.JCudnn.cudnnCreateActivationDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnCreateConvolutionDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnCreateFilterDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnCreatePoolingDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnCreateTensorDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnDestroyConvolutionDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnDestroyFilterDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnDestroyPoolingDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnDestroyTensorDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnGetConvolutionBackwardDataWorkspaceSize;
-import static 
jcuda.jcudnn.JCudnn.cudnnGetConvolutionBackwardFilterWorkspaceSize;
-import static jcuda.jcudnn.JCudnn.cudnnGetConvolutionForwardWorkspaceSize;
-import static jcuda.jcudnn.JCudnn.cudnnPoolingBackward;
-import static jcuda.jcudnn.JCudnn.cudnnPoolingForward;
-import static jcuda.jcudnn.JCudnn.cudnnSetActivationDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnSetConvolution2dDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnSetFilter4dDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnSetPooling2dDescriptor;
-import static jcuda.jcudnn.JCudnn.cudnnSetTensor4dDescriptor;
-import static jcuda.jcudnn.cudnnActivationMode.CUDNN_ACTIVATION_RELU;
-import static jcuda.jcudnn.cudnnConvolutionMode.CUDNN_CROSS_CORRELATION;
-import static jcuda.jcudnn.cudnnDataType.CUDNN_DATA_DOUBLE;
-import static jcuda.jcudnn.cudnnNanPropagation.CUDNN_PROPAGATE_NAN;
-import static jcuda.jcudnn.cudnnPoolingMode.CUDNN_POOLING_MAX;
-import static jcuda.jcudnn.cudnnTensorFormat.CUDNN_TENSOR_NCHW;
-import static jcuda.jcusparse.JCusparse.cusparseDcsrgemm;
-import static jcuda.jcusparse.JCusparse.cusparseDcsrmv;
-import static 
jcuda.jcusparse.cusparseOperation.CUSPARSE_OPERATION_NON_TRANSPOSE;
-import static jcuda.jcusparse.cusparseOperation.CUSPARSE_OPERATION_TRANSPOSE;
-import static jcuda.runtime.JCuda.cudaDeviceSynchronize;
-import static jcuda.runtime.JCuda.cudaMemcpy;
-import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToDevice;
-import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
-import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
-import static 
org.apache.sysml.runtime.instructions.gpu.context.JCudaObject.allocate;
-import static 
org.apache.sysml.runtime.instructions.gpu.context.JCudaObject.cudaFreeHelper;
-
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas2;
+import jcuda.jcublas.cublasFillMode;
+import jcuda.jcublas.cublasHandle;
+import jcuda.jcublas.cublasOperation;
+import jcuda.jcudnn.cudnnActivationDescriptor;
+import jcuda.jcudnn.cudnnConvolutionDescriptor;
+import jcuda.jcudnn.cudnnConvolutionFwdPreference;
+import jcuda.jcudnn.cudnnFilterDescriptor;
+import jcuda.jcudnn.cudnnHandle;
+import jcuda.jcudnn.cudnnPoolingDescriptor;
+import jcuda.jcudnn.cudnnTensorDescriptor;
+import jcuda.jcusparse.JCusparse;
+import jcuda.jcusparse.cusparseHandle;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
@@ -111,21 +84,48 @@ import 
org.apache.sysml.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysml.utils.GPUStatistics;
 import org.apache.sysml.utils.Statistics;
 
-import jcuda.Pointer;
-import jcuda.Sizeof;
-import jcuda.jcublas.JCublas2;
-import jcuda.jcublas.cublasFillMode;
-import jcuda.jcublas.cublasHandle;
-import jcuda.jcublas.cublasOperation;
-import jcuda.jcudnn.cudnnActivationDescriptor;
-import jcuda.jcudnn.cudnnConvolutionDescriptor;
-import jcuda.jcudnn.cudnnConvolutionFwdPreference;
-import jcuda.jcudnn.cudnnFilterDescriptor;
-import jcuda.jcudnn.cudnnHandle;
-import jcuda.jcudnn.cudnnPoolingDescriptor;
-import jcuda.jcudnn.cudnnTensorDescriptor;
-import jcuda.jcusparse.JCusparse;
-import jcuda.jcusparse.cusparseHandle;
+import static jcuda.jcublas.cublasOperation.CUBLAS_OP_N;
+import static jcuda.jcublas.cublasOperation.CUBLAS_OP_T;
+import static jcuda.jcudnn.JCudnn.cudnnActivationForward;
+import static jcuda.jcudnn.JCudnn.cudnnConvolutionBackwardData;
+import static jcuda.jcudnn.JCudnn.cudnnConvolutionBackwardFilter;
+import static jcuda.jcudnn.JCudnn.cudnnConvolutionForward;
+import static jcuda.jcudnn.JCudnn.cudnnCreateActivationDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnCreateConvolutionDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnCreateFilterDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnCreatePoolingDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnCreateTensorDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnDestroyConvolutionDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnDestroyFilterDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnDestroyPoolingDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnDestroyTensorDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnGetConvolutionBackwardDataWorkspaceSize;
+import static 
jcuda.jcudnn.JCudnn.cudnnGetConvolutionBackwardFilterWorkspaceSize;
+import static jcuda.jcudnn.JCudnn.cudnnGetConvolutionForwardWorkspaceSize;
+import static jcuda.jcudnn.JCudnn.cudnnPoolingBackward;
+import static jcuda.jcudnn.JCudnn.cudnnPoolingForward;
+import static jcuda.jcudnn.JCudnn.cudnnSetActivationDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnSetConvolution2dDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnSetFilter4dDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnSetPooling2dDescriptor;
+import static jcuda.jcudnn.JCudnn.cudnnSetTensor4dDescriptor;
+import static jcuda.jcudnn.cudnnActivationMode.CUDNN_ACTIVATION_RELU;
+import static jcuda.jcudnn.cudnnConvolutionMode.CUDNN_CROSS_CORRELATION;
+import static jcuda.jcudnn.cudnnDataType.CUDNN_DATA_DOUBLE;
+import static jcuda.jcudnn.cudnnNanPropagation.CUDNN_PROPAGATE_NAN;
+import static jcuda.jcudnn.cudnnPoolingMode.CUDNN_POOLING_MAX;
+import static jcuda.jcudnn.cudnnTensorFormat.CUDNN_TENSOR_NCHW;
+import static jcuda.jcusparse.JCusparse.cusparseDcsrgemm;
+import static jcuda.jcusparse.JCusparse.cusparseDcsrmv;
+import static 
jcuda.jcusparse.cusparseOperation.CUSPARSE_OPERATION_NON_TRANSPOSE;
+import static jcuda.jcusparse.cusparseOperation.CUSPARSE_OPERATION_TRANSPOSE;
+import static jcuda.runtime.JCuda.cudaDeviceSynchronize;
+import static jcuda.runtime.JCuda.cudaMemcpy;
+import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToDevice;
+import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
+import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
+import static 
org.apache.sysml.runtime.instructions.gpu.context.JCudaObject.allocate;
+import static 
org.apache.sysml.runtime.instructions.gpu.context.JCudaObject.cudaFreeHelper;
 
 //FIXME move could to respective instructions, this is not a block library
 public class LibMatrixCUDA {
@@ -297,10 +297,6 @@ public class LibMatrixCUDA {
                finally {
                        long t3=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t3 = 
System.nanoTime();
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
 
                        if(srcTensorDesc != null)
                                cudnnDestroyTensorDescriptor(srcTensorDesc);
@@ -508,10 +504,7 @@ public class LibMatrixCUDA {
                finally {
                        long t3=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t3 = 
System.nanoTime();
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
+
                        if(workSpace != null && sizeInBytes != 0)
                                cudaFreeHelper(instName, workSpace);
 
@@ -611,10 +604,6 @@ public class LibMatrixCUDA {
                        long t3=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t3 = 
System.nanoTime();
 
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
                        if(workSpace != null && sizeInBytes != 0)
                                cudaFreeHelper(instName, workSpace);
 
@@ -690,12 +679,12 @@ public class LibMatrixCUDA {
                        
((JCudaObject)image.getGPUObject()).sparseToDense(instName);
                }
                Pointer x = 
((JCudaObject)image.getGPUObject()).jcudaDenseMatrixPtr;
-               MatrixObject temp = new MatrixObject(image);
-               temp.getGPUObject().acquireDeviceModifyDense();
+               //MatrixObject temp = new MatrixObject(image);
+               //temp.getGPUObject().acquireDeviceModifyDense();
                Pointer y = 
((JCudaObject)image.getGPUObject()).jcudaDenseMatrixPtr;
                performReLU(instName, x, y, N, C, H, W);
                performMaxpooling(instName, y, outputBlock, N, C, H, W, K, R, 
S, pad_h, pad_w, stride_h, stride_w, P, Q);
-               ((JCudaObject)temp.getGPUObject()).clearData(); // deallocate 
the temporary data
+               //((JCudaObject)temp.getGPUObject()).clearData(); // deallocate 
the temporary data
        }
        
        private static void performMaxpooling(String instName, Pointer x,
@@ -736,10 +725,6 @@ public class LibMatrixCUDA {
                finally {
                        long t3=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t3 = 
System.nanoTime();
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
 
                        if(yDesc != null)
                                cudnnDestroyTensorDescriptor(yDesc);
@@ -837,10 +822,7 @@ public class LibMatrixCUDA {
                finally {
                        long t4=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t4 = 
System.nanoTime();
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
+
                        if(y != null)
                                cudaFreeHelper(instName, y);
 
@@ -885,10 +867,6 @@ public class LibMatrixCUDA {
                finally {
                        long t1=0;
                        if (GPUStatistics.DISPLAY_STATISTICS) t1 = 
System.nanoTime();
-                       if(alpha != null)
-                               cudaFreeHelper(instName, alpha);
-                       if(beta != null)
-                               cudaFreeHelper(instName, beta);
 
                        if(srcTensorDesc != null)
                                cudnnDestroyTensorDescriptor(srcTensorDesc);
@@ -1151,8 +1129,8 @@ public class LibMatrixCUDA {
                        int colsA = (int)left.getNumColumns();
 
 
-                       long t1=0, t2=0;
-                       long t0 = System.nanoTime();
+                       long t0=0,t1=0, t2=0;
+                       if (DMLScript.STATISTICS) t0 = System.nanoTime();
                        Pointer AT = JCudaObject.transpose(ADense, rowsA, 
colsA, colsA, rowsA);
                        if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_TRANSPOSE_LIB, System.nanoTime() - t0);
 
@@ -1160,8 +1138,8 @@ public class LibMatrixCUDA {
                        CSRPointer A = 
JCudaObject.columnMajorDenseToRowMajorSparse(cusparseHandle, rowsA, colsA, AT);
                        if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE, System.nanoTime() - t1);
 
-                       
GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
-                       GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
                        sparseSparseMatmult(instName, output, transA, transB, 
m, n, k, A, B);
 
                        if (GPUStatistics.DISPLAY_STATISTICS) t2 = 
System.nanoTime();
@@ -1175,15 +1153,15 @@ public class LibMatrixCUDA {
                        // BDenseTransposed is a column major matrix
                        // Note the arguments to denseDenseMatmult to 
accommodate for this.
                        long t0=0, t1=0;
-                       if (GPUStatistics.DISPLAY_STATISTICS) t0 = 
System.nanoTime();
+                       if (DMLScript.STATISTICS) t0 = System.nanoTime();
                        Pointer BDenseTransposed = 
B.toColumnMajorDenseMatrix(cusparseHandle, cublasHandle, 
(int)right.getNumRows(), (int)right.getNumColumns());
                        if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
-                       
GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
-                       
GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
+                       if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
 
                        if (GPUStatistics.DISPLAY_STATISTICS) t1 = 
System.nanoTime();
                        boolean allocated = 
output.getGPUObject().acquireDeviceModifyDense();   // To allocate the dense 
matrix
-                       if (allocated) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t1);
+                       if (GPUStatistics.DISPLAY_STATISTICS && allocated) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t1);
                        Pointer C = 
((JCudaObject)output.getGPUObject()).jcudaDenseMatrixPtr;
                        denseDenseMatmult(instName, C,
                                                        (int) 
left.getNumRows(), (int) left.getNumColumns(),
@@ -1233,7 +1211,7 @@ public class LibMatrixCUDA {
                                int rowsB = (int)right.getNumRows();
                                int colsB = (int)right.getNumColumns();
 
-                               if (GPUStatistics.DISPLAY_STATISTICS) t0 = 
System.nanoTime();
+                               if (DMLScript.STATISTICS) t0 = 
System.nanoTime();
                                Pointer BT = JCudaObject.transpose(BDense, 
rowsB, colsB, colsB, rowsB);
                                if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_TRANSPOSE_LIB, System.nanoTime() - t0);
 
@@ -1241,8 +1219,8 @@ public class LibMatrixCUDA {
                                CSRPointer B = 
JCudaObject.columnMajorDenseToRowMajorSparse(cusparseHandle, rowsB, colsB, BT);
                                if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE, System.nanoTime() - t1);
 
-                               
GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
-                               
GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseTime.getAndAdd(System.nanoTime() - t0);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaDenseToSparseCount.getAndAdd(1);
 
                                sparseSparseMatmult(instName, output, transA, 
transB, m, n, k, A, B);
 
@@ -1256,15 +1234,15 @@ public class LibMatrixCUDA {
                                // Convert left to dense and do a cuBlas matmul
                                // ADenseTransposed is a column major matrix
                                // Note the arguments to denseDenseMatmult to 
accommodate for this.
-                               if (GPUStatistics.DISPLAY_STATISTICS) t0 = 
System.nanoTime();
+                               if (DMLScript.STATISTICS) t0 = 
System.nanoTime();
                                Pointer ADenseTransposed = 
A.toColumnMajorDenseMatrix(cusparseHandle, cublasHandle, 
(int)left.getNumRows(), (int)left.getNumColumns());
                                if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
-                               
GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
-                               
GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseTime.getAndAdd(System.nanoTime() - t0);
+                               if (DMLScript.STATISTICS) 
GPUStatistics.cudaSparseToDenseCount.getAndAdd(System.nanoTime() - t0);
 
                                if (GPUStatistics.DISPLAY_STATISTICS) t1 = 
System.nanoTime();
                                boolean allocated = 
output.getGPUObject().acquireDeviceModifyDense();   // To allocate the dense 
matrix
-                               if (allocated) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t1);
+                               if (allocated && 
GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t1);
 
                                Pointer C = 
((JCudaObject)output.getGPUObject()).jcudaDenseMatrixPtr;
                                denseDenseMatmult(instName, C,
@@ -1860,7 +1838,7 @@ public class LibMatrixCUDA {
         * @param out           output matrix on GPU
         * @param rlen  row length
         * @param clen  column length
-        * @throws DMLRuntimeException
+        * @throws DMLRuntimeException if error
         */
        private static void squareMatrix(String instName, Pointer in, Pointer 
out, int rlen, int clen) throws DMLRuntimeException {
                ScalarOperator power2op = new 
RightScalarOperator(Power.getPowerFnObject(), 2);
@@ -2376,7 +2354,7 @@ public class LibMatrixCUDA {
         * @param instName name of the invoking instruction to record{@link 
Statistics}.
         * @param constant              scalar value with which to fill the 
matrix
         * @param outputName    (internal) name of the matrix that is to be 
filled
-        * @throws DMLRuntimeException
+        * @throws DMLRuntimeException if error
         */
        private static void setOutputToConstant(ExecutionContext ec, String 
instName, double constant, String outputName) throws DMLRuntimeException {
                if(constant == 0) {
@@ -2691,7 +2669,9 @@ public class LibMatrixCUDA {
                long t0=0;
                if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
                Pair<MatrixObject, Boolean> mb = 
ec.getDenseMatrixOutputForGPUInstruction(name);
-               if (mb.getValue()) if (GPUStatistics.DISPLAY_STATISTICS) 
GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
+               if (mb.getValue())
+                       if (GPUStatistics.DISPLAY_STATISTICS)
+                               GPUStatistics.maintainCPMiscTimes(instName, 
GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
                return mb.getKey();
        }
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/utils/GPUStatistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/GPUStatistics.java 
b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
index 8347902..1211a97 100644
--- a/src/main/java/org/apache/sysml/utils/GPUStatistics.java
+++ b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
@@ -52,10 +52,12 @@ public class GPUStatistics {
 
   public static AtomicLong cudaAllocTime = new AtomicLong(0);             // 
time spent in allocating memory on the GPU
   public static AtomicLong cudaDeAllocTime = new AtomicLong(0);           // 
time spent in deallocating memory on the GPU
+  public static AtomicLong cudaMemSet0Time = new AtomicLong(0);           // 
time spent in setting memory to 0 on the GPU (part of reusing and for new 
allocates)
   public static AtomicLong cudaToDevTime = new AtomicLong(0);             // 
time spent in copying data from host (CPU) to device (GPU) memory
   public static AtomicLong cudaFromDevTime = new AtomicLong(0);           // 
time spent in copying data from device to host
   public static AtomicLong cudaAllocCount = new AtomicLong(0);
   public static AtomicLong cudaDeAllocCount = new AtomicLong(0);
+  public static AtomicLong cudaMemSet0Count = new AtomicLong(0);
   public static AtomicLong cudaToDevCount = new AtomicLong(0);
   public static AtomicLong cudaFromDevCount = new AtomicLong(0);
   public static AtomicLong cudaEvictionCount = new AtomicLong(0);
@@ -82,6 +84,8 @@ public class GPUStatistics {
     cudaLibrariesInitTime = 0;
     cudaAllocTime.set(0);
     cudaDeAllocTime.set(0);
+    cudaMemSet0Time.set(0);
+    cudaMemSet0Count.set(0);
     cudaToDevTime.set(0);
     cudaFromDevTime.set(0);
     cudaAllocCount.set(0);
@@ -187,14 +191,16 @@ public class GPUStatistics {
     sb.append("CUDA/CuLibraries init time:\t" + String.format("%.3f", 
cudaInitTime*1e-9) + "/"
             + String.format("%.3f", cudaLibrariesInitTime*1e-9) + " sec.\n");
     sb.append("Number of executed GPU inst:\t" + getNoOfExecutedGPUInst() + 
".\n");
-    sb.append("GPU mem tx time  (alloc/dealloc/toDev/fromDev):\t"
+    sb.append("GPU mem tx time  (alloc/dealloc/set0/toDev/fromDev):\t"
             + String.format("%.3f", cudaAllocTime.get()*1e-9) + "/"
             + String.format("%.3f", cudaDeAllocTime.get()*1e-9) + "/"
+            + String.format("%.3f", cudaMemSet0Time.get()*1e-9) + "/"
             + String.format("%.3f", cudaToDevTime.get()*1e-9) + "/"
             + String.format("%.3f", cudaFromDevTime.get()*1e-9)  + " sec.\n");
-    sb.append("GPU mem tx count (alloc/dealloc/toDev/fromDev/evict):\t"
+    sb.append("GPU mem tx count (alloc/dealloc/set0/toDev/fromDev/evict):\t"
             + cudaAllocCount.get() + "/"
             + cudaDeAllocCount.get() + "/"
+            + cudaMemSet0Count.get() + "/"
             + cudaSparseConversionCount.get() + "/"
             + cudaToDevCount.get() + "/"
             + cudaFromDevCount.get() + "/"

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/main/java/org/apache/sysml/utils/LRUCacheMap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/LRUCacheMap.java 
b/src/main/java/org/apache/sysml/utils/LRUCacheMap.java
new file mode 100644
index 0000000..0c8449a
--- /dev/null
+++ b/src/main/java/org/apache/sysml/utils/LRUCacheMap.java
@@ -0,0 +1,71 @@
+package org.apache.sysml.utils;
+
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * An access ordered LRU Cache Map which conforms to the {@link Map} interface
+ * while also providing the ability to get the least recently used entry
+ * @param <K> the key type
+ * @param <V> the value type
+ */
+public class LRUCacheMap<K,V> extends LinkedHashMap<K,V> {
+
+  /**
+   * Creates an access-ordered {@link LRUCacheMap}
+   */
+  public LRUCacheMap() {
+    // An access-ordered LinkedHashMap is instantiated with the default 
initial capacity and load factors
+    super(16, 0.75f, true);
+  }
+
+  // Private variables to assist in capturing the lease recently used entry
+  private boolean evictLRU = false;
+  private Map.Entry<K,V> lastEvictedEntry = null;
+
+  /**
+   * Removes and gets the least recently used entry
+   * @return  the lease recently used entry
+   * @throws DMLRuntimeException if the internal state is somehow corrupted
+   */
+  public Map.Entry<K,V> removeAndGetLRUEntry() throws DMLRuntimeException {
+    lastEvictedEntry = null;
+    if (size() <= 0){
+      return null;
+    }
+
+    // The idea is to set removing the eldest entry to true and then putting 
in a dummy
+    // entry (null, null). the removeEldestEntry will capture the eldest entry 
and is available
+    // to return via a class member variable.
+    evictLRU = true;
+    V previous = super.put(null, null);
+    remove(null);
+    if (previous != null){
+      throw new DMLRuntimeException("ERROR : Internal state of LRUCacheMap 
invalid - a value for the key 'null' is already present");
+    }
+    evictLRU = false;
+    Map.Entry<K,V> toRet = lastEvictedEntry;
+    return toRet;
+  }
+
+  @Override
+  protected boolean removeEldestEntry(Map.Entry<K,V> eldest) {
+    if (evictLRU) {
+      lastEvictedEntry = eldest;
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public V put (K k, V v){
+    if (k == null)
+      throw new IllegalArgumentException("ERROR: an entry with a null key was 
tried to be inserted in to the LRUCacheMap");
+    return super.put (k, v);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/80225f01/src/test/java/org/apache/sysml/test/utils/LRUCacheMapTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/utils/LRUCacheMapTest.java 
b/src/test/java/org/apache/sysml/test/utils/LRUCacheMapTest.java
new file mode 100644
index 0000000..d076266
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/utils/LRUCacheMapTest.java
@@ -0,0 +1,102 @@
+package org.apache.sysml.test.utils;
+
+import org.apache.sysml.utils.LRUCacheMap;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Map;
+
+public class LRUCacheMapTest {
+
+  @Test
+  public void test1() throws Exception {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put("k1", 10l);
+    m.put("k2", 20l);
+    m.put("k3", 30l);
+    m.put("k4", 40l);
+
+    Map.Entry<String, Long> e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k1", e.getKey());
+  }
+
+  @Test
+  public void test2() throws Exception {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put("k1", 10l);
+    m.put("k2", 20l);
+    m.put("k3", 30l);
+    m.put("k4", 40l);
+    m.get("k1");
+
+    Map.Entry<String, Long> e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k2", e.getKey());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void test3() {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put(null, 10l);
+  }
+
+  @Test
+  public void test4() throws Exception {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put("k1", 10l);
+    m.put("k2", 20l);
+    m.put("k3", 30l);
+    m.put("k4", 40l);
+    m.remove("k1");
+    m.remove("k2");
+
+    Map.Entry<String, Long> e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k3", e.getKey());
+  }
+
+  @Test
+  public void test5() throws Exception {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put("k1", 10l);
+    m.put("k2", 20l);
+    m.put("k1", 30l);
+
+    Map.Entry<String, Long> e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k2", e.getKey());
+  }
+
+  @Test
+  public void test6() throws Exception {
+    LRUCacheMap<String, Long> m = new LRUCacheMap<String, Long>();
+    m.put("k1", 10l);
+    m.put("k2", 20l);
+    m.put("k3", 30l);
+    m.put("k4", 40l);
+    m.put("k5", 50l);
+    m.put("k6", 60l);
+    m.put("k7", 70l);
+    m.put("k8", 80l);
+    m.get("k4");
+
+
+    Map.Entry<String, Long> e;
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k1", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k2", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k3", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k5", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k6", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k7", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k8", e.getKey());
+    e = m.removeAndGetLRUEntry();
+    Assert.assertEquals("k4", e.getKey());
+
+  }
+
+
+}
\ No newline at end of file

Reply via email to