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/27 20:09:58 UTC

[3/5] systemml git commit: [SYSTEMML-1325] Cleanup static variables in DMLScript

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/MRJobInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/MRJobInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/MRJobInstruction.java
index 9d6f133..6f2e731 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/MRJobInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/MRJobInstruction.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.lops.DataGen;
 import org.apache.sysml.lops.Lop;
@@ -1269,7 +1270,7 @@ public class MRJobInstruction extends Instruction
 
 	@Override
 	public void processInstruction(ExecutionContext ec) {
-		if ( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE)
+		if ( ConfigurationManager.getExecutionMode() == RUNTIME_PLATFORM.SINGLE_NODE)
 			throw new DMLRuntimeException("MapReduce jobs cannot be executed when execution mode = singlenode");
 		
 		//execute MR job

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
index ed95f7d..2da792e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateUnaryCPInstruction.java
@@ -19,8 +19,8 @@
 
 package org.apache.sysml.runtime.instructions.cp;
 
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -102,7 +102,7 @@ public class AggregateUnaryCPInstruction extends UnaryCPInstruction
 				//Note: check on matrix characteristics to cover incorrect length (-1*-1 -> 1)
 				if( !mc.dimsKnown() ) //invalid nrow/ncol/length
 				{
-					if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE 
+					if( ConfigurationManager.getExecutionMode() == RUNTIME_PLATFORM.SINGLE_NODE 
 						|| (input1.getDataType() == DataType.FRAME && OptimizerUtils.isHadoopExecutionMode()) )
 					{
 						if( OptimizerUtils.isHadoopExecutionMode() )

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/ComputationCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ComputationCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ComputationCPInstruction.java
index 6af1d2c..381155c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ComputationCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ComputationCPInstruction.java
@@ -19,8 +19,8 @@
 
 package org.apache.sysml.runtime.instructions.cp;
 
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -58,7 +58,7 @@ public abstract class ComputationCPInstruction extends CPInstruction {
 	}
 
 	protected boolean checkGuardedRepresentationChange( MatrixBlock in1, MatrixBlock in2, MatrixBlock out ) {
-		if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE
+		if( ConfigurationManager.getExecutionMode() == RUNTIME_PLATFORM.SINGLE_NODE
 			&& !CacheableData.isCachingActive() )
 			return true;
 		double memIn1 = (in1 != null) ? in1.getInMemorySize() : 0;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/DnnCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/DnnCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/DnnCPInstruction.java
index e54b430..7bed33f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/DnnCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/DnnCPInstruction.java
@@ -22,7 +22,7 @@ package org.apache.sysml.runtime.instructions.cp;
 import java.util.ArrayList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
@@ -589,7 +589,7 @@ public class DnnCPInstruction extends UnaryCPInstruction {
 	 * @param sparsity sparsity of intermediate matrix used per thread
 	 */
 	private void resetNumThreads(DnnParameters params, int numRows, int numCols, double sparsity) {
-		if(DMLScript.USE_ACCELERATOR) {
+		if(ConfigurationManager.isGPU()) {
 			double memBudget1Thread = OptimizerUtils.estimateSizeExactSparsity(numRows, numCols, sparsity);
 			int limitedDegreeOfParallelism = (int) Math.floor(_intermediateMemoryBudget / memBudget1Thread);
 			if(params.numThreads > limitedDegreeOfParallelism) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
index 8e47fce..40c0647 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
@@ -24,6 +24,7 @@ import java.util.HashSet;
 import java.util.List;
 
 import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DataIdentifier;
@@ -151,7 +152,7 @@ public class FunctionCallCPInstruction extends CPInstruction {
 		// Create a symbol table under a new execution context for the function invocation,
 		// and copy the function arguments into the created table. 
 		ExecutionContext fn_ec = ExecutionContextFactory.createContext(false, ec.getProgram());
-		if (DMLScript.USE_ACCELERATOR) {
+		if (ConfigurationManager.isGPU()) {
 			fn_ec.setGPUContexts(ec.getGPUContexts());
 			fn_ec.getGPUContext(0).initializeThread();
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixIndexingCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixIndexingCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixIndexingCPInstruction.java
index 4e5d4c0..34a7476 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixIndexingCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixIndexingCPInstruction.java
@@ -19,7 +19,7 @@
 
 package org.apache.sysml.runtime.instructions.cp;
 
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.lops.LeftIndex;
 import org.apache.sysml.lops.RightIndex;
 import org.apache.sysml.parser.Expression.DataType;
@@ -82,7 +82,7 @@ public final class MatrixIndexingCPInstruction extends IndexingCPInstruction {
 		else if ( opcode.equalsIgnoreCase(LeftIndex.OPCODE))
 		{
 			UpdateType updateType = mo.getUpdateType();
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				if( updateType.isInPlace() )
 					Statistics.incrementTotalLixUIP();
 				Statistics.incrementTotalLix();

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/ParamservBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParamservBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParamservBuiltinCPInstruction.java
index b6bb6fb..bc3ca24 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParamservBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParamservBuiltinCPInstruction.java
@@ -55,7 +55,7 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.spark.network.server.TransportServer;
 import org.apache.spark.util.LongAccumulator;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.recompile.Recompiler;
 import org.apache.sysml.lops.LopProperties;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -119,7 +119,7 @@ public class ParamservBuiltinCPInstruction extends ParameterizedBuiltinCPInstruc
 	}
 
 	private void runOnSpark(SparkExecutionContext sec, PSModeType mode) {
-		Timing tSetup = DMLScript.STATISTICS ? new Timing(true) : null;
+		Timing tSetup = ConfigurationManager.isStatistics() ? new Timing(true) : null;
 
 		int workerNum = getWorkerNum(mode);
 		String updFunc = getParam(PS_UPDATE_FUN);
@@ -167,7 +167,7 @@ public class ParamservBuiltinCPInstruction extends ParameterizedBuiltinCPInstruc
 			getFrequency(), getEpochs(), getBatchSize(), program, clsMap, sec.getSparkContext().getConf(),
 			server.getPort(), aSetup, aWorker, aUpdate, aIndex, aGrad, aRPC, aBatch, aEpoch);
 
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			Statistics.accPSSetupTime((long) tSetup.stop());
 
 		MatrixObject features = sec.getMatrixObject(getParam(PS_FEATURES));
@@ -182,7 +182,7 @@ public class ParamservBuiltinCPInstruction extends ParameterizedBuiltinCPInstruc
 		}
 
 		// Accumulate the statistics for remote workers
-		if (DMLScript.STATISTICS) {
+		if (ConfigurationManager.isStatistics()) {
 			Statistics.accPSSetupTime(aSetup.value());
 			Statistics.incWorkerNumber(aWorker.value());
 			Statistics.accPSLocalModelUpdateTime(aUpdate.value());
@@ -196,7 +196,7 @@ public class ParamservBuiltinCPInstruction extends ParameterizedBuiltinCPInstruc
 	}
 
 	private void runLocally(ExecutionContext ec, PSModeType mode) {
-		Timing tSetup = DMLScript.STATISTICS ? new Timing(true) : null;
+		Timing tSetup = ConfigurationManager.isStatistics() ? new Timing(true) : null;
 		int workerNum = getWorkerNum(mode);
 		BasicThreadFactory factory = new BasicThreadFactory.Builder()
 			.namingPattern("workers-pool-thread-%d").build();
@@ -230,7 +230,7 @@ public class ParamservBuiltinCPInstruction extends ParameterizedBuiltinCPInstruc
 		PSScheme scheme = getScheme();
 		partitionLocally(scheme, ec, workers);
 
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			Statistics.accPSSetupTime((long) tSetup.stop());
 
 		if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
index b425707..8e78442 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/VariableCPInstruction.java
@@ -27,7 +27,6 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.lops.Lop;
@@ -488,7 +487,7 @@ public class VariableCPInstruction extends CPInstruction {
 				mobj.enableCleanup(!getInput1().getName()
 					.startsWith(org.apache.sysml.lops.Data.PREAD_PREFIX));
 				ec.setVariable(getInput1().getName(), mobj);
-				if(DMLScript.STATISTICS && _updateType.isInPlace())
+				if(ConfigurationManager.isStatistics() && _updateType.isInPlace())
 					Statistics.incrementTotalUIPVar();
 			}
 			else if( getInput1().getDataType() == DataType.FRAME ) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
index e774dcd..e736a1c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/DnnGPUInstruction.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 
 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.controlprogram.context.ExecutionContext;
@@ -673,8 +672,8 @@ public class DnnGPUInstruction extends GPUInstruction {
 				cudnnInput, out0Pointer, c0Pointer, cudnnWPointer, doutName, dcyName,  // input
 				dxName, dwName, dbName, dhxName, dcxName, // output 
 				return_sequences, N, M, D, T);
-		gCtx.cudaFreeHelper(instructionName, cudnnWPointer, DMLScript.EAGER_CUDA_FREE);
-		gCtx.cudaFreeHelper(instructionName, cudnnInput, DMLScript.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instructionName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instructionName, cudnnInput, gCtx.EAGER_CUDA_FREE);
 		
 		// release inputs/outputs
 		ec.releaseMatrixInputForGPUInstruction(_input4.getName());
@@ -725,8 +724,8 @@ public class DnnGPUInstruction extends GPUInstruction {
 		Pointer c0Pointer = LibMatrixCUDA.getDensePointer(gCtx, getMatrixInputForGPUInstruction(ec, _input5.getName()), instructionName); 
 		
 		LibMatrixCuDNN.lstm(ec, gCtx, instructionName, cudnnInput, cudnnWPointer, out0Pointer, c0Pointer, return_sequences, _output.getName(), _output2.getName(), N, M, D, T);
-		gCtx.cudaFreeHelper(instructionName, cudnnWPointer, DMLScript.EAGER_CUDA_FREE);
-		gCtx.cudaFreeHelper(instructionName, cudnnInput, DMLScript.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instructionName, cudnnWPointer, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(instructionName, cudnnInput, gCtx.EAGER_CUDA_FREE);
 		
 		// release inputs/outputs
 		ec.releaseMatrixInputForGPUInstruction(_input4.getName());

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/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 e3c444a..7f3b017 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
@@ -21,7 +21,7 @@ package org.apache.sysml.runtime.instructions.gpu;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.lops.runtime.RunMRJobs;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
@@ -201,10 +201,10 @@ public abstract class GPUInstruction extends Instruction {
 	
 	@Override
 	public void postprocessInstruction(ExecutionContext ec) {
-		if(DMLScript.SYNCHRONIZE_GPU) {
-			long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+		if(GPUContext.SYNCHRONIZE_GPU) {
+			long t0 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 			jcuda.runtime.JCuda.cudaDeviceSynchronize();
-			if(DMLScript.FINEGRAINED_STATISTICS)
+			if(ConfigurationManager.isFinegrainedStatistics())
 				GPUStatistics.maintainCPMiscTimes(getExtendedOpcode(), GPUInstruction.MISC_TIMER_CUDA_SYNC, System.nanoTime() - t0);
 		}
 		if(LOG.isDebugEnabled()) {
@@ -235,9 +235,9 @@ public abstract class GPUInstruction extends Instruction {
 	 * @return	the matrix object
 	 */
 	protected MatrixObject getDenseMatrixOutputForGPUInstruction(ExecutionContext ec, String name, long numRows, long numCols) {
-		long t0 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+		long t0 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 		Pair<MatrixObject, Boolean> mb = ec.getDenseMatrixOutputForGPUInstruction(name, numRows, numCols);
-		if (DMLScript.FINEGRAINED_STATISTICS && mb.getValue()) GPUStatistics.maintainCPMiscTimes(getExtendedOpcode(), GPUInstruction.MISC_TIMER_ALLOCATE_DENSE_OUTPUT, System.nanoTime() - t0);
+		if (ConfigurationManager.isFinegrainedStatistics() && 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/ae268a9e/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 d9be663..d7e38b9 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
@@ -34,7 +34,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.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
@@ -180,7 +180,7 @@ public class CSRPointer {
 	public static void copyToDevice(GPUContext gCtx, CSRPointer dest, int rows, long nnz, int[] rowPtr, int[] colInd, double[] values) {
 		CSRPointer r = dest;
 		long t0 = 0;
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			t0 = System.nanoTime();
 		r.nnz = nnz;
 		if(rows < 0) throw new DMLRuntimeException("Incorrect input parameter: rows=" + rows);
@@ -191,9 +191,9 @@ public class CSRPointer {
 		LibMatrixCUDA.cudaSupportFunctions.hostToDevice(gCtx, values, r.val, null);
 		cudaMemcpy(r.rowPtr, Pointer.to(rowPtr), getIntSizeOf(rows + 1), cudaMemcpyHostToDevice);
 		cudaMemcpy(r.colInd, Pointer.to(colInd), getIntSizeOf(nnz), cudaMemcpyHostToDevice);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaToDevTime.add(System.nanoTime() - t0);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaToDevCount.add(3);
 	}
 	
@@ -458,7 +458,7 @@ public class CSRPointer {
 	 */
 	public Pointer toColumnMajorDenseMatrix(cusparseHandle cusparseHandle, cublasHandle cublasHandle, int rows,
 			int cols, String instName) {
-		long t0 = DMLScript.FINEGRAINED_STATISTICS && instName != null ? System.nanoTime() : 0;
+		long t0 = ConfigurationManager.isFinegrainedStatistics() && 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);
@@ -471,7 +471,7 @@ public class CSRPointer {
 		} else {
 			LOG.debug("in CSRPointer, the values array, row pointers array or column indices array was null");
 		}
-		if (DMLScript.FINEGRAINED_STATISTICS && instName != null) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
+		if (ConfigurationManager.isFinegrainedStatistics() && instName != null) GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, System.nanoTime() - t0);
 		return A;
 	}
 
@@ -480,7 +480,7 @@ public class CSRPointer {
 	 *
 	 */
 	public void deallocate() {
-		deallocate(DMLScript.EAGER_CUDA_FREE);
+		deallocate(getGPUContext().EAGER_CUDA_FREE);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CudaMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CudaMemoryAllocator.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CudaMemoryAllocator.java
index e74bea3..77baa53 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CudaMemoryAllocator.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/CudaMemoryAllocator.java
@@ -20,7 +20,8 @@ package org.apache.sysml.runtime.instructions.gpu.context;
 
 import static jcuda.runtime.JCuda.cudaMemGetInfo;
 
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
 
 import jcuda.CudaException;
 import jcuda.Pointer;
@@ -31,6 +32,8 @@ import static jcuda.runtime.JCuda.cudaFree;
 
 public class CudaMemoryAllocator implements GPUMemoryAllocator {
 	
+	private final double GPU_MEMORY_UTILIZATION_FACTOR = ConfigurationManager.getDMLConfig().getDoubleValue(DMLConfig.GPU_MEMORY_UTILIZATION_FACTOR);
+	
 	/**
 	 * Allocate memory on the device. 
 	 * 
@@ -77,7 +80,7 @@ public class CudaMemoryAllocator implements GPUMemoryAllocator {
 		long free[] = { 0 };
 		long total[] = { 0 };
 		cudaMemGetInfo(free, total);
-		return (long) (free[0] * DMLScript.GPU_MEMORY_UTILIZATION_FACTOR);
+		return (long) (free[0] * GPU_MEMORY_UTILIZATION_FACTOR);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/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 180a60f..3275099 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
@@ -33,7 +33,9 @@ import static jcuda.runtime.JCuda.cudaSetDeviceFlags;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.EvictionPolicy;
+import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.utils.GPUStatistics;
@@ -88,6 +90,25 @@ public class GPUContext {
 	
 	private GPUMemoryManager memoryManager;
 	
+	// whether to synchronize GPU after every instruction 
+	// global for all GPUContext for simplicity, but initialized every time to check if the configuration has been updated
+	// Note: cudaDeviceSynchronize is static method of JCuda
+	public static boolean SYNCHRONIZE_GPU = false;
+	// whether to perform eager CUDA free on rmvar
+	public final boolean EAGER_CUDA_FREE = ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.EAGER_CUDA_FREE);
+	
+	public static final EvictionPolicy GPU_EVICTION_POLICY;
+	static {
+		String evictionPolicy = ConfigurationManager.getDMLConfig().getTextValue(DMLConfig.GPU_EVICTION_POLICY).toUpperCase();
+		EvictionPolicy policyToUse = EvictionPolicy.MIN_EVICT;
+		try {
+			policyToUse = EvictionPolicy.valueOf(evictionPolicy);
+		} catch(IllegalArgumentException e) {
+			LOG.warn("Unsupported eviction policy:" + evictionPolicy + ". Using min_evict instead.");
+		}
+		GPU_EVICTION_POLICY = policyToUse;
+	}
+	
 	public GPUMemoryManager getMemoryManager() {
 		return memoryManager;
 	}
@@ -97,14 +118,15 @@ public class GPUContext {
 		cudaSetDevice(deviceNum);
 
 		cudaSetDeviceFlags(cudaDeviceScheduleBlockingSync);
+		SYNCHRONIZE_GPU = ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.SYNCHRONIZE_GPU);
 
 		long start = -1;
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			start = System.nanoTime();
 		initializeCudaLibraryHandles();
 		
 
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaLibrariesInitTime = System.nanoTime() - start;
 
 		memoryManager = new GPUMemoryManager(this);

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
index b619fa9..ba98b3f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPULazyCudaFreeMemoryManager.java
@@ -26,7 +26,7 @@ import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.utils.GPUStatistics;
@@ -56,11 +56,11 @@ public class GPULazyCudaFreeMemoryManager {
 		if (rmvarGPUPointers.containsKey(size)) {
 			if(LOG.isTraceEnabled())
 				LOG.trace("Getting rmvar-ed pointers for size:" + size);
-			boolean measureTime = opcode != null && DMLScript.FINEGRAINED_STATISTICS; 
+			boolean measureTime = opcode != null && ConfigurationManager.isFinegrainedStatistics(); 
 			long t0 = measureTime ? System.nanoTime() : 0;
 			Pointer A = remove(rmvarGPUPointers, size); // remove from rmvarGPUPointers as you are not calling cudaFree
 			long totalTime = System.nanoTime() - t0;
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				GPUStatistics.cudaAllocReuseCount.increment();
 			}
 			if(measureTime) {
@@ -81,7 +81,7 @@ public class GPULazyCudaFreeMemoryManager {
 	 * @param startTime start time
 	 */
 	void addMiscTime(String opcode, String instructionLevelTimer, long startTime) {
-		if (opcode != null && DMLScript.FINEGRAINED_STATISTICS)
+		if (opcode != null && ConfigurationManager.isFinegrainedStatistics())
 			GPUStatistics.maintainCPMiscTimes(opcode, instructionLevelTimer, System.nanoTime() - startTime);
 	}
 	
@@ -104,13 +104,13 @@ public class GPULazyCudaFreeMemoryManager {
 		Optional<Long> toClear = rmvarGPUPointers.entrySet().stream().filter(e -> e.getValue().size() > 0).map(e -> e.getKey())
 				.filter(size -> size >= minSize).min((s1, s2) -> s1 < s2 ? -1 : 1);
 		if(toClear.isPresent()) {
-			boolean measureTime = opcode != null && DMLScript.FINEGRAINED_STATISTICS;
+			boolean measureTime = opcode != null && ConfigurationManager.isFinegrainedStatistics();
 			long t0 = measureTime ?  System.nanoTime() : 0;
 			Pointer A = remove(rmvarGPUPointers, toClear.get()); // remove from rmvarGPUPointers as you are not calling cudaFree
 			if(measureTime) {
 				gpuManager.addMiscTime(opcode, GPUInstruction.MISC_TIMER_REUSE, t0);
 			}
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				GPUStatistics.cudaAllocReuseCount.increment();
 			}
 			return A;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
index 5d0e4bc..2e43b99 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/GPUMemoryManager.java
@@ -35,6 +35,7 @@ import java.util.stream.Collectors;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -54,6 +55,8 @@ public class GPUMemoryManager {
 	private static final boolean DEBUG_MEMORY_LEAK = false;
 	private static final int [] DEBUG_MEMORY_LEAK_STACKTRACE_DEPTH = {5, 6, 7, 8, 9, 10}; // Avoids printing too much text while debuggin
 	
+	private final boolean PRINT_GPU_MEMORY_INFO = ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.PRINT_GPU_MEMORY_INFO);
+	
 	protected final GPUMemoryAllocator allocator;
 	/*****************************************************************************************/
 	// GPU Memory is divided into three major sections:
@@ -131,14 +134,15 @@ public class GPUMemoryManager {
 	public GPUMemoryManager(GPUContext gpuCtx) {
 		matrixMemoryManager = new GPUMatrixMemoryManager(this);
 		lazyCudaFreeMemoryManager = new GPULazyCudaFreeMemoryManager(this);
-		if(DMLScript.GPU_MEMORY_ALLOCATOR.equals("cuda")) {
+		String allocatorType = ConfigurationManager.getDMLConfig().getTextValue(DMLConfig.GPU_MEMORY_ALLOCATOR);
+		if(allocatorType.equals("cuda")) {
 			allocator = new CudaMemoryAllocator();
 		}
-		else if(DMLScript.GPU_MEMORY_ALLOCATOR.equals("unified_memory")) {
+		else if(allocatorType.equals("unified_memory")) {
 			allocator = new UnifiedMemoryAllocator();
 		}
 		else {
-			throw new RuntimeException("Unsupported value (" + DMLScript.GPU_MEMORY_ALLOCATOR + ") for the configuration " + DMLConfig.GPU_MEMORY_ALLOCATOR 
+			throw new RuntimeException("Unsupported value (" + allocatorType + ") for the configuration " + DMLConfig.GPU_MEMORY_ALLOCATOR 
 					+ ". Supported values are cuda, unified_memory.");
 		}
 		long free[] = { 0 };
@@ -167,30 +171,30 @@ public class GPUMemoryManager {
 	 * @return allocated pointer
 	 */
 	private Pointer cudaMallocNoWarn(Pointer A, long size, String printDebugMessage) {
-		long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long t0 = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 		try {
 			allocator.allocate(A, size);
 			allPointers.put(A, new PointerInfo(size));
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				long totalTime = System.nanoTime() - t0;
 				GPUStatistics.cudaAllocSuccessTime.add(totalTime);
 				GPUStatistics.cudaAllocSuccessCount.increment();
 				GPUStatistics.cudaAllocTime.add(totalTime);
 				GPUStatistics.cudaAllocCount.increment();
 			}
-			if(printDebugMessage != null && (DMLScript.PRINT_GPU_MEMORY_INFO || LOG.isTraceEnabled()) )  {
+			if(printDebugMessage != null && (PRINT_GPU_MEMORY_INFO || LOG.isTraceEnabled()) )  {
 				LOG.info("Success: " + printDebugMessage + ":" + byteCountToDisplaySize(size));
 			}
 			return A;
 		} catch(jcuda.CudaException e) {
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				long totalTime = System.nanoTime() - t0;
 				GPUStatistics.cudaAllocFailedTime.add(System.nanoTime() - t0);
 				GPUStatistics.cudaAllocFailedCount.increment();
 				GPUStatistics.cudaAllocTime.add(totalTime);
 				GPUStatistics.cudaAllocCount.increment();
 			}
-			if(printDebugMessage != null && (DMLScript.PRINT_GPU_MEMORY_INFO || LOG.isTraceEnabled()) )  {
+			if(printDebugMessage != null && (PRINT_GPU_MEMORY_INFO || LOG.isTraceEnabled()) )  {
 				LOG.info("Failed: " + printDebugMessage + ":" + byteCountToDisplaySize(size));
 				LOG.info("GPU Memory info " + printDebugMessage + ":" + toString());
 			}
@@ -280,7 +284,7 @@ public class GPUMemoryManager {
 		
 		// Step 5: Try eviction/clearing exactly one with size restriction
 		if(A == null) {
-			long t0 =  DMLScript.STATISTICS ? System.nanoTime() : 0;
+			long t0 =  ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 			Optional<GPUObject> sizeBasedUnlockedGPUObjects = matrixMemoryManager.gpuObjects.stream()
 						.filter(gpuObj -> !gpuObj.isLocked() && matrixMemoryManager.getWorstCaseContiguousMemorySize(gpuObj) >= size)
 						.min((o1, o2) -> worstCaseContiguousMemorySizeCompare(o1, o2));
@@ -289,7 +293,7 @@ public class GPUMemoryManager {
 				A = cudaMallocNoWarn(tmpA, size, null);
 				if(A == null)
 					LOG.warn("cudaMalloc failed after clearing/evicting based on size.");
-				if(DMLScript.STATISTICS) {
+				if(ConfigurationManager.isStatistics()) {
 					long totalTime = System.nanoTime() - t0;
 					GPUStatistics.cudaEvictTime.add(totalTime);
 					GPUStatistics.cudaEvictSizeTime.add(totalTime);
@@ -301,7 +305,7 @@ public class GPUMemoryManager {
 		
 		// Step 6: Try eviction/clearing one-by-one based on the given policy without size restriction
 		if(A == null) {
-			long t0 =  DMLScript.STATISTICS ? System.nanoTime() : 0;
+			long t0 =  ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 			long currentAvailableMemory = allocator.getAvailableMemory();
 			boolean canFit = false;
 			// ---------------------------------------------------------------
@@ -322,10 +326,10 @@ public class GPUMemoryManager {
 					// This was the bottleneck for ResNet200 experiments with batch size > 32 on P100+Intel
 					A = cudaMallocNoWarn(tmpA, size, null); 
 				}
-				if(DMLScript.STATISTICS) 
+				if(ConfigurationManager.isStatistics()) 
 					GPUStatistics.cudaEvictCount.increment();
 			}
-			if(DMLScript.STATISTICS) {
+			if(ConfigurationManager.isStatistics()) {
 				long totalTime = System.nanoTime() - t0;
 				GPUStatistics.cudaEvictTime.add(totalTime);
 			}
@@ -346,7 +350,7 @@ public class GPUMemoryManager {
 					+ toString());
 		}
 		
-		long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long t0 = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 		cudaMemset(A, 0, size);
 		addMiscTime(opcode, GPUStatistics.cudaMemSet0Time, GPUStatistics.cudaMemSet0Count, GPUInstruction.MISC_TIMER_SET_ZERO, t0);
 		return A;
@@ -405,7 +409,7 @@ public class GPUMemoryManager {
 			allPointers.remove(toFree);
 			lazyCudaFreeMemoryManager.removeIfPresent(size, toFree);
 			allocator.free(toFree);
-			if(DMLScript.SYNCHRONIZE_GPU)
+			if(GPUContext.SYNCHRONIZE_GPU)
 				jcuda.runtime.JCuda.cudaDeviceSynchronize(); // Force a device synchronize after free-ing the pointer for debugging
 		}
 		else {
@@ -426,7 +430,7 @@ public class GPUMemoryManager {
 		if(LOG.isTraceEnabled())
 			LOG.trace("Free-ing the pointer with eager=" + eager);
 		if (eager) {
-			long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;
+			long t0 = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 			guardedCudaFree(toFree);
 			addMiscTime(opcode, GPUStatistics.cudaDeAllocTime, GPUStatistics.cudaDeAllocCount, GPUInstruction.MISC_TIMER_CUDA_FREE, t0);
 		}
@@ -515,11 +519,11 @@ public class GPUMemoryManager {
 	 * @param startTime start time
 	 */
 	private void addMiscTime(String opcode, LongAdder globalGPUTimer, LongAdder globalGPUCounter, String instructionLevelTimer, long startTime) {
-		if(DMLScript.STATISTICS) {
+		if(ConfigurationManager.isStatistics()) {
 			long totalTime = System.nanoTime() - startTime;
 			globalGPUTimer.add(totalTime);
 			globalGPUCounter.add(1);
-			if (opcode != null && DMLScript.FINEGRAINED_STATISTICS)
+			if (opcode != null && ConfigurationManager.isFinegrainedStatistics())
 				GPUStatistics.maintainCPMiscTimes(opcode, instructionLevelTimer, totalTime);
 		}
 	}
@@ -532,7 +536,7 @@ public class GPUMemoryManager {
 	 * @param startTime start time
 	 */
 	void addMiscTime(String opcode, String instructionLevelTimer, long startTime) {
-		if (opcode != null && DMLScript.FINEGRAINED_STATISTICS)
+		if (opcode != null && ConfigurationManager.isFinegrainedStatistics())
 			GPUStatistics.maintainCPMiscTimes(opcode, instructionLevelTimer, System.nanoTime() - startTime);
 	}
 	
@@ -656,7 +660,7 @@ public class GPUMemoryManager {
 				return 1;
 			} else {
 				// Both are unlocked
-				if (DMLScript.GPU_EVICTION_POLICY == DMLScript.EvictionPolicy.ALIGN_MEMORY) {
+				if (GPUContext.GPU_EVICTION_POLICY == DMLScript.EvictionPolicy.ALIGN_MEMORY) {
 					if(!p1.isDensePointerNull() && !p2.isDensePointerNull()) {
 						long p1Ptr = new CustomPointer(p1.getDensePointer()).getNativePointer();
 						long p2Ptr = new CustomPointer(p2.getDensePointer()).getNativePointer();
@@ -676,7 +680,7 @@ public class GPUMemoryManager {
 						return minEvictCompare(p1, p2);
 					}
 				}
-				else if (DMLScript.GPU_EVICTION_POLICY == DMLScript.EvictionPolicy.MIN_EVICT) {
+				else if (GPUContext.GPU_EVICTION_POLICY == DMLScript.EvictionPolicy.MIN_EVICT) {
 					return minEvictCompare(p1, p2);
 				} else {
 					return Long.compare(p2.timestamp.get(), p1.timestamp.get());

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/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 a783138..cfab0d4 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
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.LongAdder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.instructions.cp.CPInstruction;
@@ -197,10 +198,10 @@ public class GPUObject {
 	}
 
 	private void cudaFreeHelper(Pointer toFree) throws DMLRuntimeException {
-		getGPUContext().cudaFreeHelper(null, toFree, DMLScript.EAGER_CUDA_FREE);
+		getGPUContext().cudaFreeHelper(null, toFree, gpuContext.EAGER_CUDA_FREE);
 	}
 
-	GPUContext getGPUContext() {
+	public GPUContext getGPUContext() {
 		return gpuContext;
 	}
 
@@ -274,8 +275,8 @@ public class GPUObject {
 				C.colInd);
 		//cudaDeviceSynchronize();
 
-		gCtx.cudaFreeHelper(null, nnzPerRowPtr, DMLScript.EAGER_CUDA_FREE);
-		gCtx.cudaFreeHelper(null, nnzTotalDevHostPtr, DMLScript.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(null, nnzPerRowPtr, gCtx.EAGER_CUDA_FREE);
+		gCtx.cudaFreeHelper(null, nnzTotalDevHostPtr, gCtx.EAGER_CUDA_FREE);
 
 		return C;
 	}
@@ -315,7 +316,7 @@ public class GPUObject {
 			LOG.trace("GPU : dense -> sparse on " + this + ", GPUContext=" + getGPUContext());
 		}
 		long t0 = 0;
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			t0 = System.nanoTime();
 		cusparseHandle cusparseHandle = getGPUContext().getCusparseHandle();
 		if (cusparseHandle == null)
@@ -331,9 +332,9 @@ public class GPUObject {
 				columnMajorDenseToRowMajorSparse(getGPUContext(), cusparseHandle, getDensePointer(), rows,
 						cols));
 		// TODO: What if mat.getNnz() is -1 ?
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaDenseToSparseTime.add(System.nanoTime() - t0);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaDenseToSparseCount.add(1);
 	}
 
@@ -398,20 +399,20 @@ public class GPUObject {
 			LOG.trace("GPU : sparse -> dense on " + this + ", GPUContext=" + getGPUContext());
 		}
 		long start = 0, end = 0;
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			start = System.nanoTime();
 		if (getJcudaSparseMatrixPtr() == null || !isAllocated())
 			throw new DMLRuntimeException("Expected allocated sparse matrix before sparseToDense() call");
 
 		sparseToColumnMajorDense();
 		denseColumnMajorToRowMajor();
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			end = System.nanoTime();
-		if (instructionName != null && DMLScript.FINEGRAINED_STATISTICS)
+		if (instructionName != null && ConfigurationManager.isFinegrainedStatistics())
 			GPUStatistics.maintainCPMiscTimes(instructionName, GPUInstruction.MISC_TIMER_SPARSE_TO_DENSE, end - start);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaSparseToDenseTime.add(end - start);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaSparseToDenseCount.add(1);
 	}
 
@@ -525,7 +526,7 @@ public class GPUObject {
 				if(!recomputeDenseNNZ)
 					return -1;
 				
-				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+				long t1 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 				GPUContext gCtx = getGPUContext();
 				cusparseHandle cusparseHandle = gCtx.getCusparseHandle();
 				cusparseMatDescr matDescr = CSRPointer.getDefaultCuSparseMatrixDescriptor();
@@ -545,9 +546,9 @@ public class GPUObject {
 					throw new DMLRuntimeException(
 							"cusparseDnnz did not calculate the correct number of nnz on the GPU");
 				}
-				gCtx.cudaFreeHelper(instName, nnzPerRowPtr, DMLScript.EAGER_CUDA_FREE);
-				gCtx.cudaFreeHelper(instName, nnzTotalDevHostPtr, DMLScript.EAGER_CUDA_FREE);
-				if(DMLScript.FINEGRAINED_STATISTICS) {
+				gCtx.cudaFreeHelper(instName, nnzPerRowPtr, gpuContext.EAGER_CUDA_FREE);
+				gCtx.cudaFreeHelper(instName, nnzTotalDevHostPtr, gpuContext.EAGER_CUDA_FREE);
+				if(ConfigurationManager.isFinegrainedStatistics()) {
 					GPUStatistics.maintainCPMiscTimes(instName, CPInstruction.MISC_TIMER_RECOMPUTE_NNZ, System.nanoTime()-t1);
 			}
 				return nnzC[0];
@@ -690,7 +691,7 @@ public class GPUObject {
 	 * Updates the locks depending on the eviction policy selected
 	 */
 	private void updateReleaseLocks() {
-		DMLScript.EvictionPolicy evictionPolicy = DMLScript.GPU_EVICTION_POLICY;
+		DMLScript.EvictionPolicy evictionPolicy = GPUContext.GPU_EVICTION_POLICY;
 		switch (evictionPolicy) {
 			case LRU:
 				timestamp.set(System.nanoTime());
@@ -785,12 +786,12 @@ public class GPUObject {
 			LOG.trace("GPU : copyFromHostToDevice, on " + this + ", GPUContext=" + getGPUContext());
 		}
 		long start = 0;
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			start = System.nanoTime();
 
-		long acqrTime = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+		long acqrTime = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 		MatrixBlock tmp = mat.acquireRead();
-		if(DMLScript.FINEGRAINED_STATISTICS) {
+		if(ConfigurationManager.isFinegrainedStatistics()) {
 			if(tmp.isInSparseFormat())
 				GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_GET_SPARSE_MB, System.nanoTime()-acqrTime);
 			else
@@ -829,23 +830,23 @@ public class GPUObject {
 					csrBlock = (SparseBlockCSR) block;
 				} else if (block instanceof SparseBlockCOO) {
 					// TODO - should we do this on the GPU using cusparse<t>coo2csr() ?
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						t0 = System.nanoTime();
 					SparseBlockCOO cooBlock = (SparseBlockCOO) block;
 					csrBlock = new SparseBlockCSR(toIntExact(mat.getNumRows()), cooBlock.rowIndexes(),
 							cooBlock.indexes(), cooBlock.values());
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						GPUStatistics.cudaSparseConversionTime.add(System.nanoTime() - t0);
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						GPUStatistics.cudaSparseConversionCount.increment();
 				} else if (block instanceof SparseBlockMCSR) {
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						t0 = System.nanoTime();
 					SparseBlockMCSR mcsrBlock = (SparseBlockMCSR) block;
 					csrBlock = new SparseBlockCSR(mcsrBlock.getRows(), toIntExact(mcsrBlock.size()));
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						GPUStatistics.cudaSparseConversionTime.add(System.nanoTime() - t0);
-					if (DMLScript.STATISTICS)
+					if (ConfigurationManager.isStatistics())
 						GPUStatistics.cudaSparseConversionCount.increment();
 				} else {
 					throw new DMLRuntimeException("Unsupported sparse matrix format for CUDA operations");
@@ -858,10 +859,10 @@ public class GPUObject {
 			allocateSparseMatrixOnDevice();
 
 			if (copyToDevice) {
-				long t1 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+				long t1 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 				CSRPointer.copyToDevice(getGPUContext(), getJcudaSparseMatrixPtr(), tmp.getNumRows(), tmp.getNonZeros(), rowPtr, colInd,
 						values);
-				if(DMLScript.FINEGRAINED_STATISTICS) 
+				if(ConfigurationManager.isFinegrainedStatistics()) 
 					GPUStatistics.maintainCPMiscTimes(opcode, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, System.nanoTime() - t1);
 			}
 		} else {
@@ -877,9 +878,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 = DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+				long t1 = ConfigurationManager.isFinegrainedStatistics() ? System.nanoTime() : 0;
 				cudaMemset(getDensePointer(), 0, getDatatypeSizeOf(mat.getNumRows() * mat.getNumColumns()));
-				if(DMLScript.FINEGRAINED_STATISTICS) 
+				if(ConfigurationManager.isFinegrainedStatistics()) 
 					GPUStatistics.maintainCPMiscTimes(opcode, GPUInstruction.MISC_TIMER_SET_ZERO, System.nanoTime() - t1);
 			}
 			else {
@@ -891,9 +892,9 @@ public class GPUObject {
 
 		mat.release();
 
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaToDevTime.add(System.nanoTime() - start);
-		if (DMLScript.STATISTICS)
+		if (ConfigurationManager.isStatistics())
 			GPUStatistics.cudaToDevCount.add(1);
 	}
 
@@ -954,7 +955,7 @@ public class GPUObject {
 		}
 		
 		MatrixBlock tmp = null;
-		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long start = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 		if (!isDensePointerNull()) {
 			tmp = new MatrixBlock(toIntExact(mat.getNumRows()), toIntExact(mat.getNumColumns()), false);
 			tmp.allocateDenseBlock();
@@ -979,7 +980,7 @@ public class GPUObject {
 		}
 		mat.acquireModify(tmp);
 		mat.release();
-		if (DMLScript.STATISTICS && !isEviction) {
+		if (ConfigurationManager.isStatistics() && !isEviction) {
 			// Eviction time measure in malloc
 			long totalTime = System.nanoTime() - start;
 			int count = !isDensePointerNull() ? 1 : 3;

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaKernels.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaKernels.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaKernels.java
index 9a0b4c5..c940c15 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaKernels.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/JCudaKernels.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.util.HashMap;
 
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.io.IOUtilFunctions;
 import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
@@ -108,7 +107,7 @@ public class JCudaKernels {
 		checkResult(cuLaunchKernel(function, config.gridDimX, config.gridDimY, config.gridDimZ, config.blockDimX,
 				config.blockDimY, config.blockDimZ, config.sharedMemBytes, config.stream, Pointer.to(kernelParams),
 				null));
-		if(DMLScript.SYNCHRONIZE_GPU)
+		if(GPUContext.SYNCHRONIZE_GPU)
 			JCuda.cudaDeviceSynchronize();
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/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
index 27a6256..e122de2 100644
--- 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
@@ -23,6 +23,9 @@ 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.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysml.runtime.matrix.data.LibMatrixCUDA;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.utils.GPUStatistics;
@@ -36,6 +39,19 @@ public class ShadowBuffer {
 	GPUObject gpuObj;
 	float[] shadowPointer = null;
 	private static boolean _warnedAboutShadowBuffer = false;
+	private static long EVICTION_SHADOW_BUFFER_CURR_BYTES = 0;
+	private static long EVICTION_SHADOW_BUFFER_MAX_BYTES;
+	static {
+		if(DMLScript.FLOATING_POINT_PRECISION.equals("double")) {
+			EVICTION_SHADOW_BUFFER_MAX_BYTES = 0;
+		}
+		else {
+			double shadowBufferSize = ConfigurationManager.getDMLConfig().getDoubleValue(DMLConfig.EVICTION_SHADOW_BUFFERSIZE);
+			if(shadowBufferSize < 0 || shadowBufferSize > 1) 
+				throw new RuntimeException("Incorrect value (" + shadowBufferSize + ") for the configuration:" + DMLConfig.EVICTION_SHADOW_BUFFERSIZE);
+			EVICTION_SHADOW_BUFFER_MAX_BYTES = (long) (((double)InfrastructureAnalyzer.getLocalMaxMemory())*shadowBufferSize);
+		}
+	}
 	
 	public ShadowBuffer(GPUObject gpuObj) {
 		this.gpuObj = gpuObj;
@@ -55,14 +71,14 @@ public class ShadowBuffer {
 	 * @param instName name of the instruction
 	 */
 	public void moveFromDevice(String instName) {
-		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long start = ConfigurationManager.isStatistics() ? 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;
+		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) {
+		if (ConfigurationManager.isStatistics()) {
 			// Eviction time measure in malloc
 			long totalTime = System.nanoTime() - start;
 			GPUStatistics.cudaFromDevToShadowTime.add(totalTime);
@@ -75,7 +91,7 @@ public class ShadowBuffer {
 	 * Move the data from shadow buffer to Matrix object
 	 */
 	public void moveToHost() {
-		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long start = ConfigurationManager.isStatistics() ? System.nanoTime() : 0;
 		MatrixBlock tmp = new MatrixBlock(GPUObject.toIntExact(gpuObj.mat.getNumRows()), GPUObject.toIntExact(gpuObj.mat.getNumColumns()), false);
 		tmp.allocateDenseBlock();
 		double [] tmpArr = tmp.getDenseBlockValues();
@@ -86,7 +102,7 @@ public class ShadowBuffer {
 		gpuObj.mat.release();
 		clearShadowPointer();
 		gpuObj.dirty = false;
-		if (DMLScript.STATISTICS) {
+		if (ConfigurationManager.isStatistics()) {
 			long totalTime = System.nanoTime() - start;
 			GPUStatistics.cudaFromShadowToHostTime.add(totalTime);
 			GPUStatistics.cudaFromShadowToHostCount.increment();
@@ -100,12 +116,12 @@ public class ShadowBuffer {
 	 * Move the data from shadow buffer to GPU
 	 */
 	public void moveToDevice() {
-		long start = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		long start = ConfigurationManager.isStatistics() ? 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) {
+		if (ConfigurationManager.isStatistics()) {
 			long totalTime = System.nanoTime() - start;
 			GPUStatistics.cudaFromShadowToDevTime.add(totalTime);
 			GPUStatistics.cudaFromShadowToDevCount.increment();
@@ -122,7 +138,7 @@ public class ShadowBuffer {
 	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;
+			boolean ret = EVICTION_SHADOW_BUFFER_CURR_BYTES + numBytes <= 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;
@@ -139,7 +155,7 @@ public class ShadowBuffer {
 	 */
 	public void clearShadowPointer() {
 		if(shadowPointer != null) {
-			DMLScript.EVICTION_SHADOW_BUFFER_CURR_BYTES -= shadowPointer.length*Sizeof.FLOAT;
+			EVICTION_SHADOW_BUFFER_CURR_BYTES -= shadowPointer.length*Sizeof.FLOAT;
 		}
 		shadowPointer = null;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/UnifiedMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/UnifiedMemoryAllocator.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/UnifiedMemoryAllocator.java
index 71c6fc3..3a77d28 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/UnifiedMemoryAllocator.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/context/UnifiedMemoryAllocator.java
@@ -23,7 +23,8 @@ import static jcuda.runtime.JCuda.cudaMallocManaged;
 import static jcuda.runtime.JCuda.cudaMemGetInfo;
 import static jcuda.runtime.cudaError.cudaSuccess;
 import static jcuda.runtime.JCuda.cudaMemAttachGlobal;
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
 
 import jcuda.CudaException;
 import jcuda.Pointer;
@@ -31,6 +32,8 @@ import jcuda.runtime.cudaError;
 
 public class UnifiedMemoryAllocator  implements GPUMemoryAllocator {
 
+	private final double GPU_MEMORY_UTILIZATION_FACTOR = ConfigurationManager.getDMLConfig().getDoubleValue(DMLConfig.GPU_MEMORY_UTILIZATION_FACTOR);
+	
 	/**
 	 * Allocate memory on the device. 
 	 * 
@@ -78,12 +81,12 @@ public class UnifiedMemoryAllocator  implements GPUMemoryAllocator {
 	 * @return the available memory in bytes
 	 */
 	public long getAvailableMemory() {
-		if(maxAvailableMemory < 0 || gpuUtilizationFactor != DMLScript.GPU_MEMORY_UTILIZATION_FACTOR) {
+		if(maxAvailableMemory < 0 || gpuUtilizationFactor != GPU_MEMORY_UTILIZATION_FACTOR) {
 			long free[] = { 0 };
 			long total[] = { 0 };
 			cudaMemGetInfo(free, total);
-			maxAvailableMemory = (long) (total[0] * DMLScript.GPU_MEMORY_UTILIZATION_FACTOR);
-			gpuUtilizationFactor = DMLScript.GPU_MEMORY_UTILIZATION_FACTOR;
+			maxAvailableMemory = (long) (total[0] * GPU_MEMORY_UTILIZATION_FACTOR);
+			gpuUtilizationFactor = GPU_MEMORY_UTILIZATION_FACTOR;
 		}
 		return maxAvailableMemory;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/src/main/java/org/apache/sysml/runtime/instructions/spark/RandSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/RandSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/RandSPInstruction.java
index 3ec3e6b..2b9adcb 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/RandSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/RandSPInstruction.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.runtime.instructions.spark;
 
 import java.io.IOException;
+
 import java.io.PrintWriter;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -39,7 +40,7 @@ import org.apache.spark.util.random.SamplingUtils;
 
 import scala.Tuple2;
 
-import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.hops.DataGenOp;
 import org.apache.sysml.hops.Hop.DataGenMethod;
@@ -254,7 +255,7 @@ public class RandSPInstruction extends UnarySPInstruction {
 
 		//step 2: potential in-memory rand operations if applicable
 		if( isMemAvail(lrows, lcols, sparsity, minValue, maxValue) 
-			&&  DMLScript.rtplatform != RUNTIME_PLATFORM.SPARK )
+			&&  ConfigurationManager.getExecutionMode() != RUNTIME_PLATFORM.SPARK )
 		{
 			RandomMatrixGenerator rgen = LibMatrixDatagen.createRandomMatrixGenerator(
 				pdf, (int)lrows, (int)lcols, rowsInBlock, colsInBlock, 

http://git-wip-us.apache.org/repos/asf/systemml/blob/ae268a9e/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 8b7a06f..1311b96 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
@@ -24,7 +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.conf.ConfigurationManager;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.instructions.gpu.GPUInstruction;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
@@ -164,7 +164,7 @@ public class DoublePrecisionCudaSupportFunctions implements CudaSupportFunctions
 
 	@Override
 	public void deviceToHost(GPUContext gCtx, Pointer src, double[] dest, String instName, boolean isEviction) {
-		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = ConfigurationManager.isFinegrainedStatistics()  && instName != null? System.nanoTime() : 0;
 		if(src == null)
 			throw new DMLRuntimeException("The source pointer in deviceToHost is null");
 		if(dest == null)
@@ -173,15 +173,15 @@ public class DoublePrecisionCudaSupportFunctions implements CudaSupportFunctions
 			LOG.debug("deviceToHost: src of size " + gCtx.getMemoryManager().getSizeAllocatedGPUPointer(src) + " (in bytes) -> dest of size " + (dest.length*Double.BYTES)  + " (in bytes).");
 		}
 		cudaMemcpy(Pointer.to(dest), src, ((long)dest.length)*Sizeof.DOUBLE, cudaMemcpyDeviceToHost);
-		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
+		if(ConfigurationManager.isFinegrainedStatistics() && 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) {
-		long t1 = DMLScript.FINEGRAINED_STATISTICS  && instName != null? System.nanoTime() : 0;
+		long t1 = ConfigurationManager.isFinegrainedStatistics()  && instName != null? System.nanoTime() : 0;
 		cudaMemcpy(dest, Pointer.to(src), ((long)src.length)*Sizeof.DOUBLE, cudaMemcpyHostToDevice);
-		if(DMLScript.FINEGRAINED_STATISTICS && instName != null) 
+		if(ConfigurationManager.isFinegrainedStatistics() && instName != null) 
 			GPUStatistics.maintainCPMiscTimes(instName, GPUInstruction.MISC_TIMER_HOST_TO_DEVICE, System.nanoTime() - t1);
 	}
 }