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

[1/3] systemml git commit: [SYSTEMML-1796] Provide fine-grained statistics for SystemML

Repository: systemml
Updated Branches:
  refs/heads/master 6133be23e -> 648eb21d6


http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
index 16051dc..835e491 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
@@ -61,6 +61,7 @@ import org.apache.sysml.runtime.functionobjects.RevIndex;
 import org.apache.sysml.runtime.functionobjects.SortIndex;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
 import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
+import org.apache.sysml.runtime.instructions.cp.CPInstruction;
 import org.apache.sysml.runtime.instructions.cp.KahanObject;
 import org.apache.sysml.runtime.instructions.cp.ScalarObject;
 import org.apache.sysml.runtime.io.IOUtilFunctions;
@@ -86,6 +87,7 @@ import org.apache.sysml.runtime.util.FastBufferedDataInputStream;
 import org.apache.sysml.runtime.util.FastBufferedDataOutputStream;
 import org.apache.sysml.runtime.util.IndexRange;
 import org.apache.sysml.runtime.util.UtilFunctions;
+import org.apache.sysml.utils.GPUStatistics;
 import org.apache.sysml.utils.NativeHelper;
 import org.apache.sysml.utils.Statistics;
 
@@ -992,6 +994,10 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		return evalSparseFormatOnDisk(lrlen, lclen, nonZeros);
 	}
 	
+	public void examSparsity() throws DMLRuntimeException {
+		examSparsity(null);
+	}
+	
 	/**
 	 * Evaluates if this matrix block should be in sparse format in
 	 * memory. Depending on the current representation, the state of the
@@ -999,10 +1005,11 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	 * Note that this consumes for the time of execution memory for both 
 	 * representations.  
 	 * 
+	 * @param opcode  extended opcode
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	@SuppressWarnings("unused")
-	public void examSparsity() 
+	public void examSparsity(String opcode) 
 		throws DMLRuntimeException
 	{
 		long start = DISPLAY_STATISTICS && DMLScript.STATISTICS ? System.nanoTime() : 0;
@@ -1016,9 +1023,9 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		//change representation if required (also done for 
 		//empty blocks in order to set representation flags)
 		if( sparse && !sparseDst)
-			sparseToDense();
+			sparseToDense(opcode);
 		else if( !sparse && sparseDst )
-			denseToSparse();
+			denseToSparse(opcode);
 		
 		Statistics.examSparsityTime += DISPLAY_STATISTICS && DMLScript.STATISTICS ? (System.nanoTime() - start) : 0;
 	}
@@ -1075,7 +1082,14 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	// basic block handling functions	
 
 	void denseToSparse() 
+	{
+		denseToSparse(null);
+	}
+	
+	void denseToSparse(String opcode) 
 	{	
+		long t1 = opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+		
 		//set target representation
 		sparse = true;
 		
@@ -1115,11 +1129,20 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		//update nnz and cleanup dense block
 		nonZeros = nnz;
 		denseBlock = null;
+		if(opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS) {
+			long t2 = System.nanoTime();
+			GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_DENSE_TO_SPARSE, t2-t1);
+		}
+	}
+	
+	public void sparseToDense() throws DMLRuntimeException {
+		sparseToDense(null);
 	}
 
-	public void sparseToDense() 
+	public void sparseToDense(String opcode) 
 		throws DMLRuntimeException 
-	{	
+	{
+		long t1 = opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		//set target representation
 		sparse = false;
 		
@@ -1153,6 +1176,10 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		
 		//cleanup sparse rows
 		sparseBlock = null;
+		if(opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS) {
+			long t2 = System.nanoTime();
+			GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_SPARSE_TO_DENSE, t2-t1);
+		}
 	}
 
 	/**
@@ -3613,13 +3640,22 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 			LibMatrixMult.matrixMultPermute(this, m2, ret1, ret2);
 
 	}
+	
+	public final MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, IndexRange ixrange, MatrixBlock ret, UpdateType update) throws DMLRuntimeException {
+		return leftIndexingOperations(rhsMatrix, ixrange, ret, update, null);
+	}
 
-	public final MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, IndexRange ixrange, MatrixBlock ret, UpdateType update) 
+	public final MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, IndexRange ixrange, MatrixBlock ret, UpdateType update, String opcode) 
 		throws DMLRuntimeException 
 	{
 		return leftIndexingOperations(
 				rhsMatrix, (int)ixrange.rowStart, (int)ixrange.rowEnd, 
-				(int)ixrange.colStart, (int)ixrange.colEnd, ret, update);
+				(int)ixrange.colStart, (int)ixrange.colEnd, ret, update, opcode);
+	}
+	
+	public MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, int rl, int ru, 
+			int cl, int cu, MatrixBlock ret, UpdateType update) throws DMLRuntimeException {
+		return leftIndexingOperations(rhsMatrix, rl, ru, cl, cu, ret, update, null);
 	}
 	
 	/**
@@ -3641,7 +3677,7 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
 	public MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, int rl, int ru, 
-			int cl, int cu, MatrixBlock ret, UpdateType update) 
+			int cl, int cu, MatrixBlock ret, UpdateType update, String opcode) 
 		throws DMLRuntimeException 
 	{	
 		// Check the validity of bounds
@@ -3678,9 +3714,9 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 			//ensure that the current block adheres to the sparsity estimate
 			//and thus implicitly the memory budget used by the compiler
 			if( result.sparse && !sp )
-				result.sparseToDense();
+				result.sparseToDense(opcode);
 			else if( !result.sparse && sp )
-				result.denseToSparse();	
+				result.denseToSparse(opcode);	
 			
 			//ensure right sparse block representation to prevent serialization
 			if( result.sparse && update != UpdateType.INPLACE_PINNED ) {
@@ -3702,6 +3738,8 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 			result.quickSetValue(rl, cl, src.quickGetValue(0, 0));
 		}		
 		else { //general case
+			long t1 = opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+			boolean isCSRCopy = false;
 			//handle csr sparse blocks separately to avoid repeated shifting on column-wise access
 			if( !result.isEmptyBlock(false) && result.sparse && result.sparseBlock instanceof SparseBlockCSR ) {
 				SparseBlockCSR sblock = (SparseBlockCSR) result.sparseBlock;
@@ -3710,11 +3748,19 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 				else //dense
 					sblock.setIndexRange(rl, ru+1, cl, cu+1, src.getDenseBlock(), 0, src.getNumRows()*src.getNumColumns());
 				result.nonZeros = sblock.size();
+				isCSRCopy = true;
 			}
 			//copy submatrix into result
 			else {
 				result.copy(rl, ru, cl, cu, src, true);
 			}
+			if(opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS) {
+				long t2 = System.nanoTime();
+				if(isCSRCopy)
+					GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_CSR_LIX_COPY, t2-t1);
+				else
+					GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_LIX_COPY, t2-t1);
+			}
 		}
 
 		return result;

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/utils/GPUStatistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/GPUStatistics.java b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
index 6b85d92..c0f1d5e 100644
--- a/src/main/java/org/apache/sysml/utils/GPUStatistics.java
+++ b/src/main/java/org/apache/sysml/utils/GPUStatistics.java
@@ -28,6 +28,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.sysml.api.DMLScript;
+
 /**
  * Measures performance numbers when GPU mode is enabled
  * Printed as part of {@link Statistics}.
@@ -119,7 +121,7 @@ public class GPUStatistics {
    */
   public synchronized static void maintainCPMiscTimes( String instructionName, String miscTimer, long timeNanos, long incrementCount)
   {
-    if (!DISPLAY_STATISTICS)
+    if (!(DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS))
       return;
 
     HashMap<String, Long> miscTimesMap = _cpInstMiscTime.get(instructionName);

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/utils/Statistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/Statistics.java b/src/main/java/org/apache/sysml/utils/Statistics.java
index 808267d..a72b89e 100644
--- a/src/main/java/org/apache/sysml/utils/Statistics.java
+++ b/src/main/java/org/apache/sysml/utils/Statistics.java
@@ -533,7 +533,7 @@ public class Statistics
 		final String instCol = "Instruction";
 		final String timeSCol = "Time(s)";
 		final String countCol = "Count";
-		final String gpuCol = "GPU";
+		final String gpuCol = "Misc Timers";
 		StringBuilder sb = new StringBuilder();
 		int numHittersToDisplay = Math.min(num, len);
 		int maxNumLen = String.valueOf(numHittersToDisplay).length();
@@ -557,7 +557,7 @@ public class Statistics
 		sb.append(String.format(
 				" %" + maxNumLen + "s  %-" + maxInstLen + "s  %" + maxTimeSLen + "s  %" + maxCountLen + "s", numCol,
 				instCol, timeSCol, countCol));
-		if (GPUStatistics.DISPLAY_STATISTICS) {
+		if (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) {
 			sb.append("  ");
 			sb.append(gpuCol);
 		}
@@ -575,7 +575,7 @@ public class Statistics
 					(i + 1), instruction, timeSString, count));
 
 			// Add the miscellaneous timer info
-			if (GPUStatistics.DISPLAY_STATISTICS) {
+			if (GPUStatistics.DISPLAY_STATISTICS || DMLScript.FINEGRAINED_STATISTICS) {
 				sb.append("  ");
 				sb.append(GPUStatistics.getStringForCPMiscTimesPerInstruction(instruction));
 			}


[2/3] systemml git commit: [SYSTEMML-1796] Provide fine-grained statistics for SystemML

Posted by ni...@apache.org.
http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 840b39e..f12cb7f 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
@@ -220,8 +220,8 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 	
 	public void processReluBackwardInstruction(ExecutionContext ec) throws DMLRuntimeException {
 		// (X > 0) * dout
-		MatrixBlock input = ec.getMatrixInput(input1.getName());
-		MatrixBlock dout = ec.getMatrixInput(_in2.getName());
+		MatrixBlock input = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock dout = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 		MatrixBlock outputBlock =  new MatrixBlock(input.getNumRows(), input.getNumColumns(), (input.isInSparseFormat() || dout.isInSparseFormat()));
 		
 		if( !input.isEmpty() && !dout.isEmpty() ) {
@@ -230,14 +230,14 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 		}
 		
 		// release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(_in2.getName());
-		ec.setMatrixOutput(getOutputVariableName(), outputBlock);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(getOutputVariableName(), outputBlock, getExtendedOpcode());
 	}
 	
 	public void processBiasAddInstruction(ExecutionContext ec) throws DMLRuntimeException {
-		MatrixBlock input = ec.getMatrixInput(input1.getName());
-		MatrixBlock bias = ec.getMatrixInput(_in2.getName());
+		MatrixBlock input = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock bias = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 		MatrixBlock outputBlock = null;
 		
 		if(bias.getNumColumns() != 1) {
@@ -258,14 +258,14 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 		}
 		
 		// release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(_in2.getName());
-		ec.setMatrixOutput(getOutputVariableName(), outputBlock);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(getOutputVariableName(), outputBlock, getExtendedOpcode());
 	}
 	
 	public void processBiasMultiplyInstruction(ExecutionContext ec) throws DMLRuntimeException {
-		MatrixBlock input = ec.getMatrixInput(input1.getName());
-		MatrixBlock bias = ec.getMatrixInput(_in2.getName());
+		MatrixBlock input = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock bias = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 		MatrixBlock outputBlock = null;
 		
 		if(bias.getNumColumns() != 1) {
@@ -284,9 +284,9 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 		}
 		
 		// release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(_in2.getName());
-		ec.setMatrixOutput(getOutputVariableName(), outputBlock);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(getOutputVariableName(), outputBlock, getExtendedOpcode());
 	}
 	
 	// Assumption: enableNative && NativeHelper.isNativeLibraryLoaded() is true
@@ -319,7 +319,7 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 		
 		// acquire inputs
 		MatrixBlock outputBlock = null;
-		MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		int pad_h = getScalarInput(ec, _padding, 0);
 		int pad_w = getScalarInput(ec, _padding, 1);
 		int stride_h = getScalarInput(ec, _stride, 0);
@@ -351,7 +351,7 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 			}
 		}
 		else if (instOpcode.equalsIgnoreCase("maxpooling_backward") || instOpcode.equalsIgnoreCase("relu_maxpooling_backward")) {
-			MatrixBlock dout = ec.getMatrixInput(_in2.getName());
+			MatrixBlock dout = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 			if(matBlock.isEmpty() || dout.isEmpty()) {
 				outputBlock = new MatrixBlock(N, C*H*W, true);
 			}
@@ -362,10 +362,10 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 				else
 					LibMatrixDNN.maxpoolingBackward(matBlock, dout, outputBlock, params, true);
 			}
-			ec.releaseMatrixInput(_in2.getName());
+			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}
 		else if (instOpcode.equalsIgnoreCase("conv2d")) {
-			MatrixBlock filter = ec.getMatrixInput(_in2.getName());
+			MatrixBlock filter = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 			if(filter.isEmpty() || matBlock.isEmpty()) {
 				outputBlock = new MatrixBlock(N, K*P*Q, true);
 			}
@@ -376,11 +376,11 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 				else
 					LibMatrixDNN.conv2d(matBlock, filter, outputBlock, params);
 			}
-			ec.releaseMatrixInput(_in2.getName());
+			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}
 		else if (instOpcode.equalsIgnoreCase("conv2d_bias_add")) {
-			MatrixBlock filter = ec.getMatrixInput(_in3.getName());
-			MatrixBlock bias = ec.getMatrixInput(_in2.getName());
+			MatrixBlock filter = ec.getMatrixInput(_in3.getName(), getExtendedOpcode());
+			MatrixBlock bias = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 			if(bias.getNumRows() != params.K || bias.getNumColumns() != 1) {
 				throw new DMLRuntimeException("Incorrect shape of bias matrix: [" + bias.getNumRows() + " " + bias.getNumColumns() + "]. "
 						+ "Expected: [" + params.K + ", 1]");
@@ -408,11 +408,11 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 				else
 					LibMatrixDNN.conv2d(matBlock, filter, outputBlock, params);
 			}
-			ec.releaseMatrixInput(_in3.getName());
-			ec.releaseMatrixInput(_in2.getName());
+			ec.releaseMatrixInput(_in3.getName(), getExtendedOpcode());
+			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}
 		else if (instOpcode.equalsIgnoreCase("conv2d_backward_filter")) {
-			MatrixBlock dout = ec.getMatrixInput(_in2.getName());
+			MatrixBlock dout = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 			if(dout.isEmpty() || matBlock.isEmpty()) {
 				outputBlock = new MatrixBlock(K, C*R*S, true);
 			}
@@ -423,10 +423,10 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 				else
 					LibMatrixDNN.conv2dBackwardFilter(matBlock, dout, outputBlock, params);
 			}
-			ec.releaseMatrixInput(_in2.getName());
+			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}
 		else if (instOpcode.equalsIgnoreCase("conv2d_backward_data")) {
-			MatrixBlock dout = ec.getMatrixInput(_in2.getName());
+			MatrixBlock dout = ec.getMatrixInput(_in2.getName(), getExtendedOpcode());
 			if(dout.isEmpty() || matBlock.isEmpty()) {
 				outputBlock = new MatrixBlock(N, C * H * W, true);
 			}
@@ -437,15 +437,15 @@ public class ConvolutionCPInstruction extends UnaryCPInstruction
 				else
 					LibMatrixDNN.conv2dBackwardData(matBlock, dout, outputBlock, params);
 			}
-			ec.releaseMatrixInput(_in2.getName());
+			ec.releaseMatrixInput(_in2.getName(), getExtendedOpcode());
 		}
 		else {
 			throw new DMLRuntimeException("Unsupported op code " + instOpcode);
 		}
 		
 		// release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.setMatrixOutput(getOutputVariableName(), outputBlock);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(getOutputVariableName(), outputBlock, getExtendedOpcode());
 	}
 	
 	private MatrixBlock getDenseOutputBlock(int numRows, int numCols) throws DMLRuntimeException {

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 cbb06aa..d2e43e2 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
@@ -80,8 +80,8 @@ public class CovarianceCPInstruction extends BinaryCPInstruction
 	public void processInstruction(ExecutionContext ec) 
 		throws DMLRuntimeException
 	{	
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		String output_name = output.getName(); 
 		
 		COVOperator cov_op = (COVOperator)_optr;
@@ -92,19 +92,19 @@ public class CovarianceCPInstruction extends BinaryCPInstruction
 			// Unweighted: cov.mvar0.mvar1.out
 			covobj = matBlock1.covOperations(cov_op, matBlock2);
 			
-			ec.releaseMatrixInput(input1.getName());
-			ec.releaseMatrixInput(input2.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+			ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		}
 		else 
 		{
 			// Weighted: cov.mvar0.mvar1.weights.out
-	        MatrixBlock wtBlock = ec.getMatrixInput(input3.getName());
+	        MatrixBlock wtBlock = ec.getMatrixInput(input3.getName(), getExtendedOpcode());
 			
 			covobj = matBlock1.covOperations(cov_op, matBlock2, wtBlock);
 			
-			ec.releaseMatrixInput(input1.getName());
-			ec.releaseMatrixInput(input2.getName());
-			ec.releaseMatrixInput(input3.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+			ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
+			ec.releaseMatrixInput(input3.getName(), getExtendedOpcode());
 		}
 		
 		double val = covobj.getRequiredResult(_optr);

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 7cd4f77..73e4cf1 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
@@ -327,6 +327,6 @@ public class DataGenCPInstruction extends UnaryCPInstruction
 		}
 		
 		//release created output
-		ec.setMatrixOutput(output.getName(), soresBlock);
+		ec.setMatrixOutput(output.getName(), soresBlock, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 2b75460..68151d2 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
@@ -97,6 +97,6 @@ public class DataPartitionCPInstruction extends UnaryCPInstruction
 		}
 		
 		//release input
-		ec.releaseMatrixInput(input1.getName());		
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());		
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 3fee0d1..273dd2a 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
@@ -75,20 +75,20 @@ public class MMChainCPInstruction extends UnaryCPInstruction
 		throws DMLRuntimeException 
 	{
 		//get inputs
-		MatrixBlock X = ec.getMatrixInput(input1.getName());
-		MatrixBlock v = ec.getMatrixInput(input2.getName());
+		MatrixBlock X = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock v = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		MatrixBlock w = (_type==ChainType.XtwXv || _type==ChainType.XtXvy) ? 
-				ec.getMatrixInput(input3.getName()) : null;
+				ec.getMatrixInput(input3.getName(), getExtendedOpcode()) : null;
 
 		//execute mmchain operation 
 		 MatrixBlock out = (MatrixBlock) X.chainMatrixMultOperations(v, w, new MatrixBlock(), _type, _numThreads);
 				
 		//set output and release inputs
-		ec.setMatrixOutput(output.getName(), out);
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		if( w !=null )
-			ec.releaseMatrixInput(input3.getName());
+			ec.releaseMatrixInput(input3.getName(), getExtendedOpcode());
 	}
 	
 	public ChainType getMMChainType()

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 ccc472b..361da7f 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
@@ -63,14 +63,14 @@ public class MMTSJCPInstruction extends UnaryCPInstruction
 		throws DMLRuntimeException 
 	{
 		//get inputs
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 
 		//execute operations 
 		MatrixBlock ret = (MatrixBlock) matBlock1.transposeSelfMatrixMultOperations(new MatrixBlock(), _type, _numThreads );
 		
 		//set output and release inputs
-		ec.setMatrixOutput(output.getName(), ret);
-		ec.releaseMatrixInput(input1.getName());
+		ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 	}
 	
 	public MMTSJType getMMTSJType()

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixAppendCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixAppendCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixAppendCPInstruction.java
index 4c48ed7..ec6bf09 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixAppendCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixAppendCPInstruction.java
@@ -35,8 +35,8 @@ public final class MatrixAppendCPInstruction extends AppendCPInstruction
 		throws DMLRuntimeException 
 	{
 		//get inputs
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 	
 		//check input dimensions
 		if( _type == AppendType.CBIND && matBlock1.getNumRows() != matBlock2.getNumRows() ) {
@@ -52,8 +52,8 @@ public final class MatrixAppendCPInstruction extends AppendCPInstruction
 		MatrixBlock ret = matBlock1.appendOperations(matBlock2, new MatrixBlock(), _type==AppendType.CBIND);
 		
 		//set output and release inputs 
-		ec.setMatrixOutput(output.getName(), ret);
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 003b85d..7817c06 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
@@ -43,20 +43,20 @@ public class MatrixBuiltinCPInstruction extends BuiltinUnaryCPInstruction
 		String opcode = getOpcode();
 		if(LibCommonsMath.isSupportedUnaryOperation(opcode)) {
 			MatrixBlock retBlock = LibCommonsMath.unaryOperations(ec.getMatrixObject(input1.getName()),getOpcode());
-			ec.setMatrixOutput(output_name, retBlock);
+			ec.setMatrixOutput(output_name, retBlock, getExtendedOpcode());
 		}
 		else {
-			MatrixBlock inBlock = ec.getMatrixInput(input1.getName());
+			MatrixBlock inBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 			MatrixBlock retBlock = (MatrixBlock) (inBlock.unaryOperations(u_op, new MatrixBlock()));
 		
-			ec.releaseMatrixInput(input1.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 			
 			// Ensure right dense/sparse output representation (guarded by released input memory)
 			if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
 	 			retBlock.examSparsity();
 	 		}
 			
-			ec.setMatrixOutput(output_name, retBlock);
+			ec.setMatrixOutput(output_name, retBlock, getExtendedOpcode());
 		}		
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 4d99a32..b2e416c 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
@@ -61,19 +61,19 @@ public final class MatrixIndexingCPInstruction extends IndexingCPInstruction
 			else //via slicing the in-memory matrix
 			{
 				//execute right indexing operation
-				MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+				MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 				resultBlock = matBlock.sliceOperations(ixrange, new MatrixBlock());	
 				
 				//unpin rhs input
-				ec.releaseMatrixInput(input1.getName());
+				ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 				
 				//ensure correct sparse/dense output representation
 				//(memory guarded by release of input)
-				resultBlock.examSparsity();
+				resultBlock.examSparsity(getExtendedOpcode());
 			}	
 			
 			//unpin output
-			ec.setMatrixOutput(output.getName(), resultBlock);
+			ec.setMatrixOutput(output.getName(), resultBlock, getExtendedOpcode());
 		}
 		//left indexing
 		else if ( opcode.equalsIgnoreCase("leftIndex"))
@@ -86,14 +86,14 @@ public final class MatrixIndexingCPInstruction extends IndexingCPInstruction
 				Statistics.incrementTotalLix();
 			}
 			
-			MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+			MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 			MatrixBlock resultBlock = null;
 			
 			if(input2.getDataType() == DataType.MATRIX) //MATRIX<-MATRIX
 			{
-				MatrixBlock rhsMatBlock = ec.getMatrixInput(input2.getName());
-				resultBlock = matBlock.leftIndexingOperations(rhsMatBlock, ixrange, new MatrixBlock(), updateType);
-				ec.releaseMatrixInput(input2.getName());
+				MatrixBlock rhsMatBlock = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
+				resultBlock = matBlock.leftIndexingOperations(rhsMatBlock, ixrange, new MatrixBlock(), updateType, getExtendedOpcode());
+				ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 			}
 			else //MATRIX<-SCALAR 
 			{
@@ -105,14 +105,14 @@ public final class MatrixIndexingCPInstruction extends IndexingCPInstruction
 			}
 
 			//unpin lhs input
-			ec.releaseMatrixInput(input1.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 			
 			//ensure correct sparse/dense output representation
 			//(memory guarded by release of input)
-			resultBlock.examSparsity();
+			resultBlock.examSparsity(getExtendedOpcode());
 			
 			//unpin output
-			ec.setMatrixOutput(output.getName(), resultBlock, updateType);
+			ec.setMatrixOutput(output.getName(), resultBlock, updateType, getExtendedOpcode());
 		}
 		else
 			throw new DMLRuntimeException("Invalid opcode (" + opcode +") encountered in MatrixIndexingCPInstruction.");		

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
index 208b688..7216418 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
@@ -43,16 +43,16 @@ public class MatrixMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		throws DMLRuntimeException
 	{
 		// Read input matrices
-        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
+        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		// Perform computation using input matrices, and produce the result matrix
 		BinaryOperator bop = (BinaryOperator) _optr;
 		MatrixBlock retBlock = (MatrixBlock) (inBlock1.binaryOperations (bop, inBlock2, new MatrixBlock()));
 		
 		// Release the memory occupied by input matrices
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
@@ -60,6 +60,6 @@ public class MatrixMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		}
 		
 		// Attach result matrix with MatrixObject associated with output_name
-		ec.setMatrixOutput(output.getName(), retBlock);
+		ec.setMatrixOutput(output.getName(), retBlock, getExtendedOpcode());
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 daba3bc..348b9d6 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
@@ -48,7 +48,7 @@ public class MatrixMatrixBuiltinCPInstruction extends BuiltinBinaryCPInstruction
         
         if ( LibCommonsMath.isSupportedMatrixMatrixOperation(opcode) ) {
         	MatrixBlock solution = LibCommonsMath.matrixMatrixOperations(ec.getMatrixObject(input1.getName()), (MatrixObject)ec.getVariable(input2.getName()), opcode);
-    		ec.setMatrixOutput(output.getName(), solution);
+    		ec.setMatrixOutput(output.getName(), solution, getExtendedOpcode());
         	return;
         }
 		
@@ -56,19 +56,19 @@ public class MatrixMatrixBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		String output_name = output.getName();
 		BinaryOperator bop = (BinaryOperator) _optr;
 		
-		MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock1.binaryOperations(bop, inBlock2, new MatrixBlock());
 	
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
  			retBlock.examSparsity();
  		}
         
-		ec.setMatrixOutput(output_name, retBlock);
+		ec.setMatrixOutput(output_name, retBlock, getExtendedOpcode());
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
index f972748..fd6bd8b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
@@ -42,22 +42,22 @@ public class MatrixMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 	public void processInstruction(ExecutionContext ec) 
 		throws DMLRuntimeException
 	{
-        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
+        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		String output_name = output.getName();
 		BinaryOperator bop = (BinaryOperator) _optr;
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock1.binaryOperations(bop, inBlock2, new MatrixBlock());
 
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
  			retBlock.examSparsity();
  		}
 		
-		ec.setMatrixOutput(output_name, retBlock);
+		ec.setMatrixOutput(output_name, retBlock, getExtendedOpcode());
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 95f5af7..132428f 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
@@ -68,7 +68,7 @@ public class MatrixReshapeCPInstruction extends UnaryCPInstruction
 		throws DMLRuntimeException 
 	{
 		//get inputs
-		MatrixBlock in = ec.getMatrixInput(input1.getName());
+		MatrixBlock in = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		int rows = (int)ec.getScalarInput(_opRows.getName(), _opRows.getValueType(), _opRows.isLiteral()).getLongValue(); //save cast
 		int cols = (int)ec.getScalarInput(_opCols.getName(), _opCols.getValueType(), _opCols.isLiteral()).getLongValue(); //save cast
 		BooleanObject byRow = (BooleanObject) ec.getScalarInput(_opByRow.getName(), ValueType.BOOLEAN, _opByRow.isLiteral());
@@ -78,8 +78,8 @@ public class MatrixReshapeCPInstruction extends UnaryCPInstruction
 		out = LibMatrixReorg.reshape(in, out, rows, cols, byRow.getBooleanValue());
 		
 		//set output and release inputs
-		ec.setMatrixOutput(output.getName(), out);
-		ec.releaseMatrixInput(input1.getName());
+		ec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 cd140ef..3cd075b 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
@@ -46,7 +46,7 @@ public class MatrixScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName(), getExtendedOpcode());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator)	_optr;
@@ -54,13 +54,13 @@ public class MatrixScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 
-		ec.releaseMatrixInput(mat.getName());
+		ec.releaseMatrixInput(mat.getName(), getExtendedOpcode());
 		
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
  			retBlock.examSparsity();
  		}
 		
-		ec.setMatrixOutput(output.getName(), retBlock);
+		ec.setMatrixOutput(output.getName(), retBlock, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 17f9350..54d206e 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
@@ -111,7 +111,7 @@ public class MultiReturnBuiltinCPInstruction extends ComputationCPInstruction
 
 		
 		for(int i=0; i < _outputs.size(); i++) {
-			ec.setMatrixOutput(_outputs.get(i).getName(), out[i]);
+			ec.setMatrixOutput(_outputs.get(i).getName(), out[i], getExtendedOpcode());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 574f7ff..429bb00 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
@@ -85,7 +85,7 @@ public class MultiReturnParameterizedBuiltinCPInstruction extends ComputationCPI
 		
 		//release input and outputs
 		ec.releaseFrameInput(input1.getName());
-		ec.setMatrixOutput(getOutput(0).getName(), data);
+		ec.setMatrixOutput(getOutput(0).getName(), data, getExtendedOpcode());
 		ec.setFrameOutput(getOutput(1).getName(), meta);
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 99f031c..75deaae 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
@@ -60,8 +60,8 @@ public class PMMJCPInstruction extends ComputationCPInstruction
 		throws DMLRuntimeException 
 	{
 		//get inputs
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		int rlen = (int)ec.getScalarInput(input3.getName(), input3.getValueType(), input3.isLiteral()).getLongValue();
 		
 		//execute operations
@@ -69,8 +69,8 @@ public class PMMJCPInstruction extends ComputationCPInstruction
 		matBlock1.permutationMatrixMultOperations(matBlock2, ret, null, _numThreads);
 		
 		//set output and release inputs
-		ec.setMatrixOutput(output.getName(), ret);
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 847d5f9..8a3d517 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
@@ -177,11 +177,11 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 		} 
 		else if ( opcode.equalsIgnoreCase("groupedagg") ) {
 			// acquire locks
-			MatrixBlock target = ec.getMatrixInput(params.get(Statement.GAGG_TARGET));
-			MatrixBlock groups = ec.getMatrixInput(params.get(Statement.GAGG_GROUPS));
+			MatrixBlock target = ec.getMatrixInput(params.get(Statement.GAGG_TARGET), getExtendedOpcode());
+			MatrixBlock groups = ec.getMatrixInput(params.get(Statement.GAGG_GROUPS), getExtendedOpcode());
 			MatrixBlock weights= null;
 			if ( params.get(Statement.GAGG_WEIGHTS) != null )
-				weights = ec.getMatrixInput(params.get(Statement.GAGG_WEIGHTS));
+				weights = ec.getMatrixInput(params.get(Statement.GAGG_WEIGHTS), getExtendedOpcode());
 			
 			int ngroups = -1;
 			if ( params.get(Statement.GAGG_NUM_GROUPS) != null) {
@@ -192,19 +192,19 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			int k = Integer.parseInt(params.get("k")); //num threads
 			MatrixBlock soresBlock = groups.groupedAggOperations(target, weights, new MatrixBlock(), ngroups, _optr, k);
 			
-			ec.setMatrixOutput(output.getName(), soresBlock);
+			ec.setMatrixOutput(output.getName(), soresBlock, getExtendedOpcode());
 			// release locks
 			target = groups = weights = null;
-			ec.releaseMatrixInput(params.get(Statement.GAGG_TARGET));
-			ec.releaseMatrixInput(params.get(Statement.GAGG_GROUPS));
+			ec.releaseMatrixInput(params.get(Statement.GAGG_TARGET), getExtendedOpcode());
+			ec.releaseMatrixInput(params.get(Statement.GAGG_GROUPS), getExtendedOpcode());
 			if ( params.get(Statement.GAGG_WEIGHTS) != null )
-				ec.releaseMatrixInput(params.get(Statement.GAGG_WEIGHTS));
+				ec.releaseMatrixInput(params.get(Statement.GAGG_WEIGHTS), getExtendedOpcode());
 			
 		}
 		else if ( opcode.equalsIgnoreCase("rmempty") ) {
 			// acquire locks
-			MatrixBlock target = ec.getMatrixInput(params.get("target"));
-			MatrixBlock select = params.containsKey("select")? ec.getMatrixInput(params.get("select")):null;
+			MatrixBlock target = ec.getMatrixInput(params.get("target"), getExtendedOpcode());
+			MatrixBlock select = params.containsKey("select")? ec.getMatrixInput(params.get("select"), getExtendedOpcode()):null;
 			
 			// compute the result
 			String margin = params.get("margin");
@@ -217,14 +217,14 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 				throw new DMLRuntimeException("Unspupported margin identifier '"+margin+"'.");
 			
 			//release locks
-			ec.setMatrixOutput(output.getName(), soresBlock);
-			ec.releaseMatrixInput(params.get("target"));
+			ec.setMatrixOutput(output.getName(), soresBlock, getExtendedOpcode());
+			ec.releaseMatrixInput(params.get("target"), getExtendedOpcode());
 			if (params.containsKey("select"))
-				ec.releaseMatrixInput(params.get("select"));
+				ec.releaseMatrixInput(params.get("select"), getExtendedOpcode());
 		}
 		else if ( opcode.equalsIgnoreCase("replace") ) {
 			// acquire locks
-			MatrixBlock target = ec.getMatrixInput(params.get("target"));
+			MatrixBlock target = ec.getMatrixInput(params.get("target"), getExtendedOpcode());
 			
 			// compute the result
 			double pattern = Double.parseDouble( params.get("pattern") );
@@ -232,12 +232,12 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			MatrixBlock ret = (MatrixBlock) target.replaceOperations(new MatrixBlock(), pattern, replacement);
 			
 			//release locks
-			ec.setMatrixOutput(output.getName(), ret);
-			ec.releaseMatrixInput(params.get("target"));
+			ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
+			ec.releaseMatrixInput(params.get("target"), getExtendedOpcode());
 		}
 		else if ( opcode.equalsIgnoreCase("rexpand") ) {
 			// acquire locks
-			MatrixBlock target = ec.getMatrixInput(params.get("target"));
+			MatrixBlock target = ec.getMatrixInput(params.get("target"), getExtendedOpcode());
 			
 			// compute the result
 			double maxVal = Double.parseDouble( params.get("max") );
@@ -249,8 +249,8 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 				new MatrixBlock(), maxVal, dirVal, cast, ignore, numThreads);
 			
 			//release locks
-			ec.setMatrixOutput(output.getName(), ret);
-			ec.releaseMatrixInput(params.get("target"));
+			ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
+			ec.releaseMatrixInput(params.get("target"), getExtendedOpcode());
 		}
 		else if ( opcode.equalsIgnoreCase("transformapply")) {
 			//acquire locks
@@ -263,13 +263,13 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			MatrixBlock mbout = encoder.apply(data, new MatrixBlock(data.getNumRows(), data.getNumColumns(), false));
 			
 			//release locks
-			ec.setMatrixOutput(output.getName(), mbout);
+			ec.setMatrixOutput(output.getName(), mbout, getExtendedOpcode());
 			ec.releaseFrameInput(params.get("target"));
 			ec.releaseFrameInput(params.get("meta"));
 		}
 		else if ( opcode.equalsIgnoreCase("transformdecode")) {			
 			//acquire locks
-			MatrixBlock data = ec.getMatrixInput(params.get("target"));
+			MatrixBlock data = ec.getMatrixInput(params.get("target"), getExtendedOpcode());
 			FrameBlock meta = ec.getFrameInput(params.get("meta"));
 			String[] colnames = meta.getColumnNames();
 			
@@ -279,7 +279,7 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			
 			//release locks
 			ec.setFrameOutput(output.getName(), fbout);
-			ec.releaseMatrixInput(params.get("target"));
+			ec.releaseMatrixInput(params.get("target"), getExtendedOpcode());
 			ec.releaseFrameInput(params.get("meta"));
 		}
 		else if ( opcode.equalsIgnoreCase("transformmeta")) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 12bc465..06d752e 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
@@ -57,8 +57,8 @@ public class PlusMultCPInstruction extends ArithmeticBinaryCPInstruction
 		String output_name = output.getName();
 
 		//get all the inputs
-		MatrixBlock matrix1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock matrix2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matrix1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock matrix2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		ScalarObject scalar = ec.getScalarInput(input3.getName(), input3.getValueType(), input3.isLiteral()); 
 		
 		//execution
@@ -66,9 +66,9 @@ public class PlusMultCPInstruction extends ArithmeticBinaryCPInstruction
 		MatrixBlock out = (MatrixBlock) matrix1.binaryOperations((BinaryOperator) _optr, matrix2, new MatrixBlock());
 		
 		//release the matrices
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
-		ec.setMatrixOutput(output_name, out);
+		ec.setMatrixOutput(output_name, out, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 693de24..eb9e256 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
@@ -105,7 +105,7 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction
 			case VALUEPICK: 
 				if( _inmem ) //INMEM VALUEPICK
 				{
-					MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+					MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 
 					if ( input2.getDataType() == DataType.SCALAR ) {
 						ScalarObject quantile = ec.getScalarInput(input2.getName(), input2.getValueType(), input2.isLiteral());
@@ -113,13 +113,13 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction
 						ec.setScalarOutput(output.getName(), new DoubleObject(picked));
 					} 
 					else {
-						MatrixBlock quantiles = ec.getMatrixInput(input2.getName());
+						MatrixBlock quantiles = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 						MatrixBlock resultBlock = (MatrixBlock) matBlock.pickValues(quantiles, new MatrixBlock());
 						quantiles = null;
-						ec.releaseMatrixInput(input2.getName());
-						ec.setMatrixOutput(output.getName(), resultBlock);
+						ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
+						ec.setMatrixOutput(output.getName(), resultBlock, getExtendedOpcode());
 					}
-					ec.releaseMatrixInput(input1.getName());										
+					ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());										
 				}
 				else //MR VALUEPICK
 				{
@@ -145,9 +145,9 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction
 			case MEDIAN:
 				if( _inmem ) //INMEM MEDIAN
 				{
-					double picked = ec.getMatrixInput(input1.getName()).median();
+					double picked = ec.getMatrixInput(input1.getName(), getExtendedOpcode()).median();
 					ec.setScalarOutput(output.getName(), new DoubleObject(picked));
-					ec.releaseMatrixInput(input1.getName());
+					ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 					break;
 				}
 				else //MR MEDIAN
@@ -173,9 +173,9 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction
 			case IQM:
 				if( _inmem ) //INMEM IQM
 				{
-					MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
+					MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 					double iqm = matBlock1.interQuartileMean();
-					ec.releaseMatrixInput(input1.getName());
+					ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 					ec.setScalarOutput(output.getName(), new DoubleObject(iqm));
 				}
 				else //MR IQM

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 37c4997..d35f26b 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
@@ -86,21 +86,21 @@ public class QuantileSortCPInstruction extends UnaryCPInstruction
 			throws DMLRuntimeException 
 	{
 		//acquire inputs matrices
-		MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		MatrixBlock wtBlock = null;
  		if (input2 != null) {
-			wtBlock = ec.getMatrixInput(input2.getName());
+			wtBlock = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		}
 		
  		//process core instruction
 		MatrixBlock resultBlock = (MatrixBlock) matBlock.sortOperations(wtBlock, new MatrixBlock());
 		
 		//release inputs
-		ec.releaseMatrixInput(input1.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 		if (input2 != null)
-			ec.releaseMatrixInput(input2.getName());
+			ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		//set and release output
-		ec.setMatrixOutput(output.getName(), resultBlock);
+		ec.setMatrixOutput(output.getName(), resultBlock, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 a7fa3b5..4fdea1b 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
@@ -109,9 +109,9 @@ public class QuaternaryCPInstruction extends ComputationCPInstruction
 	{
 		QuaternaryOperator qop = (QuaternaryOperator) _optr;
 		
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
-		MatrixBlock matBlock3 = ec.getMatrixInput(input3.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
+		MatrixBlock matBlock3 = ec.getMatrixInput(input3.getName(), getExtendedOpcode());
 		MatrixBlock matBlock4 = null;
 		if( qop.hasFourInputs() ) {
 			if (input4.getDataType() == DataType.SCALAR) {
@@ -120,7 +120,7 @@ public class QuaternaryCPInstruction extends ComputationCPInstruction
 				matBlock4.quickSetValue(0, 0, eps);
 			}
 			else {
-				matBlock4 = ec.getMatrixInput(input4.getName());
+				matBlock4 = ec.getMatrixInput(input4.getName(), getExtendedOpcode());
 			}
 		}
 		
@@ -128,23 +128,23 @@ public class QuaternaryCPInstruction extends ComputationCPInstruction
 		MatrixValue out = matBlock1.quaternaryOperations(qop, matBlock2, matBlock3, matBlock4, new MatrixBlock(), _numThreads);
 		
 		//release inputs and output
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
-		ec.releaseMatrixInput(input3.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input3.getName(), getExtendedOpcode());
 		if( qop.wtype1 != null || qop.wtype4 != null ) { //wsloss/wcemm
 			if( (qop.wtype1 != null && qop.wtype1.hasFourInputs()) ||
 				(qop.wtype4 != null && qop.wtype4.hasFourInputs()) )
 				if (input4.getDataType() == DataType.MATRIX) {
-					ec.releaseMatrixInput(input4.getName());
+					ec.releaseMatrixInput(input4.getName(), getExtendedOpcode());
 				}
 			ec.setVariable(output.getName(), new DoubleObject(out.getValue(0, 0)));
 		}
 		else { //wsigmoid / wdivmm / wumm
 			if( qop.wtype3 != null && qop.wtype3.hasFourInputs() )
 				if (input4.getDataType() == DataType.MATRIX) {
-					ec.releaseMatrixInput(input4.getName());
+					ec.releaseMatrixInput(input4.getName(), getExtendedOpcode());
 				}
-			ec.setMatrixOutput(output.getName(), (MatrixBlock)out);
+			ec.setMatrixOutput(output.getName(), (MatrixBlock)out, getExtendedOpcode());
 		}
 	}	
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 872a97f..a4cfd38 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
@@ -117,7 +117,7 @@ public class ReorgCPInstruction extends UnaryCPInstruction
 			throws DMLRuntimeException 
 	{
 		//acquire inputs
-		MatrixBlock matBlock = ec.getMatrixInput(input1.getName());		
+		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());		
 		ReorgOperator r_op = (ReorgOperator) _optr;
 		if( r_op.fn instanceof SortIndex ) {
 			//additional attributes for sort
@@ -131,8 +131,8 @@ public class ReorgCPInstruction extends UnaryCPInstruction
 		MatrixBlock soresBlock = (MatrixBlock) (matBlock.reorgOperations(r_op, new MatrixBlock(), 0, 0, 0));
         
 		//release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.setMatrixOutput(output.getName(), soresBlock);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(output.getName(), soresBlock, getExtendedOpcode());
 	}
 	
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 50ed490..eb9a4e5 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
@@ -46,7 +46,7 @@ public class ScalarMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName(), getExtendedOpcode());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 
 		ScalarOperator sc_op = (ScalarOperator) _optr;
@@ -54,13 +54,13 @@ public class ScalarMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		
-		ec.releaseMatrixInput(mat.getName());
+		ec.releaseMatrixInput(mat.getName(), getExtendedOpcode());
 		
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
  			retBlock.examSparsity();
  		}
 		
-		ec.setMatrixOutput(output.getName(), retBlock);
+		ec.setMatrixOutput(output.getName(), retBlock, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 7a5b0af..e0ff4a0 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
@@ -46,7 +46,7 @@ public class ScalarMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName(), getExtendedOpcode());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator) _optr;
@@ -54,13 +54,13 @@ public class ScalarMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 		
 		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		
-		ec.releaseMatrixInput(mat.getName());
+		ec.releaseMatrixInput(mat.getName(), getExtendedOpcode());
 
 		// Ensure right dense/sparse output representation (guarded by released input memory)
 		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
  			retBlock.examSparsity();
  		}
 		
-		ec.setMatrixOutput(output.getName(), retBlock);
+		ec.setMatrixOutput(output.getName(), retBlock, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 0d0140d..7c16242 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
@@ -75,7 +75,7 @@ public class SpoofCPInstruction extends ComputationCPInstruction
 		ArrayList<ScalarObject> scalars = new ArrayList<ScalarObject>();
 		for (CPOperand input : _in) {
 			if(input.getDataType()==DataType.MATRIX)
-				inputs.add(ec.getMatrixInput(input.getName()));
+				inputs.add(ec.getMatrixInput(input.getName(), getExtendedOpcode()));
 			else if(input.getDataType()==DataType.SCALAR) {
 				//note: even if literal, it might be compiled as scalar placeholder
 				scalars.add(ec.getScalarInput(input.getName(), input.getValueType(), input.isLiteral()));
@@ -86,7 +86,7 @@ public class SpoofCPInstruction extends ComputationCPInstruction
 		if( output.getDataType() == DataType.MATRIX) {
 			MatrixBlock out = new MatrixBlock();
 			_op.execute(inputs, scalars, out, _numThreads);
-			ec.setMatrixOutput(output.getName(), out);
+			ec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
 		}
 		else if (output.getDataType() == DataType.SCALAR) {
 			ScalarObject out = _op.execute(inputs, scalars, _numThreads);
@@ -96,6 +96,6 @@ public class SpoofCPInstruction extends ComputationCPInstruction
 		// release input matrices
 		for (CPOperand input : _in)
 			if(input.getDataType()==DataType.MATRIX)
-				ec.releaseMatrixInput(input.getName());
+				ec.releaseMatrixInput(input.getName(), getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 bac796e..055d17c 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
@@ -107,6 +107,6 @@ public class StringInitCPInstruction extends UnaryCPInstruction
 		outBlk.examSparsity();
 		
 		//put output into symbol table
-		ec.setMatrixOutput(outName, outBlk);
+		ec.setMatrixOutput(outName, outBlk, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 0c48ba1..8327dcd 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
@@ -97,7 +97,7 @@ public class TernaryCPInstruction extends ComputationCPInstruction
 	public void processInstruction(ExecutionContext ec) 
 		throws DMLRuntimeException {
 		
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		MatrixBlock matBlock2=null, wtBlock=null;
 		double cst1, cst2;
 		
@@ -126,19 +126,19 @@ public class TernaryCPInstruction extends ComputationCPInstruction
 		switch(ctableOp) {
 		case CTABLE_TRANSFORM: //(VECTOR)
 			// F=ctable(A,B,W)
-			matBlock2 = ec.getMatrixInput(input2.getName());
-			wtBlock = ec.getMatrixInput(input3.getName());
+			matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
+			wtBlock = ec.getMatrixInput(input3.getName(), getExtendedOpcode());
 			matBlock1.ternaryOperations((SimpleOperator)_optr, matBlock2, wtBlock, resultMap, resultBlock);
 			break;
 		case CTABLE_TRANSFORM_SCALAR_WEIGHT: //(VECTOR/MATRIX)
 			// F = ctable(A,B) or F = ctable(A,B,1)
-			matBlock2 = ec.getMatrixInput(input2.getName());
+			matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 			cst1 = ec.getScalarInput(input3.getName(), input3.getValueType(), input3.isLiteral()).getDoubleValue();
 			matBlock1.ternaryOperations((SimpleOperator)_optr, matBlock2, cst1, _ignoreZeros, resultMap, resultBlock);
 			break;
 		case CTABLE_EXPAND_SCALAR_WEIGHT: //(VECTOR)
 			// F = ctable(seq,A) or F = ctable(seq,B,1)
-			matBlock2 = ec.getMatrixInput(input2.getName());
+			matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 			cst1 = ec.getScalarInput(input3.getName(), input3.getValueType(), input3.isLiteral()).getDoubleValue();
 			// only resultBlock.rlen known, resultBlock.clen set in operation
 			matBlock1.ternaryOperations((SimpleOperator)_optr, matBlock2, cst1, resultBlock);
@@ -151,7 +151,7 @@ public class TernaryCPInstruction extends ComputationCPInstruction
 			break;
 		case CTABLE_TRANSFORM_WEIGHTED_HISTOGRAM: //(VECTOR)
 			// F=ctable(A,1,W)
-			wtBlock = ec.getMatrixInput(input3.getName());
+			wtBlock = ec.getMatrixInput(input3.getName(), getExtendedOpcode());
 			cst1 = ec.getScalarInput(input2.getName(), input2.getValueType(), input2.isLiteral()).getDoubleValue();
 			matBlock1.ternaryOperations((SimpleOperator)_optr, cst1, wtBlock, resultMap, resultBlock);
 			break;
@@ -161,11 +161,11 @@ public class TernaryCPInstruction extends ComputationCPInstruction
 		}
 		
 		if(input1.getDataType() == DataType.MATRIX)
-			ec.releaseMatrixInput(input1.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 		if(input2.getDataType() == DataType.MATRIX)
-			ec.releaseMatrixInput(input2.getName());
+			ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		if(input3.getDataType() == DataType.MATRIX)
-			ec.releaseMatrixInput(input3.getName());
+			ec.releaseMatrixInput(input3.getName(), getExtendedOpcode());
 		
 		if ( resultBlock == null ){
 			//we need to respect potentially specified output dimensions here, because we might have 
@@ -178,6 +178,6 @@ public class TernaryCPInstruction extends ComputationCPInstruction
 		else
 			resultBlock.examSparsity();
 		
-		ec.setMatrixOutput(output.getName(), resultBlock);
+		ec.setMatrixOutput(output.getName(), resultBlock, getExtendedOpcode());
 	}	
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 fcd2d56..746ee04 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
@@ -87,19 +87,19 @@ public class UaggOuterChainCPInstruction extends UnaryCPInstruction
 		MatrixBlock mbLeft = null, mbRight = null, mbOut = null;		
 		//get the main data input
 		if( rightCached ) { 
-			mbLeft = ec.getMatrixInput(input1.getName());
-			mbRight = ec.getMatrixInput(input2.getName());
+			mbLeft = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+			mbRight = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		}
 		else { 
-			mbLeft = ec.getMatrixInput(input2.getName());
-			mbRight = ec.getMatrixInput(input1.getName());
+			mbLeft = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
+			mbRight = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		}
 		
 		mbOut = mbLeft.uaggouterchainOperations(mbLeft, mbRight, mbOut, _bOp, _uaggOp);
 
 		//release locks
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		if( _uaggOp.aggOp.correctionExists )
 			mbOut.dropLastRowsOrColums(_uaggOp.aggOp.correctionLocation);
@@ -122,7 +122,7 @@ public class UaggOuterChainCPInstruction extends UnaryCPInstruction
 		{	
 			//Additional memory requirement to convert from dense to sparse can be leveraged from released memory needed for input data above.
 			mbOut.examSparsity();
-			ec.setMatrixOutput(output_name, mbOut);
+			ec.setMatrixOutput(output_name, mbOut, getExtendedOpcode());
 		}
 		
 	}		

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 3b9ef66..336cfbf 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
@@ -585,11 +585,11 @@ public class VariableCPInstruction extends CPInstruction
 						ScalarObjectFactory.createScalarObject(fBlock.getSchema()[0], value));
 			}
 			else { //assume DataType.MATRIX otherwise
-				MatrixBlock mBlock = ec.getMatrixInput(getInput1().getName());
+				MatrixBlock mBlock = ec.getMatrixInput(getInput1().getName(), getExtendedOpcode());
 				if( mBlock.getNumRows()!=1 || mBlock.getNumColumns()!=1 )
 					throw new DMLRuntimeException("Dimension mismatch - unable to cast matrix '"+getInput1().getName()+"' of dimension ("+mBlock.getNumRows()+" x "+mBlock.getNumColumns()+") to scalar.");
 				double value = mBlock.getValue(0,0);
-				ec.releaseMatrixInput(getInput1().getName());
+				ec.releaseMatrixInput(getInput1().getName(), getExtendedOpcode());
 				ec.setScalarOutput(output.getName(), new DoubleObject(value));
 			}
 			break;
@@ -605,7 +605,7 @@ public class VariableCPInstruction extends CPInstruction
 				out = new MatrixBlock(1,1,false);
 				out.quickSetValue(0, 0, scalarInput.getDoubleValue());		
 			}
-			ec.setMatrixOutput(output.getName(), out);
+			ec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
 			break;
 		}
 		case CastAsFrameVariable:{
@@ -617,9 +617,9 @@ public class VariableCPInstruction extends CPInstruction
 				out.set(0, 0, scalarInput.getStringValue());	
 			}
 			else { //DataType.FRAME
-				MatrixBlock min = ec.getMatrixInput(getInput1().getName());
+				MatrixBlock min = ec.getMatrixInput(getInput1().getName(), getExtendedOpcode());
 				out = DataConverter.convertToFrameBlock(min);
-				ec.releaseMatrixInput(getInput1().getName());
+				ec.releaseMatrixInput(getInput1().getName(), getExtendedOpcode());
 			}
 			ec.setFrameOutput(output.getName(), out);
 			break;

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cpfile/MatrixIndexingCPFileInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cpfile/MatrixIndexingCPFileInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cpfile/MatrixIndexingCPFileInstruction.java
index a1e9ca1..70bff67 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cpfile/MatrixIndexingCPFileInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cpfile/MatrixIndexingCPFileInstruction.java
@@ -137,7 +137,7 @@ public final class MatrixIndexingCPFileInstruction extends IndexingCPInstruction
 			{
 				//will return an empty matrix partition 
 				MatrixBlock resultBlock = mo.readMatrixPartition( ixrange );
-				ec.setMatrixOutput(output.getName(), resultBlock);
+				ec.setMatrixOutput(output.getName(), resultBlock, getExtendedOpcode());
 			}
 		}
 		else

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateTernarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateTernarySPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateTernarySPInstruction.java
index 6a494b7..7ac1e5b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateTernarySPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateTernarySPInstruction.java
@@ -111,7 +111,7 @@ public class AggregateTernarySPInstruction extends ComputationSPInstruction
 			
 			//put output block into symbol table (no lineage because single block)
 			//this also includes implicit maintenance of matrix characteristics
-			sec.setMatrixOutput(output.getName(), ret);		
+			sec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());		
 		}
 		else //tack+* multi block
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateUnarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateUnarySPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateUnarySPInstruction.java
index 73f67a3..352a72e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateUnarySPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/AggregateUnarySPInstruction.java
@@ -106,7 +106,7 @@ public class AggregateUnarySPInstruction extends UnarySPInstruction
 			
 			//put output block into symbol table (no lineage because single block)
 			//this also includes implicit maintenance of matrix characteristics
-			sec.setMatrixOutput(output.getName(), out3);
+			sec.setMatrixOutput(output.getName(), out3, getExtendedOpcode());
 		}
 		else //MULTI_BLOCK or NONE
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/ConvolutionSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ConvolutionSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ConvolutionSPInstruction.java
index fcf90e9..c3101c2 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ConvolutionSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ConvolutionSPInstruction.java
@@ -237,8 +237,8 @@ public class ConvolutionSPInstruction extends UnarySPInstruction {
 	}
 	
 	private Broadcast<MatrixBlock> getBroadcast(SparkExecutionContext sec, String name) throws DMLRuntimeException {
-		MatrixBlock mb = sec.getMatrixInput( name );
-		sec.releaseMatrixInput(name);
+		MatrixBlock mb = sec.getMatrixInput( name, getExtendedOpcode() );
+		sec.releaseMatrixInput(name, getExtendedOpcode());
 		return sec.getSparkContext().broadcast(mb);
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/CpmmSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CpmmSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CpmmSPInstruction.java
index 4a29c5e..4b46a96 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CpmmSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CpmmSPInstruction.java
@@ -107,7 +107,7 @@ public class CpmmSPInstruction extends BinarySPInstruction
 			
 			//put output block into symbol table (no lineage because single block)
 			//this also includes implicit maintenance of matrix characteristics
-			sec.setMatrixOutput(output.getName(), out2);	
+			sec.setMatrixOutput(output.getName(), out2, getExtendedOpcode());	
 		}
 		else //DEFAULT: MULTI_BLOCK
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmChainSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmChainSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmChainSPInstruction.java
index 89cdaa5..f5a6312 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmChainSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmChainSPInstruction.java
@@ -134,7 +134,7 @@ public class MapmmChainSPInstruction extends SPInstruction
 		
 		//put output block into symbol table (no lineage because single block)
 		//this also includes implicit maintenance of matrix characteristics
-		sec.setMatrixOutput(_output.getName(), out);
+		sec.setMatrixOutput(_output.getName(), out, getExtendedOpcode());
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmSPInstruction.java
index c1fdea6..14e04ee 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/MapmmSPInstruction.java
@@ -143,7 +143,7 @@ public class MapmmSPInstruction extends BinarySPInstruction
 			
 			//put output block into symbol table (no lineage because single block)
 			//this also includes implicit maintenance of matrix characteristics
-			sec.setMatrixOutput(output.getName(), out2);
+			sec.setMatrixOutput(output.getName(), out2, getExtendedOpcode());
 		}
 		else //MULTI_BLOCK or NONE
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
index 0d4d3c4..b8aab2c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
@@ -116,10 +116,10 @@ public class MatrixIndexingSPInstruction  extends IndexingSPInstruction
 			JavaPairRDD<MatrixIndexes,MatrixBlock> in1 = sec.getBinaryBlockRDDHandleForVariable( input1.getName() );
 			
 			if( isSingleBlockLookup(mcIn, ixrange) ) {
-				sec.setMatrixOutput(output.getName(), singleBlockIndexing(in1, mcIn, mcOut, ixrange));
+				sec.setMatrixOutput(output.getName(), singleBlockIndexing(in1, mcIn, mcOut, ixrange), getExtendedOpcode());
 			}
 			else if( isMultiBlockLookup(in1, mcIn, mcOut, ixrange) ) {
-				sec.setMatrixOutput(output.getName(), multiBlockIndexing(in1, mcIn, mcOut, ixrange));
+				sec.setMatrixOutput(output.getName(), multiBlockIndexing(in1, mcIn, mcOut, ixrange), getExtendedOpcode());
 			}
 			else { //rdd output for general case
 				JavaPairRDD<MatrixIndexes,MatrixBlock> out = generalCaseRightIndexing(in1, mcIn, mcOut, ixrange, _aggType);

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
index 179ef9e..e62dd60 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ParameterizedBuiltinSPInstruction.java
@@ -213,7 +213,7 @@ public class ParameterizedBuiltinSPInstruction  extends ComputationSPInstruction
 				
 				//put output block into symbol table (no lineage because single block)
 				//this also includes implicit maintenance of matrix characteristics
-				sec.setMatrixOutput(output.getName(), out2);
+				sec.setMatrixOutput(output.getName(), out2, getExtendedOpcode());
 			}
 			//multi-block aggregation
 			else {
@@ -369,7 +369,7 @@ public class ParameterizedBuiltinSPInstruction  extends ComputationSPInstruction
 			else //special case: empty output (ensure valid dims)
 			{
 				MatrixBlock out = new MatrixBlock(rows?1:(int)mcIn.getRows(), rows?(int)mcIn.getCols():1, true); 
-				sec.setMatrixOutput(output.getName(), out);
+				sec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
 			}
 		}
 		else if ( opcode.equalsIgnoreCase("replace") ) 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 e96cd63..9d89ae1 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
@@ -345,7 +345,7 @@ public class RandSPInstruction extends UnarySPInstruction
 					sparsity, minValue, maxValue, pdfParams);
 			MatrixBlock mb = MatrixBlock.randOperations(rgen, lSeed);
 			
-			sec.setMatrixOutput(output.getName(), mb);
+			sec.setMatrixOutput(output.getName(), mb, getExtendedOpcode());
 			Statistics.decrementNoOfExecutedSPInst();
 			return;
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/SpoofSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/SpoofSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/SpoofSPInstruction.java
index 2be7b2c..1d360a1 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/SpoofSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/SpoofSPInstruction.java
@@ -161,7 +161,7 @@ public class SpoofSPInstruction extends SPInstruction
 					.mapToPair(new MultiAggregateFunction(_class.getName(), _classBytes, bcMatrices, scalars))
 					.values().fold(new MatrixBlock(), new MultiAggAggregateFunction(aggOps) );
 			
-			sec.setMatrixOutput(_out.getName(), tmpMB);
+			sec.setMatrixOutput(_out.getName(), tmpMB, getExtendedOpcode());
 			return;
 		}
 		else if(_class.getSuperclass() == SpoofOuterProduct.class) // outer product operator
@@ -205,7 +205,7 @@ public class SpoofSPInstruction extends SPInstruction
 			if( op.getRowType().isColumnAgg() || op.getRowType()==RowType.FULL_AGG ) {
 				MatrixBlock tmpMB = RDDAggregateUtils.sumStable(out);
 				if( op.getRowType().isColumnAgg() )
-					sec.setMatrixOutput(_out.getName(), tmpMB);
+					sec.setMatrixOutput(_out.getName(), tmpMB, getExtendedOpcode());
 				else
 					sec.setScalarOutput(_out.getName(), 
 						new DoubleObject(tmpMB.quickGetValue(0, 0)));

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/Tsmm2SPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/Tsmm2SPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/Tsmm2SPInstruction.java
index 1f1b3e4..26461a6 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/Tsmm2SPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/Tsmm2SPInstruction.java
@@ -114,7 +114,7 @@ public class Tsmm2SPInstruction extends UnarySPInstruction
 		      
 			//put output block into symbol table (no lineage because single block)
 			//this also includes implicit maintenance of matrix characteristics
-			sec.setMatrixOutput(output.getName(), out);
+			sec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
 		}
 		else {
 			//output individual output blocks and aggregate by key (no action)

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/TsmmSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/TsmmSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/TsmmSPInstruction.java
index 4386b02..61a9130 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/TsmmSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/TsmmSPInstruction.java
@@ -82,7 +82,7 @@ public class TsmmSPInstruction extends UnarySPInstruction
 		      
 		//put output block into symbol table (no lineage because single block)
 		//this also includes implicit maintenance of matrix characteristics
-		sec.setMatrixOutput(output.getName(), out);
+		sec.setMatrixOutput(output.getName(), out, getExtendedOpcode());
 	}
 
 	private static class RDDTSMMFunction implements Function<Tuple2<MatrixIndexes,MatrixBlock>, MatrixBlock> 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/UaggOuterChainSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/UaggOuterChainSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/UaggOuterChainSPInstruction.java
index d6cc8e8..8f74b9d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/UaggOuterChainSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/UaggOuterChainSPInstruction.java
@@ -132,8 +132,8 @@ public class UaggOuterChainSPInstruction extends BinarySPInstruction
 		if (LibMatrixOuterAgg.isSupportedUaggOp(_uaggOp, _bOp))
 		{
 			//create sorted broadcast matrix 
-			MatrixBlock mb = sec.getMatrixInput(bcastVar);
-			sec.releaseMatrixInput(bcastVar);
+			MatrixBlock mb = sec.getMatrixInput(bcastVar, getExtendedOpcode());
+			sec.releaseMatrixInput(bcastVar, getExtendedOpcode());
 			bcastVar = null; //prevent lineage tracking
 			double[] vmb = DataConverter.convertToDoubleVector(mb);
 			Broadcast<int[]> bvi = null;
@@ -166,7 +166,7 @@ public class UaggOuterChainSPInstruction extends BinarySPInstruction
 			tmp.dropLastRowsOrColums(_aggOp.correctionLocation);
 
 			//put output block into symbol table (no lineage because single block)
-			sec.setMatrixOutput(output.getName(), tmp);
+			sec.setMatrixOutput(output.getName(), tmp, getExtendedOpcode());
 		}
 		else //R/C AGG (output is rdd)
 		{			

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/spark/ZipmmSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ZipmmSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ZipmmSPInstruction.java
index e21eefa..e61e39b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ZipmmSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ZipmmSPInstruction.java
@@ -103,7 +103,7 @@ public class ZipmmSPInstruction extends BinarySPInstruction
 		
 		//put output block into symbol table (no lineage because single block)
 		//this also includes implicit maintenance of matrix characteristics
-		sec.setMatrixOutput(output.getName(), out2);	
+		sec.setMatrixOutput(output.getName(), out2, getExtendedOpcode());	
 	}
 
 	private static class ZipMultiplyFunction implements Function<Tuple2<MatrixBlock,MatrixBlock>, MatrixBlock> 


[3/3] systemml git commit: [SYSTEMML-1796] Provide fine-grained statistics for SystemML

Posted by ni...@apache.org.
[SYSTEMML-1796] Provide fine-grained statistics for SystemML

Though SystemML provides explain and statistics tooling functionality to
understand the execution plan and the performance, most users (even
committers) are not able to explain why SystemML's performance is so good
or bad. This is likely because SystemML's optimizer performs sophisticated
dataflow analysis, cost optimization as well as static and dynamic
rewrites. New features/backends such as code generation, automatic fusion,
compression and GPU backend adds additional level of complexity to the
optimizer. This commit is a small step in the direction of explaining
SystemML's behavior.

Instead of examining statistics output and then
trying to correlate with the explain plan, this commit provides
instruction-level statistics with the corresponding DML snippet (file
beginLine:beginCol-endLine-endCol).

Output with the configuration property systemml.stats.finegrained set to true:
```
Heavy hitter instructions:
  #  Instruction               Time(s)  Count  Misc Timers
  1  solve [148:0-148:0]         0.027      1  aqmd[0.000s,1]
  2  tsmm [137:4-137:13]         0.008      1  rlsi[0.000s,1], aqmd[0.000s,1], aqrd[0.000s,1]
  3  uak+ [161:10-161:10]        0.007      1  aqrd[0.000s,1], rlsi[0.000s,1]
  4  ba+* [138:4-138:13]         0.005      1  aqmd[0.000s,1], aqrd[0.000s,2], rlsi[0.000s,2]
  5  rand [89:0-89:0]            0.004      1  aqmd[0.000s,1]
  6  * [133:9-133:24]            0.002      1  aqmd[0.000s,1], aqrd[0.000s,1], rlsi[0.000s,1]
  7  r' [138:4-138:13]           0.001      2  aqmd[0.000s,2], aqrd[0.000s,2], rlsi[0.000s,2]
  8  append [97:4-97:4]          0.001      1  rlswr[0.000s,1], aqmd[0.000s,1], rlsev[0.000s,0], aqrd[0.000s,2], rlsi[0.000s,2]
  9  + [144:4-144:21]            0.001      1  aqmd[0.000s,1], aqrd[0.000s,2], rlsi[0.000s,2]
 10  leftIndex [104:4-104:26]    0.001      1  aqmd[0.000s,1], rlsi[0.000s,1], aqrd[0.000s,1]
```

aqrd and aqmd refers to dense input and outputs respectively. Similarly, aqrs and aqms refers to sparse input and outputs. It can also show both:
```
1  leftIndex [106:4-106:42]                                                       141.928   28552  aqrs[0.024s,23165], rls[0.066s,57104], csrlix[116.652s,22649], s2d[9.297s,258], aqrd[14.097s,33939], lixcp[1.410s,5903]
```

Closes #577.


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

Branch: refs/heads/master
Commit: 648eb21d66f9cd8727090cdf950986765a7e6ee8
Parents: 6133be2
Author: Niketan Pansare <np...@us.ibm.com>
Authored: Fri Jul 21 09:09:03 2017 -0800
Committer: Niketan Pansare <np...@us.ibm.com>
Committed: Fri Jul 21 10:09:03 2017 -0700

----------------------------------------------------------------------
 conf/SystemML-config.xml.template               |  3 +
 .../java/org/apache/sysml/api/DMLScript.java    |  1 +
 .../apache/sysml/api/ScriptExecutorUtils.java   |  1 +
 .../java/org/apache/sysml/conf/DMLConfig.java   |  4 +-
 .../sysml/debug/DMLDebuggerProgramInfo.java     |  6 +-
 .../java/org/apache/sysml/hops/BinaryOp.java    | 36 +++++------
 src/main/java/org/apache/sysml/hops/Hop.java    | 15 +++--
 .../java/org/apache/sysml/hops/TernaryOp.java   | 16 ++---
 .../java/org/apache/sysml/hops/UnaryOp.java     |  2 +-
 .../sysml/hops/rewrite/HopRewriteUtils.java     |  2 +-
 .../org/apache/sysml/lops/CombineBinary.java    |  2 +-
 .../org/apache/sysml/lops/CombineTernary.java   |  2 +-
 .../org/apache/sysml/lops/CombineUnary.java     |  2 +-
 src/main/java/org/apache/sysml/lops/Lop.java    |  7 ++-
 .../java/org/apache/sysml/lops/SortKeys.java    |  4 +-
 .../java/org/apache/sysml/lops/compile/Dag.java |  2 +-
 .../org/apache/sysml/parser/DMLProgram.java     | 10 +--
 .../org/apache/sysml/parser/DMLTranslator.java  | 52 +++++++--------
 .../ExternalFunctionProgramBlock.java           |  2 +-
 .../runtime/controlprogram/ProgramBlock.java    |  6 +-
 .../controlprogram/caching/CacheableData.java   | 38 +++++++++--
 .../controlprogram/caching/LazyWriteBuffer.java | 21 ++++++-
 .../context/ExecutionContext.java               | 49 +++++++++++----
 .../parfor/opt/OptimizerRuleBased.java          |  2 +-
 .../sysml/runtime/instructions/Instruction.java | 41 +++++++++---
 .../cp/AggregateBinaryCPInstruction.java        | 10 +--
 .../cp/AggregateTernaryCPInstruction.java       | 14 ++---
 .../cp/AggregateUnaryCPInstruction.java         |  6 +-
 .../runtime/instructions/cp/CPInstruction.java  | 16 +++++
 .../cp/CentralMomentCPInstruction.java          |  8 +--
 .../cp/CompressionCPInstruction.java            |  6 +-
 .../cp/ConvolutionCPInstruction.java            | 60 +++++++++---------
 .../cp/CovarianceCPInstruction.java             | 16 ++---
 .../instructions/cp/DataGenCPInstruction.java   |  2 +-
 .../cp/DataPartitionCPInstruction.java          |  2 +-
 .../instructions/cp/MMChainCPInstruction.java   | 14 ++---
 .../instructions/cp/MMTSJCPInstruction.java     |  6 +-
 .../cp/MatrixAppendCPInstruction.java           | 10 +--
 .../cp/MatrixBuiltinCPInstruction.java          |  8 +--
 .../cp/MatrixIndexingCPInstruction.java         | 22 +++----
 .../cp/MatrixMatrixArithmeticCPInstruction.java | 10 +--
 .../cp/MatrixMatrixBuiltinCPInstruction.java    | 12 ++--
 .../cp/MatrixMatrixRelationalCPInstruction.java | 10 +--
 .../cp/MatrixReshapeCPInstruction.java          |  6 +-
 .../cp/MatrixScalarBuiltinCPInstruction.java    |  6 +-
 .../cp/MultiReturnBuiltinCPInstruction.java     |  2 +-
 ...ReturnParameterizedBuiltinCPInstruction.java |  2 +-
 .../instructions/cp/PMMJCPInstruction.java      | 10 +--
 .../cp/ParameterizedBuiltinCPInstruction.java   | 42 ++++++-------
 .../instructions/cp/PlusMultCPInstruction.java  | 10 +--
 .../cp/QuantilePickCPInstruction.java           | 18 +++---
 .../cp/QuantileSortCPInstruction.java           | 10 +--
 .../cp/QuaternaryCPInstruction.java             | 20 +++---
 .../instructions/cp/ReorgCPInstruction.java     |  6 +-
 .../cp/ScalarMatrixArithmeticCPInstruction.java |  6 +-
 .../cp/ScalarMatrixRelationalCPInstruction.java |  6 +-
 .../instructions/cp/SpoofCPInstruction.java     |  6 +-
 .../cp/StringInitCPInstruction.java             |  2 +-
 .../instructions/cp/TernaryCPInstruction.java   | 20 +++---
 .../cp/UaggOuterChainCPInstruction.java         | 14 ++---
 .../instructions/cp/VariableCPInstruction.java  | 10 +--
 .../cpfile/MatrixIndexingCPFileInstruction.java |  2 +-
 .../spark/AggregateTernarySPInstruction.java    |  2 +-
 .../spark/AggregateUnarySPInstruction.java      |  2 +-
 .../spark/ConvolutionSPInstruction.java         |  4 +-
 .../instructions/spark/CpmmSPInstruction.java   |  2 +-
 .../spark/MapmmChainSPInstruction.java          |  2 +-
 .../instructions/spark/MapmmSPInstruction.java  |  2 +-
 .../spark/MatrixIndexingSPInstruction.java      |  4 +-
 .../ParameterizedBuiltinSPInstruction.java      |  4 +-
 .../instructions/spark/RandSPInstruction.java   |  2 +-
 .../instructions/spark/SpoofSPInstruction.java  |  4 +-
 .../instructions/spark/Tsmm2SPInstruction.java  |  2 +-
 .../instructions/spark/TsmmSPInstruction.java   |  2 +-
 .../spark/UaggOuterChainSPInstruction.java      |  6 +-
 .../instructions/spark/ZipmmSPInstruction.java  |  2 +-
 .../sysml/runtime/matrix/data/MatrixBlock.java  | 66 +++++++++++++++++---
 .../org/apache/sysml/utils/GPUStatistics.java   |  4 +-
 .../java/org/apache/sysml/utils/Statistics.java |  6 +-
 79 files changed, 526 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/conf/SystemML-config.xml.template
----------------------------------------------------------------------
diff --git a/conf/SystemML-config.xml.template b/conf/SystemML-config.xml.template
index ff724b4..11e86ed 100644
--- a/conf/SystemML-config.xml.template
+++ b/conf/SystemML-config.xml.template
@@ -69,6 +69,9 @@
    <!-- enables native blas for matrix multiplication and convolution, experimental feature (options: auto, mkl, openblas, none) -->
    <native.blas>none</native.blas>
 
+   <!-- prints finegrained statistics information -->
+   <systemml.stats.finegrained>false</systemml.stats.finegrained>
+   
    <!-- prints extra statistics information for GPU -->
    <systemml.stats.extraGPU>false</systemml.stats.extraGPU>
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/api/DMLScript.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/DMLScript.java b/src/main/java/org/apache/sysml/api/DMLScript.java
index 515d632..2674af4 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -160,6 +160,7 @@ public class DMLScript
 
 	public static RUNTIME_PLATFORM  rtplatform          = DMLOptions.defaultOptions.execMode;    // the execution mode
 	public static boolean           STATISTICS          = DMLOptions.defaultOptions.stats;       // whether to print statistics
+	public static boolean           FINEGRAINED_STATISTICS  = DMLOptions.defaultOptions.stats;       // whether to print statistics
 	public static int               STATISTICS_COUNT    = DMLOptions.defaultOptions.statsCount;  // statistics maximum heavy hitter count
 	public static boolean           ENABLE_DEBUG_MODE   = DMLOptions.defaultOptions.debug;       // debug mode
 	public static ExplainType       EXPLAIN             = DMLOptions.defaultOptions.explainType; // explain type

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
index 2895aa4..0e0950e 100644
--- a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
+++ b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
@@ -76,6 +76,7 @@ public class ScriptExecutorUtils {
 		// in digging into performance problems are recorded and displayed
 		GPUStatistics.DISPLAY_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_GPU_STATS);
 		LibMatrixDNN.DISPLAY_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_DNN_STATS);
+		DMLScript.FINEGRAINED_STATISTICS = dmlconf.getBooleanValue(DMLConfig.EXTRA_FINEGRAINED_STATS);
 
 		// Sets the maximum number of GPUs per process, -1 for all available
 		// GPUs

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/conf/DMLConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/conf/DMLConfig.java b/src/main/java/org/apache/sysml/conf/DMLConfig.java
index 5b5b8ea..c248098 100644
--- a/src/main/java/org/apache/sysml/conf/DMLConfig.java
+++ b/src/main/java/org/apache/sysml/conf/DMLConfig.java
@@ -75,6 +75,7 @@ public class DMLConfig
 	public static final String CODEGEN              = "codegen.enabled"; //boolean
 	public static final String CODEGEN_PLANCACHE    = "codegen.plancache"; //boolean
 	public static final String CODEGEN_LITERALS     = "codegen.literals"; //1..heuristic, 2..always
+	public static final String EXTRA_FINEGRAINED_STATS = "systemml.stats.finegrained"; //boolean
 	public static final String EXTRA_GPU_STATS      = "systemml.stats.extraGPU"; //boolean
 	public static final String EXTRA_DNN_STATS      = "systemml.stats.extraDNN"; //boolean
 	public static final String MAX_GPUS_PER_PROCESS = "systemml.gpu.perProcessMax"; // boolean, maximum number of gpus to use, -1 for all
@@ -117,6 +118,7 @@ public class DMLConfig
 		_defaultVals.put(CODEGEN_PLANCACHE,      "true" );
 		_defaultVals.put(CODEGEN_LITERALS,       "1" );
 		_defaultVals.put(NATIVE_BLAS,            "none" );
+		_defaultVals.put(EXTRA_FINEGRAINED_STATS,"false" );
 		_defaultVals.put(EXTRA_GPU_STATS,        "false" );
 		_defaultVals.put(EXTRA_DNN_STATS,        "false" );
 
@@ -403,7 +405,7 @@ public class DMLConfig
 				YARN_APPMASTER, YARN_APPMASTERMEM, YARN_MAPREDUCEMEM, 
 				CP_PARALLEL_OPS, CP_PARALLEL_IO, NATIVE_BLAS,
 				COMPRESSED_LINALG, CODEGEN, CODEGEN_LITERALS, CODEGEN_PLANCACHE,
-				EXTRA_GPU_STATS, EXTRA_DNN_STATS
+				EXTRA_GPU_STATS, EXTRA_DNN_STATS, EXTRA_FINEGRAINED_STATS
 		}; 
 		
 		StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/debug/DMLDebuggerProgramInfo.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/debug/DMLDebuggerProgramInfo.java b/src/main/java/org/apache/sysml/debug/DMLDebuggerProgramInfo.java
index d943a69..92a9495 100644
--- a/src/main/java/org/apache/sysml/debug/DMLDebuggerProgramInfo.java
+++ b/src/main/java/org/apache/sysml/debug/DMLDebuggerProgramInfo.java
@@ -336,9 +336,9 @@ public class DMLDebuggerProgramInfo
 				}
 				//set MR job line number
 				if (min == 0 || min == Integer.MAX_VALUE)
-					currMRInst.setLocation(prevLineNum, prevLineNum, -1, -1); //last seen instruction line number
+					currMRInst.setLocation(null, prevLineNum, prevLineNum, -1, -1); //last seen instruction line number
 				else
-					currMRInst.setLocation(min, min, -1, -1); //minimum instruction line number for this MR job
+					currMRInst.setLocation(null, min, min, -1, -1); //minimum instruction line number for this MR job
 				//insert current MR instruction into corresponding source code line
 				if (!disassembler.containsKey(currMRInst.getLineNum()))
 					disassembler.put(currMRInst.getLineNum(), new ArrayList<Instruction>());
@@ -348,7 +348,7 @@ public class DMLDebuggerProgramInfo
 			{
 				//if CP instruction line number is not set, then approximate to last seen line number
 				if (currInst.getLineNum() == 0)
-					currInst.setLocation(prevLineNum, prevLineNum, -1, -1);
+					currInst.setLocation(null, prevLineNum, prevLineNum, -1, -1);
 				//insert current CP instruction into corresponding source code line
 				if (!disassembler.containsKey(currInst.getLineNum()))
 					disassembler.put(currInst.getLineNum(), new ArrayList<Instruction>());

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/hops/BinaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/BinaryOp.java b/src/main/java/org/apache/sysml/hops/BinaryOp.java
index 2c88a9e..d072afd 100644
--- a/src/main/java/org/apache/sysml/hops/BinaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/BinaryOp.java
@@ -318,7 +318,7 @@ public class BinaryOp extends Hop
 			pick.getOutputParameters().setDimensions(getDim1(),
 					getDim2(), getRowsInBlock(), getColsInBlock(), getNnz());
 			
-			pick.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			pick.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 
 			setLops(pick);
 		}
@@ -344,7 +344,7 @@ public class BinaryOp extends Hop
 			pick.getOutputParameters().setDimensions(getDim1(),
 					getDim2(), getRowsInBlock(), getColsInBlock(), getNnz());
 			
-			pick.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			pick.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 
 			setLops(pick);
 		}
@@ -465,7 +465,7 @@ public class BinaryOp extends Hop
 			pick.getOutputParameters().setDimensions(getDim1(),
 					getDim2(), getRowsInBlock(), getColsInBlock(), getNnz());
 			
-			pick.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			pick.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 
 			setLops(pick);
 		}
@@ -683,7 +683,7 @@ public class BinaryOp extends Hop
 						          < OptimizerUtils.getLocalMemBudget()) ? ExecType.CP : ExecType.MR; //operator selection
 						dcInput = new DataPartition(dcInput, DataType.MATRIX, ValueType.DOUBLE, etPart, (right.getDim2()==1)?PDataPartitionFormat.ROW_BLOCK_WISE_N:PDataPartitionFormat.COLUMN_BLOCK_WISE_N);
 						dcInput.getOutputParameters().setDimensions(right.getDim1(), right.getDim2(), right.getRowsInBlock(), right.getColsInBlock(), right.getNnz());
-						dcInput.setAllPositions(right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
+						dcInput.setAllPositions(right.getFilename(), right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
 					}					
 					
 					BinaryM binary = new BinaryM(left.constructLops(), dcInput, HopsOpOp2LopsB.get(op),
@@ -1083,11 +1083,11 @@ public class BinaryOp extends Hop
 					          < OptimizerUtils.getLocalMemBudget()) ? ExecType.CP : ExecType.MR; //operator selection
 					dcInput = new DataPartition(dcInput, DataType.MATRIX, ValueType.DOUBLE, etPart, PDataPartitionFormat.ROW_BLOCK_WISE_N);
 					dcInput.getOutputParameters().setDimensions(right.getDim1(), right.getDim2(), right.getRowsInBlock(), right.getColsInBlock(), right.getNnz());
-					dcInput.setAllPositions(right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
+					dcInput.setAllPositions(right.getFilename(), right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
 				}					
 				
 				AppendM appM = new AppendM(left.constructLops(), dcInput, offset, dt, vt, cbind, needPart, ExecType.MR);
-				appM.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+				appM.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 				appM.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
 				ret = appM;
 				break;
@@ -1097,15 +1097,15 @@ public class BinaryOp extends Hop
 				//group
 				Group group1 = new Group(left.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, vt);
 				group1.getOutputParameters().setDimensions(m1_dim1, m1_dim2, brlen, bclen, left.getNnz());
-				group1.setAllPositions(left.getBeginLine(), left.getBeginColumn(), left.getEndLine(), left.getEndColumn());
+				group1.setAllPositions(left.getFilename(), left.getBeginLine(), left.getBeginColumn(), left.getEndLine(), left.getEndColumn());
 				
 				Group group2 = new Group(right.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, vt);
 				group1.getOutputParameters().setDimensions(m2_dim1, m2_dim2, brlen, bclen, right.getNnz());
-				group1.setAllPositions(right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
+				group1.setAllPositions(right.getFilename(), right.getBeginLine(), right.getBeginColumn(), right.getEndLine(), right.getEndColumn());
 				
 				AppendR appR = new AppendR(group1, group2, dt, vt, cbind, ExecType.MR);
 				appR.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
-				appR.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+				appR.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 				
 				ret = appR;
 				break;
@@ -1117,17 +1117,17 @@ public class BinaryOp extends Hop
 				
 				AppendG appG = new AppendG(left.constructLops(), right.constructLops(),	offset, offset2, dt, vt, cbind, ExecType.MR);
 				appG.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
-				appG.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+				appG.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 				
 				//group
 				Group group1 = new Group(appG, Group.OperationTypes.Sort, DataType.MATRIX, vt);
 				group1.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
-				group1.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+				group1.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 				
 				//aggregate
 				Aggregate agg1 = new Aggregate(group1, Aggregate.OperationTypes.Sum, DataType.MATRIX, vt, ExecType.MR);
 				agg1.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
-				agg1.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+				agg1.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 				ret = agg1;
 				break;
 			}	
@@ -1178,7 +1178,7 @@ public class BinaryOp extends Hop
 				throw new HopsException("Invalid SP append method: "+am);
 		}
 		
-		ret.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+		ret.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 		
 		
 		return ret;
@@ -1225,23 +1225,23 @@ public class BinaryOp extends Hop
 		
 		Group group1 = new Group(left.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, vt);
 		group1.getOutputParameters().setDimensions(m1_dim1, m1_dim2, brlen, bclen, left.getNnz());
-		group1.setAllPositions(left.getBeginLine(), left.getBeginColumn(), left.getEndLine(), left.getEndColumn());
+		group1.setAllPositions(left.getFilename(), left.getBeginLine(), left.getBeginColumn(), left.getEndLine(), left.getEndColumn());
 		
 		Group group2 = new Group(right1.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, vt);
 		group1.getOutputParameters().setDimensions(m2_dim1, m2_dim2, brlen, bclen, right1.getNnz());
-		group1.setAllPositions(right1.getBeginLine(), right1.getBeginColumn(), right1.getEndLine(), right1.getEndColumn());
+		group1.setAllPositions(right1.getFilename(), right1.getBeginLine(), right1.getBeginColumn(), right1.getEndLine(), right1.getEndColumn());
 		
 		Group group3 = new Group(right2.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, vt);
 		group1.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, right2.getNnz());
-		group1.setAllPositions(right2.getBeginLine(), right2.getBeginColumn(), right2.getEndLine(), right2.getEndColumn());
+		group1.setAllPositions(right2.getFilename(), right2.getBeginLine(), right2.getBeginColumn(), right2.getEndLine(), right2.getEndColumn());
 		
 		AppendR appR1 = new AppendR(group1, group2, dt, vt, cbind, ExecType.MR);
 		appR1.getOutputParameters().setDimensions(m1_dim1, m41_dim2, brlen, bclen, m41_nnz);
-		appR1.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+		appR1.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 		
 		AppendR appR2 = new AppendR(appR1, group3, dt, vt, cbind, ExecType.MR);
 		appR1.getOutputParameters().setDimensions(m1_dim1, m42_dim2, brlen, bclen, m42_nnz);
-		appR1.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
+		appR1.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(), current.getEndColumn());
 	
 		return appR2;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/hops/Hop.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/Hop.java b/src/main/java/org/apache/sysml/hops/Hop.java
index 4529d04..7aa58b7 100644
--- a/src/main/java/org/apache/sysml/hops/Hop.java
+++ b/src/main/java/org/apache/sysml/hops/Hop.java
@@ -444,7 +444,7 @@ public abstract class Hop
 		}
 		
 		offset.getOutputParameters().setDimensions(0, 0, 0, 0, -1);
-		offset.setAllPositions(hop.getBeginLine(), hop.getBeginColumn(), hop.getEndLine(), hop.getEndColumn());
+		offset.setAllPositions(hop.getFilename(), hop.getBeginLine(), hop.getBeginColumn(), hop.getEndLine(), hop.getEndColumn());
 		
 		return offset;
 	}
@@ -1802,13 +1802,16 @@ public abstract class Hop
 	///////////////////////////////////////////////////////////////////////////
 	public int _beginLine, _beginColumn;
 	public int _endLine, _endColumn;
+	public String _filename;
 	
 	public void setBeginLine(int passed)    { _beginLine = passed;   }
 	public void setBeginColumn(int passed) 	{ _beginColumn = passed; }
 	public void setEndLine(int passed) 		{ _endLine = passed;   }
 	public void setEndColumn(int passed)	{ _endColumn = passed; }
+	public void setFilename(String passed) { _filename = passed; }
 	
-	public void setAllPositions(int blp, int bcp, int elp, int ecp){
+	public void setAllPositions(String filename, int blp, int bcp, int elp, int ecp){
+		_filename = filename;
 		_beginLine	 = blp; 
 		_beginColumn = bcp; 
 		_endLine 	 = elp;
@@ -1819,9 +1822,13 @@ public abstract class Hop
 	public int getBeginColumn() { return _beginColumn; }
 	public int getEndLine() 	{ return _endLine;   }
 	public int getEndColumn()	{ return _endColumn; }
+	public String getFilename()	{ return _filename; }
 	
 	public String printErrorLocation(){
-		return "ERROR: line " + _beginLine + ", column " + _beginColumn + " -- ";
+		if(_filename != null)
+			return "ERROR: " + _filename + " line " + _beginLine + ", column " + _beginColumn + " -- ";
+		else
+			return "ERROR: line " + _beginLine + ", column " + _beginColumn + " -- ";
 	}
 
 	/**
@@ -1831,7 +1838,7 @@ public abstract class Hop
 	 */
 	protected void setLineNumbers(Lop lop)
 	{
-		lop.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+		lop.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 	}
 	
 } // end class

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/hops/TernaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/TernaryOp.java b/src/main/java/org/apache/sysml/hops/TernaryOp.java
index e1bef3e..22895e5 100644
--- a/src/main/java/org/apache/sysml/hops/TernaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/TernaryOp.java
@@ -422,7 +422,7 @@ public class TernaryOp extends Hop
 			Ternary tertiary = new Ternary(inputLops, tertiaryOp, getDataType(), getValueType(), ignoreZeros, et);
 			
 			tertiary.getOutputParameters().setDimensions(_dim1, _dim2, getRowsInBlock(), getColsInBlock(), -1);
-			tertiary.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			tertiary.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 			
 			//force blocked output in CP (see below), otherwise binarycell
 			if ( et == ExecType.SPARK ) {
@@ -447,7 +447,7 @@ public class TernaryOp extends Hop
 			group1.getOutputParameters().setDimensions(getDim1(),
 					getDim2(), getRowsInBlock(), getColsInBlock(), getNnz());
 			
-			group1.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			group1.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 
 			Ternary tertiary = null;
 			// create "group" lops for MATRIX inputs
@@ -462,7 +462,7 @@ public class TernaryOp extends Hop
 					group2.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group2.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					group2.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 					
 					group3 = new Group(
 							inputLops[2],
@@ -471,7 +471,7 @@ public class TernaryOp extends Hop
 					group3.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group3.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					group3.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 					
 					if ( inputLops.length == 3 )
 						tertiary = new Ternary(
@@ -495,7 +495,7 @@ public class TernaryOp extends Hop
 					group2.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group2.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					group2.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 					
 					if ( inputLops.length == 3)
 						tertiary = new Ternary(
@@ -579,7 +579,7 @@ public class TernaryOp extends Hop
 					group3.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group3.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					group3.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 					
 					if ( inputLops.length == 3)
 						tertiary = new Ternary(
@@ -618,14 +618,14 @@ public class TernaryOp extends Hop
 						tertiary, Group.OperationTypes.Sort, getDataType(),
 						getValueType());
 				group4.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
-				group4.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+				group4.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 	
 				Aggregate agg1 = new Aggregate(
 						group4, HopsAgg2Lops.get(AggOp.SUM), getDataType(),
 						getValueType(), ExecType.MR);
 				agg1.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
 	
-				agg1.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+				agg1.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
 
 				// kahamSum is used for aggregation but inputs do not have
 				// correction values

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/hops/UnaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/UnaryOp.java b/src/main/java/org/apache/sysml/hops/UnaryOp.java
index 61ebedf..35902b7 100644
--- a/src/main/java/org/apache/sysml/hops/UnaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/UnaryOp.java
@@ -290,7 +290,7 @@ public class UnaryOp extends Hop implements MultiThreadedHop
 
 			Data lit = Data.createLiteralLop(ValueType.DOUBLE, Double.toString(0.25));
 			
-			lit.setAllPositions(this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			lit.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
             			
 			PickByCount pick = new PickByCount(
 					sort, lit, DataType.MATRIX, getValueType(),

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
index ec8e20d..14bcc1e 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/HopRewriteUtils.java
@@ -645,7 +645,7 @@ public class HopRewriteUtils
 	}
 	
 	public static void copyLineNumbers( Hop src, Hop dest ) {
-		dest.setAllPositions(src.getBeginLine(), src.getBeginColumn(), src.getEndLine(), src.getEndColumn());
+		dest.setAllPositions(src.getFilename(), src.getBeginLine(), src.getBeginColumn(), src.getEndLine(), src.getEndColumn());
 	}
 	
 	public static void updateHopCharacteristics( Hop hop, long brlen, long bclen, Hop src )

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/CombineBinary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/CombineBinary.java b/src/main/java/org/apache/sysml/lops/CombineBinary.java
index 1c98b11..f3c757e 100644
--- a/src/main/java/org/apache/sysml/lops/CombineBinary.java
+++ b/src/main/java/org/apache/sysml/lops/CombineBinary.java
@@ -124,7 +124,7 @@ public class CombineBinary extends Lop
 		}
 		
 		CombineBinary comn = new CombineBinary(op, input1, input2, dt, vt);
-		comn.setAllPositions(input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
+		comn.setAllPositions(input1.getFilename(), input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
 		return comn;
 	}
  

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/CombineTernary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/CombineTernary.java b/src/main/java/org/apache/sysml/lops/CombineTernary.java
index e580045..9200f5c 100644
--- a/src/main/java/org/apache/sysml/lops/CombineTernary.java
+++ b/src/main/java/org/apache/sysml/lops/CombineTernary.java
@@ -115,7 +115,7 @@ public class CombineTernary extends Lop
 		}
 
 		CombineTernary comn = new CombineTernary(op, input1, input2, input3, dt, vt);
-		comn.setAllPositions(input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
+		comn.setAllPositions(input1.getFilename(), input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
 		return comn;
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/CombineUnary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/CombineUnary.java b/src/main/java/org/apache/sysml/lops/CombineUnary.java
index 98f0a99..4882e3c 100644
--- a/src/main/java/org/apache/sysml/lops/CombineUnary.java
+++ b/src/main/java/org/apache/sysml/lops/CombineUnary.java
@@ -90,7 +90,7 @@ public class CombineUnary extends Lop
 		}
 		
 		CombineUnary comn = new CombineUnary(input1, dt, vt);
-		comn.setAllPositions(input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndLine());
+		comn.setAllPositions(input1.getFilename(), input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndLine());
 		return comn;
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/Lop.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/Lop.java b/src/main/java/org/apache/sysml/lops/Lop.java
index c92fb57..1e88561 100644
--- a/src/main/java/org/apache/sysml/lops/Lop.java
+++ b/src/main/java/org/apache/sysml/lops/Lop.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
 import org.apache.sysml.lops.LopProperties.ExecLocation;
 import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.lops.compile.Dag;
@@ -620,13 +619,16 @@ public abstract class Lop
 	///////////////////////////////////////////////////////////////////////////
 	public int _beginLine, _beginColumn;
 	public int _endLine, _endColumn;
+	public String _filename;
 	
 	public void setBeginLine(int passed)    { _beginLine = passed;   }
 	public void setBeginColumn(int passed) 	{ _beginColumn = passed; }
 	public void setEndLine(int passed) 		{ _endLine = passed;   }
 	public void setEndColumn(int passed)	{ _endColumn = passed; }
+	public void setFilename(String passed) { _filename = passed; }
 	
-	public void setAllPositions(int blp, int bcp, int elp, int ecp){
+	public void setAllPositions(String filename, int blp, int bcp, int elp, int ecp){
+		_filename = filename;
 		_beginLine	 = blp; 
 		_beginColumn = bcp; 
 		_endLine 	 = elp;
@@ -637,6 +639,7 @@ public abstract class Lop
 	public int getBeginColumn() { return _beginColumn; }
 	public int getEndLine() 	{ return _endLine;   }
 	public int getEndColumn()	{ return _endColumn; }
+	public String getFilename()	{ return _filename; }
 	
 	public String printErrorLocation(){
 		return "ERROR: line " + _beginLine + ", column " + _beginColumn + " -- ";

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/SortKeys.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/SortKeys.java b/src/main/java/org/apache/sysml/lops/SortKeys.java
index e2c6b6a..6344eda 100644
--- a/src/main/java/org/apache/sysml/lops/SortKeys.java
+++ b/src/main/java/org/apache/sysml/lops/SortKeys.java
@@ -152,7 +152,7 @@ public class SortKeys extends Lop
 		}
 		
 		SortKeys retVal = new SortKeys(input1, op, dt, vt, et);
-		retVal.setAllPositions(input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
+		retVal.setAllPositions(input1.getFilename(), input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
 		return retVal;
 	}
 
@@ -172,7 +172,7 @@ public class SortKeys extends Lop
 		}
 		
 		SortKeys retVal = new SortKeys(input1, input2, op, dt, vt, et);
-		retVal.setAllPositions(input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
+		retVal.setAllPositions(input1.getFilename(), input1.getBeginLine(), input1.getBeginColumn(), input1.getEndLine(), input1.getEndColumn());
 		return retVal;
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/lops/compile/Dag.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/compile/Dag.java b/src/main/java/org/apache/sysml/lops/compile/Dag.java
index ddee17d..1da1ba6 100644
--- a/src/main/java/org/apache/sysml/lops/compile/Dag.java
+++ b/src/main/java/org/apache/sysml/lops/compile/Dag.java
@@ -366,7 +366,7 @@ public class Dag<N extends Lop>
 		for (String varName : sb.liveIn().getVariableNames()) {
 			if (!sb.liveOut().containsVariable(varName)) {
 				inst = VariableCPInstruction.prepareRemoveInstruction(varName);
-				inst.setLocation(sb.getEndLine(), sb.getEndLine(), -1, -1);
+				inst.setLocation(sb.getFilename(), sb.getEndLine(), sb.getEndLine(), -1, -1);
 				
 				deleteInst.add(inst);
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/parser/DMLProgram.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLProgram.java b/src/main/java/org/apache/sysml/parser/DMLProgram.java
index 2f0d682..15d46c7 100644
--- a/src/main/java/org/apache/sysml/parser/DMLProgram.java
+++ b/src/main/java/org/apache/sysml/parser/DMLProgram.java
@@ -284,7 +284,7 @@ public class DMLProgram
 			retPB.setStatementBlock(sb);
 			
 			// add location information
-			retPB.setAllPositions(sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
 		}
 		
 		// process If Statement - add runtime program blocks to program
@@ -351,7 +351,7 @@ public class DMLProgram
 			retPB.setStatementBlock(sb);
 			
 			// add location information
-			retPB.setAllPositions(sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
 		}
 		
 		// process For Statement - add runtime program blocks to program
@@ -429,7 +429,7 @@ public class DMLProgram
 			retPB.setStatementBlock(sb);
 			
 			// add location information
-			retPB.setAllPositions(sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
 		}
 		
 		// process function statement block - add runtime program blocks to program
@@ -511,7 +511,7 @@ public class DMLProgram
 			retPB = rtpb;
 			
 			// add location information
-			retPB.setAllPositions(sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
 		}
 		else {
 	
@@ -548,7 +548,7 @@ public class DMLProgram
 			retPB.setStatementBlock(sb);
 			
 			// add location information
-			retPB.setAllPositions(sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
 		}
 		
 		return retPB;

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/parser/DMLTranslator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLTranslator.java b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
index 42ab12e..61f7217 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -877,7 +877,7 @@ public class DMLTranslator
 					long actualDim1 = (var instanceof IndexedIdentifier) ? ((IndexedIdentifier)var).getOrigDim1() : var.getDim1();
 					long actualDim2 = (var instanceof IndexedIdentifier) ? ((IndexedIdentifier)var).getOrigDim2() : var.getDim2();
 					DataOp read = new DataOp(var.getName(), var.getDataType(), var.getValueType(), DataOpTypes.TRANSIENTREAD, null, actualDim1, actualDim2, var.getNnz(), var.getRowsInBlock(), var.getColumnsInBlock());
-					read.setAllPositions(var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
+					read.setAllPositions(var.getFilename(), var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
 					ids.put(varName, read);
 				}
 			}
@@ -946,7 +946,7 @@ public class DMLTranslator
 						Hop ae = processExpression(source, target, ids);
 						Hop printHop = new UnaryOp(target.getName(), target.getDataType(), target.getValueType(), op,
 								ae);
-						printHop.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(),
+						printHop.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(),
 								current.getEndColumn());
 						output.add(printHop);
 					} else if (ptype == PRINTTYPE.STOP) {
@@ -955,7 +955,7 @@ public class DMLTranslator
 						Hop ae = processExpression(source, target, ids);
 						Hop stopHop = new UnaryOp(target.getName(), target.getDataType(), target.getValueType(), op,
 								ae);
-						stopHop.setAllPositions(current.getBeginLine(), current.getBeginColumn(), current.getEndLine(),
+						stopHop.setAllPositions(current.getFilename(), current.getBeginLine(), current.getBeginColumn(), current.getEndLine(),
 								current.getEndColumn());
 						output.add(stopHop);
 					} else if (ptype == PRINTTYPE.PRINTF) {
@@ -1000,7 +1000,7 @@ public class DMLTranslator
 						if ((statementId != null) && (statementId.intValue() == i)) {
 							DataOp transientwrite = new DataOp(target.getName(), target.getDataType(), target.getValueType(), ae, DataOpTypes.TRANSIENTWRITE, null);
 							transientwrite.setOutputParams(ae.getDim1(), ae.getDim2(), ae.getNnz(), ae.getUpdateType(), ae.getRowsInBlock(), ae.getColsInBlock());
-							transientwrite.setAllPositions(target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndLine());
+							transientwrite.setAllPositions(target.getFilename(), target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndLine());
 							updatedLiveOut.addVariable(target.getName(), target);
 							output.add(transientwrite);
 						}
@@ -1031,7 +1031,7 @@ public class DMLTranslator
 						if ((statementId != null) && (statementId.intValue() == i)) {
 							DataOp transientwrite = new DataOp(target.getName(), target.getDataType(), target.getValueType(), ae, DataOpTypes.TRANSIENTWRITE, null);
 							transientwrite.setOutputParams(origDim1, origDim2, ae.getNnz(), ae.getUpdateType(), ae.getRowsInBlock(), ae.getColsInBlock());
-							transientwrite.setAllPositions(target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
+							transientwrite.setAllPositions(target.getFilename(), target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
 							updatedLiveOut.addVariable(target.getName(), target);
 							output.add(transientwrite);
 						}
@@ -1241,7 +1241,7 @@ public class DMLTranslator
 				
 				read = new DataOp(var.getName(), var.getDataType(), var.getValueType(), DataOpTypes.TRANSIENTREAD,
 						null, actualDim1, actualDim2, var.getNnz(), var.getRowsInBlock(), var.getColumnsInBlock());
-				read.setAllPositions(var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
+				read.setAllPositions(var.getFilename(), var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
 			}
 			_ids.put(varName, read);
 		}
@@ -1339,7 +1339,7 @@ public class DMLTranslator
 						long actualDim2 = (var instanceof IndexedIdentifier) ? ((IndexedIdentifier)var).getOrigDim2() : var.getDim2();
 						read = new DataOp(var.getName(), var.getDataType(), var.getValueType(), DataOpTypes.TRANSIENTREAD,
 								null, actualDim1, actualDim2,  var.getNnz(), var.getRowsInBlock(),  var.getColumnsInBlock());
-						read.setAllPositions(var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
+						read.setAllPositions(var.getFilename(), var.getBeginLine(), var.getBeginColumn(), var.getEndLine(), var.getEndColumn());
 					}
 					_ids.put(varName, read);
 				}
@@ -1417,28 +1417,28 @@ public class DMLTranslator
 			else if (source instanceof IntIdentifier) {
 				IntIdentifier sourceInt = (IntIdentifier) source;
 				LiteralOp litop = new LiteralOp(sourceInt.getValue());
-				litop.setAllPositions(sourceInt.getBeginLine(), sourceInt.getBeginColumn(), sourceInt.getEndLine(), sourceInt.getEndColumn());
+				litop.setAllPositions(sourceInt.getFilename(), sourceInt.getBeginLine(), sourceInt.getBeginColumn(), sourceInt.getEndLine(), sourceInt.getEndColumn());
 				setIdentifierParams(litop, sourceInt);
 				return litop;
 			} 
 			else if (source instanceof DoubleIdentifier) {
 				DoubleIdentifier sourceDouble = (DoubleIdentifier) source;
 				LiteralOp litop = new LiteralOp(sourceDouble.getValue());
-				litop.setAllPositions(sourceDouble.getBeginLine(), sourceDouble.getBeginColumn(), sourceDouble.getEndLine(), sourceDouble.getEndColumn());
+				litop.setAllPositions(source.getFilename(), sourceDouble.getBeginLine(), sourceDouble.getBeginColumn(), sourceDouble.getEndLine(), sourceDouble.getEndColumn());
 				setIdentifierParams(litop, sourceDouble);
 				return litop;
 			}
 			else if (source instanceof BooleanIdentifier) {
 				BooleanIdentifier sourceBoolean = (BooleanIdentifier) source;
 				LiteralOp litop = new LiteralOp(sourceBoolean.getValue());
-				litop.setAllPositions(sourceBoolean.getBeginLine(), sourceBoolean.getBeginColumn(), sourceBoolean.getEndLine(), sourceBoolean.getEndColumn());
+				litop.setAllPositions(sourceBoolean.getFilename(), sourceBoolean.getBeginLine(), sourceBoolean.getBeginColumn(), sourceBoolean.getEndLine(), sourceBoolean.getEndColumn());
 				setIdentifierParams(litop, sourceBoolean);
 				return litop;
 			} 
 			else if (source instanceof StringIdentifier) {
 				StringIdentifier sourceString = (StringIdentifier) source;
 				LiteralOp litop = new LiteralOp(sourceString.getValue());
-				litop.setAllPositions(sourceString.getBeginLine(), sourceString.getBeginColumn(), sourceString.getEndLine(), sourceString.getEndColumn());
+				litop.setAllPositions(sourceString.getFilename(), sourceString.getBeginLine(), sourceString.getBeginColumn(), sourceString.getEndLine(), sourceString.getEndColumn());
 				setIdentifierParams(litop, sourceString);
 				return litop;
 			} 
@@ -1505,7 +1505,7 @@ public class DMLTranslator
 				rowUpperHops = new LiteralOp(target.getOrigDim1());
 			else {
 				rowUpperHops = new UnaryOp(target.getName(), DataType.SCALAR, ValueType.INT, Hop.OpOp1.NROW, hops.get(target.getName()));
-				rowUpperHops.setAllPositions(target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
+				rowUpperHops.setAllPositions(target.getFilename(), target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
 			}
 		}
 		if (target.getColLowerBound() != null)
@@ -1543,7 +1543,7 @@ public class DMLTranslator
 		
 		setIdentifierParams(leftIndexOp, target);
 	
-		leftIndexOp.setAllPositions(target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
+		leftIndexOp.setAllPositions(target.getFilename(), target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
 		leftIndexOp.setDim1(target.getOrigDim1());
 		leftIndexOp.setDim2(target.getOrigDim2());
 	
@@ -1570,7 +1570,7 @@ public class DMLTranslator
 				rowUpperHops = new LiteralOp(source.getOrigDim1());
 			else {
 				rowUpperHops = new UnaryOp(source.getName(), DataType.SCALAR, ValueType.INT, Hop.OpOp1.NROW, hops.get(source.getName()));
-				rowUpperHops.setAllPositions(source.getBeginLine(),source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+				rowUpperHops.setAllPositions(source.getFilename(), source.getBeginLine(),source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 			}
 		}
 		if (source.getColLowerBound() != null)
@@ -1599,7 +1599,7 @@ public class DMLTranslator
 				hops.get(source.getName()), rowLowerHops, rowUpperHops, colLowerHops, colUpperHops,
 				source.getRowLowerEqualsUpper(), source.getColLowerEqualsUpper());
 	
-		indexOp.setAllPositions(indexOp.getBeginLine(), indexOp.getBeginColumn(), indexOp.getEndLine(), indexOp.getEndColumn());
+		indexOp.setAllPositions(target.getFilename(), target.getBeginLine(), target.getBeginColumn(), target.getEndLine(), target.getEndColumn());
 		setIdentifierParams(indexOp, target);
 		
 		return indexOp;
@@ -1657,7 +1657,7 @@ public class DMLTranslator
 			throw new ParseException("Unsupported parsing of binary expression: "+source.getOpCode());
 		}
 		setIdentifierParams(currBop, source.getOutput());
-		currBop.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBop.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		return currBop;
 		
 	}
@@ -1701,7 +1701,7 @@ public class DMLTranslator
 			op = OpOp2.NOTEQUAL;
 		}
 		currBop = new BinaryOp(target.getName(), target.getDataType(), target.getValueType(), op, left, right);
-		currBop.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBop.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		return currBop;
 	}
 
@@ -1735,7 +1735,7 @@ public class DMLTranslator
 		
 		if (source.getRight() == null) {
 			Hop currUop = new UnaryOp(target.getName(), target.getDataType(), target.getValueType(), Hop.OpOp1.NOT, left);
-			currUop.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+			currUop.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 			return currUop;
 		} 
 		else {
@@ -1751,7 +1751,7 @@ public class DMLTranslator
 				throw new RuntimeException(source.printErrorLocation() + "Unknown boolean operation " + source.getOpCode());
 			}
 			currBop = new BinaryOp(target.getName(), target.getDataType(), target.getValueType(), op, left, right);
-			currBop.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+			currBop.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 			// setIdentifierParams(currBop,source.getOutput());
 			return currBop;
 		}
@@ -1830,9 +1830,9 @@ public class DMLTranslator
 		// set properties for created hops based on outputs of source expression
 		for ( int i=0; i < source.getOutputs().length; i++ ) {
 			setIdentifierParams( outputs.get(i), source.getOutputs()[i]);
-			outputs.get(i).setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+			outputs.get(i).setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		}
-		currBuiltinOp.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBuiltinOp.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 
 		return currBuiltinOp;
 	}
@@ -1952,7 +1952,7 @@ public class DMLTranslator
 		
 		setIdentifierParams(currBuiltinOp, source.getOutput());
 		
-		currBuiltinOp.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBuiltinOp.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		
 		return currBuiltinOp;
 	}
@@ -2040,7 +2040,7 @@ public class DMLTranslator
 		setIdentifierParams(currBuiltinOp, source.getOutput());
 		if( source.getOpCode()==DataExpression.DataOp.READ )
 			((DataOp)currBuiltinOp).setInputBlockSizes(target.getRowsInBlock(), target.getColumnsInBlock());
-		currBuiltinOp.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBuiltinOp.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		
 		return currBuiltinOp;
 	}
@@ -2102,9 +2102,9 @@ public class DMLTranslator
 		// set properties for created hops based on outputs of source expression
 		for ( int i=0; i < source.getOutputs().length; i++ ) {
 			setIdentifierParams( outputs.get(i), source.getOutputs()[i]);
-			outputs.get(i).setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+			outputs.get(i).setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		}
-		currBuiltinOp.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBuiltinOp.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 
 		return currBuiltinOp;
 	}
@@ -2776,7 +2776,7 @@ public class DMLTranslator
 			// Since the dimension of output doesnot match that of input variable for these operations
 			setIdentifierParams(currBuiltinOp, source.getOutput());
 		}
-		currBuiltinOp.setAllPositions(source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
+		currBuiltinOp.setAllPositions(source.getFilename(), source.getBeginLine(), source.getBeginColumn(), source.getEndLine(), source.getEndColumn());
 		return currBuiltinOp;
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/ExternalFunctionProgramBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/ExternalFunctionProgramBlock.java b/src/main/java/org/apache/sysml/runtime/controlprogram/ExternalFunctionProgramBlock.java
index 3c72ca9..7b364d7 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/ExternalFunctionProgramBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/ExternalFunctionProgramBlock.java
@@ -304,7 +304,7 @@ public class ExternalFunctionProgramBlock extends FunctionProgramBlock
 		else if (getOutputParams().size() > 0)
 			einst.setLocation(getOutputParams().get(0));
 		else
-			einst.setLocation(this._beginLine, this._endLine, this._beginColumn, this._endColumn);
+			einst.setLocation(this.getFilename(), this._beginLine, this._endLine, this._beginColumn, this._endColumn);
 		
 		_inst.add(einst);
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java b/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
index 209c566..43e93f4 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
@@ -409,21 +409,25 @@ public class ProgramBlock
 	// store position information for program blocks
 	///////////////////////////////////////////////////////////////////////////
 
+	public String _filename;
 	public int _beginLine, _beginColumn;
 	public int _endLine, _endColumn;
 
+	public void setFilename(String passed)    { _filename = passed;   }
 	public void setBeginLine(int passed)    { _beginLine = passed;   }
 	public void setBeginColumn(int passed) 	{ _beginColumn = passed; }
 	public void setEndLine(int passed) 		{ _endLine = passed;   }
 	public void setEndColumn(int passed)	{ _endColumn = passed; }
 
-	public void setAllPositions(int blp, int bcp, int elp, int ecp){
+	public void setAllPositions(String filename, int blp, int bcp, int elp, int ecp){
+		_filename = filename;
 		_beginLine	 = blp;
 		_beginColumn = bcp;
 		_endLine 	 = elp;
 		_endColumn 	 = ecp;
 	}
 
+	public String getFilename()	{ return _filename;   }
 	public int getBeginLine()	{ return _beginLine;   }
 	public int getBeginColumn() { return _beginColumn; }
 	public int getEndLine() 	{ return _endLine;   }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/caching/CacheableData.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/caching/CacheableData.java b/src/main/java/org/apache/sysml/runtime/controlprogram/caching/CacheableData.java
index a9f604f..dbc8e07 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/caching/CacheableData.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/caching/CacheableData.java
@@ -39,6 +39,7 @@ import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.caching.LazyWriteBuffer.RPolicy;
 import org.apache.sysml.runtime.controlprogram.parfor.util.IDSequence;
+import org.apache.sysml.runtime.instructions.cp.CPInstruction;
 import org.apache.sysml.runtime.instructions.cp.Data;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContext;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUObject;
@@ -51,10 +52,12 @@ import org.apache.sysml.runtime.matrix.MatrixFormatMetaData;
 import org.apache.sysml.runtime.matrix.MetaData;
 import org.apache.sysml.runtime.matrix.data.FileFormatProperties;
 import org.apache.sysml.runtime.matrix.data.InputInfo;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.NumItemsByEachReducerMetaData;
 import org.apache.sysml.runtime.matrix.data.OutputInfo;
 import org.apache.sysml.runtime.util.LocalFileUtils;
 import org.apache.sysml.runtime.util.MapReduceTool;
+import org.apache.sysml.utils.GPUStatistics;
 
 
 /**
@@ -522,6 +525,10 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 		return _data;
 	}
 	
+	public T acquireModify(T newData) throws DMLRuntimeException {
+		return acquireModify(newData, null);
+	}
+	
 	/**
 	 * Acquires the exclusive "write" lock for a thread that wants to throw away the
 	 * old cache block data and link up with new cache block data. Abandons the old data
@@ -531,10 +538,11 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 	 * Out-Status: MODIFY.
 	 * 
 	 * @param newData new data
+	 * @param opcode extended instruction opcode
 	 * @return cacheable data
 	 * @throws DMLRuntimeException if error occurs
 	 */
-	public synchronized T acquireModify(T newData)
+	public synchronized T acquireModify(T newData, String opcode)
 		throws DMLRuntimeException
 	{
 		if( LOG.isTraceEnabled() )
@@ -562,11 +570,24 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 		if( DMLScript.STATISTICS ){
 			long t1 = System.nanoTime();
 			CacheStatistics.incrementAcquireMTime(t1-t0);
+			if(DMLScript.FINEGRAINED_STATISTICS && opcode != null) {
+				if(_data instanceof MatrixBlock) {
+					MatrixBlock currObject = (MatrixBlock)_data;
+					if(currObject.isInSparseFormat())
+						GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_ACQ_MODIFY_SPARSE_MB, t1-t0);
+					else
+						GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_ACQ_MODIFY_DENSE_MB, t1-t0);
+				}
+			}
 		}
 		
 		return _data;
 	}
 	
+	public void release() throws CacheException {
+		release(null);
+	}
+	
 	/**
 	 * Releases the shared ("read-only") or exclusive ("write") lock.  Updates
 	 * size information, last-access time, metadata, etc.
@@ -579,7 +600,7 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 	 * 
 	 * @throws CacheException if CacheException occurs
 	 */
-	public synchronized void release() 
+	public synchronized void release(String opcode) 
 		throws CacheException
 	{
 		if( LOG.isTraceEnabled() )
@@ -613,7 +634,7 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 				//evict blob
 				String filePath = getCacheFilePathAndName();
 				try {
-					LazyWriteBuffer.writeBlock(filePath, _data);
+					LazyWriteBuffer.writeBlock(filePath, _data, opcode);
 				}
 				catch (Exception e)
 				{
@@ -721,6 +742,12 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 		exportData(fName, outputFormat, -1, formatProperties);
 	}
 	
+	public synchronized void exportData (String fName, String outputFormat, int replication, FileFormatProperties formatProperties)
+			throws CacheException
+	{
+		exportData(fName, outputFormat, replication, formatProperties, null);
+	}
+	
 	/**
 	 * Synchronized because there might be parallel threads (parfor local) that
 	 * access the same object (in case it was created before the loop).
@@ -736,9 +763,10 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 	 * @param outputFormat format
 	 * @param replication ?
 	 * @param formatProperties file format properties
+	 * @param opcode instruction opcode if available
 	 * @throws CacheException if CacheException occurs
 	 */
-	public synchronized void exportData (String fName, String outputFormat, int replication, FileFormatProperties formatProperties)
+	public synchronized void exportData (String fName, String outputFormat, int replication, FileFormatProperties formatProperties, String opcode)
 		throws CacheException
 	{
 		if( LOG.isTraceEnabled() )
@@ -816,7 +844,7 @@ public abstract class CacheableData<T extends CacheBlock> extends Data
 			}
 			finally
 			{
-				release();
+				release(opcode);
 			}
 		}
 		else if( pWrite ) // pwrite with same output format

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/caching/LazyWriteBuffer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/caching/LazyWriteBuffer.java b/src/main/java/org/apache/sysml/runtime/controlprogram/caching/LazyWriteBuffer.java
index f0eb926..c045961 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/caching/LazyWriteBuffer.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/caching/LazyWriteBuffer.java
@@ -28,7 +28,9 @@ import java.util.concurrent.Executors;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.instructions.cp.CPInstruction;
 import org.apache.sysml.runtime.util.LocalFileUtils;
+import org.apache.sysml.utils.GPUStatistics;
 
 public class LazyWriteBuffer 
 {
@@ -56,7 +58,7 @@ public class LazyWriteBuffer
 		_limit = (long)(CacheableData.CACHING_BUFFER_SIZE * maxMem);
 	}
 
-	public static void writeBlock( String fname, CacheBlock cb ) 
+	public static void writeBlock( String fname, CacheBlock cb, String opcode ) 
 		throws IOException
 	{	
 		//obtain basic meta data of cache block
@@ -72,6 +74,7 @@ public class LazyWriteBuffer
 			ByteBuffer bbuff = new ByteBuffer( lSize );
 			int numEvicted = 0;
 			
+			long t1 = DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			//modify buffer pool
 			synchronized( _mQueue )
 			{
@@ -99,21 +102,33 @@ public class LazyWriteBuffer
 				_mQueue.addLast(fname, bbuff);
 				_size += lSize;	
 			}
+			long t2 = DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			
 			//serialize matrix (outside synchronized critical path)
-			bbuff.serializeBlock(cb); 
+			bbuff.serializeBlock(cb);
 			
 			if( DMLScript.STATISTICS ) {
+				if(DMLScript.FINEGRAINED_STATISTICS && opcode != null) {
+					long t3 = DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+					GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_RELEASE_EVICTION, t2-t1, numEvicted);
+					GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_RELEASE_BUFF_WRITE, t3-t2, 1);
+				}
 				CacheStatistics.incrementFSBuffWrites();
 				CacheStatistics.incrementFSWrites(numEvicted);
 			}
 		}	
 		else
 		{
+			long t1 = DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 			//write directly to local FS (bypass buffer if too large)
 			LocalFileUtils.writeCacheBlockToLocal(fname, cb);
-			if( DMLScript.STATISTICS )
+			if( DMLScript.STATISTICS ) {
+				if(DMLScript.FINEGRAINED_STATISTICS && opcode != null) {
+					long t2 = DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+					GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_RELEASE_BUFF_WRITE, t2-t1, 1);
+				}
 				CacheStatistics.incrementFSWrites();
+			}
 		}	
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/context/ExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/context/ExecutionContext.java b/src/main/java/org/apache/sysml/runtime/controlprogram/context/ExecutionContext.java
index 2b97067..8d27e3b 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/context/ExecutionContext.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/context/ExecutionContext.java
@@ -39,6 +39,7 @@ import org.apache.sysml.runtime.controlprogram.caching.FrameObject;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject.UpdateType;
 import org.apache.sysml.runtime.instructions.Instruction;
+import org.apache.sysml.runtime.instructions.cp.CPInstruction;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.cp.Data;
 import org.apache.sysml.runtime.instructions.cp.FunctionCallCPInstruction;
@@ -54,6 +55,7 @@ import org.apache.sysml.runtime.matrix.data.FrameBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.Pair;
 import org.apache.sysml.runtime.util.MapReduceTool;
+import org.apache.sysml.utils.GPUStatistics;
 
 
 public class ExecutionContext {
@@ -231,13 +233,34 @@ public class ExecutionContext {
 	}
 	
 	/**
+	 * Pins a matrix variable into memory, update the finegrained statistics and returns the internal matrix block.
+	 * 
+	 * @param varName variable name
+	 * @param opcode  extended opcode
+	 * @return matrix block
+	 * @throws DMLRuntimeException if DMLRuntimeException occurs
+	 */
+	public MatrixBlock getMatrixInput(String varName, String opcode) throws DMLRuntimeException {
+		long t1 = opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
+		MatrixBlock mb = getMatrixInput(varName);
+		if(opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS) {
+			long t2 = System.nanoTime();
+			if(mb.isInSparseFormat())
+				GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_GET_SPARSE_MB, t2-t1);
+			else
+				GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_GET_DENSE_MB, t2-t1);
+		}
+		return mb;
+	}
+	
+	/**
 	 * Pins a matrix variable into memory and returns the internal matrix block.
 	 * 
 	 * @param varName variable name
 	 * @return matrix block
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
-	public MatrixBlock getMatrixInput(String varName) 
+	private MatrixBlock getMatrixInput(String varName) 
 		throws DMLRuntimeException 
 	{	
 		MatrixObject mo = getMatrixObject(varName);
@@ -340,16 +363,19 @@ public class ExecutionContext {
 	}
 	
 	/**
-	 * Unpins a currently pinned matrix variable. 
+	 * Unpins a currently pinned matrix variable and update fine-grained statistics. 
 	 * 
 	 * @param varName variable name
 	 * @throws DMLRuntimeException if DMLRuntimeException occurs
 	 */
-	public void releaseMatrixInput(String varName) 
-		throws DMLRuntimeException 
-	{
+	public void releaseMatrixInput(String varName, String opcode) throws DMLRuntimeException {
+		long t1 = opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS ? System.nanoTime() : 0;
 		MatrixObject mo = getMatrixObject(varName);
-		mo.release();
+		mo.release(opcode);
+		if(opcode != null && DMLScript.STATISTICS && DMLScript.FINEGRAINED_STATISTICS) {
+			long t2 = System.nanoTime();
+			GPUStatistics.maintainCPMiscTimes(opcode, CPInstruction.MISC_TIMER_RELEASE_INPUT_MB, t2-t1);
+		}
 	}
 	
 	public void releaseMatrixInputForGPUInstruction(String varName)
@@ -417,17 +443,18 @@ public class ExecutionContext {
 		}
 		mo.getGPUObject(getGPUContext(0)).releaseOutput();
 	}
+	
 
-	public void setMatrixOutput(String varName, MatrixBlock outputData) 
+	public void setMatrixOutput(String varName, MatrixBlock outputData, String opcode) 
 			throws DMLRuntimeException 
 	{
 		MatrixObject mo = getMatrixObject(varName);
-		mo.acquireModify(outputData);
-	    mo.release();
+		mo.acquireModify(outputData, opcode);
+	    mo.release(opcode);
 	    setVariable(varName, mo);
 	}
 
-	public void setMatrixOutput(String varName, MatrixBlock outputData, UpdateType flag) 
+	public void setMatrixOutput(String varName, MatrixBlock outputData, UpdateType flag, String opcode) 
 		throws DMLRuntimeException 
 	{
 		if( flag.isInPlace() ) {
@@ -437,7 +464,7 @@ public class ExecutionContext {
 		}
 		
 		//default case
-		setMatrixOutput(varName, outputData);
+		setMatrixOutput(varName, outputData, opcode);
 	}
 
 	public void setFrameOutput(String varName, FrameBlock outputData) 

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
index bbd8fb8..118aede 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
@@ -2830,7 +2830,7 @@ public class OptimizerRuleBased extends Optimizer
 				//replace existing matrix object with empty matrix
 				MatrixObject mo = (MatrixObject)dat;
 				ec.cleanupMatrixObject(mo);
-				ec.setMatrixOutput(rvar, new MatrixBlock((int)mo.getNumRows(), (int)mo.getNumColumns(),false));
+				ec.setMatrixOutput(rvar, new MatrixBlock((int)mo.getNumRows(), (int)mo.getNumColumns(),false), null);
 				
 				//keep track of cleaned result variables
 				cleanedVars.add(rvar);

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/Instruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/Instruction.java b/src/main/java/org/apache/sysml/runtime/instructions/Instruction.java
index 0681f14..ad8cb92 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/Instruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/Instruction.java
@@ -21,7 +21,7 @@ package org.apache.sysml.runtime.instructions;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.DataIdentifier;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -53,9 +53,11 @@ public abstract class Instruction
 	protected INSTRUCTION_TYPE type = null;
 	protected String instString = null;
 	protected String instOpcode = null;
+	private String extendedOpcode = null;
 	private long instID = -1;
 	
 	//originating script positions
+	protected String filename = null;
 	protected int beginLine = -1;
 	protected int endLine = -1;  
 	protected int beginCol = -1; 
@@ -77,7 +79,8 @@ public abstract class Instruction
 	 * @param beginCol beginning column position
 	 * @param endCol ending column position
 	 */
-	public void setLocation ( int beginLine, int endLine,  int beginCol, int endCol) {
+	public void setLocation ( String filename, int beginLine, int endLine,  int beginCol, int endCol) {
+		this.filename = filename;
 		this.beginLine = beginLine;
 		this.endLine = endLine;
 		this.beginCol = beginCol;
@@ -86,6 +89,7 @@ public abstract class Instruction
 	
 	public void setLocation(Lop lop) {
 		if(lop != null) {
+			this.filename = lop.getFilename();
 			this.beginLine = lop._beginLine;
 			this.endLine = lop._endLine;
 			this.beginCol = lop._beginColumn;
@@ -95,6 +99,7 @@ public abstract class Instruction
 	
 	public void setLocation(DataIdentifier id) {
 		if(id != null) {
+			this.filename = id.getFilename();
 			this.beginLine = id.getBeginLine();
 			this.endLine = id.getEndLine();
 			this.beginCol = id.getBeginColumn();
@@ -104,6 +109,7 @@ public abstract class Instruction
 	
 	public void setLocation(Instruction oldInst) {
 		if(oldInst != null) {
+			this.filename = oldInst.filename;
 			this.beginLine = oldInst.beginLine;
 			this.endLine = oldInst.endLine;
 			this.beginCol = oldInst.beginCol;
@@ -152,12 +158,31 @@ public abstract class Instruction
 	}
 	
 	public String getExtendedOpcode() {
-		if( type == INSTRUCTION_TYPE.SPARK )
-			return SP_INST_PREFIX + getOpcode();
-		else if( type == INSTRUCTION_TYPE.GPU )
-			return GPU_INST_PREFIX + getOpcode();
-		else
-			return getOpcode();
+		if(extendedOpcode != null)
+			return extendedOpcode;
+		if(DMLScript.FINEGRAINED_STATISTICS) {
+			String scriptInfo;
+			if(filename != null)
+				scriptInfo = " [" + filename + " " + beginLine + ":" + beginCol + "-" + endLine + ":" + endCol + "]";
+			else
+				scriptInfo = " [" + beginLine + ":" + beginCol + "-" + endLine + ":" + endCol + "]";
+			if( type == INSTRUCTION_TYPE.SPARK )
+				extendedOpcode = SP_INST_PREFIX + getOpcode() + scriptInfo;
+			else if( type == INSTRUCTION_TYPE.GPU )
+				extendedOpcode = GPU_INST_PREFIX + getOpcode() + scriptInfo;
+			else
+				extendedOpcode = getOpcode() + scriptInfo;
+		}
+		else {
+			// This ensures that there is no overhead if finegrained statistics is disabled
+			if( type == INSTRUCTION_TYPE.SPARK )
+				extendedOpcode = SP_INST_PREFIX + getOpcode();
+			else if( type == INSTRUCTION_TYPE.GPU )
+				extendedOpcode = GPU_INST_PREFIX + getOpcode();
+			else
+				extendedOpcode = getOpcode();
+		}
+		return extendedOpcode;
 	}
 
 	public boolean requiresLabelUpdate()

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateBinaryCPInstruction.java
index 359728f..2b09843 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateBinaryCPInstruction.java
@@ -70,8 +70,8 @@ public class AggregateBinaryCPInstruction extends BinaryCPInstruction
 		throws DMLRuntimeException
 	{	
 		//get inputs
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+		MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 		
 		//compute matrix multiplication
 		AggregateBinaryOperator ab_op = (AggregateBinaryOperator) _optr;
@@ -79,8 +79,8 @@ public class AggregateBinaryCPInstruction extends BinaryCPInstruction
 		MatrixBlock ret = (MatrixBlock) main.aggregateBinaryOperations(matBlock1, matBlock2, new MatrixBlock(), ab_op);
 		
 		//release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
-		ec.setMatrixOutput(output.getName(), ret);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateTernaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateTernaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateTernaryCPInstruction.java
index 147436e..949462a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateTernaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/AggregateTernaryCPInstruction.java
@@ -62,23 +62,23 @@ public class AggregateTernaryCPInstruction extends ComputationCPInstruction
 	public void processInstruction(ExecutionContext ec) 
 		throws DMLRuntimeException
 	{		
-		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
+        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
         MatrixBlock matBlock3 = input3.isLiteral() ? null : //matrix or literal 1
-        						ec.getMatrixInput(input3.getName());
+        						ec.getMatrixInput(input3.getName(), getExtendedOpcode());
 			
 		AggregateTernaryOperator ab_op = (AggregateTernaryOperator) _optr;
 		MatrixBlock ret = matBlock1.aggregateTernaryOperations(
 				matBlock1, matBlock2, matBlock3, new MatrixBlock(), ab_op, true);
 			
 		//release inputs/outputs
-		ec.releaseMatrixInput(input1.getName());
-		ec.releaseMatrixInput(input2.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		if( !input3.isLiteral() )
-			ec.releaseMatrixInput(input3.getName());
+			ec.releaseMatrixInput(input3.getName(), getExtendedOpcode());
 		if( output.getDataType().isScalar() )
 			ec.setScalarOutput(output.getName(), new DoubleObject(ret.quickGetValue(0, 0)));
 		else
-			ec.setMatrixOutput(output.getName(), ret);	
+			ec.setMatrixOutput(output.getName(), ret, getExtendedOpcode());	
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/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 8dd372a..5edce07 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
@@ -141,19 +141,19 @@ public class AggregateUnaryCPInstruction extends UnaryCPInstruction
 		else 
 		{
 			/* Default behavior for AggregateUnary Instruction */
-			MatrixBlock matBlock = ec.getMatrixInput(input1.getName());		
+			MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());		
 			AggregateUnaryOperator au_op = (AggregateUnaryOperator) _optr;
 			
 			MatrixBlock resultBlock = (MatrixBlock) matBlock.aggregateUnaryOperations(au_op, new MatrixBlock(), matBlock.getNumRows(), matBlock.getNumColumns(), new MatrixIndexes(1, 1), true);
 			
-			ec.releaseMatrixInput(input1.getName());
+			ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 			
 			if(output.getDataType() == DataType.SCALAR){
 				DoubleObject ret = new DoubleObject(output_name, resultBlock.getValue(0, 0));
 				ec.setScalarOutput(output_name, ret);
 			} else{
 				// since the computed value is a scalar, allocate a "temp" output matrix
-				ec.setMatrixOutput(output_name, resultBlock);
+				ec.setMatrixOutput(output_name, resultBlock, getExtendedOpcode());
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/CPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/CPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/CPInstruction.java
index 590dacb..c879481 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/CPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/CPInstruction.java
@@ -42,6 +42,22 @@ public abstract class CPInstruction extends Instruction
 	
 	protected boolean _requiresLabelUpdate = false;
 	
+	// Generic miscellaneous timers that are applicable to all CP (and few SP) instructions 
+	public final static String MISC_TIMER_GET_SPARSE_MB =          		"aqrs";	// time spent in bringing input sparse matrix block
+	public final static String MISC_TIMER_GET_DENSE_MB =          		"aqrd";	// time spent in bringing input dense matrix block
+	public final static String MISC_TIMER_ACQ_MODIFY_SPARSE_MB =        "aqms";	// time spent in bringing output sparse matrix block
+	public final static String MISC_TIMER_ACQ_MODIFY_DENSE_MB =         "aqmd";	// time spent in bringing output dense matrix block
+	public final static String MISC_TIMER_RELEASE_INPUT_MB =      		"rlsi";	// time spent in release input matrix block
+	public final static String MISC_TIMER_RELEASE_EVICTION =			"rlsev";// time spent in buffer eviction of release operation
+	public final static String MISC_TIMER_RELEASE_BUFF_WRITE =			"rlswr";// time spent in buffer write in release operation
+	public final static String MISC_TIMER_SPARSE_TO_DENSE =				"s2d";  // time spent in sparse to dense conversion
+	public final static String MISC_TIMER_DENSE_TO_SPARSE =				"d2s";  // time spent in sparse to dense conversion
+	
+	// Instruction specific miscellaneous timers that were found as potential bottlenecks in one of performance analysis.
+	// SystemML committers have to be judicious about adding them by weighing the tradeoffs between reuse in future analysis and unnecessary overheads.
+	public final static String MISC_TIMER_CSR_LIX_COPY =				"csrlix";// time spent in CSR-specific method to address performance issues due to repeated re-shifting on update-in-place.
+	public final static String MISC_TIMER_LIX_COPY =					"lixcp";// time spent in range copy
+	
 	public CPInstruction(String opcode, String istr) {
 		type = INSTRUCTION_TYPE.CONTROL_PROGRAM;
 		instString = istr;

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/CentralMomentCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/CentralMomentCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/CentralMomentCPInstruction.java
index e8ca315..917e984 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/CentralMomentCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/CentralMomentCPInstruction.java
@@ -102,7 +102,7 @@ public class CentralMomentCPInstruction extends AggregateUnaryCPInstruction
 		 * order and update the CMOperator, if needed.
 		 */
 		
-		MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
+		MatrixBlock matBlock = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 
 		CPOperand scalarInput = (input3==null ? input2 : input3);
 		ScalarObject order = ec.getScalarInput(scalarInput.getName(), scalarInput.getValueType(), scalarInput.isLiteral()); 
@@ -117,12 +117,12 @@ public class CentralMomentCPInstruction extends AggregateUnaryCPInstruction
 			cmobj = matBlock.cmOperations(cm_op);
 		}
 		else {
-			MatrixBlock wtBlock = ec.getMatrixInput(input2.getName());
+			MatrixBlock wtBlock = ec.getMatrixInput(input2.getName(), getExtendedOpcode());
 			cmobj = matBlock.cmOperations(cm_op, wtBlock);
-			ec.releaseMatrixInput(input2.getName());
+			ec.releaseMatrixInput(input2.getName(), getExtendedOpcode());
 		}
 		
-		ec.releaseMatrixInput(input1.getName());
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
 		
 		double val = cmobj.getRequiredResult(_optr);
 		DoubleObject ret = new DoubleObject(output_name, val);

http://git-wip-us.apache.org/repos/asf/systemml/blob/648eb21d/src/main/java/org/apache/sysml/runtime/instructions/cp/CompressionCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/CompressionCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/CompressionCPInstruction.java
index 5230945..e9826e1 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/CompressionCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/CompressionCPInstruction.java
@@ -51,14 +51,14 @@ public class CompressionCPInstruction extends UnaryCPInstruction
 		throws DMLRuntimeException
 	{
 		//get matrix block input
-		MatrixBlock in = ec.getMatrixInput(input1.getName());
+		MatrixBlock in = ec.getMatrixInput(input1.getName(), getExtendedOpcode());
 		
 		//compress the matrix block
 		CompressedMatrixBlock cmb = new CompressedMatrixBlock(in);
 		cmb.compress(OptimizerUtils.getConstrainedNumThreads(-1));
 		
 		//set output and release input
-		ec.releaseMatrixInput(input1.getName());
-		ec.setMatrixOutput(output.getName(), cmb);
+		ec.releaseMatrixInput(input1.getName(), getExtendedOpcode());
+		ec.setMatrixOutput(output.getName(), cmb, getExtendedOpcode());
 	}
 }