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);
        }
 }

Reply via email to