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 2017/11/14 19:07:10 UTC

systemml git commit: [MINOR] Merge extra GPU and extra DNN flags into single "sysml.stats.finegrained" flag to simplify usage

Repository: systemml
Updated Branches:
  refs/heads/master 3da574684 -> de69afdc8


[MINOR] Merge extra GPU and extra DNN flags into single "sysml.stats.finegrained" flag to simplify usage

Closes #701.


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

Branch: refs/heads/master
Commit: de69afdc84ad99e2f2a41ac03f69f2f1e64e921f
Parents: 3da5746
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Tue Nov 14 11:06:18 2017 -0800
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Tue Nov 14 11:06:18 2017 -0800

----------------------------------------------------------------------
 conf/SystemML-config.xml.template               |   8 +-
 .../apache/sysml/api/ScriptExecutorUtils.java   |   6 +-
 .../java/org/apache/sysml/conf/DMLConfig.java   |   6 +-
 .../instructions/gpu/GPUInstruction.java        |   8 +-
 .../instructions/gpu/context/CSRPointer.java    |   4 +-
 .../instructions/gpu/context/GPUContext.java    |  10 +-
 .../instructions/gpu/context/GPUObject.java     |  14 +-
 .../DoublePrecisionCudaSupportFunctions.java    |   9 +-
 .../runtime/matrix/data/LibMatrixCUDA.java      | 148 +++++++++----------
 .../runtime/matrix/data/LibMatrixCuDNN.java     |  53 +++----
 .../LibMatrixCuDNNConvolutionAlgorithm.java     |  17 ++-
 .../data/LibMatrixCuDNNInputRowFetcher.java     |   6 +-
 .../runtime/matrix/data/LibMatrixCuMatMult.java |  20 +--
 .../sysml/runtime/matrix/data/LibMatrixDNN.java |  12 +-
 .../LibMatrixDNNConv2dBackwardDataHelper.java   |  10 +-
 .../LibMatrixDNNConv2dBackwardFilterHelper.java |  20 +--
 .../matrix/data/LibMatrixDNNConv2dHelper.java   |  20 +--
 .../SinglePrecisionCudaSupportFunctions.java    |   9 +-
 .../org/apache/sysml/utils/GPUStatistics.java   |   5 +-
 .../java/org/apache/sysml/utils/Statistics.java |   8 +-
 20 files changed, 190 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/conf/SystemML-config.xml.template
----------------------------------------------------------------------
diff --git a/conf/SystemML-config.xml.template b/conf/SystemML-config.xml.template
index 8452e75..8a4a5d6 100644
--- a/conf/SystemML-config.xml.template
+++ b/conf/SystemML-config.xml.template
@@ -75,15 +75,9 @@
    <!-- enables native blas for matrix multiplication and convolution, experimental feature (options: auto, mkl, openblas, none) -->
    <sysml.native.blas>none</sysml.native.blas>
 
-   <!-- prints finegrained statistics information -->
+   <!-- prints finegrained statistics information (includes extra GPU information and extra statistics information for Deep Neural Networks done in CP mode) -->
    <sysml.stats.finegrained>false</sysml.stats.finegrained>
    
-   <!-- prints extra statistics information for GPU -->
-   <sysml.stats.extraGPU>false</sysml.stats.extraGPU>
-
-   <!-- prints extra statistics information for Deep Neural Networks done in CP mode -->
-   <sysml.stats.extraDNN>false</sysml.stats.extraDNN>
-
     <!-- sets the GPUs to use per process, -1 for all GPUs, a specific GPU number (5), a range (eg: 0-2) or a comma separated list (eg: 0,2,4)-->
     <sysml.gpu.availableGPUs>-1</sysml.gpu.availableGPUs>
     

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
index 51ab6a1..cb39340 100644
--- a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
+++ b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
@@ -30,8 +30,6 @@ import org.apache.sysml.runtime.controlprogram.Program;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContextPool;
-import org.apache.sysml.runtime.matrix.data.LibMatrixDNN;
-import org.apache.sysml.utils.GPUStatistics;
 import org.apache.sysml.utils.Statistics;
 
 public class ScriptExecutorUtils {
@@ -75,9 +73,7 @@ public class ScriptExecutorUtils {
 			throws DMLRuntimeException {
 		// Whether extra statistics useful for developers and others interested
 		// in digging into performance problems are recorded and displayed
-		GPUStatistics.DISPLAY_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_GPU_STATS);
-		LibMatrixDNN.DISPLAY_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_DNN_STATS);
-		DMLScript.FINEGRAINED_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_FINEGRAINED_STATS);
+		DMLScript.FINEGRAINED_STATISTICS = DMLScript.STATISTICS && dmlconf.getBooleanValue(DMLConfig.EXTRA_FINEGRAINED_STATS);
 		DMLScript.SYNCHRONIZE_GPU = dmlconf.getBooleanValue(DMLConfig.SYNCHRONIZE_GPU);
 		DMLScript.EAGER_CUDA_FREE = dmlconf.getBooleanValue(DMLConfig.EAGER_CUDA_FREE);
 		DMLScript.STATISTICS_MAX_WRAP_LEN = dmlconf.getIntValue(DMLConfig.STATS_MAX_WRAP_LEN);

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 e8bde56..42037a1 100644
--- a/src/main/java/org/apache/sysml/conf/DMLConfig.java
+++ b/src/main/java/org/apache/sysml/conf/DMLConfig.java
@@ -84,8 +84,6 @@ public class DMLConfig
 	
 	public static final String EXTRA_FINEGRAINED_STATS = "sysml.stats.finegrained"; //boolean
 	public static final String STATS_MAX_WRAP_LEN   = "sysml.stats.maxWrapLength"; //int
-	public static final String EXTRA_GPU_STATS      = "sysml.stats.extraGPU"; //boolean
-	public static final String EXTRA_DNN_STATS      = "sysml.stats.extraDNN"; //boolean
 	public static final String AVAILABLE_GPUS       = "sysml.gpu.availableGPUs"; // String to specify which GPUs to use (a range, all GPUs, comma separated list or a specific GPU)
 	public static final String SYNCHRONIZE_GPU      = "sysml.gpu.sync.postProcess"; // boolean: whether to synchronize GPUs after every instruction 
 	public static final String EAGER_CUDA_FREE		= "sysml.gpu.eager.cudaFree"; // boolean: whether to perform eager CUDA free on rmvar
@@ -134,8 +132,6 @@ public class DMLConfig
 		_defaultVals.put(NATIVE_BLAS,            "none" );
 		_defaultVals.put(EXTRA_FINEGRAINED_STATS,"false" );
 		_defaultVals.put(STATS_MAX_WRAP_LEN,     "30" );
-		_defaultVals.put(EXTRA_GPU_STATS,        "false" );
-		_defaultVals.put(EXTRA_DNN_STATS,        "false" );
 		_defaultVals.put(GPU_MEMORY_UTILIZATION_FACTOR,      "0.9" );
 		_defaultVals.put(AVAILABLE_GPUS,         "-1");
 		_defaultVals.put(SYNCHRONIZE_GPU,        "true" );
@@ -422,7 +418,7 @@ public class DMLConfig
 				CP_PARALLEL_OPS, CP_PARALLEL_IO, NATIVE_BLAS,
 				COMPRESSED_LINALG, 
 				CODEGEN, CODEGEN_COMPILER, CODEGEN_OPTIMIZER, CODEGEN_PLANCACHE, CODEGEN_LITERALS,
-				EXTRA_GPU_STATS, EXTRA_DNN_STATS, EXTRA_FINEGRAINED_STATS, STATS_MAX_WRAP_LEN,
+				EXTRA_FINEGRAINED_STATS, STATS_MAX_WRAP_LEN,
 				AVAILABLE_GPUS, SYNCHRONIZE_GPU, EAGER_CUDA_FREE, FLOATING_POINT_PRECISION
 		}; 
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 f5d2f46..1883582 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
@@ -200,9 +200,9 @@ public abstract class GPUInstruction extends Instruction {
 					throws DMLRuntimeException
 	{
 		if(DMLScript.SYNCHRONIZE_GPU) {
-			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			jcuda.runtime.JCuda.cudaDeviceSynchronize();
-			if(GPUStatistics.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				GPUStatistics.maintainCPMiscTimes(getExtendedOpcode(), GPUInstruction.MISC_TIMER_CUDA_SYNC, System.nanoTime() - t0);
 			}
 		}
@@ -238,9 +238,9 @@ public abstract class GPUInstruction extends Instruction {
 	 * @throws DMLRuntimeException	if an error occurs
 	 */
 	protected MatrixObject getDenseMatrixOutputForGPUInstruction(ExecutionContext ec, String name, long numRows, long numCols) throws DMLRuntimeException {
-		long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		Pair<MatrixObject, Boolean> mb = ec.getDenseMatrixOutputForGPUInstruction(name, numRows, numCols);
-		if (GPUStatistics.DISPLAY_STATISTICS && mb.getValue()) GPUStatistics.maintainCPMiscTimes(getExtendedOpcode(), GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS && mb.getValue()) GPUStatistics.maintainCPMiscTimes(getExtendedOpcode(), GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
 		return mb.getKey();
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
index 53f1a19..d165970 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CSRPointer.java
@@ -497,7 +497,7 @@ public class CSRPointer {
 	 */
 	public Pointer toColumnMajorDenseMatrix(cusparseHandle cusparseHandle, cublasHandle cublasHandle, int rows,
 			int cols, String instName) throws DMLRuntimeException {
-		long t0 = GPUStatistics.DISPLAY_STATISTICS && instName != null ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS && instName != null ? System.nanoTime() : 0;
 		LOG.trace("GPU : sparse -> column major dense (inside CSRPointer) on " + this + ", GPUContext="
 				+ getGPUContext());
 		long size = ((long) rows) * getDataTypeSizeOf((long) cols);
@@ -510,7 +510,7 @@ public class CSRPointer {
 		} else {
 			LOG.debug("in CSRPointer, the values array, row pointers array or column indices array was null");
 		}
-		if (GPUStatistics.DISPLAY_STATISTICS && instName != null) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS && instName != null) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
 		return A;
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 0a2f6c4..4b47117 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
@@ -298,7 +298,7 @@ public class GPUContext {
 						"GPU : in allocate from instruction " + instructionName + ", found free block of size " + (size
 								/ 1024.0) + " Kbytes from previously allocated block on " + this);
 			}
-			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+			if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 				t0 = System.nanoTime();
 			Set<Pointer> freeList = freeCUDASpaceMap.get(size);
 
@@ -308,7 +308,7 @@ public class GPUContext {
 
 			if (freeList.isEmpty())
 				freeCUDASpaceMap.remove(size);
-			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+			if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics
 						.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_REUSE, System.nanoTime() - t0);
 		} else {
@@ -326,7 +326,7 @@ public class GPUContext {
 				GPUStatistics.cudaAllocTime.add(System.nanoTime() - t0);
 			if (DMLScript.STATISTICS)
 				GPUStatistics.cudaAllocCount.add(statsCount);
-			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+			if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_ALLOCATE,
 						System.nanoTime() - t0);
 		}
@@ -340,7 +340,7 @@ public class GPUContext {
 		cudaMemset(A, 0, size);
 		if (DMLScript.STATISTICS)
 			end = System.nanoTime();
-		if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+		if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_SET_ZERO, end - t1);
 		if (DMLScript.STATISTICS)
 			GPUStatistics.cudaMemSet0Time.add(end - t1);
@@ -413,7 +413,7 @@ public class GPUContext {
 				GPUStatistics.cudaDeAllocTime.add(System.nanoTime() - t0);
 			if (DMLScript.STATISTICS)
 				GPUStatistics.cudaDeAllocCount.add(1);
-			if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+			if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_CUDA_FREE,
 						System.nanoTime() - t0);
 		} else {

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 7e01166..c4a16fc 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
@@ -401,7 +401,7 @@ public class GPUObject {
 		denseColumnMajorToRowMajor();
 		if (DMLScript.STATISTICS)
 			end = System.nanoTime();
-		if (instructionName != null && GPUStatistics.DISPLAY_STATISTICS)
+		if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, end - start);
 		if (DMLScript.STATISTICS)
 			GPUStatistics.cudaSparseToDenseTime.add(end - start);
@@ -756,9 +756,9 @@ public class GPUObject {
 		if (DMLScript.STATISTICS)
 			start = System.nanoTime();
 
-		long acqrTime = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long acqrTime = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		MatrixBlock tmp = mat.acquireRead();
-		if(GPUStatistics.DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			if(tmp.isInSparseFormat())
 				GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_GET_SPARSE_MB, System.nanoTime()-acqrTime);
 			else
@@ -826,10 +826,10 @@ public class GPUObject {
 			allocateSparseMatrixOnDevice();
 
 			if (copyToDevice) {
-				long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				CSRPointer.copyToDevice(getGPUContext(), getJcudaSparseMatrixPtr(), tmp.getNumRows(), tmp.getNonZeros(), rowPtr, colInd,
 						values);
-				if(GPUStatistics.DISPLAY_STATISTICS) 
+				if(DMLScript.FINEGRAINED_STATISTICS) 
 					GPUStatistics.maintainCPMiscTimes(opcode, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, System.nanoTime() - t1);
 			}
 		} else {
@@ -845,9 +845,9 @@ public class GPUObject {
 			if (tmp.getNonZeros() == 0) {
 				// Minor optimization: No need to allocate empty error for CPU 
 				// data = new double[tmp.getNumRows() * tmp.getNumColumns()];
-				long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				cudaMemset(getJcudaDenseMatrixPtr(), 0, getDatatypeSizeOf(mat.getNumRows() * mat.getNumColumns()));
-				if(GPUStatistics.DISPLAY_STATISTICS) 
+				if(DMLScript.FINEGRAINED_STATISTICS) 
 					GPUStatistics.maintainCPMiscTimes(opcode, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t1);
 			}
 			else {

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/DoublePrecisionCudaSupportFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/DoublePrecisionCudaSupportFunctions.java b/src/main/java/org/apache/sysml/runtime/matrix/data/DoublePrecisionCudaSupportFunctions.java
index be5ade7..fb70c13 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/DoublePrecisionCudaSupportFunctions.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/DoublePrecisionCudaSupportFunctions.java
@@ -22,6 +22,7 @@ import static jcuda.runtime.JCuda.cudaMemcpy;
 import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyDeviceToHost;
 import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
 
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
@@ -159,17 +160,17 @@ public class DoublePrecisionCudaSupportFunctions implements CudaSupportFunctions
 
 	@Override
 	public void deviceToHost(GPUContext gCtx, Pointer src, double[] dest, String instName, boolean isEviction) throws DMLRuntimeException {
-		long t1 = GPUStatistics.DISPLAY_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
 		cudaMemcpy(Pointer.to(dest), src, ((long)dest.length)*Sizeof.DOUBLE, cudaMemcpyDeviceToHost);
-		if(GPUStatistics.DISPLAY_STATISTICS && instName != null) 
+		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_HOST, System.nanoTime() - t1);
 	}
 
 	@Override
 	public void hostToDevice(GPUContext gCtx, double[] src, Pointer dest, String instName) throws DMLRuntimeException {
-		long t1 = GPUStatistics.DISPLAY_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
 		cudaMemcpy(dest, Pointer.to(src), ((long)src.length)*Sizeof.DOUBLE, cudaMemcpyHostToDevice);
-		if(GPUStatistics.DISPLAY_STATISTICS && instName != null) 
+		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, System.nanoTime() - t1);
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 fd99eb3..59a9e49 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
@@ -315,11 +315,11 @@ public class LibMatrixCUDA {
 		Pointer outputPointer = getDensePointer(gCtx, outputBlock, instName);
 
 		long t1=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("relu_backward",
 				ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
 				imagePointer, doutPointer, outputPointer, toInt(rows), toInt(cols));
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_BACKWARD_KERNEL, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_BACKWARD_KERNEL, System.nanoTime() - t1);
 
 	}
 	
@@ -387,11 +387,11 @@ public class LibMatrixCUDA {
 		Pointer biasPointer = bias.getGPUObject(gCtx).getJcudaDenseMatrixPtr();
 		Pointer outputPointer = outputBlock.getGPUObject(gCtx).getJcudaDenseMatrixPtr();
 		long t1 = 0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("bias_multiply",
 				ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(rows), toInt(cols)),
 				imagePointer, biasPointer, outputPointer, toInt(rows), toInt(cols), toInt(PQ));
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_BIAS_ADD_LIB, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_BIAS_ADD_LIB, System.nanoTime() - t1);
 
 	}
 
@@ -441,11 +441,11 @@ public class LibMatrixCUDA {
 		}
 		int PQ = cols / k;
 		long t1 = 0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("bias_add",
 				ExecutionConfig.getConfigForSimpleMatrixOperations(rows, cols),
 				image, bias, output, rows, cols, PQ);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_BIAS_ADD_LIB, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_BIAS_ADD_LIB, System.nanoTime() - t1);
 	}
 	
 
@@ -512,13 +512,13 @@ public class LibMatrixCUDA {
 
 		long t0=0, t1=0;
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		cudaSupportFunctions.cublassyrk(getCublasHandle(gCtx), cublasFillMode.CUBLAS_FILL_MODE_LOWER,transa, m, k, one(), A, lda, zero(), C, ldc);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SYRK_LIB, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SYRK_LIB, System.nanoTime() - t0);
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		copyUpperToLowerTriangle(gCtx, instName, output);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_UPPER_TO_LOWER_TRIANGLE_KERNEL, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_UPPER_TO_LOWER_TRIANGLE_KERNEL, System.nanoTime() - t1);
 	}
 
 	/**
@@ -930,19 +930,19 @@ public class LibMatrixCUDA {
 
 		long t1=0,t2=0;
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem), in, tempOut, n);
 		//cudaDeviceSynchronize;
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ALL_KERNEL, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ALL_KERNEL, System.nanoTime() - t1);
 
 		int s = blocks;
 		while (s > 1) {
 			tmp = getKernelParamsForReduceAll(gCtx, s);
 			blocks = tmp[0]; threads = tmp[1]; sharedMem = tmp[2];
-			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t2 = System.nanoTime();
 			getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
 					tempOut, tempOut, s);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ALL_KERNEL, System.nanoTime() - t2);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ALL_KERNEL, System.nanoTime() - t2);
 			s = (s + (threads*2-1)) / (threads*2);
 		}
 		double[] result = {-1f};
@@ -971,11 +971,11 @@ public class LibMatrixCUDA {
 		int blocks = tmp[0], threads = tmp[1], sharedMem = tmp[2];
 
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
 				in, out, rows, cols);
 		//cudaDeviceSynchronize;
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ROW_KERNEL, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_ROW_KERNEL, System.nanoTime() - t0);
 
 	}
 
@@ -999,11 +999,11 @@ public class LibMatrixCUDA {
 		int blocks = tmp[0], threads = tmp[1], sharedMem = tmp[2];
 
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel(kernelFunction, new ExecutionConfig(blocks, threads, sharedMem),
 				in, out, rows, cols);
 		//cudaDeviceSynchronize;
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_COL_KERNEL, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_REDUCE_COL_KERNEL, System.nanoTime() - t0);
 	}
 
 	/**
@@ -1329,11 +1329,11 @@ public class LibMatrixCUDA {
 		int isLeftScalar = (op instanceof LeftScalarOperator) ? 1 : 0;
 		int size = rlenA * clenA;
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("matrix_scalar_op",
 				ExecutionConfig.getConfigForSimpleVectorOperations(size),
 				a, scalar, c, size, getBinaryOp(op.fn), isLeftScalar);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_SCALAR_OP_KERNEL, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_SCALAR_OP_KERNEL, System.nanoTime() - t0);
 	}
 
 	/**
@@ -1433,11 +1433,11 @@ public class LibMatrixCUDA {
 			LOG.trace("GPU : matrix_matrix_cellwise_op" + ", GPUContext=" + gCtx);
 		}
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		getCudaKernels(gCtx).launchKernel("matrix_matrix_cellwise_op",
 				ExecutionConfig.getConfigForSimpleMatrixOperations(maxRlen, maxClen),
 				a, b, c, maxRlen, maxClen, vecStatusA, vecStatusB, getBinaryOp(op.fn));
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_MATRIX_CELLWISE_OP_KERNEL, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MATRIX_MATRIX_CELLWISE_OP_KERNEL, System.nanoTime() - t0);
 	}
 
 	/**
@@ -1529,11 +1529,11 @@ public class LibMatrixCUDA {
 			int rlen = toInt(out.getNumRows());
 			int clen = toInt(out.getNumColumns());
 			long t0 = 0;
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				t0 = System.nanoTime();
 			int size = rlen * clen;
 			getCudaKernels(gCtx).launchKernel("fill", ExecutionConfig.getConfigForSimpleVectorOperations(size), A, constant, size);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_FILL_KERNEL, System.nanoTime() - t0);
 		}
 	}
@@ -1549,10 +1549,10 @@ public class LibMatrixCUDA {
 	 */
 	private static void deviceCopy(String instName, Pointer src, Pointer dest, int rlen, int clen) throws DMLRuntimeException {
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		int size = rlen * clen * sizeOfDataType;
 		cudaMemcpy(dest, src, size, cudaMemcpyDeviceToDevice);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_DEVICE, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_DEVICE, System.nanoTime() - t0);
 	}
 
 	/**
@@ -1633,19 +1633,19 @@ public class LibMatrixCUDA {
 			// Invoke cuSparse when either are in sparse format
 			// Perform sparse-sparse dgeam
 			if (!isInSparseFormat(gCtx, in1)) {
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					t0 = System.nanoTime();
 				in1.getGPUObject(gCtx).denseToSparse();
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE,
 							System.nanoTime() - t0);
 			}
 			CSRPointer A = in1.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
 			if (!isInSparseFormat(gCtx, in2)) {
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					t0 = System.nanoTime();
 				in2.getGPUObject(gCtx).denseToSparse();
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_TO_SPARSE,
 							System.nanoTime() - t0);
 			}
@@ -1668,21 +1668,21 @@ public class LibMatrixCUDA {
 							"Transpose in cusparseDcsrgeam not supported for sparse matrices on GPU");
 				}
 
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					t1 = System.nanoTime();
 				CSRPointer C = CSRPointer.allocateForDgeam(gCtx, getCusparseHandle(gCtx), A, B, m, n);
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_ALLOCATE_LIB,
 							System.nanoTime() - t1);
 
 				out.getGPUObject(gCtx).setSparseMatrixCudaPointer(C);
 				//long sizeOfC = CSRPointer.estimateSize(C.nnz, out.getNumRows());
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					t0 = System.nanoTime();
 				cudaSupportFunctions.cusparsecsrgeam(getCusparseHandle(gCtx), m, n, alphaPtr, A.descr, toInt(A.nnz), A.val, A.rowPtr, A.colInd, betaPtr,
 						B.descr, toInt(B.nnz), B.val, B.rowPtr, B.colInd, C.descr, C.val, C.rowPtr, C.colInd);
 				//cudaDeviceSynchronize;
-				if (GPUStatistics.DISPLAY_STATISTICS)
+				if (DMLScript.FINEGRAINED_STATISTICS)
 					GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_DGEAM_LIB,
 							System.nanoTime() - t0);
 			}
@@ -1709,9 +1709,9 @@ public class LibMatrixCUDA {
 			getDenseMatrixOutputForGPUInstruction(ec, instName, outputName, outRLen, outCLen);	// Allocated the dense output matrix
 			Pointer C = getDensePointer(gCtx, out, instName);
 
-			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 			cudaSupportFunctions.cublasgeam(getCublasHandle(gCtx), transa, transb, m, n, alphaPtr, A, lda, betaPtr, B, ldb, C, ldc);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_DGEAM_LIB, System.nanoTime() - t0);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DENSE_DGEAM_LIB, System.nanoTime() - t0);
 		}
 	}
 
@@ -1823,7 +1823,7 @@ public class LibMatrixCUDA {
 	 */
 	protected static void sliceDenseDense(GPUContext gCtx, String instName, Pointer inPointer, Pointer outPointer, 
 			int rl, int ru, int cl, int cu, int inClen) throws DMLRuntimeException {
-		long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		long retClen = cu - cl + 1;
 		if (inClen == retClen) {
 			cudaMemcpy(outPointer, inPointer.withByteOffset(rl * inClen * sizeOfDataType), (ru - rl + 1) * inClen
@@ -1833,7 +1833,7 @@ public class LibMatrixCUDA {
 			getCudaKernels(gCtx).launchKernel("slice_dense_dense", ExecutionConfig.getConfigForSimpleVectorOperations(toInt(retRlen*retClen)),
 					inPointer, outPointer, rl, ru, cl, cu, inClen,  retRlen, retClen);
 		}
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RIX_DENSE_OP, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RIX_DENSE_OP, System.nanoTime() - t0);
 	}
 	
 	/**
@@ -1857,7 +1857,7 @@ public class LibMatrixCUDA {
 		if(size == 0) return;
 		
 		int retRlen = ru - rl + 1;
-		long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		int retClen = cu - cl + 1;
 		
 		String kernel = null; String timer = null;
@@ -1879,7 +1879,7 @@ public class LibMatrixCUDA {
 		// We can generalize this later to output sparse matrix.
 		getCudaKernels(gCtx).launchKernel(kernel, ExecutionConfig.getConfigForSimpleVectorOperations(size),
 				inPointer.val, inPointer.rowPtr, inPointer.colInd, outPointer, rl, ru, cl, cu, retClen);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, timer, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, timer, System.nanoTime() - t0);
 	}
 	
 	/**
@@ -1924,11 +1924,11 @@ public class LibMatrixCUDA {
 		int maxRows = toInt(Math.max(rowsA, rowsB));
 		int maxCols = toInt(Math.max(colsA, colsB));
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx)
 		.launchKernel("cbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
 				rowsA, colsA, rowsB, colsB);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CBIND_KERNEL, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CBIND_KERNEL, System.nanoTime() - t1);
 
 	}
 
@@ -1959,11 +1959,11 @@ public class LibMatrixCUDA {
 		int maxRows = Math.max(rowsA, rowsB);
 		int maxCols = Math.max(colsA, colsB);
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 		getCudaKernels(gCtx)
 		.launchKernel("rbind", ExecutionConfig.getConfigForSimpleMatrixOperations(maxRows, maxCols), A, B, C,
 				rowsA, colsA, rowsB, colsB);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RBIND_KERNEL, System.nanoTime() - t1);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RBIND_KERNEL, System.nanoTime() - t1);
 
 	}
 
@@ -2295,10 +2295,10 @@ public class LibMatrixCUDA {
 			Pointer output = getDensePointer(gCtx, out, instName);
 			Pointer input = getDensePointer(gCtx, in1, instName);
 			int size = toInt(in1.getNumColumns() * in1.getNumRows());
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 			getCudaKernels(gCtx).launchKernel(kernel, ExecutionConfig.getConfigForSimpleVectorOperations(size),
 					input, output, size);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, kernelTimer, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, kernelTimer, System.nanoTime() - t1);
 		}
 	}
 
@@ -2337,13 +2337,13 @@ public class LibMatrixCUDA {
 			// becomes
 			// C <- A
 			// C <- alpha*B + C
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 			cudaMemcpy(C, A, n*((long)sizeOfDataType), cudaMemcpyDeviceToDevice);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_DEVICE, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_DEVICE, System.nanoTime() - t1);
 
-			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t2 = System.nanoTime();
 			cudaSupportFunctions.cublasaxpy(getCublasHandle(gCtx), toInt(n), alphaPtr, B, 1, C, 1);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DAXPY_LIB, System.nanoTime() - t2);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DAXPY_LIB, System.nanoTime() - t2);
 		}
 		else {
 			if(LOG.isTraceEnabled()) {
@@ -2353,12 +2353,12 @@ public class LibMatrixCUDA {
 			// Matrix-Vector daxpy
 			// Note: Vector-Matrix operation is not supported
 			// daxpy_matrix_vector(double* A,  double* B, double alpha, double* ret, int rlenA, int clenA, int rlenB, int clenB)
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 			int rlenA = toInt(in1.getNumRows()); int clenA =  toInt(in1.getNumColumns());
 			int rlenB = toInt(in2.getNumRows()); int clenB =  toInt(in2.getNumColumns());
 			getCudaKernels(gCtx).launchKernel("daxpy_matrix_vector", ExecutionConfig.getConfigForSimpleMatrixOperations(rlenA, clenA),
 					A, B, constant, C, rlenA, clenA, rlenB, clenB);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DAXPY_MV_KERNEL, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DAXPY_MV_KERNEL, System.nanoTime() - t1);
 		}
 	}
 
@@ -2406,20 +2406,20 @@ public class LibMatrixCUDA {
 		// convert dense matrices to row major
 		// Operation in cuSolver and cuBlas are for column major dense matrices
 		// and are destructive to the original input
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		GPUObject ATobj = (GPUObject) Aobj.clone();
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		ATobj.denseRowMajorToColumnMajor();
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
 		Pointer A = ATobj.getJcudaDenseMatrixPtr();
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		GPUObject bTobj = (GPUObject) bobj.clone();
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_OBJECT_CLONE, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		bTobj.denseRowMajorToColumnMajor();
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ROW_TO_COLUMN_MAJOR, System.nanoTime() - t0);
 
 
 		Pointer b = bTobj.getJcudaDenseMatrixPtr();
@@ -2428,18 +2428,18 @@ public class LibMatrixCUDA {
 		// http://docs.nvidia.com/cuda/cusolver/#ormqr-example1
 
 		// step 3: query working space of geqrf and ormqr
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		int[] lwork = {0};
 		cudaSupportFunctions.cusolverDngeqrf_bufferSize(gCtx.getCusolverDnHandle(), m, n, A, m, lwork);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR_BUFFER, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR_BUFFER, System.nanoTime() - t0);
 
 		// step 4: compute QR factorization
 		Pointer work = gCtx.allocate(instName, lwork[0] * sizeOfDataType);
 		Pointer tau = gCtx.allocate(instName, m * sizeOfDataType);
 		Pointer devInfo = gCtx.allocate(Sizeof.INT);
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		cudaSupportFunctions.cusolverDngeqrf(gCtx.getCusolverDnHandle(), m, n, A, m, tau, work, lwork[0], devInfo);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_QR, System.nanoTime() - t0);
 
 		int[] qrError = {-1};
 		cudaMemcpy(Pointer.to(qrError), devInfo, Sizeof.INT, cudaMemcpyDeviceToHost);
@@ -2448,24 +2448,24 @@ public class LibMatrixCUDA {
 		}
 
 		// step 5: compute Q^T*B
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		cudaSupportFunctions.cusolverDnormqr(gCtx.getCusolverDnHandle(), cublasSideMode.CUBLAS_SIDE_LEFT, cublasOperation.CUBLAS_OP_T, m, 1, n, A, m, tau, b, m, work, lwork[0], devInfo);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ORMQR, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ORMQR, System.nanoTime() - t0);
 		cudaMemcpy(Pointer.to(qrError), devInfo, Sizeof.INT, cudaMemcpyDeviceToHost);
 		if (qrError[0] != 0) {
 			throw new DMLRuntimeException("GPU : Error in call to ormqr (to compuete Q^T*B after QR factorization) as part of solve, argument " + qrError[0] + " was wrong");
 		}
 
 		// step 6: compute x = R \ Q^T*B
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		cudaSupportFunctions.cublastrsm(gCtx.getCublasHandle(),
 			cublasSideMode.CUBLAS_SIDE_LEFT, cublasFillMode.CUBLAS_FILL_MODE_UPPER, cublasOperation.CUBLAS_OP_N, cublasDiagType.CUBLAS_DIAG_NON_UNIT,
 			n, 1, dataTypePointerTo(1.0), A, m, b, m);
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_TRSM, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_TRSM, System.nanoTime() - t0);
 
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		bTobj.denseColumnMajorToRowMajor();
-		if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_COLUMN_TO_ROW_MAJOR, System.nanoTime() - t0);
+		if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_COLUMN_TO_ROW_MAJOR, System.nanoTime() - t0);
 
 		// TODO  : Find a way to assign bTobj directly to the output and set the correct flags so as to not crash
 		// There is an avoidable copy happening here
@@ -2497,10 +2497,10 @@ public class LibMatrixCUDA {
 	 */
 	protected static MatrixObject getDenseMatrixOutputForGPUInstruction(ExecutionContext ec, String instName, String name, long numRows, long numCols) throws DMLRuntimeException {
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		Pair<MatrixObject, Boolean> mb = ec.getDenseMatrixOutputForGPUInstruction(name, numRows, numCols);
 		if (mb.getValue())
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
 		return mb.getKey();
 	}
@@ -2519,10 +2519,10 @@ public class LibMatrixCUDA {
 	 */
 	private static MatrixObject getSparseMatrixOutputForGPUInstruction(ExecutionContext ec, long numRows, long numCols, long nnz, String instName, String name) throws DMLRuntimeException {
 		long t0=0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 		Pair<MatrixObject, Boolean> mb = ec.getSparseMatrixOutputForGPUInstruction(name, numRows, numCols, nnz);
 		if (mb.getValue())
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ALLOCATE_SPARSE_OUTPUT, System.nanoTime() - t0);
 		return mb.getKey();
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
index 5935285..4e23953 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNN.java
@@ -43,6 +43,7 @@ import jcuda.jcudnn.cudnnTensorDescriptor;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
@@ -220,13 +221,13 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 		}
 		try {
 			long t1 = 0;
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 			int status = cudnnConvolutionForward(getCudnnHandle(gCtx), one(),
 					algo.nchwTensorDesc, image,
 					algo.filterDesc, filter,
 					algo.convDesc, algo.algo, algo.workSpace, algo.sizeInBytes, zero(),
 					algo.nkpqTensorDesc, output);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_FORWARD_LIB, System.nanoTime() - t1);
 			if (status != cudnnStatus.CUDNN_STATUS_SUCCESS) {
 				throw new DMLRuntimeException("Could not executed cudnnConvolutionForward: " + cudnnStatus.stringFor(status));
@@ -292,9 +293,9 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 						// Perform one-input conv2dBackwardFilter
 						Pointer tempdwPointer = gCtx.allocate(KCRS*sizeOfDataType);
 						for(int n = 0; n < N; n++) {
-							long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+							long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 							cudaMemset(tempdwPointer, 0, KCRS*sizeOfDataType);
-							if(GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t0);
+							if(DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t0);
 							// Perform one-input conv2dBackwardFilter
 							cudnnConv2dBackwardFilter(gCtx, instName, imgFetcher.getNthRow(n), doutFetcher.getNthRow(n), tempdwPointer, algo);
 							getCudaKernels(gCtx).launchKernel("inplace_add",
@@ -330,10 +331,10 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 			LOG.trace("GPU : conv2dBackwardFilter" + ", GPUContext=" + gCtx);
 		}
 		try {
-			long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			int status = cudnnConvolutionBackwardFilter(getCudnnHandle(gCtx), one(), algo.nchwTensorDesc, imagePointer,
 					algo.nkpqTensorDesc, doutPointer, algo.convDesc, algo.algo, algo.workSpace, algo.sizeInBytes, zero(), algo.filterDesc, dwPointer);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_BACKWARD_FILTER_LIB, System.nanoTime() - t1);
 			if (status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
 				throw new DMLRuntimeException("Could not executed cudnnConvolutionBackwardFilter: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
@@ -424,10 +425,10 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 			LOG.trace("GPU : conv2dBackwardData" + ", GPUContext=" + gCtx);
 		}
 		try {
-			long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			int status = cudnnConvolutionBackwardData(getCudnnHandle(gCtx), one(), algo.filterDesc, w,
 					algo.nkpqTensorDesc, dy, algo.convDesc, algo.algo, algo.workSpace, algo.sizeInBytes, zero(), algo.nchwTensorDesc, dx);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_BACKWARD_DATA_LIB, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CONVOLUTION_BACKWARD_DATA_LIB, System.nanoTime() - t1);
 
 			if(status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
 				throw new DMLRuntimeException("Could not executed cudnnConvolutionBackwardData: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
@@ -499,11 +500,11 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 				LibMatrixCuDNNPoolingDescriptors.cudnnMaxpoolingDescriptors(gCtx, instName, N, C, H, W, K, R, S, 
 						pad_h, pad_w, stride_h, stride_w, P, Q)) {
 			long t1=0,t2=0;
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
-			if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) t2 = System.nanoTime();
 			int status = cudnnPoolingForward(getCudnnHandle(gCtx), desc.poolingDesc, one(), desc.xDesc, x, zero(), desc.yDesc, y);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_FORWARD_LIB, System.nanoTime() - t2);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_FORWARD_LIB, System.nanoTime() - t2);
 			if(status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
 				throw new DMLRuntimeException("Could not executed cudnnPoolingForward: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
 			}
@@ -598,20 +599,20 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 			long t1=0, t2=0, t3=0;
 			int status;
 			if(!isMaxPoolOutputProvided) {
-				if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
+				if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
 				long numBytes = N*C*P*Q*sizeOfDataType;
 				y = gCtx.allocate(numBytes);
-				if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
-				if (GPUStatistics.DISPLAY_STATISTICS) t2 = System.nanoTime();
+				if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
+				if (DMLScript.FINEGRAINED_STATISTICS) t2 = System.nanoTime();
 				status = cudnnPoolingForward(getCudnnHandle(gCtx), desc.poolingDesc, one(), desc.xDesc, x, zero(), desc.yDesc, y);
-				if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_FORWARD_LIB, System.nanoTime() - t2);
+				if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_FORWARD_LIB, System.nanoTime() - t2);
 				if(status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
 					throw new DMLRuntimeException("Could not executed cudnnPoolingForward before cudnnPoolingBackward: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
 				}
 			}
-			if (GPUStatistics.DISPLAY_STATISTICS) t3 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t3 = System.nanoTime();
 			status = cudnnPoolingBackward(getCudnnHandle(gCtx), desc.poolingDesc, one(), desc.yDesc, y, desc.dyDesc, dy, desc.xDesc, x, zero(), desc.dxDesc, dx);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_BACKWARD_LIB, System.nanoTime() - t3);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_MAXPOOLING_BACKWARD_LIB, System.nanoTime() - t3);
 
 			if(status != jcuda.jcudnn.cudnnStatus.CUDNN_STATUS_SUCCESS) {
 				throw new DMLRuntimeException("Could not executed cudnnPoolingBackward: " + jcuda.jcudnn.cudnnStatus.stringFor(status));
@@ -621,10 +622,10 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 		}
 		finally {
 			long t4=0;
-			if (GPUStatistics.DISPLAY_STATISTICS) t4 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t4 = System.nanoTime();
 			if(!isMaxPoolOutputProvided)
 				gCtx.cudaFreeHelper(instName, y);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_CLEANUP, System.nanoTime() - t4);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_CLEANUP, System.nanoTime() - t4);
 		}
 	}
 
@@ -641,18 +642,18 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 			cudnnCreateActivationDescriptor(activationDescriptor);
 			double dummy = -1;
 			cudnnSetActivationDescriptor(activationDescriptor, CUDNN_ACTIVATION_RELU, CUDNN_PROPAGATE_NAN, dummy);
-			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 			cudnnActivationForward(getCudnnHandle(gCtx), activationDescriptor,
 					one(), srcTensorDesc, srcData,
 					zero(), dstTensorDesc, dstData);
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ACTIVATION_FORWARD_LIB, System.nanoTime() - t0);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_ACTIVATION_FORWARD_LIB, System.nanoTime() - t0);
 		} catch (CudaException e) {
 			throw new DMLRuntimeException("Error in conv2d in GPUContext " + gCtx.toString() + " from Thread " + Thread.currentThread().toString(), e);
 		}
 		finally {
 			long t1=0;
-			if (GPUStatistics.DISPLAY_STATISTICS) t1 = System.nanoTime();
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_CLEANUP, System.nanoTime() - t1);
+			if (DMLScript.FINEGRAINED_STATISTICS) t1 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_CLEANUP, System.nanoTime() - t1);
 		}
 	}
 
@@ -678,13 +679,13 @@ public class LibMatrixCuDNN extends LibMatrixCUDA {
 				LOG.trace("GPU : relu custom kernel" + ", GPUContext=" + gCtx);
 			}
 			// Invokes relu(double* A,  double* ret, int rlen, int clen)
-			if (GPUStatistics.DISPLAY_STATISTICS) t0 = System.nanoTime();
+			if (DMLScript.FINEGRAINED_STATISTICS) t0 = System.nanoTime();
 			Pointer dstData = getDensePointerForCuDNN(gCtx, output, instName);
 			Pointer srcData = getDensePointerForCuDNN(gCtx, in, instName); // TODO: FIXME: Add sparse kernel support for relu
 			getCudaKernels(gCtx).launchKernel("relu",
 					ExecutionConfig.getConfigForSimpleMatrixOperations(toInt(N), toInt(CHW)),
 					srcData, dstData, toInt(N), toInt(CHW));
-			if (GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_KERNEL, System.nanoTime() - t0);
+			if (DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_RELU_KERNEL, System.nanoTime() - t0);
 		}
 		else {
 			cudnnTensorDescriptor tensorDescriptor = new cudnnTensorDescriptor();

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNConvolutionAlgorithm.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNConvolutionAlgorithm.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNConvolutionAlgorithm.java
index ee22541..17cd610 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNConvolutionAlgorithm.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNConvolutionAlgorithm.java
@@ -19,6 +19,7 @@
 
 package org.apache.sysml.runtime.matrix.data;
 
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
@@ -85,7 +86,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 	@Override
 	public void close() {
 		long t3 = 0;
-		if (GPUStatistics.DISPLAY_STATISTICS) t3 = System.nanoTime();
+		if (DMLScript.FINEGRAINED_STATISTICS) t3 = System.nanoTime();
 		if(nchwTensorDesc != null)
 			cudnnDestroyTensorDescriptor(nchwTensorDesc);
 		if(nkpqTensorDesc != null)
@@ -96,7 +97,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 			cudnnDestroyConvolutionDescriptor(convDesc);
 		if(sizeInBytes != 0)
 			gCtx.cudaFreeHelper(instName, workSpace);
-		if(GPUStatistics.DISPLAY_STATISTICS)
+		if(DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_CLEANUP, System.nanoTime() - t3);
 	}
 	
@@ -125,7 +126,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 	public static LibMatrixCuDNNConvolutionAlgorithm cudnnGetConvolutionForwardAlgorithm(
 			GPUContext gCtx, String instName, int N, int C, int H, int W, int K, int R, int S, 
 			int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q, long workspaceLimit) throws DMLRuntimeException {
-		long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		LibMatrixCuDNNConvolutionAlgorithm ret = new LibMatrixCuDNNConvolutionAlgorithm(gCtx, instName, N, C, H, W, K, R, S, 
 				pad_h, pad_w, stride_h, stride_w, P, Q);
 		int[] algos = {-1};
@@ -139,7 +140,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 			ret.workSpace = gCtx.allocate(sizeInBytesArray[0]);
 		ret.sizeInBytes = sizeInBytesArray[0];
 		ret.algo = algos[0];
-		if (GPUStatistics.DISPLAY_STATISTICS)
+		if (DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 		return ret;
 	}
@@ -169,7 +170,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 	public static LibMatrixCuDNNConvolutionAlgorithm cudnnGetConvolutionBackwardFilterAlgorithm(
 			GPUContext gCtx, String instName, int N, int C, int H, int W, int K, int R, int S, 
 			int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q, long workspaceLimit) throws DMLRuntimeException {
-		long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		LibMatrixCuDNNConvolutionAlgorithm ret = new LibMatrixCuDNNConvolutionAlgorithm(gCtx, instName, N, C, H, W, K, R, S, 
 				pad_h, pad_w, stride_h, stride_w, P, Q);
 		
@@ -186,7 +187,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 		ret.sizeInBytes = sizeInBytesArray[0];
 		ret.algo = algos[0];
 		
-		if (GPUStatistics.DISPLAY_STATISTICS)
+		if (DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 		return ret;
 	}
@@ -216,7 +217,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 	public static LibMatrixCuDNNConvolutionAlgorithm cudnnGetConvolutionBackwardDataAlgorithm(
 			GPUContext gCtx, String instName, int N, int C, int H, int W, int K, int R, int S, 
 			int pad_h, int pad_w, int stride_h, int stride_w, int P, int Q, long workspaceLimit) throws DMLRuntimeException {
-		//long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		//long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		LibMatrixCuDNNConvolutionAlgorithm ret = new LibMatrixCuDNNConvolutionAlgorithm(gCtx, instName, N, C, H, W, K, R, S, 
 				pad_h, pad_w, stride_h, stride_w, P, Q);
 		
@@ -237,7 +238,7 @@ public class LibMatrixCuDNNConvolutionAlgorithm implements java.lang.AutoCloseab
 //			ret.workSpace = gCtx.allocate(sizeInBytesArray[0]);
 //		ret.sizeInBytes = sizeInBytesArray[0];
 //		ret.algo = algos[0];
-//		if (GPUStatistics.DISPLAY_STATISTICS)
+//		if (DMLScript.FINEGRAINED_STATISTICS)
 //			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_CUDNN_INIT, System.nanoTime() - t1);
 		return ret;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNInputRowFetcher.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNInputRowFetcher.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNInputRowFetcher.java
index 5121c87..5a7cad3 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNInputRowFetcher.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuDNNInputRowFetcher.java
@@ -20,6 +20,8 @@ package org.apache.sysml.runtime.matrix.data;
 
 import static jcuda.runtime.JCuda.cudaMemset;
 import jcuda.Pointer;
+
+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.instructions.gpu.GPUInstruction;
@@ -59,10 +61,10 @@ public class LibMatrixCuDNNInputRowFetcher extends LibMatrixCUDA implements java
 	public Pointer getNthRow(int n) throws DMLRuntimeException {
 		if(isInputInSparseFormat) {
 			jcuda.runtime.JCuda.cudaDeviceSynchronize();
-			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			cudaMemset(outPointer, 0, numColumns*sizeOfDataType);
 			jcuda.runtime.JCuda.cudaDeviceSynchronize();
-			if(GPUStatistics.DISPLAY_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t0);
+			if(DMLScript.FINEGRAINED_STATISTICS) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t0);
 			LibMatrixCUDA.sliceSparseDense(gCtx, instName, (CSRPointer)inPointer, outPointer, n, n, 0, LibMatrixCUDA.toInt(numColumns-1), numColumns);
 		}
 		else {

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
index d962027..ce0ad5b 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixCuMatMult.java
@@ -164,19 +164,19 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			// and output
 			CSRPointer A = left.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
 			CSRPointer B = right.getGPUObject(gCtx).getJcudaSparseMatrixPtr();
-			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			CSRPointer C = CSRPointer.allocateForMatrixMultiply(gCtx, getCusparseHandle(gCtx), A, transa, B, transb,
 					params.m, params.n, params.k);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_ALLOCATE_LIB,
 						System.nanoTime() - t0);
 
 			// Step 3: Invoke the kernel
-			long t1 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			cudaSupportFunctions.cusparsecsrgemm(getCusparseHandle(gCtx), transa, transb, params.m, params.n, params.k, A.descr,
 					(int) A.nnz, A.val, A.rowPtr, A.colInd, B.descr, (int) B.nnz, B.val, B.rowPtr, B.colInd, C.descr,
 					C.val, C.rowPtr, C.colInd);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_MATRIX_SPARSE_MATRIX_LIB,
 						System.nanoTime() - t1);
 			output.getGPUObject(gCtx).setSparseMatrixCudaPointer(C);
@@ -284,14 +284,14 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 		denseSparseMatMult(getCusparseHandle(gCtx), instName, output, B, A, params);
 		if (outRLen != 1 && outCLen != 1) {
 			// Transpose: C = t(output)
-			long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+			long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			cudaSupportFunctions.cublasgeam(gCtx.getCublasHandle(), cublasOperation.CUBLAS_OP_T, cublasOperation.CUBLAS_OP_T,
 					toInt(outCLen), toInt(outRLen), one(), output, toInt(outRLen), zero(), new Pointer(),
 					toInt(outRLen), C, toInt(outCLen));
 			if (!DMLScript.EAGER_CUDA_FREE)
 				JCuda.cudaDeviceSynchronize();
 			gCtx.cudaFreeHelper(output, DMLScript.EAGER_CUDA_FREE);
-			if (GPUStatistics.DISPLAY_STATISTICS)
+			if (DMLScript.FINEGRAINED_STATISTICS)
 				GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_TRANSPOSE_LIB, System.nanoTime()
 						- t0);
 		}
@@ -319,7 +319,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 	 */
 	private static void denseSparseMatMult(cusparseHandle handle, String instName, Pointer C, Pointer A, CSRPointer B,
 			CuMatMultParameters param) throws DMLRuntimeException {
-		long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		String kernel = GPUInstruction.MISC_TIMER_SPARSE_MATRIX_DENSE_MATRIX_LIB;
 		// Ignoring sparse vector dense matrix multiplication and dot product
 		boolean isVector = (param.leftNumRows == 1 && !param.isLeftTransposed)
@@ -343,7 +343,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 			cudaSupportFunctions.cusparsecsrmm2(handle, transa, transb, m, param.n, k, toInt(B.nnz), one(), B.descr, B.val,
 					B.rowPtr, B.colInd, A, param.ldb, zero(), C, param.ldc);
 		}
-		if (GPUStatistics.DISPLAY_STATISTICS)
+		if (DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instName, kernel, System.nanoTime() - t0);
 	}
 
@@ -370,7 +370,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 	 */
 	private static void denseDenseMatMult(cublasHandle handle, String instName, Pointer C, Pointer A, Pointer B,
 			CuMatMultParameters param) throws DMLRuntimeException {
-		long t0 = GPUStatistics.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		String kernel = null;
 		param.rowToColumnMajor();
 		param.validate();
@@ -412,7 +412,7 @@ public class LibMatrixCuMatMult extends LibMatrixCUDA {
 					zero(), C, param.ldc);
 			kernel = GPUInstruction.MISC_TIMER_DENSE_MATRIX_DENSE_MATRIX_LIB;
 		}
-		if (GPUStatistics.DISPLAY_STATISTICS)
+		if (DMLScript.FINEGRAINED_STATISTICS)
 			GPUStatistics.maintainCPMiscTimes(instName, kernel, System.nanoTime() - t0);
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
index 096574a..67d4a1a 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNN.java
@@ -68,8 +68,6 @@ public class LibMatrixDNN {
 	protected static final Log LOG =  LogFactory.getLog(LibMatrixDNN.class.getName());
 	
 	//library configurations and external contracts
-	public static boolean DISPLAY_STATISTICS = false; //conv2d summaries in stats output
-	
 	// ------------------------------------------------------------------------------------------------
 	private static AtomicLong conv2dSparseCount = new AtomicLong(0);
 	private static AtomicLong conv2dDenseCount = new AtomicLong(0);
@@ -89,7 +87,7 @@ public class LibMatrixDNN {
 	static AtomicLong loopedConvBwdDataCol2ImTime = new AtomicLong(0);
 	
 	public static void appendStatistics(StringBuilder sb) {
-		if(DMLScript.STATISTICS && DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			sb.append("LibMatrixDNN dense count (conv/bwdF/bwdD/im2col/maxBwd):\t" 
 					+ conv2dDenseCount.get() + "/"
 					+ conv2dBwdFilterDenseCount.get() + "/"
@@ -230,7 +228,7 @@ public class LibMatrixDNN {
 		if(params.stride_h <= 0 || params.stride_w <= 0) 
 			throw new DMLRuntimeException("Only positive strides supported:" + params.stride_h + ", " + params.stride_w);
 		
-		if(DMLScript.STATISTICS && DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			if(filter.isInSparseFormat() || dout.isInSparseFormat()) {
 				conv2dBwdDataSparseCount.addAndGet(1);
 			}
@@ -255,7 +253,7 @@ public class LibMatrixDNN {
 		if(params.stride_h <= 0 || params.stride_w <= 0) 
 			throw new DMLRuntimeException("Only positive strides supported:" + params.stride_h + ", " + params.stride_w);
 		
-		if(DMLScript.STATISTICS && DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			if(input.isInSparseFormat() || dout.isInSparseFormat()) {
 				conv2dBwdFilterSparseCount.addAndGet(1);
 			}
@@ -281,7 +279,7 @@ public class LibMatrixDNN {
 		if(params.stride_h <= 0 || params.stride_w <= 0) 
 			throw new DMLRuntimeException("Only positive strides supported:" + params.stride_h + ", " + params.stride_w);
 		
-		if(DMLScript.STATISTICS && DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			if(input.isInSparseFormat() || filter.isInSparseFormat()) {
 				conv2dSparseCount.addAndGet(1);
 			}
@@ -314,7 +312,7 @@ public class LibMatrixDNN {
 			throw new DMLRuntimeException("Incorrect dout dimensions in maxpooling_backward:" + input.getNumRows() + " " + input.getNumColumns() + " " + params.N + " " + params.K*params.P*params.Q);
 		}
 		
-		if(DMLScript.STATISTICS && DISPLAY_STATISTICS) {
+		if(DMLScript.FINEGRAINED_STATISTICS) {
 			if(input.isInSparseFormat() || dout.isInSparseFormat()) {
 				maxPoolBwdSparseCount.addAndGet(1);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
index 960cea6..55f263f 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardDataHelper.java
@@ -89,20 +89,20 @@ public class LibMatrixDNNConv2dBackwardDataHelper {
 				// rotate180(dout[n,]) => dout_reshaped
 				rotate180Worker.execute(n, 0);
 				// dout_reshaped %*% filter => temp
-				long t1 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				outMM.reset(PQ, CRS, false);
 				LibMatrixDNNHelper.singleThreadedMatMult(outRotate, filter, outMM, !outRotate.sparse, false, _params);
-				long t2 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t2 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				// col2im(temp) => output[n,] 
 				LibMatrixDNNHelper.doCol2imOverSingleImage(n, outMM, _params);
-				long t3 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t3 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
-				if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+				if(DMLScript.FINEGRAINED_STATISTICS) {
 					time1 += t2 - t1;
 					time2 += t3 - t2;
 				}
 			}
-			if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				LibMatrixDNN.loopedConvBwdDataMatMultTime.addAndGet(time1);
 				LibMatrixDNN.loopedConvBwdDataCol2ImTime.addAndGet(time2);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardFilterHelper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardFilterHelper.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardFilterHelper.java
index 9698725..4a94838 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardFilterHelper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dBackwardFilterHelper.java
@@ -99,24 +99,24 @@ public class LibMatrixDNNConv2dBackwardFilterHelper {
 				rotate180Worker.execute(n, 0);
 				
 				// im2col(input) => _im2ColOutBlock
-				long t1 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				im2ColWorker.execute(n);
-				long t2 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t2 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
 				outMM.reset(CRS, K, false);
 				LibMatrixDNNHelper.singleThreadedMatMult(im2ColOutBlock, outRotate, outMM, !im2ColOutBlock.sparse, !outRotate.sparse, _params);
-				long t3 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t3 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
 				if( !outMM.isEmptyBlock() ) //accumulate row results
 					LibMatrixMult.vectAdd(outMM.getDenseBlock(), partRet, 0, 0, K*CRS);
 				
-				if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+				if(DMLScript.FINEGRAINED_STATISTICS) {
 					time1 += t2 - t1;
 					time2 += t3 - t2;
 				}
 			}
 			inplaceTransAdd(partRet, _params);
-			if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				LibMatrixDNN.loopedConvBwdFilterIm2ColTime.addAndGet(time1);
 				LibMatrixDNN.loopedConvBwdFilterMatMultTime.addAndGet(time2);
 			}
@@ -150,27 +150,27 @@ public class LibMatrixDNNConv2dBackwardFilterHelper {
 				rotate180Worker.execute(n, 0);
 				
 				// im2col(input) => _im2ColOutBlock
-				long t1 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				im2ColWorker.execute(n);
-				long t2 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t2 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
 				outMM.reset(K, CRS, false);
 				//Timing time = new Timing(true);
 				LibMatrixDNNHelper.singleThreadedMatMult(outRotate, im2ColOutBlock, 
 					outMM, !outRotate.sparse, !im2ColOutBlock.sparse, _params);
-				long t3 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t3 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
 				if( !outMM.isEmptyBlock() ) //accumulate row results
 					LibMatrixMult.vectAdd(outMM.getDenseBlock(), partRet, 0, 0, K*CRS);
 				
-				if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+				if(DMLScript.FINEGRAINED_STATISTICS) {
 					time1 += t2 - t1;
 					time2 += t3 - t2;
 				}
 			}
 			//no need to transpose because t(t(out)) cancel out
 			inplaceAdd(partRet, _params);
-			if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				LibMatrixDNN.loopedConvBwdFilterIm2ColTime.addAndGet(time1);
 				LibMatrixDNN.loopedConvBwdFilterMatMultTime.addAndGet(time2);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dHelper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dHelper.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dHelper.java
index 6a0205e..b390906 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dHelper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixDNNConv2dHelper.java
@@ -56,16 +56,16 @@ public class LibMatrixDNNConv2dHelper {
 			for(int n = _rl; n < _ru; n++)  {
 				for(int c = 0; c < _params.C; c++)  {
 					// im2col(input) => _im2ColOutBlock
-					long t1 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+					long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 					im2ColWorker.execute(n, c);
-					long t2 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+					long t2 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 					
 					// filter %*% _im2ColOutBlock => matMultOutBlock
 					MatrixBlock matMultOutBlock = new MatrixBlock(K, PQ, false);
 					LibMatrixDNNHelper.singleThreadedMatMult(_filters.get(c), im2ColOutBlock, matMultOutBlock, false, true, _params);
-					long t3 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+					long t3 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 					
-					if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+					if(DMLScript.FINEGRAINED_STATISTICS) {
 						time1 += t2 - t1;
 						time2 += t3 - t2;
 					}
@@ -77,7 +77,7 @@ public class LibMatrixDNNConv2dHelper {
 				if(_params.bias != null)
 					LibMatrixDNNHelper.addBias(n, _params.output.getDenseBlock(), _params.bias.getDenseBlock(), K, PQ);
 			}
-			if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				LibMatrixDNN.loopedConvIm2ColTime.addAndGet(time1);
 				LibMatrixDNN.loopedConvMatMultTime.addAndGet(time2);
 			}
@@ -137,16 +137,16 @@ public class LibMatrixDNNConv2dHelper {
 			long time1 = 0; long time2 = 0;
 			for(int n = _rl; n < _ru; n++)  {
 				// im2col(input) => _im2ColOutBlock
-				long t1 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				im2ColWorker.execute(n);
-				long t2 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t2 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
 				// filter %*% _im2ColOutBlock => matMultOutBlock
 				outMM.reset(outMM.rlen, outMM.clen, false);
 				LibMatrixDNNHelper.singleThreadedMatMult(_params.input2, outIm2col, outMM, false, true, _params);
-				long t3 = DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS ? System.nanoTime() : 0;
+				long t3 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 				
-				if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+				if(DMLScript.FINEGRAINED_STATISTICS) {
 					time1 += t2 - t1;
 					time2 += t3 - t2;
 				}
@@ -159,7 +159,7 @@ public class LibMatrixDNNConv2dHelper {
 					LibMatrixDNNHelper.addBias(n, _params.output.getDenseBlock(), _params.bias.getDenseBlock(), K, PQ);
 			}
 			
-			if(DMLScript.STATISTICS && LibMatrixDNN.DISPLAY_STATISTICS) {
+			if(DMLScript.FINEGRAINED_STATISTICS) {
 				LibMatrixDNN.loopedConvIm2ColTime.addAndGet(time1);
 				LibMatrixDNN.loopedConvMatMultTime.addAndGet(time2);
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java b/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
index 96a3887..8429d35 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/SinglePrecisionCudaSupportFunctions.java
@@ -24,6 +24,7 @@ import static jcuda.runtime.cudaMemcpyKind.cudaMemcpyHostToDevice;
 
 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.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
@@ -163,7 +164,7 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 	
 	@Override
 	public void deviceToHost(GPUContext gCtx, Pointer src, double[] dest, String instName, boolean isEviction) throws DMLRuntimeException {
-		long t1 = GPUStatistics.DISPLAY_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
 		// We invoke transfer matrix from device to host in two cases:
 		// 1. During eviction of unlocked matrices
 		// 2. During acquireHostRead
@@ -187,7 +188,7 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 				dest[i] = floatData[i];
 			}
 		}
-		if(GPUStatistics.DISPLAY_STATISTICS && instName != null) 
+		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_DEVICE_TO_HOST, System.nanoTime() - t1);
 	}
 
@@ -195,7 +196,7 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 	public void hostToDevice(GPUContext gCtx, double[] src, Pointer dest, String instName) throws DMLRuntimeException {
 		LOG.debug("Potential OOM: Allocated additional space in hostToDevice");
 		// TODO: Perform conversion on GPU using double2float and float2double kernels
-		long t1 = GPUStatistics.DISPLAY_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
 		if(PERFORM_CONVERSION_ON_DEVICE) {
 			Pointer deviceDoubleData = gCtx.allocate(((long)src.length)*Sizeof.DOUBLE);
 			cudaMemcpy(deviceDoubleData, Pointer.to(src), ((long)src.length)*Sizeof.DOUBLE, cudaMemcpyHostToDevice);
@@ -210,7 +211,7 @@ public class SinglePrecisionCudaSupportFunctions implements CudaSupportFunctions
 			cudaMemcpy(dest, Pointer.to(floatData), ((long)src.length)*Sizeof.FLOAT, cudaMemcpyHostToDevice);
 		}
 		
-		if(GPUStatistics.DISPLAY_STATISTICS && instName != null) 
+		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, System.nanoTime() - t1);
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/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 2668ad9..33ab953 100644
--- a/src/main/java/org/apache/sysml/utils/GPUStatistics.java
+++ b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
@@ -35,9 +35,6 @@ import org.apache.sysml.api.DMLScript;
  * Printed as part of {@link Statistics}.
  */
 public class GPUStatistics {
-	// Whether or not extra per-instruction statistics will be recorded and shown for the GPU
-	public static boolean DISPLAY_STATISTICS = false;
-
 	private static int iNoOfExecutedGPUInst = 0;
 
 	public static long cudaInitTime = 0;
@@ -117,7 +114,7 @@ public class GPUStatistics {
 	 */
 	public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos, long incrementCount)
 	{
-		if (!(DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS))
+		if (!(DMLScript.FINEGRAINED_STATISTICS))
 			return;
 
 		HashMap<String, Long> miscTimesMap = _cpInstMiscTime.get(instructionName);

http://git-wip-us.apache.org/repos/asf/systemml/blob/de69afdc/src/main/java/org/apache/sysml/utils/Statistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/Statistics.java b/src/main/java/org/apache/sysml/utils/Statistics.java
index 5cc0650..44bb232 100644
--- a/src/main/java/org/apache/sysml/utils/Statistics.java
+++ b/src/main/java/org/apache/sysml/utils/Statistics.java
@@ -601,7 +601,7 @@ public class Statistics
 		sb.append(String.format(
 				" %" + maxNumLen + "s  %-" + maxInstLen + "s  %" + maxTimeSLen + "s  %" + maxCountLen + "s", numCol,
 				instCol, timeSCol, countCol));
-		if (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) {
+		if (DMLScript.FINEGRAINED_STATISTICS) {
 			sb.append("  ");
 			sb.append(gpuCol);
 		}
@@ -618,15 +618,15 @@ public class Statistics
 			int numLines = wrappedInstruction.length;
 			String [] miscTimers = null;
 			
-			if (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) {
+			if (DMLScript.FINEGRAINED_STATISTICS) {
 				miscTimers = wrap(GPUStatistics.getStringForCPMiscTimesPerInstruction(instruction), DMLScript.STATISTICS_MAX_WRAP_LEN);
 				numLines = Math.max(numLines, miscTimers.length);
 			}
 			
-			String miscFormatString = (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) ? " %" + DMLScript.STATISTICS_MAX_WRAP_LEN + "s" : "%s";
+			String miscFormatString = (DMLScript.FINEGRAINED_STATISTICS) ? " %" + DMLScript.STATISTICS_MAX_WRAP_LEN + "s" : "%s";
 			for(int wrapIter = 0; wrapIter < numLines; wrapIter++) {
 				String instStr = (wrapIter < wrappedInstruction.length) ? wrappedInstruction[wrapIter] : "";
-				String miscTimerStr = ( (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) && wrapIter < miscTimers.length) ? miscTimers[wrapIter] : ""; 
+				String miscTimerStr = ( (DMLScript.FINEGRAINED_STATISTICS) && wrapIter < miscTimers.length) ? miscTimers[wrapIter] : ""; 
 				if(wrapIter == 0) {
 					// Display instruction count
 					sb.append(String.format(