You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2017/11/07 20:27:19 UTC

[2/3] systemml git commit: [SYSTEMML-1988] Fix cloning of prepared scripts, rework CP runtime

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/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 572290d..1e71e63 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
@@ -24,21 +24,21 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public abstract class ComputationCPInstruction extends CPInstruction {
 
-	public CPOperand output;
-	public CPOperand input1, input2, input3;
+	public final CPOperand output;
+	public final CPOperand input1, input2, input3;
 
-	protected ComputationCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
+	protected ComputationCPInstruction(CPType type, Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String istr) {
-		super(op, opcode, istr);
+		super(type, op, opcode, istr);
 		input1 = in1;
 		input2 = in2;
 		input3 = null;
 		output = out;
 	}
 
-	protected ComputationCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
+	protected ComputationCPInstruction(CPType type, Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
 			String opcode, String istr) {
-		super(op, opcode, istr);
+		super(type, op, opcode, istr);
 		input1 = in1;
 		input2 = in2;
 		input3 = in3;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
index 36422d9..60ca720 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ConvolutionCPInstruction.java
@@ -28,97 +28,74 @@ import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.KahanPlus;
-import org.apache.sysml.runtime.functionobjects.SwapIndex;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.ConvolutionParameters;
 import org.apache.sysml.runtime.matrix.data.LibMatrixDNN;
 import org.apache.sysml.runtime.matrix.data.LibMatrixNative;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.SparseBlock;
-import org.apache.sysml.runtime.matrix.operators.ReorgOperator;
 import org.apache.sysml.runtime.util.ConvolutionUtils;
 import org.apache.sysml.utils.NativeHelper;
 
 public class ConvolutionCPInstruction extends UnaryCPInstruction {
-	private CPOperand _in2;
-	private CPOperand _in3;
-	private ArrayList<CPOperand> _input_shape;
-	private ArrayList<CPOperand> _filter_shape;
-	private ArrayList<CPOperand> _stride = new ArrayList<>();
-	private ArrayList<CPOperand> _padding = new ArrayList<>();
-	private int _numThreads = -1;	private double _intermediateMemoryBudget = 0;
 	private static final Log LOG = LogFactory.getLog(ConvolutionCPInstruction.class.getName());
 	private static boolean warnedUnderUtilitization = false;
 	
+	private final CPOperand _in2;
+	private final CPOperand _in3;
+	private final ArrayList<CPOperand> _input_shape;
+	private final ArrayList<CPOperand> _filter_shape;
+	private final ArrayList<CPOperand> _stride;
+	private final ArrayList<CPOperand> _padding;
+	private final int _numThreads;
+	private final double _intermediateMemoryBudget;
+	
+	public ConvolutionCPInstruction(CPOperand in, CPOperand in2, CPOperand in3, CPOperand out, 
+			ArrayList<CPOperand> stride, ArrayList<CPOperand> padding, ArrayList<CPOperand> input_shape,
+			ArrayList<CPOperand> filter_shape, int numThreads, double intermediateMemoryBudget, String opcode, String istr) {
+		super(CPType.Convolution, null, in, out, opcode, istr);
+		_in2 = in2;
+		_in3 = in3;
+		_stride = stride;
+		_padding = padding;
+		_input_shape = input_shape;
+		_filter_shape = filter_shape;
+		_numThreads = numThreads;
+		_intermediateMemoryBudget = intermediateMemoryBudget;
+	}
+	
 	public ConvolutionCPInstruction(CPOperand in, CPOperand in2, CPOperand out, String opcode, String istr, int numThreads, double intermediateMemoryBudget) throws DMLRuntimeException {
-		super(new ReorgOperator(SwapIndex.getSwapIndexFnObject()), in, out,
-				opcode, istr);
+		this(in, in2, null, out, null, null, null, null, numThreads, intermediateMemoryBudget, opcode, istr);
 		if( !(opcode.equals("bias_add") || opcode.equals("relu_backward") || opcode.equals("bias_multiply") ) ) {
 			throw new DMLRuntimeException("Incorrect usage. Expected the opcode to be bias_add or bias_multiply or relu_backward, but found " + opcode);
 		}
-		_in2 = in2;
-		_cptype = CPINSTRUCTION_TYPE.Convolution;
-		_numThreads = numThreads;
-		_intermediateMemoryBudget = intermediateMemoryBudget;
 	}
 	
 	public ConvolutionCPInstruction(CPOperand in, CPOperand in2, CPOperand in3, CPOperand out, String opcode, String istr, int numThreads, double intermediateMemoryBudget) throws DMLRuntimeException {
-		super(new ReorgOperator(SwapIndex.getSwapIndexFnObject()), in, out,
-				opcode, istr);
+		this(in, in2, in3, out, null, null, null, null, numThreads, intermediateMemoryBudget, opcode, istr);
 		if( !opcode.equals("channel_sums") ) {
 			throw new DMLRuntimeException("Incorrect usage. Expected the opcode to be channel_sums, but found " + opcode);
 		}
-		_in2 = in2;
-		_in3 = in3;
-		_cptype = CPINSTRUCTION_TYPE.Convolution;
-		_numThreads = numThreads;
-		_intermediateMemoryBudget = intermediateMemoryBudget;
 	}
 
 	private ConvolutionCPInstruction(CPOperand in, CPOperand out, String opcode, String istr,
 			ArrayList<CPOperand> stride, ArrayList<CPOperand> padding, ArrayList<CPOperand> input_shape,
 			ArrayList<CPOperand> filter_shape, int numThreads, double intermediateMemoryBudget) {
-		super(new ReorgOperator(SwapIndex.getSwapIndexFnObject()), in, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.Convolution;
-		_stride = stride;
-		_padding = padding;
-		_input_shape = input_shape;
-		_filter_shape = filter_shape;
-		_numThreads = numThreads;
-		_intermediateMemoryBudget = intermediateMemoryBudget;
+		this(in, null, null, out, stride, padding, input_shape, filter_shape, numThreads, intermediateMemoryBudget, opcode, istr);
 	}
 	
 	public ConvolutionCPInstruction(CPOperand in, CPOperand in2, CPOperand out, String opcode,
 			String istr, ArrayList<CPOperand> stride,
 			ArrayList<CPOperand> padding, ArrayList<CPOperand> input_shape,
 			ArrayList<CPOperand> filter_shape, int numThreads, double intermediateMemoryBudget) {
-		super(new ReorgOperator(SwapIndex.getSwapIndexFnObject()), in, out,
-				opcode, istr);
-		_in2 = in2;
-		_cptype = CPINSTRUCTION_TYPE.Convolution;
-		_stride = stride;
-		_padding = padding;
-		_input_shape = input_shape;
-		_filter_shape = filter_shape;
-		_numThreads = numThreads;
-		_intermediateMemoryBudget = intermediateMemoryBudget;
+		this(in, in2, null, out, stride, padding, input_shape, filter_shape, numThreads, intermediateMemoryBudget, opcode, istr);
 	}
 	
 	public ConvolutionCPInstruction(CPOperand in, CPOperand in2, CPOperand in3, CPOperand out, String opcode,
 			String istr, ArrayList<CPOperand> stride,
 			ArrayList<CPOperand> padding, ArrayList<CPOperand> input_shape,
 			ArrayList<CPOperand> filter_shape, int numThreads, double intermediateMemoryBudget) {
-		super(new ReorgOperator(SwapIndex.getSwapIndexFnObject()), in, out,
-				opcode, istr);
-		_in2 = in2;
-		_in3 = in3;
-		_cptype = CPINSTRUCTION_TYPE.Convolution;
-		_stride = stride;
-		_padding = padding;
-		_input_shape = input_shape;
-		_filter_shape = filter_shape;
-		_numThreads = numThreads;
-		_intermediateMemoryBudget = intermediateMemoryBudget;
+		this(in, in2, in3, out, stride, padding, input_shape, filter_shape, numThreads, intermediateMemoryBudget, opcode, istr);
 	}
 
 	public static ConvolutionCPInstruction parseInstruction(String str)

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/CovarianceCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/CovarianceCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/CovarianceCPInstruction.java
index d7eb24f..9457c65 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/CovarianceCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/CovarianceCPInstruction.java
@@ -33,14 +33,12 @@ public class CovarianceCPInstruction extends BinaryCPInstruction {
 
 	private CovarianceCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String istr) {
-		super(op, in1, in2, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.AggregateBinary;
+		super(CPType.AggregateBinary, op, in1, in2, out, opcode, istr);
 	}
 
 	private CovarianceCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
 			String opcode, String istr) {
-		super(op, in1, in2, in3, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.AggregateBinary;
+		super(CPType.AggregateBinary, op, in1, in2, in3, out, opcode, istr);
 	}
 
 	public static CovarianceCPInstruction parseInstruction( String str ) 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/DataGenCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/DataGenCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/DataGenCPInstruction.java
index 46f2116..236f2ae 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/DataGenCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/DataGenCPInstruction.java
@@ -36,30 +36,24 @@ public class DataGenCPInstruction extends UnaryCPInstruction {
 
 	private DataGenMethod method = DataGenMethod.INVALID;
 
-	private long rows;
-	private long cols;
-	private int rowsInBlock;
-	private int colsInBlock;
-	private double minValue;
-	private double maxValue;
-	private double sparsity;
-	private String pdf, pdfParams;
-	private long seed = 0;
+	private final long rows, cols;
+	private final int rowsInBlock, colsInBlock;
+	private final double minValue, maxValue, sparsity;
+	private final String pdf, pdfParams;
+	private final long seed;
 
 	// sequence specific attributes
-	private double seq_from;
-	private double seq_to;
-	private double seq_incr;
+	private final double seq_from, seq_to, seq_incr;
 
 	// sample specific attributes
-	private boolean replace;
-	private int numThreads = -1;
-
-	private DataGenCPInstruction(Operator op, DataGenMethod mthd, CPOperand in, CPOperand out, long rows, long cols,
-			int rpb, int cpb, double minValue, double maxValue, double sparsity, long seed,
-			String probabilityDensityFunction, String pdfParams, int k, String opcode, String istr) {
-		super(op, in, out, opcode, istr);
+	private final boolean replace;
+	private final int numThreads;
 
+	private DataGenCPInstruction(Operator op, DataGenMethod mthd, CPOperand in, CPOperand out, 
+			long rows, long cols, int rpb, int cpb, double minValue, double maxValue, double sparsity, long seed,
+			String probabilityDensityFunction, String pdfParams, int k, 
+			double seqFrom, double seqTo, double seqIncr, boolean replace, String opcode, String istr) {
+		super(CPType.Rand, op, in, out, opcode, istr);
 		this.method = mthd;
 		this.rows = rows;
 		this.cols = cols;
@@ -70,94 +64,60 @@ public class DataGenCPInstruction extends UnaryCPInstruction {
 		this.sparsity = sparsity;
 		this.seed = seed;
 		this.pdf = probabilityDensityFunction;
-		this.numThreads = k;
 		this.pdfParams = pdfParams;
+		this.numThreads = k;
+		this.seq_from = seqFrom;
+		this.seq_to = seqTo;
+		this.seq_incr = seqIncr;
+		this.replace = replace;
+	}
+	
+	private DataGenCPInstruction(Operator op, DataGenMethod mthd, CPOperand in, CPOperand out, long rows, long cols,
+			int rpb, int cpb, double minValue, double maxValue, double sparsity, long seed,
+			String probabilityDensityFunction, String pdfParams, int k, String opcode, String istr) {
+		this(op, mthd, in, out, rows, cols, rpb, cpb, minValue, maxValue, sparsity, seed, 
+			probabilityDensityFunction, pdfParams, k, -1, -1, -1, false, opcode, istr);
 	}
 
 	private DataGenCPInstruction(Operator op, DataGenMethod mthd, CPOperand in, CPOperand out, long rows, long cols,
 			int rpb, int cpb, double maxValue, boolean replace, long seed, String opcode, String istr) {
-		super(op, in, out, opcode, istr);
-
-		this.method = mthd;
-		this.rows = rows;
-		this.cols = cols;
-		this.rowsInBlock = rpb;
-		this.colsInBlock = cpb;
-		this.maxValue = maxValue;
-		this.replace = replace;
-		this.seed = seed;
+		this(op, mthd, in, out, rows, cols, rpb, cpb, 0, maxValue, 1.0, seed, 
+			null, null, 1, -1, -1, -1, replace, opcode, istr);
 	}
 
 	private DataGenCPInstruction(Operator op, DataGenMethod mthd, CPOperand in, CPOperand out, long rows, long cols,
 			int rpb, int cpb, double seqFrom, double seqTo, double seqIncr, String opcode, String istr) {
-		super(op, in, out, opcode, istr);
-
-		this.method = mthd;
-		this.rows = rows;
-		this.cols = cols;
-		this.rowsInBlock = rpb;
-		this.colsInBlock = cpb;
-		this.seq_from = seqFrom;
-		this.seq_to = seqTo;
-		this.seq_incr = seqIncr;
+		this(op, mthd, in, out, rows, cols, rpb, cpb, 0, 1, 1.0, -1, 
+			null, null, 1, seqFrom, seqTo, seqIncr, false, opcode, istr);
 	}
 
 	public long getRows() {
 		return rows;
 	}
 
-	public void setRows(long rows) {
-		this.rows = rows;
-	}
-
 	public long getCols() {
 		return cols;
 	}
-
-	public void setCols(long cols) {
-		this.cols = cols;
-	}
-
 	public int getRowsInBlock() {
 		return rowsInBlock;
 	}
 
-	public void setRowsInBlock(int rowsInBlock) {
-		this.rowsInBlock = rowsInBlock;
-	}
-
 	public int getColsInBlock() {
 		return colsInBlock;
 	}
 
-	public void setColsInBlock(int colsInBlock) {
-		this.colsInBlock = colsInBlock;
-	}
-
 	public double getMinValue() {
 		return minValue;
 	}
 
-	public void setMinValue(double minValue) {
-		this.minValue = minValue;
-	}
-
 	public double getMaxValue() {
 		return maxValue;
 	}
 
-	public void setMaxValue(double maxValue) {
-		this.maxValue = maxValue;
-	}
-
 	public double getSparsity() {
 		return sparsity;
 	}
 
-	public void setSparsity(double sparsity) {
-		this.sparsity = sparsity;
-	}
-
 	public static DataGenCPInstruction parseInstruction(String str) 
 		throws DMLRuntimeException 
 	{
@@ -299,12 +259,12 @@ public class DataGenCPInstruction extends UnaryCPInstruction {
 		else if ( method == DataGenMethod.SEQ ) 
 		{
 			//handle default 1 to -1 for special case of from>to
-			seq_incr = LibMatrixDatagen.updateSeqIncr(seq_from, seq_to, seq_incr);
+			double lSeqIncr = LibMatrixDatagen.updateSeqIncr(seq_from, seq_to, seq_incr);
 			
 			if( LOG.isTraceEnabled() )
-				LOG.trace("Process DataGenCPInstruction seq with seqFrom="+seq_from+", seqTo="+seq_to+", seqIncr"+seq_incr);
+				LOG.trace("Process DataGenCPInstruction seq with seqFrom="+seq_from+", seqTo="+seq_to+", seqIncr"+lSeqIncr);
 			
-			soresBlock = MatrixBlock.seqOperations(seq_from, seq_to, seq_incr);
+			soresBlock = MatrixBlock.seqOperations(seq_from, seq_to, lSeqIncr);
 		}
 		else if ( method == DataGenMethod.SAMPLE ) 
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/DataPartitionCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/DataPartitionCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/DataPartitionCPInstruction.java
index 6bb6fcd..50fad3b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/DataPartitionCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/DataPartitionCPInstruction.java
@@ -38,12 +38,11 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class DataPartitionCPInstruction extends UnaryCPInstruction {
 
-	private PDataPartitionFormat _pformat = null;
+	private final PDataPartitionFormat _pformat;
 
 	private DataPartitionCPInstruction(Operator op, CPOperand in1, PDataPartitionFormat pformat, CPOperand out,
 			String opcode, String istr) {
-		super(op, in1, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MMTSJ;
+		super(CPType.Partition, op, in1, out, opcode, istr);
 		_pformat = pformat;
 	}
 
@@ -73,7 +72,7 @@ public class DataPartitionCPInstruction extends UnaryCPInstruction {
 		MatrixBlock mb = moIn.acquireRead();
 		
 		//execute operations 
-		MatrixObject moOut = (MatrixObject) ec.getVariable(output.getName());		
+		MatrixObject moOut = (MatrixObject) ec.getVariable(output.getName());
 		String fname = moOut.getFileName();
 		moOut.setPartitioned(_pformat, -1); //modify meta data output
 		try
@@ -94,6 +93,6 @@ public class DataPartitionCPInstruction extends UnaryCPInstruction {
 		}
 		
 		//release input
-		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());		
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/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 e785196..1c001f0 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
@@ -39,17 +39,15 @@ import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 
 public class FunctionCallCPInstruction extends CPInstruction {
-	private String _functionName;
-	private String _namespace;
+	private final String _functionName;
+	private final String _namespace;
 	private final CPOperand[] _boundInputs;
 	private final ArrayList<String> _boundInputNames;
 	private final ArrayList<String> _boundOutputNames;
-	private HashSet<String> _expectRetVars = null;
 
 	private FunctionCallCPInstruction(String namespace, String functName, CPOperand[] boundInputs, 
 		ArrayList<String> boundInputNames, ArrayList<String> boundOutputNames, String istr) {
-		super(null, functName, istr);
-		_cptype = CPINSTRUCTION_TYPE.External;
+		super(CPType.External, null, functName, istr);
 		_functionName = functName;
 		_namespace = namespace;
 		_boundInputs = boundInputs;
@@ -173,15 +171,13 @@ public class FunctionCallCPInstruction extends CPInstruction {
 		}
 		
 		// cleanup all returned variables w/o binding 
-		if( _expectRetVars == null ) {
-			_expectRetVars = new HashSet<>();
-			for(DataIdentifier di : fpb.getOutputParams())
-				_expectRetVars.add(di.getName());
-		}
+		HashSet<String> expectRetVars = new HashSet<>();
+		for(DataIdentifier di : fpb.getOutputParams())
+			expectRetVars.add(di.getName());
 		
 		LocalVariableMap retVars = fn_ec.getVariables();
 		for( Entry<String,Data> var : retVars.entrySet() ) {
-			if( _expectRetVars.contains(var.getKey()) )
+			if( expectRetVars.contains(var.getKey()) )
 				continue;
 			//cleanup unexpected return values to avoid leaks
 			if( var.getValue() instanceof MatrixObject )
@@ -237,17 +233,6 @@ public class FunctionCallCPInstruction extends CPInstruction {
 		return _boundOutputNames;
 	}
 
-	public void setFunctionName(String fname)
-	{
-		//update instruction string
-		String oldfname = _functionName;
-		instString = updateInstStringFunctionName(oldfname, fname);
-		
-		//set attribute
-		_functionName = fname;
-		instOpcode = fname;
-	}
-
 	public String updateInstStringFunctionName(String pattern, String replace)
 	{
 		//split current instruction

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/IndexingCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/IndexingCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/IndexingCPInstruction.java
index e3cb100..20fd852 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/IndexingCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/IndexingCPInstruction.java
@@ -30,11 +30,11 @@ import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
 import org.apache.sysml.runtime.util.IndexRange;
 
 public abstract class IndexingCPInstruction extends UnaryCPInstruction {
-	protected CPOperand rowLower, rowUpper, colLower, colUpper;
+	protected final CPOperand rowLower, rowUpper, colLower, colUpper;
 
 	protected IndexingCPInstruction(Operator op, CPOperand in, CPOperand rl, CPOperand ru, CPOperand cl, CPOperand cu,
 			CPOperand out, String opcode, String istr) {
-		super(op, in, out, opcode, istr);
+		super(CPType.MatrixIndexing, op, in, out, opcode, istr);
 		rowLower = rl;
 		rowUpper = ru;
 		colLower = cl;
@@ -43,7 +43,7 @@ public abstract class IndexingCPInstruction extends UnaryCPInstruction {
 
 	protected IndexingCPInstruction(Operator op, CPOperand lhsInput, CPOperand rhsInput, CPOperand rl, CPOperand ru,
 			CPOperand cl, CPOperand cu, CPOperand out, String opcode, String istr) {
-		super(op, lhsInput, rhsInput, out, opcode, istr);
+		super(CPType.MatrixIndexing, op, lhsInput, rhsInput, out, opcode, istr);
 		rowLower = rl;
 		rowUpper = ru;
 		colLower = cl;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MMChainCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MMChainCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MMChainCPInstruction.java
index de898fe..b69772b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MMChainCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MMChainCPInstruction.java
@@ -28,13 +28,12 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class MMChainCPInstruction extends UnaryCPInstruction {
 
-	private ChainType _type = null;
-	private int _numThreads = -1;
+	private final ChainType _type;
+	private final int _numThreads;
 
 	private MMChainCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
 			ChainType type, int k, String opcode, String istr) {
-		super(op, in1, in2, in3, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MMChain;
+		super(CPType.MMChain, op, in1, in2, in3, out, opcode, istr);
 		_type = type;
 		_numThreads = k;
 	}
@@ -43,7 +42,7 @@ public class MMChainCPInstruction extends UnaryCPInstruction {
 		throws DMLRuntimeException 
 	{
 		//parse instruction parts (without exec type)
-		String[] parts = InstructionUtils.getInstructionPartsWithValueType( str );		
+		String[] parts = InstructionUtils.getInstructionPartsWithValueType( str );
 		InstructionUtils.checkNumFields( parts, 5, 6 );
 	
 		String opcode = parts[0];

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MMTSJCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MMTSJCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MMTSJCPInstruction.java
index eb414e7..edd420a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MMTSJCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MMTSJCPInstruction.java
@@ -27,13 +27,12 @@ import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class MMTSJCPInstruction extends UnaryCPInstruction {
-	private MMTSJType _type = null;
-	private int _numThreads = 1;
+	private final MMTSJType _type;
+	private final int _numThreads;
 
 	private MMTSJCPInstruction(Operator op, CPOperand in1, MMTSJType type, CPOperand out, int k, String opcode,
 			String istr) {
-		super(op, in1, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MMTSJ;
+		super(CPType.MMTSJ, op, in1, out, opcode, istr);
 		_type = type;
 		_numThreads = k;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
index 30000f9..84e3ebb 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
@@ -28,7 +28,7 @@ import org.apache.sysml.runtime.matrix.operators.UnaryOperator;
 
 public class MatrixBuiltinCPInstruction extends BuiltinUnaryCPInstruction {
 	protected MatrixBuiltinCPInstruction(Operator op, CPOperand in, CPOperand out, String opcode, String instr) {
-		super(op, in, out, 1, opcode, instr);
+		super(op, in, out, opcode, instr);
 	}
 
 	@Override 
@@ -55,6 +55,6 @@ public class MatrixBuiltinCPInstruction extends BuiltinUnaryCPInstruction {
 	 		}
 			
 			ec.setMatrixOutput(output_name, retBlock, getExtendedOpcode());
-		}		
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinNaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinNaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinNaryCPInstruction.java
index 8ad619b..c3c04b4 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinNaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixBuiltinNaryCPInstruction.java
@@ -24,7 +24,7 @@ import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 
-public class MatrixBuiltinNaryCPInstruction extends BuiltinNary {
+public class MatrixBuiltinNaryCPInstruction extends BuiltinNaryCPInstruction {
 
 	protected MatrixBuiltinNaryCPInstruction(Operator op, String opcode, String istr, CPOperand output, CPOperand[] inputs) {
 		super(op, opcode, istr, output, inputs);

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
index 96a1e86..f247740 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
@@ -31,7 +31,7 @@ public class MatrixMatrixBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 
 	protected MatrixMatrixBuiltinCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String istr) {
-		super(op, in1, in2, out, 2, opcode, istr);
+		super(op, in1, in2, out, opcode, istr);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixReshapeCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixReshapeCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixReshapeCPInstruction.java
index 6474bd6..d13f97c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixReshapeCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixReshapeCPInstruction.java
@@ -29,15 +29,13 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class MatrixReshapeCPInstruction extends UnaryCPInstruction {
 
-	private CPOperand _opRows = null;
-	private CPOperand _opCols = null;
-	private CPOperand _opByRow = null;
+	private final CPOperand _opRows;
+	private final CPOperand _opCols;
+	private final CPOperand _opByRow;
 
 	private MatrixReshapeCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand in4,
 			CPOperand out, String opcode, String istr) {
-		super(op, in1, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MatrixReshape;
-
+		super(CPType.MatrixReshape, op, in1, out, opcode, istr);
 		_opRows = in2;
 		_opCols = in3;
 		_opByRow = in4;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
index 35c5177..964957e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
@@ -30,7 +30,7 @@ public class MatrixScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 
 	protected MatrixScalarBuiltinCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String instr) {
-		super(op, in1, in2, out, 2, opcode, instr);
+		super(op, in1, in2, out, opcode, instr);
 	}
 
 	@Override 
@@ -44,7 +44,7 @@ public class MatrixScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
index d69f08c..343a821 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
@@ -33,22 +33,15 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class MultiReturnBuiltinCPInstruction extends ComputationCPInstruction {
 
-	int arity;
 	protected ArrayList<CPOperand> _outputs;
 
 	private MultiReturnBuiltinCPInstruction(Operator op, CPOperand input1, ArrayList<CPOperand> outputs, String opcode,
 			String istr) {
-		super(op, input1, null, outputs.get(0), opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MultiReturnBuiltin;
+		super(CPType.MultiReturnBuiltin, op, input1, null, outputs.get(0), opcode, istr);
 		_outputs = outputs;
 	}
-
-	public int getArity() {
-		return arity;
-	}
 	
-	public CPOperand getOutput(int i)
-	{
+	public CPOperand getOutput(int i) {
 		return _outputs.get(i);
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
index 27013a7..c4f61bb 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
@@ -33,12 +33,11 @@ import org.apache.sysml.runtime.transform.encode.Encoder;
 import org.apache.sysml.runtime.transform.encode.EncoderFactory;
 
 public class MultiReturnParameterizedBuiltinCPInstruction extends ComputationCPInstruction {
-	protected ArrayList<CPOperand> _outputs;
+	protected final ArrayList<CPOperand> _outputs;
 
 	private MultiReturnParameterizedBuiltinCPInstruction(Operator op, CPOperand input1, CPOperand input2,
 			ArrayList<CPOperand> outputs, String opcode, String istr) {
-		super(op, input1, input2, outputs.get(0), opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.MultiReturnBuiltin;
+		super(CPType.MultiReturnBuiltin, op, input1, input2, outputs.get(0), opcode, istr);
 		_outputs = outputs;
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/PMMJCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/PMMJCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/PMMJCPInstruction.java
index ec42e05..bb0cb2a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/PMMJCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/PMMJCPInstruction.java
@@ -27,11 +27,11 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 
 public class PMMJCPInstruction extends ComputationCPInstruction {
 
-	private int _numThreads = -1;
+	private final int _numThreads;
 
 	private PMMJCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out, int k,
 			String opcode, String istr) {
-		super(op, in1, in2, in3, out, opcode, istr);
+		super(CPType.AggregateBinary, op, in1, in2, in3, out, opcode, istr);
 		_numThreads = k;
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
index f6532d7..5a7442a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
@@ -55,19 +55,13 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 	private static final String TOSTRING_SEPARATOR = " ";
 	private static final String TOSTRING_LINESEPARATOR = "\n";
 
-	private int arity;
-	protected HashMap<String, String> params;
+	protected final HashMap<String, String> params;
 
 	protected ParameterizedBuiltinCPInstruction(Operator op, HashMap<String, String> paramsMap, CPOperand out,
 			String opcode, String istr) {
-		super(op, null, null, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.ParameterizedBuiltin;
+		super(CPType.ParameterizedBuiltin, op, null, null, out, opcode, istr);
 		params = paramsMap;
 	}
-
-	public int getArity() {
-		return arity;
-	}
 	
 	public HashMap<String,String> getParameterMap() { 
 		return params; 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/PlusMultCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/PlusMultCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/PlusMultCPInstruction.java
index c49f229..27f8521 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/PlusMultCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/PlusMultCPInstruction.java
@@ -28,11 +28,11 @@ import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 
-public class PlusMultCPInstruction extends ArithmeticBinaryCPInstruction {
+public class PlusMultCPInstruction extends ComputationCPInstruction {
+	
 	private PlusMultCPInstruction(BinaryOperator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
 			String opcode, String str) {
-		super(op, in1, in2, out, opcode, str);
-		input3 = in3;
+		super(CPType.ArithmeticBinary, op, in1, in2, in3, out, opcode, str);
 	}
 
 	public static PlusMultCPInstruction parseInstruction(String str)

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantilePickCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantilePickCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantilePickCPInstruction.java
index 1ad6054..2950a1c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantilePickCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantilePickCPInstruction.java
@@ -36,8 +36,8 @@ import org.apache.sysml.runtime.util.UtilFunctions;
 
 public class QuantilePickCPInstruction extends BinaryCPInstruction {
 
-	private OperationTypes _type = null;
-	private boolean _inmem = true;
+	private final OperationTypes _type;
+	private final boolean _inmem;
 
 	private QuantilePickCPInstruction(Operator op, CPOperand in, CPOperand out, OperationTypes type, boolean inmem,
 			String opcode, String istr) {
@@ -46,8 +46,7 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction {
 
 	private QuantilePickCPInstruction(Operator op, CPOperand in, CPOperand in2, CPOperand out, OperationTypes type,
 			boolean inmem, String opcode, String istr) {
-		super(op, in, in2, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.QPick;
+		super(CPType.QPick, op, in, in2, out, opcode, istr);
 		_type = type;
 		_inmem = inmem;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantileSortCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantileSortCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantileSortCPInstruction.java
index f25c225..b7a5254 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantileSortCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuantileSortCPInstruction.java
@@ -45,8 +45,7 @@ public class QuantileSortCPInstruction extends UnaryCPInstruction {
 
 	private QuantileSortCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String istr) {
-		super(op, in1, in2, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.QSort;
+		super(CPType.QSort, op, in1, in2, out, opcode, istr);
 	}
 
 	public static QuantileSortCPInstruction parseInstruction ( String str ) 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/QuaternaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuaternaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuaternaryCPInstruction.java
index d4e89ce..b0bb392 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/QuaternaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/QuaternaryCPInstruction.java
@@ -35,13 +35,12 @@ import org.apache.sysml.runtime.matrix.operators.QuaternaryOperator;
 
 public class QuaternaryCPInstruction extends ComputationCPInstruction {
 
-	private CPOperand input4 = null;
-	private int _numThreads = -1;
+	private final CPOperand input4;
+	private final int _numThreads;
 
 	private QuaternaryCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand in4,
 			CPOperand out, int k, String opcode, String istr) {
-		super(op, in1, in2, in3, out, opcode, istr);
-
+		super(CPType.Quaternary, op, in1, in2, in3, out, opcode, istr);
 		input4 = in4;
 		_numThreads = k;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
index 29d2304..b04aa9f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
@@ -29,8 +29,7 @@ public abstract class RelationalBinaryCPInstruction extends BinaryCPInstruction
 
 	protected RelationalBinaryCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String istr) {
-		super(op, in1, in2, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.RelationalBinary;
+		super(CPType.RelationalBinary, op, in1, in2, out, opcode, istr);
 	}
 
 	public static RelationalBinaryCPInstruction parseInstruction ( String str ) throws DMLRuntimeException {

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ReorgCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ReorgCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ReorgCPInstruction.java
index c52f020..3788d68 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ReorgCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ReorgCPInstruction.java
@@ -34,9 +34,9 @@ import org.apache.sysml.runtime.matrix.operators.ReorgOperator;
 
 public class ReorgCPInstruction extends UnaryCPInstruction {
 	// sort-specific attributes (to enable variable attributes)
-	private CPOperand _col = null;
-	private CPOperand _desc = null;
-	private CPOperand _ixret = null;
+	private final CPOperand _col;
+	private final CPOperand _desc;
+	private final CPOperand _ixret;
 
 	/**
 	 * for opcodes r' and rdiag
@@ -53,8 +53,7 @@ public class ReorgCPInstruction extends UnaryCPInstruction {
 	 *            ?
 	 */
 	private ReorgCPInstruction(Operator op, CPOperand in, CPOperand out, String opcode, String istr) {
-		super(op, in, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.Reorg;
+		this(op, in, out, null, null, null, opcode, istr);
 	}
 
 	/**
@@ -77,9 +76,9 @@ public class ReorgCPInstruction extends UnaryCPInstruction {
 	 * @param istr
 	 *            ?
 	 */
-	private ReorgCPInstruction(Operator op, CPOperand in, CPOperand col, CPOperand desc, CPOperand ixret, CPOperand out,
+	private ReorgCPInstruction(Operator op, CPOperand in, CPOperand out, CPOperand col, CPOperand desc, CPOperand ixret,
 			String opcode, String istr) {
-		this(op, in, out, opcode, istr);
+		super(CPType.Reorg, op, in, out, opcode, istr);
 		_col = col;
 		_desc = desc;
 		_ixret = ixret;
@@ -117,7 +116,7 @@ public class ReorgCPInstruction extends UnaryCPInstruction {
 			CPOperand desc = new CPOperand(parts[3]);
 			CPOperand ixret = new CPOperand(parts[4]);
 			return new ReorgCPInstruction(new ReorgOperator(SortIndex.getSortIndexFnObject(1,false,false)), 
-					                      in, col, desc, ixret, out, opcode, str);
+						in, out, col, desc, ixret, opcode, str);
 		}
 		else {
 			throw new DMLRuntimeException("Unknown opcode while parsing a ReorgInstruction: " + str);
@@ -129,19 +128,19 @@ public class ReorgCPInstruction extends UnaryCPInstruction {
 			throws DMLRuntimeException 
 	{
 		//acquire inputs
-		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());		
+		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		ReorgOperator r_op = (ReorgOperator) _optr;
 		if( r_op.fn instanceof SortIndex ) {
 			//additional attributes for sort
 			int col = (int)ec.getScalarInput(_col.getName(), _col.getValueType(), _col.isLiteral()).getLongValue();
 			boolean desc = ec.getScalarInput(_desc.getName(), _desc.getValueType(), _desc.isLiteral()).getBooleanValue();
 			boolean ixret = ec.getScalarInput(_ixret.getName(), _ixret.getValueType(), _ixret.isLiteral()).getBooleanValue();
-			r_op.fn = SortIndex.getSortIndexFnObject(col, desc, ixret);
+			r_op = r_op.setFn(SortIndex.getSortIndexFnObject(col, desc, ixret));
 		}
 		
 		//execute operation
 		MatrixBlock soresBlock = (MatrixBlock) (matBlock.reorgOperations(r_op, new MatrixBlock(), 0, 0, 0));
-        
+		
 		//release inputs/outputs
 		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 		ec.setMatrixOutput(output.getName(), soresBlock, getExtendedOpcode());

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinCPInstruction.java
index 9ced515..77fb73e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinCPInstruction.java
@@ -30,7 +30,7 @@ import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
 public class ScalarBuiltinCPInstruction extends BuiltinUnaryCPInstruction {
 
 	protected ScalarBuiltinCPInstruction(Operator op, CPOperand in, CPOperand out, String opcode, String instr) {
-		super(op, in, out, 1, opcode, instr);
+		super(op, in, out, opcode, instr);
 	}
 
 	@Override 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinNaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinNaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinNaryCPInstruction.java
index 9b15696..04408a8 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinNaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarBuiltinNaryCPInstruction.java
@@ -36,7 +36,7 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
  * string.
  *
  */
-public class ScalarBuiltinNaryCPInstruction extends BuiltinNary {
+public class ScalarBuiltinNaryCPInstruction extends BuiltinNaryCPInstruction {
 
 	protected ScalarBuiltinNaryCPInstruction(Operator op, String opcode, String istr, CPOperand output, CPOperand[] inputs) {
 		super(op, opcode, istr, output, inputs);

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
index 3ad836b..e1f177f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
@@ -45,7 +45,7 @@ public class ScalarMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
index 00e6616..12f545d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
@@ -45,7 +45,7 @@ public class ScalarMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarBuiltinCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarBuiltinCPInstruction.java
index 8722358..217bb96 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ScalarScalarBuiltinCPInstruction.java
@@ -29,7 +29,7 @@ public class ScalarScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 
 	protected ScalarScalarBuiltinCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String instr) {
-		super(op, in1, in2, out, 2, opcode, instr);
+		super(op, in1, in2, out, opcode, instr);
 	}
 
 	@Override 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/SpoofCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/SpoofCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/SpoofCPInstruction.java
index 021ab34..16ab1a4 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/SpoofCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/SpoofCPInstruction.java
@@ -37,7 +37,7 @@ public class SpoofCPInstruction extends ComputationCPInstruction {
 
 	private SpoofCPInstruction(SpoofOperator op, Class<?> cla, int k, CPOperand[] in, CPOperand out, String opcode,
 			String str) {
-		super(null, null, null, out, opcode, str);
+		super(CPType.SpoofFused, null, null, null, out, opcode, str);
 		_class = cla;
 		_op = op;
 		_numThreads = k;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/StringInitCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/StringInitCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/StringInitCPInstruction.java
index f8897ba..4b89573 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/StringInitCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/StringInitCPInstruction.java
@@ -32,13 +32,13 @@ import org.apache.sysml.runtime.matrix.operators.Operator;
 public class StringInitCPInstruction extends UnaryCPInstruction {
 	private static final String DELIM = " ";
 
-	private long _rlen = -1;
-	private long _clen = -1;
-	private String _data = null;
+	private final long _rlen;
+	private final long _clen;
+	private final String _data;
 
 	private StringInitCPInstruction(Operator op, CPOperand in, CPOperand out, long rows, long cols, int rpb, int cpb,
 			String data, String opcode, String inst) {
-		super(op, in, out, opcode, inst);
+		super(CPType.StringInit, op, in, out, opcode, inst);
 		_rlen = rows;
 		_clen = cols;
 		_data = data;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/TernaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/TernaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/TernaryCPInstruction.java
index 2e970c1..6d42976 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/TernaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/TernaryCPInstruction.java
@@ -33,17 +33,17 @@ import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
 import org.apache.sysml.runtime.util.DataConverter;
 
 public class TernaryCPInstruction extends ComputationCPInstruction {
-	private String _outDim1;
-	private String _outDim2;
-	private boolean _dim1Literal;
-	private boolean _dim2Literal;
-	private boolean _isExpand;
-	private boolean _ignoreZeros;
+	private final String _outDim1;
+	private final String _outDim2;
+	private final boolean _dim1Literal;
+	private final boolean _dim2Literal;
+	private final boolean _isExpand;
+	private final boolean _ignoreZeros;
 
 	private TernaryCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
 			String outputDim1, boolean dim1Literal, String outputDim2, boolean dim2Literal, boolean isExpand,
 			boolean ignoreZeros, String opcode, String istr) {
-		super(op, in1, in2, in3, out, opcode, istr);
+		super(CPType.Ternary, op, in1, in2, in3, out, opcode, istr);
 		_outDim1 = outputDim1;
 		_dim1Literal = dim1Literal;
 		_outDim2 = outputDim2;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/UaggOuterChainCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/UaggOuterChainCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/UaggOuterChainCPInstruction.java
index 007fadc..e48f396 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/UaggOuterChainCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/UaggOuterChainCPInstruction.java
@@ -33,19 +33,14 @@ import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 
 public class UaggOuterChainCPInstruction extends UnaryCPInstruction {
-	// operators
-	private AggregateUnaryOperator _uaggOp = null;
-	private BinaryOperator _bOp = null;
+	private final AggregateUnaryOperator _uaggOp;
+	private final BinaryOperator _bOp;
 
 	private UaggOuterChainCPInstruction(BinaryOperator bop, AggregateUnaryOperator uaggop, AggregateOperator aggop,
 			CPOperand in1, CPOperand in2, CPOperand out, String opcode, String istr) {
-		super(bop, in1, in2, out, opcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.UaggOuterChain;
-
+		super(CPType.UaggOuterChain, bop, in1, in2, out, opcode, istr);
 		_uaggOp = uaggop;
 		_bOp = bop;
-
-		instString = istr;
 	}
 
 	public static UaggOuterChainCPInstruction parseInstruction(String str)

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/cp/UnaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/UnaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/UnaryCPInstruction.java
index 384357a..f3f903a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/UnaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/UnaryCPInstruction.java
@@ -27,18 +27,18 @@ import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
 
 public abstract class UnaryCPInstruction extends ComputationCPInstruction {
 
-	protected UnaryCPInstruction(Operator op, CPOperand in, CPOperand out, String opcode, String instr) {
-		this(op, in, null, null, out, opcode, instr);
+	protected UnaryCPInstruction(CPType type, Operator op, CPOperand in, CPOperand out, String opcode, String instr) {
+		this(type, op, in, null, null, out, opcode, instr);
 	}
 
-	protected UnaryCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
+	protected UnaryCPInstruction(CPType type, Operator op, CPOperand in1, CPOperand in2, CPOperand out, String opcode,
 			String instr) {
-		this(op, in1, in2, null, out, opcode, instr);
+		this(type, op, in1, in2, null, out, opcode, instr);
 	}
 
-	protected UnaryCPInstruction(Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out, String opcode,
+	protected UnaryCPInstruction(CPType type, Operator op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out, String opcode,
 			String instr) {
-		super(op, in1, in2, in3, out, opcode, instr);
+		super(type, op, in1, in2, in3, out, opcode, instr);
 	}
 
 	static String parseUnaryInstruction(String instr, CPOperand in,

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/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 2239919..9ddcdcc 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
@@ -86,7 +86,7 @@ public class VariableCPInstruction extends CPInstruction {
 		CopyVariable,
 		MoveVariable,
 		RemoveVariable,
-		RemoveVariableAndFile,		 
+		RemoveVariableAndFile,
 		CastAsScalarVariable, 
 		CastAsMatrixVariable, 
 		CastAsFrameVariable,
@@ -98,58 +98,52 @@ public class VariableCPInstruction extends CPInstruction {
 		SetFileName, 
 	}
 	
-	private static IDSequence _uniqueVarID;	
+	private static final IDSequence _uniqueVarID = new IDSequence(true);
 	private static final int CREATEVAR_FILE_NAME_VAR_POS=3;
 	
 	private final VariableOperationCode opcode;
 	private final List<CPOperand> inputs;
 	private final CPOperand output;
-	private MetaData metadata;
-	private UpdateType _updateType;
+	private final MetaData metadata;
+	private final UpdateType _updateType;
 	
 	// Frame related members
-	private String _schema;
+	private final String _schema;
 	
 	// CSV related members (used only in createvar instructions)
-	private FileFormatProperties _formatProperties;
-	
-	static {
-		_uniqueVarID  = new IDSequence(true); 
-	}
+	private final FileFormatProperties _formatProperties;
 
 	private VariableCPInstruction(VariableOperationCode op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
-			int _arity, String sopcode, String istr) {
-		super(sopcode, istr);
-		_cptype = CPINSTRUCTION_TYPE.Variable;
+			MetaData meta, FileFormatProperties fprops, String schema, UpdateType utype, String sopcode, String istr) {
+		super(CPType.Variable, sopcode, istr);
 		opcode = op;
 		inputs = new ArrayList<>();
 		addInput(in1);
 		addInput(in2);
 		addInput(in3);
 		output = out;
-
-		_formatProperties = null;
-		_schema = null;
+		metadata = meta;
+		_formatProperties = fprops;
+		_schema = schema;
+		_updateType = utype;
+	}
+	
+	private VariableCPInstruction(VariableOperationCode op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
+			String sopcode, String istr) {
+		this(op, in1, in2, in3, out, null, null, null, null, sopcode, istr);
 	}
 
 	// This version of the constructor is used only in case of CreateVariable
 	private VariableCPInstruction(VariableOperationCode op, CPOperand in1, CPOperand in2, CPOperand in3, MetaData md,
-			UpdateType updateType, int _arity, String schema, String sopcode, String istr) {
-		this(op, in1, in2, in3, (CPOperand) null, _arity, sopcode, istr);
-		metadata = md;
-		_updateType = updateType;
-		_schema = schema;
+			UpdateType updateType, String schema, String sopcode, String istr) {
+		this(op, in1, in2, in3, null, md, null, schema, updateType, sopcode, istr);
 	}
 
 	// This version of the constructor is used only in case of CreateVariable
 	private VariableCPInstruction(VariableOperationCode op, CPOperand in1, CPOperand in2, CPOperand in3, MetaData md,
-			UpdateType updateType, int _arity, FileFormatProperties formatProperties, String schema, String sopcode,
+			UpdateType updateType, FileFormatProperties formatProperties, String schema, String sopcode,
 			String istr) {
-		this(op, in1, in2, in3, (CPOperand) null, _arity, sopcode, istr);
-		metadata = md;
-		_updateType = updateType;
-		_formatProperties = formatProperties;
-		_schema = schema;
+		this(op, in1, in2, in3, null, md, formatProperties, schema, updateType, sopcode, istr);
 	}
 
 	private static VariableOperationCode getVariableOperationCode ( String str ) throws DMLRuntimeException {
@@ -235,10 +229,6 @@ public class VariableCPInstruction extends CPInstruction {
 		return _formatProperties;
 	}
 	
-	public void setFormatProperties(FileFormatProperties prop) {
-		_formatProperties = prop;
-	}
-	
 	public List<CPOperand> getInputs() {
 		return inputs;
 	}
@@ -293,7 +283,6 @@ public class VariableCPInstruction extends CPInstruction {
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
 		String opcode = parts[0];
 		VariableOperationCode voc = getVariableOperationCode(opcode);
-		int _arity = -1;
 		
 		if ( voc == VariableOperationCode.CreateVariable ){
 			if ( parts.length < 5 )  //&& parts.length != 10 )
@@ -311,9 +300,8 @@ public class VariableCPInstruction extends CPInstruction {
 				throw new DMLRuntimeException("Invalid number of operands in write instruction: " + str);
 		}
 		else {
-			_arity = getArity(voc);
 			if( voc != VariableOperationCode.RemoveVariable )
-				InstructionUtils.checkNumFields ( parts, _arity ); // no output
+				InstructionUtils.checkNumFields ( parts, getArity(voc) ); // no output
 		}
 		
 		CPOperand in1=null, in2=null, in3=null, in4=null, out=null;
@@ -389,10 +377,10 @@ public class VariableCPInstruction extends CPInstruction {
 						naStrings = parts[16];
 					fmtProperties = new CSVFileFormatProperties(hasHeader, delim, fill, fillValue, naStrings) ;
 				}
-				return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, parts.length, fmtProperties, schema, opcode, str);
+				return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, fmtProperties, schema, opcode, str);
 			}
 			else {
-				return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, parts.length, schema, opcode, str);
+				return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, schema, opcode, str);
 			}
 		case AssignVariable:
 			in1 = new CPOperand(parts[1]);
@@ -414,7 +402,7 @@ public class VariableCPInstruction extends CPInstruction {
 			
 		case RemoveVariable:
 			VariableCPInstruction rminst = new VariableCPInstruction(
-				getVariableOperationCode(opcode), null, null, null, out, _arity, opcode, str);
+				getVariableOperationCode(opcode), null, null, null, out, opcode, str);
 			for( int i=1; i<parts.length; i++ )
 				rminst.addInput(new CPOperand(parts[i], ValueType.UNKNOWN, DataType.SCALAR));
 			return rminst;
@@ -442,22 +430,21 @@ public class VariableCPInstruction extends CPInstruction {
 			in2 = new CPOperand(parts[2]);
 			in3 = new CPOperand(parts[3]);
 			
-			VariableCPInstruction inst = new VariableCPInstruction(getVariableOperationCode(opcode), in1, in2, in3, out, _arity, opcode, str); 
-			
+			FileFormatProperties fprops = null;
 			if ( in3.getName().equalsIgnoreCase("csv") ) {
 				boolean hasHeader = Boolean.parseBoolean(parts[4]);
 				String delim = parts[5];
 				boolean sparse = Boolean.parseBoolean(parts[6]);
-				FileFormatProperties formatProperties = new CSVFileFormatProperties(hasHeader, delim, sparse);
-				inst.setFormatProperties(formatProperties);
+				fprops = new CSVFileFormatProperties(hasHeader, delim, sparse);
 				in4 = new CPOperand(parts[7]); // description
-				inst.addInput(in4);
 			} else {
-				FileFormatProperties ffp = new FileFormatProperties();
-				inst.setFormatProperties(ffp);
+				fprops = new FileFormatProperties();
 				in4 = new CPOperand(parts[4]); // description
-				inst.addInput(in4);
 			}
+			VariableCPInstruction inst = new VariableCPInstruction(
+				getVariableOperationCode(opcode), in1, in2, in3, out, null, fprops, null, null, opcode, str); 
+			inst.addInput(in4);
+			
 			return inst;
 			
 		case Read:
@@ -474,7 +461,7 @@ public class VariableCPInstruction extends CPInstruction {
 			break;
 		
 		}
-		return new VariableCPInstruction(getVariableOperationCode(opcode), in1, in2, in3, out, _arity, opcode, str);
+		return new VariableCPInstruction(getVariableOperationCode(opcode), in1, in2, in3, out, opcode, str);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/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 9dd163d..f5d2f46 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
@@ -147,7 +147,7 @@ public abstract class GPUInstruction extends Instruction {
 	protected boolean _requiresLabelUpdate = false;
 
 	private GPUInstruction(String opcode, String istr) {
-		type = INSTRUCTION_TYPE.GPU;
+		type = IType.GPU;
 		instString = istr;
 
 		// prepare opcode and update requirement for repeated usage

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixArithmeticGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixArithmeticGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixArithmeticGPUInstruction.java
index 7aba2ce..4552c1f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixArithmeticGPUInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixArithmeticGPUInstruction.java
@@ -55,7 +55,7 @@ public class ScalarMatrixArithmeticGPUInstruction extends ArithmeticBinaryGPUIns
 		ec.setMetaData(_output.getName(), rlen, clen);
 		
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 		
 		LibMatrixCUDA.matrixScalarArithmetic(ec, ec.getGPUContext(0), getExtendedOpcode(), in1, _output.getName(), isTransposed, sc_op);
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixRelationalBinaryGPUInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixRelationalBinaryGPUInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixRelationalBinaryGPUInstruction.java
index 7d9e180..59e717f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixRelationalBinaryGPUInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/gpu/ScalarMatrixRelationalBinaryGPUInstruction.java
@@ -51,7 +51,7 @@ public class ScalarMatrixRelationalBinaryGPUInstruction extends RelationalBinary
 		ec.setMetaData(_output.getName(), rlen, clen);
 
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 
 		LibMatrixCUDA.matrixScalarRelational(ec, ec.getGPUContext(0), getExtendedOpcode(), in1, _output.getName(), sc_op);
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
index e0a97bf..b8b566c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
@@ -42,7 +42,7 @@ public abstract class MRInstruction extends Instruction {
 	public byte output;
 
 	protected MRInstruction(Operator op, byte out) {
-		type = INSTRUCTION_TYPE.MAPREDUCE;
+		type = IType.MAPREDUCE;
 		optr = op;
 		output = out;
 		mrtype = MRINSTRUCTION_TYPE.INVALID;

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
index 58f6821..04f4a89 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
@@ -37,9 +37,6 @@ public class ScalarInstruction extends UnaryMRInstructionBase {
 		super(op, in, out);
 		mrtype = MRINSTRUCTION_TYPE.ArithmeticBinary;
 		instString = istr;
-
-		// value dependent safe-safeness (trigger re-evaluation sparse-safe)
-		op.setConstant(op.getConstant());
 	}
 
 	public static ScalarInstruction parseInstruction ( String str )

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/spark/BinarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/BinarySPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/BinarySPInstruction.java
index 2bd622a..948371d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/BinarySPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/BinarySPInstruction.java
@@ -176,7 +176,7 @@ public abstract class BinarySPInstruction extends ComputationSPInstruction {
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		ScalarOperator sc_op = (ScalarOperator) _optr;
-		sc_op.setConstant(constant.getDoubleValue());
+		sc_op = sc_op.setConstant(constant.getDoubleValue());
 		
 		//execute scalar matrix arithmetic instruction
 		JavaPairRDD<MatrixIndexes,MatrixBlock> out = in1.mapValues( new MatrixScalarUnaryFunction(sc_op) );

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/instructions/spark/SPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/SPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/SPInstruction.java
index bafe4e8..ae7da3d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/SPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/SPInstruction.java
@@ -45,7 +45,7 @@ public abstract class SPInstruction extends Instruction {
 	protected boolean _requiresLabelUpdate = false;
 
 	protected SPInstruction(String opcode, String istr) {
-		type = INSTRUCTION_TYPE.SPARK;
+		type = IType.SPARK;
 		instString = istr;
 		instOpcode = opcode;
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateBinaryOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateBinaryOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateBinaryOperator.java
index 712325d..661c12a 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateBinaryOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateBinaryOperator.java
@@ -29,31 +29,24 @@ import org.apache.sysml.runtime.functionobjects.ValueFunction;
 
 public class AggregateBinaryOperator extends Operator implements Serializable
 {
-
 	private static final long serialVersionUID = 1666421325090925726L;
 
-	public ValueFunction binaryFn;
-	public AggregateOperator aggOp;
-	private int k; //num threads
+	public final ValueFunction binaryFn;
+	public final AggregateOperator aggOp;
+	private final int k; //num threads
 	
-	public AggregateBinaryOperator(ValueFunction inner, AggregateOperator outer)
-	{
+	public AggregateBinaryOperator(ValueFunction inner, AggregateOperator outer) {
 		//default degree of parallelism is 1 
 		//(for example in MR/Spark because we parallelize over the number of blocks)
 		this( inner, outer, 1 );
 	}
 	
-	public AggregateBinaryOperator(ValueFunction inner, AggregateOperator outer, int numThreads)
-	{
+	public AggregateBinaryOperator(ValueFunction inner, AggregateOperator outer, int numThreads) {
+		//so far, we only support matrix multiplication, and it is sparseSafe
+		super(inner instanceof Multiply && outer.increOp.fn instanceof Plus);
 		binaryFn = inner;
 		aggOp = outer;
 		k = numThreads;
-		
-		//so far, we only support matrix multiplication, and it is sparseSafe
-		if(binaryFn instanceof Multiply && aggOp.increOp.fn instanceof Plus)
-			sparseSafe=true;
-		else
-			sparseSafe=false;
 	}
 	
 	public int getNumThreads() {

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateOperator.java
index 56293e2..4e1322c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateOperator.java
@@ -33,33 +33,24 @@ import org.apache.sysml.runtime.functionobjects.ValueFunction;
 
 public class AggregateOperator  extends Operator implements Serializable
 {
-
 	private static final long serialVersionUID = 8761527329665129670L;
 
-	public double initialValue;
-	public BinaryOperator increOp;
+	public final double initialValue;
+	public final BinaryOperator increOp;
+	public final boolean correctionExists;
+	public final CorrectionLocationType correctionLocation;
 	
-	public boolean correctionExists=false;
-	public CorrectionLocationType correctionLocation=CorrectionLocationType.INVALID;
+	public AggregateOperator(double initValue, ValueFunction op) {
+		this(initValue, op, false, CorrectionLocationType.INVALID);
+	}
 	
-	public AggregateOperator(double initValue, ValueFunction op)
-	{
+	public AggregateOperator(double initValue, ValueFunction op, boolean correctionExists, CorrectionLocationType correctionLocation) {
+		//as long as (v op 0)=v, then op is sparseSafe
+		super(op instanceof Plus || op instanceof KahanPlus || op instanceof KahanPlusSq 
+			|| op instanceof Or || op instanceof Minus);
 		initialValue=initValue;
 		increOp=new BinaryOperator(op);
-		//increFn=op;
-		//as long as (v op 0)=v, then op is sparseSafe
-		if(op instanceof Plus || op instanceof KahanPlus || op instanceof KahanPlusSq
-				|| op instanceof Or || op instanceof Minus)
-			sparseSafe=true;
-		else
-			sparseSafe=false;
-	}
-	
-	public AggregateOperator(double initValue, ValueFunction op, boolean correctionExists, CorrectionLocationType correctionLocation)
-	{
-		this(initValue, op);
 		this.correctionExists=correctionExists;
 		this.correctionLocation=correctionLocation;
 	}
-	
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateTernaryOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateTernaryOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateTernaryOperator.java
index 7ffdaff..f9335de 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateTernaryOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateTernaryOperator.java
@@ -30,25 +30,23 @@ public class AggregateTernaryOperator extends Operator implements Serializable
 {
 	private static final long serialVersionUID = 4251745081160216784L;
 	
-	public ValueFunction binaryFn;
-	public AggregateOperator aggOp;
-	public IndexFunction indexFn;
-	private int k; //num threads
+	public final ValueFunction binaryFn;
+	public final AggregateOperator aggOp;
+	public final IndexFunction indexFn;
+	private final int k; //num threads
 	
 	public AggregateTernaryOperator(ValueFunction inner, AggregateOperator outer, IndexFunction ixfun) {
 		//default degree of parallelism is 1 (e.g., for distributed operations)
 		this( inner, outer, ixfun, 1 );
 	}
 	
-	public AggregateTernaryOperator(ValueFunction inner, AggregateOperator outer, IndexFunction ixfun, int numThreads)
-	{
+	public AggregateTernaryOperator(ValueFunction inner, AggregateOperator outer, IndexFunction ixfun, int numThreads) {
+		//so far we only support sum-product and its sparse-safe
+		super(true);
 		binaryFn = inner;
 		aggOp = outer;
 		indexFn = ixfun;
 		k = numThreads;
-		
-		//so far we only support sum-product and its sparse-safe
-		sparseSafe = true;
 	}
 	
 	public int getNumThreads() {

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateUnaryOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateUnaryOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateUnaryOperator.java
index 9fbc4f8..05926b9 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateUnaryOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/AggregateUnaryOperator.java
@@ -32,9 +32,9 @@ public class AggregateUnaryOperator  extends Operator
 {
 	private static final long serialVersionUID = 6690553323120787735L;
 
-	public AggregateOperator aggOp;
-	public IndexFunction indexFn;
-	private int k; //num threads
+	public final AggregateOperator aggOp;
+	public final IndexFunction indexFn;
+	private final int k; //num threads
 
 	public AggregateUnaryOperator(AggregateOperator aop, IndexFunction iop)
 	{
@@ -45,25 +45,14 @@ public class AggregateUnaryOperator  extends Operator
 
 	public AggregateUnaryOperator(AggregateOperator aop, IndexFunction iop, int numThreads)
 	{
+		super(aop.increOp.fn instanceof Plus 
+			|| aop.increOp.fn instanceof KahanPlus 
+			|| aop.increOp.fn instanceof KahanPlusSq 
+			|| aop.increOp.fn instanceof Or 
+			|| aop.increOp.fn instanceof Minus);
 		aggOp = aop;
 		indexFn = iop;
 		k = numThreads;
-		
-		//decide on sparse safe
-		if( aggOp.increOp.fn instanceof Plus || 
-			aggOp.increOp.fn instanceof KahanPlus ||
-			aggOp.increOp.fn instanceof KahanPlusSq ||
-			aggOp.increOp.fn instanceof Or ||
-			aggOp.increOp.fn instanceof Minus ) 
-		{
-			sparseSafe=true;
-		}
-		else
-			sparseSafe=false;
-	}
-	
-	public void setNumThreads(int numThreads) {
-		k = numThreads;
 	}
 	
 	public int getNumThreads(){

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/BinaryOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/BinaryOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/BinaryOperator.java
index 620a599..2ff4497 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/BinaryOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/BinaryOperator.java
@@ -49,16 +49,14 @@ public class BinaryOperator  extends Operator implements Serializable
 {
 	private static final long serialVersionUID = -2547950181558989209L;
 
-	public ValueFunction fn;
+	public final ValueFunction fn;
 	
-	public BinaryOperator(ValueFunction p)
-	{
-		fn = p;
-		
+	public BinaryOperator(ValueFunction p) {
 		//binaryop is sparse-safe iff (0 op 0) == 0
-		sparseSafe = (fn instanceof Plus || fn instanceof Multiply 
-			|| fn instanceof Minus || fn instanceof And || fn instanceof Or 
-			|| fn instanceof PlusMultiply || fn instanceof MinusMultiply);
+		super (p instanceof Plus || p instanceof Multiply 
+			|| p instanceof Minus || p instanceof And || p instanceof Or 
+			|| p instanceof PlusMultiply || p instanceof MinusMultiply);
+		fn = p;
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/CMOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/CMOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/CMOperator.java
index 7a6c17b..57fa4b4 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/CMOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/CMOperator.java
@@ -44,18 +44,19 @@ public class CMOperator extends Operator
 	public AggregateOperationTypes aggOpType;
 
 	public CMOperator(ValueFunction op, AggregateOperationTypes agg) {
+		super(true);
 		fn = op;
 		aggOpType = agg;
-		sparseSafe = true;
 	}
 
 	public AggregateOperationTypes getAggOpType() {
 		return aggOpType;
 	}
 	
-	public void setCMAggOp(int order) {
-		aggOpType = getCMAggOpType(order);
-		fn = CM.getCMFnObject(aggOpType);
+	public CMOperator setCMAggOp(int order) {
+		AggregateOperationTypes agg = getCMAggOpType(order);
+		ValueFunction fn = CM.getCMFnObject(aggOpType);
+		return new CMOperator(fn, agg);
 	}
 	
 	public static AggregateOperationTypes getCMAggOpType ( int order ) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/COVOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/COVOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/COVOperator.java
index 40a1b05..d99d34c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/COVOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/COVOperator.java
@@ -24,14 +24,12 @@ import org.apache.sysml.runtime.functionobjects.COV;
 
 public class COVOperator extends Operator 
 {
-
 	private static final long serialVersionUID = -8404264552880694469L;
 
-	public COV fn;
+	public final COV fn;
 	
-	public COVOperator(COV op)
-	{
-		fn=op;
-		sparseSafe=true; // TODO: check with YY
+	public COVOperator(COV op) {
+		super(true);
+		fn = op;
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/LeftScalarOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/LeftScalarOperator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/LeftScalarOperator.java
index b8349fc..b47fe12 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/LeftScalarOperator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/LeftScalarOperator.java
@@ -37,21 +37,15 @@ public class LeftScalarOperator extends ScalarOperator
 	private static final long serialVersionUID = 2360577666575746424L;
 	
 	public LeftScalarOperator(ValueFunction p, double cst) {
-		super(p, cst);
+		super(p, cst, (p instanceof GreaterThan && cst<=0)
+			|| (p instanceof GreaterThanEquals && cst<0)
+			|| (p instanceof LessThan && cst>=0)
+			|| (p instanceof LessThanEquals && cst>0));
 	}
 	
 	@Override
-	public void setConstant(double cst) 
-	{
-		//overwrites constant and sparse safe flag
-		super.setConstant(cst);
-		
-		//enable conditionally sparse safe operations
-		sparseSafe |= ( isSparseSafeStatic()
-			|| (fn instanceof GreaterThan && _constant<=0)
-			|| (fn instanceof GreaterThanEquals && _constant<0)
-			|| (fn instanceof LessThan && _constant>=0)
-			|| (fn instanceof LessThanEquals && _constant>0));
+	public ScalarOperator setConstant(double cst) {
+		return new LeftScalarOperator(fn, cst);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/systemml/blob/75b93f26/src/main/java/org/apache/sysml/runtime/matrix/operators/Operator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/operators/Operator.java b/src/main/java/org/apache/sysml/runtime/matrix/operators/Operator.java
index 296a1a2..43912c5 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/operators/Operator.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/operators/Operator.java
@@ -24,18 +24,15 @@ import java.io.Serializable;
 
 public class Operator implements Serializable 
 {
-
 	private static final long serialVersionUID = -7453178973186473277L;
 
-	public boolean sparseSafe=false;
+	public final boolean sparseSafe;
 
-	public Operator()
-	{
-		
+	public Operator() {
+		this(false);
 	}
 	
-	public Operator(boolean sparseSafeFlag)
-	{
+	public Operator(boolean sparseSafeFlag) {
 		sparseSafe = sparseSafeFlag;
 	}
 }