You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by ni...@apache.org on 2018/08/06 16:45:12 UTC

systemml git commit: [SYSTEMML-445] Refactored the shadow buffer and added documentation for newly added features

Repository: systemml
Updated Branches:
  refs/heads/master 11f0291d7 -> a11933002


[SYSTEMML-445] Refactored the shadow buffer and added documentation for newly added features

- Refactored the shadow buffer logic from GPUObject to ShadowBuffer class for maintenance.
- Added an additional timer to measure shadow buffer time.
- Updated the gpu documentation


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

Branch: refs/heads/master
Commit: a11933002bfa8ba4d3e50b16f69c60bb36a270f6
Parents: 11f0291
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Mon Aug 6 09:40:08 2018 -0700
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Mon Aug 6 09:42:45 2018 -0700

----------------------------------------------------------------------
 conf/SystemML-config.xml.template               |   2 +-
 docs/gpu.md                                     |  28 +++-
 .../java/org/apache/sysml/api/DMLScript.java    |   2 +-
 .../java/org/apache/sysml/conf/DMLConfig.java   |   2 +-
 .../gpu/context/GPUMatrixMemoryManager.java     |   2 +-
 .../instructions/gpu/context/GPUObject.java     |  89 +++--------
 .../instructions/gpu/context/ShadowBuffer.java  | 146 +++++++++++++++++++
 .../org/apache/sysml/utils/GPUStatistics.java   |  14 +-
 8 files changed, 206 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/conf/SystemML-config.xml.template
----------------------------------------------------------------------
diff --git a/conf/SystemML-config.xml.template b/conf/SystemML-config.xml.template
index 3ce88c1..d773f79 100644
--- a/conf/SystemML-config.xml.template
+++ b/conf/SystemML-config.xml.template
@@ -97,7 +97,7 @@
     <sysml.floating.point.precision>double</sysml.floating.point.precision>
     
     <!-- the eviction policy for the GPU bufferpool. Supported values are lru, mru, lfu, min_evict, align_memory -->
-    <sysml.gpu.eviction.policy>align_memory</sysml.gpu.eviction.policy>
+    <sysml.gpu.eviction.policy>min_evict</sysml.gpu.eviction.policy>
     
    <!-- maximum wrap length for instruction and miscellaneous timer column of statistics -->
    <sysml.stats.maxWrapLength>30</sysml.stats.maxWrapLength>

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/docs/gpu.md
----------------------------------------------------------------------
diff --git a/docs/gpu.md b/docs/gpu.md
index e9d7bca..5e13e60 100644
--- a/docs/gpu.md
+++ b/docs/gpu.md
@@ -91,4 +91,30 @@ cd gcc-5.3.0
 num_cores=`grep -c ^processor /proc/cpuinfo`
 make -j $num_cores
 sudo make install
-```
\ No newline at end of file
+```
+
+# Advanced Configuration
+
+## Using single precision
+
+By default, SystemML uses double precision to store its matrices in the GPU memory.
+To use single precision, the user needs to set the configuration property 'sysml.floating.point.precision'
+to 'single'. However, with exception of BLAS operations, SystemML always performs all CPU operations
+in double precision.
+
+## Training very deep network
+
+### Shadow buffer
+To train very deep network with double precision, no additional configurations are necessary.
+But to train very deep network with single precision, the user can speed up the eviction by 
+using shadow buffer. The fraction of the driver memory to be allocated to the shadow buffer can  
+be set by using the configuration property 'sysml.gpu.eviction.shadow.bufferSize'.
+In the current version, the shadow buffer is currently not guarded by SystemML
+and can potentially lead to OOM if the network is deep as well as wide.
+
+### Unified memory allocator
+
+By default, SystemML uses CUDA's memory allocator and performs on-demand eviction
+using the eviction policy set by the configuration property 'sysml.gpu.eviction.policy'.
+To use CUDA's unified memory allocator that performs page-level eviction instead,
+please set the configuration property 'sysml.gpu.memory.allocator' to 'unified_memory'.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/src/main/java/org/apache/sysml/api/DMLScript.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/DMLScript.java b/src/main/java/org/apache/sysml/api/DMLScript.java
index d9413a8..9a6bb9b 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -121,7 +121,7 @@ public class DMLScript
 	public static ExplainType       EXPLAIN             = DMLOptions.defaultOptions.explainType; // explain type
 	public static String            DML_FILE_PATH_ANTLR_PARSER = DMLOptions.defaultOptions.filePath; // filename of dml/pydml script
 	public static String            FLOATING_POINT_PRECISION = "double";                         // data type to use internally
-	public static EvictionPolicy    GPU_EVICTION_POLICY = EvictionPolicy.ALIGN_MEMORY;           // currently employed GPU eviction policy
+	public static EvictionPolicy    GPU_EVICTION_POLICY = EvictionPolicy.MIN_EVICT;           	// currently employed GPU eviction policy
 	public static boolean           PRINT_GPU_MEMORY_INFO = false;                               // whether to print GPU memory-related information
 	public static long            	EVICTION_SHADOW_BUFFER_MAX_BYTES = 0;                         // maximum number of bytes to use for shadow buffer
 	public static long            	EVICTION_SHADOW_BUFFER_CURR_BYTES = 0;                        // number of bytes to use for shadow buffer

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/src/main/java/org/apache/sysml/conf/DMLConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/conf/DMLConfig.java b/src/main/java/org/apache/sysml/conf/DMLConfig.java
index 1333075..5b30609 100644
--- a/src/main/java/org/apache/sysml/conf/DMLConfig.java
+++ b/src/main/java/org/apache/sysml/conf/DMLConfig.java
@@ -143,7 +143,7 @@ public class DMLConfig
 		_defaultVals.put(GPU_MEMORY_UTILIZATION_FACTOR,      "0.9" );
 		_defaultVals.put(GPU_MEMORY_ALLOCATOR,	 "cuda");
 		_defaultVals.put(AVAILABLE_GPUS,         "-1");
-		_defaultVals.put(GPU_EVICTION_POLICY,    "align_memory");
+		_defaultVals.put(GPU_EVICTION_POLICY,    "min_evict");
 		_defaultVals.put(SYNCHRONIZE_GPU,        "false" );
 		_defaultVals.put(CACHING_BUFFER_SIZE,    "0.15" );
 		_defaultVals.put(EAGER_CUDA_FREE,        "false" );

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMatrixMemoryManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMatrixMemoryManager.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMatrixMemoryManager.java
index cbb8d4e..457968b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMatrixMemoryManager.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMatrixMemoryManager.java
@@ -52,7 +52,7 @@ public class GPUMatrixMemoryManager {
 	long getWorstCaseContiguousMemorySize(GPUObject gpuObj) {
 		long ret = 0;
 		if(!gpuObj.isDensePointerNull()) {
-			if(gpuObj.shadowPointer == null)
+			if(!gpuObj.shadowBuffer.isBuffered())
 				ret = gpuManager.allPointers.get(gpuObj.getDensePointer()).getSizeInBytes();
 			else
 				ret = 0; // evicted hence no contiguous memory on GPU

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/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 26cbd97..a783138 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
@@ -43,7 +43,6 @@ import org.apache.sysml.runtime.matrix.data.SparseBlockMCSR;
 import org.apache.sysml.utils.GPUStatistics;
 
 import jcuda.Pointer;
-import jcuda.Sizeof;
 import jcuda.jcusparse.cusparseDirection;
 import jcuda.jcusparse.cusparseHandle;
 import jcuda.jcusparse.cusparseMatDescr;
@@ -63,7 +62,7 @@ public class GPUObject {
 	/**
 	 * Pointer to the underlying dense matrix block on GPU
 	 */
-	private Pointer jcudaDenseMatrixPtr = null;
+	Pointer jcudaDenseMatrixPtr = null;
 
 	/**
 	 * Pointer to the underlying sparse matrix block on GPU
@@ -98,19 +97,12 @@ public class GPUObject {
 	/**
 	 * Enclosing {@link MatrixObject} instance
 	 */
-	protected MatrixObject mat = null;
+	MatrixObject mat = null;
 	
-	float[] shadowPointer = null;
-	private static boolean _warnedAboutShadowBuffer = false;
-	public boolean canFitIntoShadowBuffer() {
-		int numBytes = toIntExact(mat.getNumRows()*mat.getNumColumns())*Sizeof.FLOAT;
-		boolean ret = DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES + numBytes <= DMLScript.EVICTION_SHADOW_BUFFER_MAX_BYTES;
-		if(!ret && !_warnedAboutShadowBuffer) {
-			LOG.warn("Shadow buffer is full, so using CP bufferpool instead. Consider increasing sysml.gpu.eviction.shadow.bufferSize.");
-			_warnedAboutShadowBuffer = true;
-		}
-		return ret;
-	}
+	/**
+	 * Shadow buffer instance
+	 */
+	final ShadowBuffer shadowBuffer;
 	
 	// ----------------------------------------------------------------------
 	// Methods used to access, set and check jcudaDenseMatrixPtr
@@ -121,11 +113,8 @@ public class GPUObject {
 	 * @return a pointer to the dense matrix
 	 */
 	public Pointer getDensePointer() {
-		if(jcudaDenseMatrixPtr == null && shadowPointer != null && getJcudaSparseMatrixPtr() == null) {
-			long numBytes = shadowPointer.length*LibMatrixCUDA.sizeOfDataType;
-			jcudaDenseMatrixPtr = gpuContext.allocate(null, numBytes);
-			cudaMemcpy(jcudaDenseMatrixPtr, Pointer.to(shadowPointer), numBytes, jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice);
-			clearShadowPointer();
+		if(jcudaDenseMatrixPtr == null && shadowBuffer.isBuffered() && getJcudaSparseMatrixPtr() == null) {
+			shadowBuffer.moveToDevice();
 		}
 		return jcudaDenseMatrixPtr;
 	}
@@ -144,17 +133,7 @@ public class GPUObject {
 	 */
 	public void clearDensePointer() {
 		jcudaDenseMatrixPtr = null;
-		clearShadowPointer();
-	}
-	
-	/**
-	 * Removes shadow pointer
-	 */
-	public void clearShadowPointer() {
-		if(shadowPointer != null) {
-			DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES -= shadowPointer.length*Sizeof.FLOAT;
-		}
-		shadowPointer = null;
+		shadowBuffer.clearShadowPointer();
 	}
 	
 	
@@ -221,7 +200,7 @@ public class GPUObject {
 		getGPUContext().cudaFreeHelper(null, toFree, DMLScript.EAGER_CUDA_FREE);
 	}
 
-	private GPUContext getGPUContext() {
+	GPUContext getGPUContext() {
 		return gpuContext;
 	}
 
@@ -322,7 +301,7 @@ public class GPUObject {
 		}
 		this.jcudaSparseMatrixPtr = sparseMatrixPtr;
 		this.isSparse = true;
-		if (!isDensePointerNull() && shadowPointer == null) {
+		if (!isDensePointerNull() && !shadowBuffer.isBuffered()) {
 			cudaFreeHelper(getDensePointer());
 			clearDensePointer();
 		}
@@ -344,7 +323,7 @@ public class GPUObject {
 		int rows = toIntExact(mat.getNumRows());
 		int cols = toIntExact(mat.getNumColumns());
 
-		if ((isDensePointerNull() && shadowPointer == null) || !isAllocated())
+		if ((isDensePointerNull() && !shadowBuffer.isBuffered()) || !isAllocated())
 			throw new DMLRuntimeException("Expected allocated dense matrix before denseToSparse() call");
 
 		denseRowMajorToColumnMajor();
@@ -462,6 +441,7 @@ public class GPUObject {
 	GPUObject(GPUContext gCtx, MatrixObject mat2) {
 		gpuContext = gCtx;
 		this.mat = mat2;
+		this.shadowBuffer = new ShadowBuffer(this);
 	}
 
 	public boolean isSparse() {
@@ -477,7 +457,7 @@ public class GPUObject {
 	}
 
 	public boolean isAllocated() {
-		boolean eitherAllocated = shadowPointer != null || !isDensePointerNull() || getJcudaSparseMatrixPtr() != null;
+		boolean eitherAllocated = shadowBuffer.isBuffered() || !isDensePointerNull() || getJcudaSparseMatrixPtr() != null;
 		return eitherAllocated;
 	}
 
@@ -939,7 +919,7 @@ public class GPUObject {
 		if(LOG.isTraceEnabled()) {
 			LOG.trace("GPU : copyFromDeviceToHost, on " + this + ", GPUContext=" + getGPUContext());
 		}
-		if(shadowPointer != null) {
+		if(shadowBuffer.isBuffered()) {
 			if(isEviction) {
 				// If already copied to shadow buffer as part of previous eviction, do nothing.
 				return;
@@ -947,44 +927,13 @@ public class GPUObject {
 			else {
 				// If already copied to shadow buffer as part of previous eviction and this is not an eviction (i.e. bufferpool call for subsequent CP/Spark instruction),
 				// then copy from shadow buffer to MatrixObject.
-				long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
-				MatrixBlock tmp = new MatrixBlock(toIntExact(mat.getNumRows()), toIntExact(mat.getNumColumns()), false);
-				tmp.allocateDenseBlock();
-				double [] tmpArr = tmp.getDenseBlockValues();
-				for(int i = 0; i < shadowPointer.length; i++) {
-					tmpArr[i] = shadowPointer[i];
-				}
-				mat.acquireModify(tmp);
-				mat.release();
-				clearShadowPointer();
-				dirty = false;
-				if (DMLScript.STATISTICS) {
-					long totalTime = System.nanoTime() - start;
-					GPUStatistics.cudaFromShadowToHostTime.add(totalTime);
-					GPUStatistics.cudaFromShadowToHostCount.increment();
-					// Part of dev -> host, not eviction
-					GPUStatistics.cudaFromDevTime.add(totalTime);
-					GPUStatistics.cudaFromDevCount.increment();
-				}
+				shadowBuffer.moveToHost();
 				return;
 			}
 		}
-		else if(LibMatrixCUDA.sizeOfDataType == jcuda.Sizeof.FLOAT && isEviction && eagerDelete && !isDensePointerNull() && canFitIntoShadowBuffer()) {
+		else if(shadowBuffer.isEligibleForBuffering(isEviction, eagerDelete)) {
 			// Perform shadow buffering if (1) single precision, (2) during eviction, (3) for dense matrices, and (4) if the given matrix can fit into the shadow buffer. 
-			long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
-			int numElems = toIntExact(mat.getNumRows()*mat.getNumColumns());
-			shadowPointer = new float[numElems];
-			DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES += shadowPointer.length*Sizeof.FLOAT;
-			cudaMemcpy(Pointer.to(shadowPointer), jcudaDenseMatrixPtr, numElems*LibMatrixCUDA.sizeOfDataType, jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost);
-			getGPUContext().cudaFreeHelper(instName, jcudaDenseMatrixPtr, eagerDelete);
-			jcudaDenseMatrixPtr = null;
-			if (DMLScript.STATISTICS) {
-				// Eviction time measure in malloc
-				long totalTime = System.nanoTime() - start;
-				GPUStatistics.cudaFromDevToShadowTime.add(totalTime);
-				GPUStatistics.cudaFromDevToShadowCount.increment();
-				
-			}
+			shadowBuffer.moveFromDevice(instName);
 			return;
 		}
 		else if (isDensePointerNull() && getJcudaSparseMatrixPtr() == null) {
@@ -1059,7 +1008,7 @@ public class GPUObject {
 			getJcudaSparseMatrixPtr().deallocate(eager);
 		}
 		clearDensePointer();
-		clearShadowPointer();
+		shadowBuffer.clearShadowPointer();
 		jcudaSparseMatrixPtr = null;
 		resetReadWriteLock();
 		getGPUContext().getMemoryManager().removeGPUObject(this);

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
new file mode 100644
index 0000000..27a6256
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/ShadowBuffer.java
@@ -0,0 +1,146 @@
+/*
+ * 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.
+ */
+package org.apache.sysml.runtime.instructions.gpu.context;
+
+import static jcuda.runtime.JCuda.cudaMemcpy;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.utils.GPUStatistics;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+
+public class ShadowBuffer {
+	private static final Log LOG = LogFactory.getLog(ShadowBuffer.class.getName());
+	
+	GPUObject gpuObj;
+	float[] shadowPointer = null;
+	private static boolean _warnedAboutShadowBuffer = false;
+	
+	public ShadowBuffer(GPUObject gpuObj) {
+		this.gpuObj = gpuObj;
+	}
+	
+	/**
+	 * Check if the gpu object is shadow buffered
+	 * 
+	 * @return true if the gpu object is shadow buffered
+	 */
+	public boolean isBuffered() {
+		return shadowPointer != null;
+	}
+	
+	/**
+	 * Move the data from GPU to shadow buffer 
+	 * @param instName name of the instruction
+	 */
+	public void moveFromDevice(String instName) {
+		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		int numElems = GPUObject.toIntExact(gpuObj.mat.getNumRows()*gpuObj.mat.getNumColumns());
+		shadowPointer = new float[numElems];
+		DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES += shadowPointer.length*Sizeof.FLOAT;
+		cudaMemcpy(Pointer.to(shadowPointer), gpuObj.jcudaDenseMatrixPtr, numElems*LibMatrixCUDA.sizeOfDataType, jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost);
+		gpuObj.getGPUContext().cudaFreeHelper(instName, gpuObj.jcudaDenseMatrixPtr, true);
+		gpuObj.jcudaDenseMatrixPtr = null;
+		if (DMLScript.STATISTICS) {
+			// Eviction time measure in malloc
+			long totalTime = System.nanoTime() - start;
+			GPUStatistics.cudaFromDevToShadowTime.add(totalTime);
+			GPUStatistics.cudaFromDevToShadowCount.increment();
+			
+		}
+	}
+	
+	/**
+	 * Move the data from shadow buffer to Matrix object
+	 */
+	public void moveToHost() {
+		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		MatrixBlock tmp = new MatrixBlock(GPUObject.toIntExact(gpuObj.mat.getNumRows()), GPUObject.toIntExact(gpuObj.mat.getNumColumns()), false);
+		tmp.allocateDenseBlock();
+		double [] tmpArr = tmp.getDenseBlockValues();
+		for(int i = 0; i < shadowPointer.length; i++) {
+			tmpArr[i] = shadowPointer[i];
+		}
+		gpuObj.mat.acquireModify(tmp);
+		gpuObj.mat.release();
+		clearShadowPointer();
+		gpuObj.dirty = false;
+		if (DMLScript.STATISTICS) {
+			long totalTime = System.nanoTime() - start;
+			GPUStatistics.cudaFromShadowToHostTime.add(totalTime);
+			GPUStatistics.cudaFromShadowToHostCount.increment();
+			// Part of dev -> host, not eviction
+			GPUStatistics.cudaFromDevTime.add(totalTime);
+			GPUStatistics.cudaFromDevCount.increment();
+		}
+	}
+	
+	/**
+	 * Move the data from shadow buffer to GPU
+	 */
+	public void moveToDevice() {
+		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long numBytes = shadowPointer.length*LibMatrixCUDA.sizeOfDataType;
+		gpuObj.jcudaDenseMatrixPtr = gpuObj.getGPUContext().allocate(null, numBytes);
+		cudaMemcpy(gpuObj.jcudaDenseMatrixPtr, Pointer.to(shadowPointer), numBytes, jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice);
+		clearShadowPointer();
+		if (DMLScript.STATISTICS) {
+			long totalTime = System.nanoTime() - start;
+			GPUStatistics.cudaFromShadowToDevTime.add(totalTime);
+			GPUStatistics.cudaFromShadowToDevCount.increment();
+		}
+	}
+	
+	/**
+	 * Checks if the GPU object is eligible for shadow buffering
+	 * 
+	 * @param isEviction true if this method is called during eviction
+	 * @param eagerDelete true if the data on device has to be eagerly deleted
+	 * @return true if the given GPU object is eligible to be shadow buffered
+	 */
+	public boolean isEligibleForBuffering(boolean isEviction, boolean eagerDelete) {
+		if(LibMatrixCUDA.sizeOfDataType == jcuda.Sizeof.FLOAT && isEviction && eagerDelete && !gpuObj.isDensePointerNull()) {
+			int numBytes = GPUObject.toIntExact(gpuObj.mat.getNumRows()*gpuObj.mat.getNumColumns())*Sizeof.FLOAT;
+			boolean ret = DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES + numBytes <= DMLScript.EVICTION_SHADOW_BUFFER_MAX_BYTES;
+			if(!ret && !_warnedAboutShadowBuffer) {
+				LOG.warn("Shadow buffer is full, so using CP bufferpool instead. Consider increasing sysml.gpu.eviction.shadow.bufferSize.");
+				_warnedAboutShadowBuffer = true;
+			}
+			return ret;
+		}
+		else {
+			return false;
+		}
+	}
+	
+	/**
+	 * Removes the content from shadow buffer
+	 */
+	public void clearShadowPointer() {
+		if(shadowPointer != null) {
+			DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES -= shadowPointer.length*Sizeof.FLOAT;
+		}
+		shadowPointer = null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/a1193300/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 12abf21..fcbc4c4 100644
--- a/src/main/java/org/apache/sysml/utils/GPUStatistics.java
+++ b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
@@ -54,6 +54,7 @@ public class GPUStatistics {
 	public static LongAdder cudaToDevTime = new LongAdder();             // time spent in copying data from host (CPU) to device (GPU) memory
 	public static LongAdder cudaFromDevTime = new LongAdder();           // time spent in copying data from device to host
 	public static LongAdder cudaFromShadowToHostTime = new LongAdder();  // time spent in copying data from shadow to host
+	public static LongAdder cudaFromShadowToDevTime = new LongAdder();  // time spent in copying data from shadow to host
 	public static LongAdder cudaFromDevToShadowTime = new LongAdder();  // time spent in copying data from device to shadow
 	public static LongAdder cudaEvictTime = new LongAdder();           	 // time spent in eviction
 	public static LongAdder cudaEvictSizeTime = new LongAdder();         // time spent in eviction
@@ -68,6 +69,7 @@ public class GPUStatistics {
 	public static LongAdder cudaToDevCount = new LongAdder();
 	public static LongAdder cudaFromDevCount = new LongAdder();
 	public static LongAdder cudaFromShadowToHostCount = new LongAdder();
+	public static LongAdder cudaFromShadowToDevCount = new LongAdder();
 	public static LongAdder cudaFromDevToShadowCount = new LongAdder();
 	public static LongAdder cudaEvictCount = new LongAdder();
 	public static LongAdder cudaEvictSizeCount = new LongAdder();
@@ -104,6 +106,7 @@ public class GPUStatistics {
 		cudaToDevTime.reset();
 		cudaFromDevTime.reset();
 		cudaFromShadowToHostTime.reset();
+		cudaFromShadowToDevTime.reset();
 		cudaFromDevToShadowTime.reset();
 		cudaEvictTime.reset();
 		cudaEvictSizeTime.reset();
@@ -118,6 +121,7 @@ public class GPUStatistics {
 		cudaToDevCount.reset();
 		cudaFromDevCount.reset();
 		cudaFromShadowToHostCount.reset();
+		cudaFromShadowToDevCount.reset();
 		cudaFromDevToShadowCount.reset();
 		cudaEvictCount.reset();
 		cudaEvictSizeCount.reset();
@@ -238,18 +242,20 @@ public class GPUStatistics {
 				+ cudaAllocReuseCount.longValue() +") / "
 				+ cudaDeAllocCount.longValue() + " / "
 				+ cudaMemSet0Count.longValue() + ".\n");
-		sb.append("GPU mem tx time  (toDev(d2f) / fromDev(f2d/s2h) / evict(d2s/size)):\t"
+		sb.append("GPU mem tx time  (toDev(d2f/s2d) / fromDev(f2d/s2h) / evict(d2s/size)):\t"
 				+ String.format("%.3f", cudaToDevTime.longValue()*1e-9) + "("
-				+ String.format("%.3f", cudaDouble2FloatTime.longValue()*1e-9)+ ") / "
+				+ String.format("%.3f", cudaDouble2FloatTime.longValue()*1e-9)+ "/"
+				+ String.format("%.3f", cudaFromShadowToDevTime.longValue()*1e-9) + ") / "
 				+ String.format("%.3f", cudaFromDevTime.longValue()*1e-9) + "("
 				+ String.format("%.3f", cudaFloat2DoubleTime.longValue()*1e-9) + "/"
 				+ String.format("%.3f", cudaFromShadowToHostTime.longValue()*1e-9) + ") / "
 				+ String.format("%.3f", cudaEvictTime.longValue()*1e-9) + "("
 				+ String.format("%.3f", cudaFromDevToShadowTime.longValue()*1e-9) + "/"
 				+ String.format("%.3f", cudaEvictSizeTime.longValue()*1e-9) + ") sec.\n");
-		sb.append("GPU mem tx count (toDev(d2f) / fromDev(f2d/s2h) / evict(d2s/size)):\t"
+		sb.append("GPU mem tx count (toDev(d2f/s2d) / fromDev(f2d/s2h) / evict(d2s/size)):\t"
 				+ cudaToDevCount.longValue() + "("
-				+ cudaDouble2FloatCount.longValue() + ") / "
+				+ cudaDouble2FloatCount.longValue() + "/" 
+				+ cudaFromShadowToDevCount.longValue() + ") / "
 				+ cudaFromDevCount.longValue() + "("
 				+ cudaFloat2DoubleCount.longValue() + "/"
 				+ cudaFromShadowToHostCount.longValue() + ") / "