You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2016/01/02 20:42:35 UTC

[1/4] incubator-systemml git commit: Performance mr grouped aggregate (broadcast-based op, local aggregation)

Repository: incubator-systemml
Updated Branches:
  refs/heads/master b308c09b9 -> 8e7b6ed3d


Performance mr grouped aggregate (broadcast-based op, local aggregation)

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

Branch: refs/heads/master
Commit: 2fa5a187dec4762b5e8d3e62b5bce059e04fc4a0
Parents: b308c09
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Jan 1 17:08:10 2016 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Fri Jan 1 17:08:10 2016 -0800

----------------------------------------------------------------------
 .../sysml/hops/ParameterizedBuiltinOp.java      | 127 +++++++++++-----
 .../org/apache/sysml/lops/GroupedAggregate.java |   8 +-
 .../apache/sysml/lops/GroupedAggregateM.java    | 131 +++++++++++++++++
 src/main/java/org/apache/sysml/lops/Lop.java    |   2 +-
 .../runtime/instructions/InstructionUtils.java  |   2 +
 .../instructions/MRInstructionParser.java       |   6 +-
 .../mr/GroupedAggregateInstruction.java         |  17 ++-
 .../mr/GroupedAggregateMInstruction.java        | 145 +++++++++++++++++++
 .../runtime/instructions/mr/MRInstruction.java  |   2 +-
 .../runtime/matrix/MatrixCharacteristics.java   |   7 +
 .../matrix/mapred/GroupedAggMRMapper.java       |  54 +++++--
 .../matrix/mapred/MRJobConfiguration.java       |  12 +-
 12 files changed, 447 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java b/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
index 44714d4..466d008 100644
--- a/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
+++ b/src/main/java/org/apache/sysml/hops/ParameterizedBuiltinOp.java
@@ -27,18 +27,23 @@ import org.apache.sysml.hops.rewrite.HopRewriteUtils;
 import org.apache.sysml.lops.Aggregate;
 import org.apache.sysml.lops.AppendR;
 import org.apache.sysml.lops.Data;
+import org.apache.sysml.lops.DataPartition;
 import org.apache.sysml.lops.Group;
 import org.apache.sysml.lops.GroupedAggregate;
+import org.apache.sysml.lops.GroupedAggregateM;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.lops.LopProperties.ExecType;
 import org.apache.sysml.lops.LopsException;
 import org.apache.sysml.lops.OutputParameters.Format;
+import org.apache.sysml.lops.PartialAggregate.CorrectionLocationType;
 import org.apache.sysml.lops.ParameterizedBuiltin;
 import org.apache.sysml.lops.RepMat;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.Statement;
+import org.apache.sysml.runtime.controlprogram.ParForProgramBlock.PDataPartitionFormat;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
+import org.apache.sysml.runtime.matrix.mapred.DistributedCacheInput;
 import org.apache.sysml.runtime.util.UtilFunctions;
 
 
@@ -249,6 +254,8 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 		//construct lops
 		if ( et == ExecType.MR ) 
 		{
+			Lop grp_agg = null;
+			
 			// construct necessary lops: combineBinary/combineTertiary and groupedAgg
 			boolean isWeighted = (_paramIndexMap.get(Statement.GAGG_WEIGHTS) != null);
 			if (isWeighted) 
@@ -266,61 +273,105 @@ public class ParameterizedBuiltinOp extends Hop implements MultiThreadedHop
 				inputlops.remove(Statement.GAGG_GROUPS);
 				inputlops.remove(Statement.GAGG_WEIGHTS);
 
+				grp_agg = new GroupedAggregate(inputlops, isWeighted, getDataType(), getValueType());
+				grp_agg.getOutputParameters().setDimensions(outputDim1, outputDim2, getRowsInBlock(), getColsInBlock(), -1);
+				
+				setRequiresReblock( true );
 			} 
 			else 
 			{
 				Hop target = getInput().get(_paramIndexMap.get(Statement.GAGG_TARGET));
 				Hop groups = getInput().get(_paramIndexMap.get(Statement.GAGG_GROUPS));
 				Lop append = null;
-				
-				if(  target.getDim2()>=target.getColsInBlock()  // multi-column-block result matrix
-					|| target.getDim2()<=0  )                   // unkown
+			
+				//physical operator selection
+				double groupsSizeP = OptimizerUtils.estimatePartitionedSizeExactSparsity(groups.getDim1(), groups.getDim2(), groups.getRowsInBlock(), groups.getColsInBlock(), groups.getNnz());
+			
+				if( groupsSizeP < OptimizerUtils.getRemoteMemBudgetMap(true) //mapgroupedagg
+					&& getInput().get(_paramIndexMap.get(Statement.GAGG_FN)) instanceof LiteralOp
+					&& ((LiteralOp)getInput().get(_paramIndexMap.get(Statement.GAGG_FN))).getStringValue().equals("sum")
+					&& inputlops.get(Statement.GAGG_NUM_GROUPS) != null ) 
 				{
-					long m1_dim1 = target.getDim1();
-					long m1_dim2 = target.getDim2();		
-					long m2_dim1 = groups.getDim1();
-					long m2_dim2 = groups.getDim2();
-					long m3_dim1 = m1_dim1; 
-					long m3_dim2 = ((m1_dim2>0 && m2_dim2>0) ? (m1_dim2 + m2_dim2) : -1);
-					long m3_nnz = (target.getNnz()>0 && groups.getNnz()>0) ? (target.getNnz() + groups.getNnz()) : -1; 
-					long brlen = target.getRowsInBlock();
-					long bclen = target.getColsInBlock();
+					//pre partitioning
+					boolean needPart = (groups.dimsKnown() && groups.getDim1()*groups.getDim2() > DistributedCacheInput.PARTITION_SIZE);  
+					if( needPart ) {
+						ExecType etPart = (OptimizerUtils.estimateSizeExactSparsity(groups.getDim1(), groups.getDim2(), 1.0) 
+								          < OptimizerUtils.getLocalMemBudget()) ? ExecType.CP : ExecType.MR; //operator selection
+						Lop dcinput = new DataPartition(groups.constructLops(), DataType.MATRIX, ValueType.DOUBLE, etPart, PDataPartitionFormat.ROW_BLOCK_WISE_N);
+						dcinput.getOutputParameters().setDimensions(groups.getDim1(), groups.getDim2(), target.getRowsInBlock(), target.getColsInBlock(), groups.getNnz());
+						setLineNumbers(dcinput);
+						
+						inputlops.put(Statement.GAGG_GROUPS, dcinput);
+					}
 					
-					Lop offset = createOffsetLop(target, true); 
-					Lop rep = new RepMat(groups.constructLops(), offset, true, groups.getDataType(), groups.getValueType());
-					setOutputDimensions(rep);
-					setLineNumbers(rep);	
+					Lop grp_agg_m = new GroupedAggregateM(inputlops, getDataType(), getValueType(), needPart, ExecType.MR);
+					grp_agg_m.getOutputParameters().setDimensions(outputDim1, outputDim2, target.getRowsInBlock(), target.getColsInBlock(), -1);
+					setLineNumbers(grp_agg_m);
 					
-					Group group1 = new Group(target.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, target.getValueType());
-					group1.getOutputParameters().setDimensions(m1_dim1, m1_dim2, brlen, bclen, target.getNnz());
-					setLineNumbers(group1);
+					//post aggregation 
+					Group grp = new Group(grp_agg_m, Group.OperationTypes.Sort, getDataType(), getValueType());
+					grp.getOutputParameters().setDimensions(outputDim1, outputDim2, target.getRowsInBlock(), target.getColsInBlock(), -1);
+					setLineNumbers(grp);
 					
-					Group group2 = new Group(rep, Group.OperationTypes.Sort, DataType.MATRIX, groups.getValueType());
-					group1.getOutputParameters().setDimensions(m2_dim1, m2_dim2, brlen, bclen, groups.getNnz());
-					setLineNumbers(group2);
+					Aggregate agg1 = new Aggregate(grp, HopsAgg2Lops.get(AggOp.SUM), getDataType(), getValueType(), ExecType.MR);
+					agg1.setupCorrectionLocation(CorrectionLocationType.NONE);  
+					agg1.getOutputParameters().setDimensions(outputDim1, outputDim2, target.getRowsInBlock(), target.getColsInBlock(), -1);			
+					grp_agg = agg1;
 					
-					append = new AppendR(group1, group2, DataType.MATRIX, ValueType.DOUBLE, true, ExecType.MR);
-					append.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
-					setLineNumbers(append);
+					//note: no reblock required
 				}
-				else //single-column-block vector or matrix
-				{
-					append = BinaryOp.constructMRAppendLop(target, groups, 
-							DataType.MATRIX, getValueType(), true, target);
+				else //general case: groupedagg
+				{				
+					if(  target.getDim2()>=target.getColsInBlock()  // multi-column-block result matrix
+						|| target.getDim2()<=0  )                   // unkown
+					{
+						long m1_dim1 = target.getDim1();
+						long m1_dim2 = target.getDim2();		
+						long m2_dim1 = groups.getDim1();
+						long m2_dim2 = groups.getDim2();
+						long m3_dim1 = m1_dim1; 
+						long m3_dim2 = ((m1_dim2>0 && m2_dim2>0) ? (m1_dim2 + m2_dim2) : -1);
+						long m3_nnz = (target.getNnz()>0 && groups.getNnz()>0) ? (target.getNnz() + groups.getNnz()) : -1; 
+						long brlen = target.getRowsInBlock();
+						long bclen = target.getColsInBlock();
+						
+						Lop offset = createOffsetLop(target, true); 
+						Lop rep = new RepMat(groups.constructLops(), offset, true, groups.getDataType(), groups.getValueType());
+						setOutputDimensions(rep);
+						setLineNumbers(rep);	
+						
+						Group group1 = new Group(target.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, target.getValueType());
+						group1.getOutputParameters().setDimensions(m1_dim1, m1_dim2, brlen, bclen, target.getNnz());
+						setLineNumbers(group1);
+						
+						Group group2 = new Group(rep, Group.OperationTypes.Sort, DataType.MATRIX, groups.getValueType());
+						group1.getOutputParameters().setDimensions(m2_dim1, m2_dim2, brlen, bclen, groups.getNnz());
+						setLineNumbers(group2);
+						
+						append = new AppendR(group1, group2, DataType.MATRIX, ValueType.DOUBLE, true, ExecType.MR);
+						append.getOutputParameters().setDimensions(m3_dim1, m3_dim2, brlen, bclen, m3_nnz);
+						setLineNumbers(append);
+					}
+					else //single-column-block vector or matrix
+					{
+						append = BinaryOp.constructMRAppendLop(target, groups, 
+								DataType.MATRIX, getValueType(), true, target);
+					}
+					
+					// add the combine lop to parameter list, with a new name "combinedinput"
+					inputlops.put(GroupedAggregate.COMBINEDINPUT, append);
+					inputlops.remove(Statement.GAGG_TARGET);
+					inputlops.remove(Statement.GAGG_GROUPS);
+
+					grp_agg = new GroupedAggregate(inputlops, isWeighted, getDataType(), getValueType());
+					grp_agg.getOutputParameters().setDimensions(outputDim1, outputDim2, getRowsInBlock(), getColsInBlock(), -1);
+
+					setRequiresReblock( true );
 				}
-				
-				// add the combine lop to parameter list, with a new name "combinedinput"
-				inputlops.put(GroupedAggregate.COMBINEDINPUT, append);
-				inputlops.remove(Statement.GAGG_TARGET);
-				inputlops.remove(Statement.GAGG_GROUPS);
 			}
 			
-			GroupedAggregate grp_agg = new GroupedAggregate(inputlops, isWeighted, getDataType(), getValueType());
-			grp_agg.getOutputParameters().setDimensions(outputDim1, outputDim2, getRowsInBlock(), getColsInBlock(), -1);
 			setLineNumbers(grp_agg);
-			
 			setLops(grp_agg);
-			setRequiresReblock( true );
 		}
 		else //CP/Spark 
 		{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/lops/GroupedAggregate.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/GroupedAggregate.java b/src/main/java/org/apache/sysml/lops/GroupedAggregate.java
index d07cfe0..04ea824 100644
--- a/src/main/java/org/apache/sysml/lops/GroupedAggregate.java
+++ b/src/main/java/org/apache/sysml/lops/GroupedAggregate.java
@@ -258,9 +258,11 @@ public class GroupedAggregate extends Lop
 	
 		sb.append( OPERAND_DELIMITOR );
 		sb.append( _weights );
-			
+		
+		sb.append( OPERAND_DELIMITOR );
+		Lop ngroups = _inputParams.get(Statement.GAGG_NUM_GROUPS);
+		sb.append( (ngroups!=null)? ngroups.prepScalarInputOperand(getExecType()) : "-1" );
+		
 		return sb.toString();
 	}
-	
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/lops/GroupedAggregateM.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/GroupedAggregateM.java b/src/main/java/org/apache/sysml/lops/GroupedAggregateM.java
new file mode 100644
index 0000000..5edba62
--- /dev/null
+++ b/src/main/java/org/apache/sysml/lops/GroupedAggregateM.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.lops;
+
+import java.util.HashMap;
+
+import org.apache.sysml.lops.LopProperties.ExecLocation;
+import org.apache.sysml.lops.LopProperties.ExecType;
+import org.apache.sysml.lops.compile.JobType;
+import org.apache.sysml.parser.Statement;
+import org.apache.sysml.parser.Expression.*;
+
+
+/**
+ * Lop to perform mr map-side grouped aggregates 
+ * (restriction: sum, w/o weights, ngroups), groups broadcasted
+ * 
+ */
+public class GroupedAggregateM extends Lop 
+{	
+	public static final String OPCODE = "mapgroupedagg";
+
+	public enum CacheType {
+		RIGHT,
+		RIGHT_PART,
+	}
+	
+	private HashMap<String, Lop> _inputParams;
+	private CacheType _cacheType = null;
+		
+	public GroupedAggregateM(HashMap<String, Lop> inputParameterLops, 
+			DataType dt, ValueType vt, boolean partitioned, ExecType et) {		
+		super(Lop.Type.GroupedAggM, dt, vt);
+		init(inputParameterLops, dt, vt, et);
+		_inputParams = inputParameterLops;
+		_cacheType = partitioned ? CacheType.RIGHT_PART : CacheType.RIGHT;
+	}
+
+	/**
+	 * 
+	 * @param inputParameterLops
+	 * @param dt
+	 * @param vt
+	 * @param et
+	 */
+	private void init(HashMap<String, Lop> inputParameterLops, 
+			DataType dt, ValueType vt, ExecType et) 
+	{
+		addInput(inputParameterLops.get(Statement.GAGG_TARGET));
+		inputParameterLops.get(Statement.GAGG_TARGET).addOutput(this);
+		addInput(inputParameterLops.get(Statement.GAGG_GROUPS));
+		inputParameterLops.get(Statement.GAGG_GROUPS).addOutput(this);
+		
+		//setup MR parameters
+		boolean breaksAlignment = true;
+		boolean aligner = false;
+		boolean definesMRJob = false;
+		lps.addCompatibility(JobType.GMR);
+		lps.addCompatibility(JobType.DATAGEN);
+		lps.setProperties( inputs, ExecType.MR, ExecLocation.Map, breaksAlignment, aligner, definesMRJob );
+	}
+
+	@Override
+	public String toString() {
+		return "Operation = MapGroupedAggregate";
+	}
+	
+	@Override
+	public String getInstructions(int input1, int input2, int output) 
+	{
+		StringBuilder sb = new StringBuilder();
+		
+		sb.append( getExecType() );
+		
+		sb.append( Lop.OPERAND_DELIMITOR );
+		sb.append( OPCODE );
+		
+		sb.append( OPERAND_DELIMITOR );
+		sb.append( getInputs().get(0).prepInputOperand(input1));
+		
+		sb.append( OPERAND_DELIMITOR );
+		sb.append( getInputs().get(1).prepInputOperand(input2));
+	
+		sb.append( OPERAND_DELIMITOR );
+		sb.append( prepOutputOperand(output) );
+	
+		sb.append( OPERAND_DELIMITOR );
+		sb.append( _inputParams.get(Statement.GAGG_NUM_GROUPS)
+				.prepScalarInputOperand(getExecType()) );
+		
+		sb.append( OPERAND_DELIMITOR );
+		sb.append( _cacheType.toString() );
+	
+		return sb.toString();
+	}
+	
+	@Override
+	public boolean usesDistributedCache() 
+	{
+		if( getExecType()==ExecType.MR )
+			return true;
+		else
+			return false;
+	}
+	
+	@Override
+	public int[] distributedCacheInputIndex() 
+	{
+		if( getExecType()==ExecType.MR )
+			return new int[]{2};
+		else
+			return new int[]{-1};
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/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 29310bc..bed9d10 100644
--- a/src/main/java/org/apache/sysml/lops/Lop.java
+++ b/src/main/java/org/apache/sysml/lops/Lop.java
@@ -50,7 +50,7 @@ public abstract class Lop
 		Grouping, 											//MR grouping
 		Append,                                             //CP/MR append (column append)
 		CombineUnary, CombineBinary, CombineTernary,        //MR combine (stitch together)
-		CentralMoment, CoVariance, GroupedAgg,
+		CentralMoment, CoVariance, GroupedAgg, GroupedAggM,
 		Transform, DataPartition, RepMat,                   //CP/MR reorganization, partitioning, replication
 		ParameterizedBuiltin,                               //CP/MR parameterized ops (name/value)
 		FunctionCallCP, 									//CP function calls 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/instructions/InstructionUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/InstructionUtils.java b/src/main/java/org/apache/sysml/runtime/instructions/InstructionUtils.java
index 5a6c197..bec5ff7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/InstructionUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/InstructionUtils.java
@@ -23,6 +23,7 @@ import java.util.StringTokenizer;
 
 import org.apache.sysml.lops.AppendM;
 import org.apache.sysml.lops.BinaryM;
+import org.apache.sysml.lops.GroupedAggregateM;
 import org.apache.sysml.lops.MapMult;
 import org.apache.sysml.lops.MapMultChain;
 import org.apache.sysml.lops.PMMJ;
@@ -299,6 +300,7 @@ public class InstructionUtils
 			   || opcode.equalsIgnoreCase(MapMultChain.OPCODE)
 			   || opcode.equalsIgnoreCase(PMMJ.OPCODE)
 			   || opcode.equalsIgnoreCase(UAggOuterChain.OPCODE)
+			   || opcode.equalsIgnoreCase(GroupedAggregateM.OPCODE)
 			   || isDistQuaternaryOpcode( opcode ) //multiple quaternary opcodes
 			   || BinaryM.isOpcode( opcode ) ) //multiple binary opcodes	
 			{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
index ee64e2a..a28c00d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
@@ -56,6 +56,7 @@ import org.apache.sysml.runtime.instructions.mr.CumulativeSplitInstruction;
 import org.apache.sysml.runtime.instructions.mr.DataGenMRInstruction;
 import org.apache.sysml.runtime.instructions.mr.DataPartitionMRInstruction;
 import org.apache.sysml.runtime.instructions.mr.GroupedAggregateInstruction;
+import org.apache.sysml.runtime.instructions.mr.GroupedAggregateMInstruction;
 import org.apache.sysml.runtime.instructions.mr.MMTSJMRInstruction;
 import org.apache.sysml.runtime.instructions.mr.MRInstruction;
 import org.apache.sysml.runtime.instructions.mr.MapMultChainInstruction;
@@ -247,7 +248,7 @@ public class MRInstructionParser extends InstructionParser
 		
 		//groupedAgg Instruction Opcodes
 		String2MRInstructionType.put( "groupedagg"  , MRINSTRUCTION_TYPE.GroupedAggregate); 
-		//String2MRInstructionType.put( "grpcm"  , MRINSTRUCTION_TYPE.GroupedAggregate); 
+		String2MRInstructionType.put( "mapgroupedagg"  , MRINSTRUCTION_TYPE.MapGroupedAggregate); 
 		
 		//rangereindexing
 		String2MRInstructionType.put( "rangeReIndex"  , MRINSTRUCTION_TYPE.RangeReIndex);
@@ -393,6 +394,9 @@ public class MRInstructionParser extends InstructionParser
 		case GroupedAggregate:
 			return (MRInstruction) GroupedAggregateInstruction.parseInstruction(str);
 		
+		case MapGroupedAggregate:
+			return (MRInstruction) GroupedAggregateMInstruction.parseInstruction(str);
+		
 		case RangeReIndex:
 			return (MRInstruction) RangeBasedReIndexInstruction.parseInstruction(str);
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
index a104ca8..effafbd 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
@@ -38,20 +38,26 @@ import org.apache.sysml.runtime.matrix.operators.CMOperator.AggregateOperationTy
 public class GroupedAggregateInstruction extends UnaryMRInstructionBase
 {
 	private boolean _weights = false;
+	private int _ngroups = -1;
 	private long _bclen = -1;
 	
-	public GroupedAggregateInstruction(Operator op, byte in, byte out, boolean weights, String istr) {
+	public GroupedAggregateInstruction(Operator op, byte in, byte out, boolean weights, int ngroups, String istr) {
 		super(op, in, out);
 		mrtype = MRINSTRUCTION_TYPE.GroupedAggregate;
 		instString = istr;
 		
 		_weights = weights;
+		_ngroups = ngroups;
 	}
 
 	public boolean hasWeights() {
 		return _weights;
 	}
 	
+	public int getNGroups() {
+		return _ngroups;
+	}
+	
 	public void setBclen(long bclen){
 		_bclen = bclen;
 	}
@@ -73,20 +79,21 @@ public class GroupedAggregateInstruction extends UnaryMRInstructionBase
 	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		String[] parts = InstructionUtils.getInstructionParts ( str );
-		if(parts.length<2)
+		if(parts.length<3)
 			throw new DMLRuntimeException("the number of fields of instruction "+str+" is less than 2!");
 		byte in, out;
 		String opcode = parts[0];
 		in = Byte.parseByte(parts[1]);
-		out = Byte.parseByte(parts[parts.length - 2]);
-		boolean weights = Boolean.parseBoolean(parts[parts.length-1]);
+		out = Byte.parseByte(parts[parts.length - 3]);
+		boolean weights = Boolean.parseBoolean(parts[parts.length-2]);
+		int ngroups = Integer.parseInt(parts[parts.length-1]);
 		
 		if ( !opcode.equalsIgnoreCase("groupedagg") ) {
 			throw new DMLRuntimeException("Invalid opcode in GroupedAggregateInstruction: " + opcode);
 		}
 		
 		Operator optr = parseGroupedAggOperator(parts[2], parts[3]);
-		return new GroupedAggregateInstruction(optr, in, out, weights, str);
+		return new GroupedAggregateInstruction(optr, in, out, weights, ngroups, str);
 	}
 	
 	public static Operator parseGroupedAggOperator(String fn, String other) throws DMLRuntimeException {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
new file mode 100644
index 0000000..2d1a238
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.runtime.instructions.mr;
+
+import java.util.ArrayList;
+
+import org.apache.sysml.lops.PartialAggregate.CorrectionLocationType;
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.DMLUnsupportedOperationException;
+import org.apache.sysml.runtime.functionobjects.KahanPlus;
+import org.apache.sysml.runtime.instructions.Instruction;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
+import org.apache.sysml.runtime.matrix.data.MatrixValue;
+import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
+import org.apache.sysml.runtime.matrix.mapred.DistributedCacheInput;
+import org.apache.sysml.runtime.matrix.mapred.IndexedMatrixValue;
+import org.apache.sysml.runtime.matrix.mapred.MRBaseForCommonInstructions;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.matrix.operators.Operator;
+
+
+public class GroupedAggregateMInstruction extends BinaryMRInstructionBase implements IDistributedCacheConsumer
+{	
+	private int _ngroups = -1;
+	
+	public GroupedAggregateMInstruction(Operator op, byte in1, byte in2, byte out, int ngroups, String istr)
+	{
+		super(op, in1, in2, out);
+		_ngroups = ngroups;
+	}
+	
+	/**
+	 * 
+	 * @param str
+	 * @return
+	 * @throws DMLRuntimeException
+	 */
+	public static Instruction parseInstruction ( String str ) 
+		throws DMLRuntimeException 
+	{
+		String[] parts = InstructionUtils.getInstructionParts ( str );
+		InstructionUtils.checkNumFields(parts, 5);
+		
+		byte in1 = Byte.parseByte(parts[1]);
+		byte in2 = Byte.parseByte(parts[2]);
+		byte out = Byte.parseByte(parts[3]);
+		int ngroups = Integer.parseInt(parts[4]);
+		//partitioning ignored
+		
+		Operator op = new AggregateOperator(0, KahanPlus.getKahanPlusFnObject(), true, CorrectionLocationType.LASTCOLUMN);
+		
+		return new GroupedAggregateMInstruction(op, in1, in2, out, ngroups, str);
+	}
+	
+	@Override
+	public void processInstruction(Class<? extends MatrixValue> valueClass,
+			CachedValueMap cachedValues, IndexedMatrixValue tempValue, IndexedMatrixValue zeroInput, 
+			int blockRowFactor, int blockColFactor)
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{	
+		ArrayList<IndexedMatrixValue> blkList = cachedValues.get(input1);
+		if( blkList == null ) 
+			return;
+		
+		for(IndexedMatrixValue in1 : blkList)
+		{
+			if(in1 == null)
+				continue;
+		
+			DistributedCacheInput dcInput = MRBaseForCommonInstructions.dcValues.get(input2);
+			
+			//get all inputs
+			MatrixIndexes ix = in1.getIndexes();
+			MatrixBlock target = (MatrixBlock)in1.getValue();
+			MatrixBlock groups = (MatrixBlock)dcInput.getDataBlock((int)ix.getRowIndex(), 1).getValue();
+			
+			//execute grouped aggregate operations
+			MatrixBlock out = groups.groupedAggOperations(target, null, new MatrixBlock(), _ngroups, getOperator());
+			
+			//output blocked result
+			int brlen = dcInput.getNumRowsPerBlock();
+			int bclen = dcInput.getNumColsPerBlock();
+			
+			if( out.getNumRows()<=brlen && out.getNumColumns()<=bclen )
+			{
+				//single output block
+				cachedValues.add(output, new IndexedMatrixValue(new MatrixIndexes(1,ix.getColumnIndex()), out));	
+			}
+			else
+			{
+				//multiple output blocks (by op def, single column block )				
+				for(int blockRow = 0; blockRow < (int)Math.ceil(out.getNumRows()/(double)brlen); blockRow++)
+				{
+					int maxRow = (blockRow*brlen + brlen < out.getNumRows()) ? brlen : out.getNumRows() - blockRow*brlen;			
+					int row_offset = blockRow*brlen;
+
+					//copy submatrix to block
+					MatrixBlock tmp = out.sliceOperations( row_offset, row_offset+maxRow-1, 
+							             0, out.getNumColumns()-1, new MatrixBlock() );
+					
+					//append block to result cache
+					cachedValues.add(output, new IndexedMatrixValue(
+							new MatrixIndexes(blockRow+1,ix.getColumnIndex()), tmp));			
+				}
+			}			
+		}	
+	}
+	
+
+	@Override //IDistributedCacheConsumer
+	public boolean isDistCacheOnlyIndex( String inst, byte index )
+	{
+		return (index==input2 && index!=input1);
+	}
+	
+	@Override //IDistributedCacheConsumer
+	public void addDistCacheIndex( String inst, ArrayList<Byte> indexes )
+	{
+		indexes.add(input2);
+	}
+	
+	public void computeOutputCharacteristics(MatrixCharacteristics mcIn, MatrixCharacteristics mcOut) {
+		mcOut.set(_ngroups, mcIn.getCols(), mcIn.getRowsPerBlock(), mcIn.getColsPerBlock());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
index b4f28a6..386eebe 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MRInstruction.java
@@ -35,7 +35,7 @@ public abstract class MRInstruction extends Instruction
 	public enum MRINSTRUCTION_TYPE { INVALID, Append, Aggregate, ArithmeticBinary, AggregateBinary, AggregateUnary, 
 		Rand, Seq, CSVReblock, CSVWrite, Transform,
 		Reblock, Reorg, Replicate, Unary, CombineBinary, CombineUnary, CombineTernary, PickByCount, Partition,
-		Ternary, Quaternary, CM_N_COV, Combine, GroupedAggregate, RangeReIndex, ZeroOut, MMTSJ, PMMJ, MatrixReshape, ParameterizedBuiltin, Sort, MapMultChain,
+		Ternary, Quaternary, CM_N_COV, Combine, MapGroupedAggregate, GroupedAggregate, RangeReIndex, ZeroOut, MMTSJ, PMMJ, MatrixReshape, ParameterizedBuiltin, Sort, MapMultChain,
 		CumsumAggregate, CumsumSplit, CumsumOffset, BinUaggChain, UaggOuterChain, RemoveEmpty}; 
 	
 	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/matrix/MatrixCharacteristics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/MatrixCharacteristics.java b/src/main/java/org/apache/sysml/runtime/matrix/MatrixCharacteristics.java
index 70e13ac..4618099 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/MatrixCharacteristics.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/MatrixCharacteristics.java
@@ -40,6 +40,7 @@ import org.apache.sysml.runtime.instructions.mr.CombineUnaryInstruction;
 import org.apache.sysml.runtime.instructions.mr.CumulativeAggregateInstruction;
 import org.apache.sysml.runtime.instructions.mr.DataGenMRInstruction;
 import org.apache.sysml.runtime.instructions.mr.GroupedAggregateInstruction;
+import org.apache.sysml.runtime.instructions.mr.GroupedAggregateMInstruction;
 import org.apache.sysml.runtime.instructions.mr.MMTSJMRInstruction;
 import org.apache.sysml.runtime.instructions.mr.MRInstruction;
 import org.apache.sysml.runtime.instructions.mr.MapMultChainInstruction;
@@ -351,6 +352,12 @@ public class MatrixCharacteristics implements Serializable
 			MatrixCharacteristics mc2 = dims.get(realIns.input2);
 			realIns.computeOutputCharacteristics(mc1, mc2, dimOut);
 		}
+		else if( ins instanceof GroupedAggregateMInstruction )
+		{
+			GroupedAggregateMInstruction realIns = (GroupedAggregateMInstruction) ins;
+			MatrixCharacteristics mc1 = dims.get(realIns.input1);
+			realIns.computeOutputCharacteristics(mc1, dimOut);
+		}
 		else if(ins instanceof BinaryInstruction || ins instanceof BinaryMInstruction || ins instanceof CombineBinaryInstruction )
 		{
 			BinaryMRInstructionBase realIns=(BinaryMRInstructionBase)ins;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/matrix/mapred/GroupedAggMRMapper.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/mapred/GroupedAggMRMapper.java b/src/main/java/org/apache/sysml/runtime/matrix/mapred/GroupedAggMRMapper.java
index 079e1fb..1a3b306 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/mapred/GroupedAggMRMapper.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/mapred/GroupedAggMRMapper.java
@@ -28,13 +28,14 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Mapper;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
+import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.instructions.mr.GroupedAggregateInstruction;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.TaggedMatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.WeightedCell;
-
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
 
 public class GroupedAggMRMapper extends MapperBase
 	implements Mapper<MatrixIndexes, MatrixValue, TaggedMatrixIndexes, WeightedCell>
@@ -51,8 +52,8 @@ public class GroupedAggMRMapper extends MapperBase
 			        OutputCollector<TaggedMatrixIndexes, WeightedCell> out, Reporter reporter) 
 	    throws IOException 
 	{
-		for(int i=0; i<representativeMatrixes.size(); i++)
-			for(GroupedAggregateInstruction ins : groupAgg_instructions.get(i))
+		for(int k=0; k<representativeMatrixes.size(); k++)
+			for(GroupedAggregateInstruction ins : groupAgg_instructions.get(k))
 			{
 				//set the tag once for the block
 				outKey.setTag(ins.output);
@@ -67,13 +68,46 @@ public class GroupedAggMRMapper extends MapperBase
 				{
 					long coloff = (key.getColumnIndex()-1)*ins.getBclen();
 					
-					for( int r=0; r<rlen; r++ ) {
-						int group = (int)block.quickGetValue(r, clen-1);
-						for( int c=0; c<clen-1; c++ ) {
-							outKeyValue.setIndexes(group, coloff+c+1);
-							outValue.setValue(block.quickGetValue(r, c));
-							outValue.setWeight(1);
-							out.collect(outKey, outValue);		
+					//local pre-aggregation for sum w/ known output dimensions
+					if(ins.getOperator() instanceof AggregateOperator && ins.getNGroups() > 0 
+						&& OptimizerUtils.isValidCPDimensions(ins.getNGroups(), block.getNumColumns()-1) ) 
+					{
+						try 
+						{
+							MatrixBlock group = block.sliceOperations(0, block.getNumRows()-1, 
+									block.getNumColumns()-1, block.getNumColumns()-1, new MatrixBlock());
+							MatrixBlock target = block.sliceOperations(0, block.getNumRows()-1, 
+									0, block.getNumColumns()-2, new MatrixBlock());
+								
+							MatrixBlock tmp = group.groupedAggOperations(target, null, new MatrixBlock(), ins.getNGroups(), ins.getOperator());
+							
+							for(int i=0; i<tmp.getNumRows(); i++) {
+								for( int j=0; j<tmp.getNumColumns(); j++ ) {
+									double tmpval = tmp.quickGetValue(i, j);
+									if( tmpval != 0 ) {
+										outKeyValue.setIndexes(i+1,coloff+j+1);
+										outValue.setValue(tmpval);
+										outValue.setWeight(1);
+										out.collect(outKey, outValue);
+									}
+								}
+							}
+						} 
+						catch(Exception ex) {
+							throw new IOException(ex);
+						}
+					}
+					//general case without pre-aggregation
+					else
+					{
+						for( int r=0; r<rlen; r++ ) {
+							int group = (int)block.quickGetValue(r, clen-1);
+							for( int c=0; c<clen-1; c++ ) {
+								outKeyValue.setIndexes(group, coloff+c+1);
+								outValue.setValue(block.quickGetValue(r, c));
+								outValue.setWeight(1);
+								out.collect(outKey, outValue);		
+							}
 						}
 					}
 				}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/2fa5a187/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java b/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
index 5f40a65..3382a0c 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/mapred/MRJobConfiguration.java
@@ -1271,7 +1271,7 @@ public class MRJobConfiguration
 		job.set(RESULT_DIMS_UNKNOWN_CONFIG, MRJobConfiguration.getIndexesString(updDimsUnknown));
 	}
 	
-	public static void setUpMultipleOutputs(JobConf job, byte[] resultIndexes, byte[] resultDimsUnknwon, String[] outputs, 
+	public static void setUpMultipleOutputs(JobConf job, byte[] resultIndexes, byte[] resultDimsUnknown, String[] outputs, 
 			OutputInfo[] outputInfos, boolean inBlockRepresentation, boolean mayContainCtable) 
 	throws Exception
 	{
@@ -1281,19 +1281,17 @@ public class MRJobConfiguration
 			throw new Exception("number of outputs and outputInfos indexes does not match");
 		
 		job.set(RESULT_INDEXES_CONFIG, MRJobConfiguration.getIndexesString(resultIndexes));
-		job.set(RESULT_DIMS_UNKNOWN_CONFIG, MRJobConfiguration.getIndexesString(resultDimsUnknwon));
+		job.set(RESULT_DIMS_UNKNOWN_CONFIG, MRJobConfiguration.getIndexesString(resultDimsUnknown));
 		job.setStrings(OUTPUT_MATRICES_DIRS_CONFIG, outputs);
 		job.setOutputCommitter(MultipleOutputCommitter.class);
 		
 		for(int i=0; i<outputs.length; i++)
 		{
 			MapReduceTool.deleteFileIfExistOnHDFS(new Path(outputs[i]), job);
-			if ( mayContainCtable && resultDimsUnknwon[i] == (byte) 1 ) 
-			{
+			if ( mayContainCtable && resultDimsUnknown[i] == (byte) 1 )  {
 				setOutputInfo(job, i, outputInfos[i], false);
 			}
-			else
-			{
+			else {
 				setOutputInfo(job, i, outputInfos[i], inBlockRepresentation);
 			}
 			MultipleOutputs.addNamedOutput(job, Integer.toString(i), 
@@ -1306,7 +1304,6 @@ public class MRJobConfiguration
 		Path tempOutputPath = new Path( constructTempOutputFilename() );
 		FileOutputFormat.setOutputPath(job, tempOutputPath);
 		MapReduceTool.deleteFileIfExistOnHDFS(tempOutputPath, job);
-		
 	}
 	
 	public static void setUpMultipleOutputs(JobConf job, byte[] resultIndexes, byte[] resultDimsUnknwon, String[] outputs, 
@@ -1428,6 +1425,7 @@ public class MRJobConfiguration
 			for(MRInstruction ins: insMapper)
 			{
 				MatrixCharacteristics.computeDimension(dims, ins);
+				
 				if( ins instanceof UnaryMRInstructionBase )
 				{
 					UnaryMRInstructionBase tempIns=(UnaryMRInstructionBase) ins;



[3/4] incubator-systemml git commit: Cleanup instruction parsing (removed unnecessary casts, unused methods)

Posted by mb...@apache.org.
Cleanup instruction parsing (removed unnecessary casts, unused methods)

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

Branch: refs/heads/master
Commit: 5878e1dbba9f7c9161d99e7fdab7e67d5b5da715
Parents: 2fa5a18
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Jan 1 17:51:03 2016 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Fri Jan 1 17:51:03 2016 -0800

----------------------------------------------------------------------
 .../instructions/CPInstructionParser.java       | 241 ++++++++---------
 .../runtime/instructions/InstructionParser.java |  19 +-
 .../instructions/MRInstructionParser.java       | 267 +++++++++----------
 .../instructions/SPInstructionParser.java       |  38 ++-
 .../cp/AggregateUnaryCPInstruction.java         |   3 +-
 .../instructions/cp/AppendCPInstruction.java    |   3 +-
 .../cp/ArithmeticBinaryCPInstruction.java       |   5 +-
 .../cp/BooleanBinaryCPInstruction.java          |   5 +-
 .../cp/BooleanUnaryCPInstruction.java           |   5 +-
 .../cp/BuiltinBinaryCPInstruction.java          |   3 +-
 .../cp/BuiltinUnaryCPInstruction.java           |   3 +-
 .../cp/CentralMomentCPInstruction.java          |   3 +-
 .../instructions/cp/DataGenCPInstruction.java   |   4 +-
 .../cp/DataPartitionCPInstruction.java          |   3 +-
 .../instructions/cp/FileCPInstruction.java      |   3 +-
 .../cp/FunctionCallCPInstruction.java           |   5 +-
 .../instructions/cp/MMChainCPInstruction.java   |   3 +-
 .../instructions/cp/MMTSJCPInstruction.java     |   3 +-
 .../cp/MatrixIndexingCPInstruction.java         |   3 +-
 .../cp/MatrixReshapeCPInstruction.java          |   3 +-
 .../cp/MultiReturnBuiltinCPInstruction.java     |   4 +-
 .../instructions/cp/PMMJCPInstruction.java      |   3 +-
 .../cp/ParameterizedBuiltinCPInstruction.java   |   4 +-
 .../cp/QuantilePickCPInstruction.java           |   3 +-
 .../cp/QuantileSortCPInstruction.java           |   3 +-
 .../cp/RelationalBinaryCPInstruction.java       |   3 +-
 .../instructions/cp/ReorgCPInstruction.java     |   3 +-
 .../cp/StringInitCPInstruction.java             |   3 +-
 .../cp/UaggOuterChainCPInstruction.java         |   3 +-
 .../instructions/cp/VariableCPInstruction.java  |   2 +-
 .../cpfile/MatrixIndexingCPFileInstruction.java |   3 +-
 .../ParameterizedBuiltinCPFileInstruction.java  |   4 +-
 .../mr/AggregateBinaryInstruction.java          |   3 +-
 .../instructions/mr/AggregateInstruction.java   |   3 +-
 .../mr/AggregateUnaryInstruction.java           |   3 +-
 .../instructions/mr/AppendGInstruction.java     |   3 +-
 .../instructions/mr/AppendInstruction.java      |   3 +-
 .../instructions/mr/AppendMInstruction.java     |   3 +-
 .../instructions/mr/AppendRInstruction.java     |   3 +-
 .../mr/BinUaggChainInstruction.java             |   3 +-
 .../instructions/mr/BinaryInstruction.java      |   6 +-
 .../instructions/mr/BinaryMInstruction.java     |   3 +-
 .../instructions/mr/CM_N_COVInstruction.java    |   6 +-
 .../instructions/mr/CSVReblockInstruction.java  |   2 +-
 .../instructions/mr/CSVWriteInstruction.java    |   2 +-
 .../mr/CombineBinaryInstruction.java            |   3 +-
 .../mr/CombineTernaryInstruction.java           |   3 +-
 .../mr/CombineUnaryInstruction.java             |   3 +-
 .../mr/CumulativeAggregateInstruction.java      |   3 +-
 .../mr/CumulativeOffsetInstruction.java         |   3 +-
 .../mr/CumulativeSplitInstruction.java          |   3 +-
 .../mr/DataPartitionMRInstruction.java          |   3 +-
 .../mr/GroupedAggregateInstruction.java         |   3 +-
 .../mr/GroupedAggregateMInstruction.java        |   3 +-
 .../instructions/mr/MMTSJMRInstruction.java     |   3 +-
 .../mr/MapMultChainInstruction.java             |   3 +-
 .../mr/MatrixReshapeMRInstruction.java          |   3 +-
 .../instructions/mr/PMMJMRInstruction.java      |   3 +-
 .../mr/ParameterizedBuiltinMRInstruction.java   |   3 +-
 .../instructions/mr/PickByCountInstruction.java |   3 +-
 .../instructions/mr/QuaternaryInstruction.java  |   3 +-
 .../instructions/mr/RandInstruction.java        |   3 +-
 .../mr/RangeBasedReIndexInstruction.java        |   3 +-
 .../instructions/mr/ReblockInstruction.java     |   2 +-
 .../mr/RemoveEmptyMRInstruction.java            |   3 +-
 .../instructions/mr/ReorgInstruction.java       |   3 +-
 .../instructions/mr/ReplicateInstruction.java   |   3 +-
 .../instructions/mr/ScalarInstruction.java      |   3 +-
 .../runtime/instructions/mr/SeqInstruction.java |   3 +-
 .../instructions/mr/TernaryInstruction.java     |   3 +-
 .../mr/UaggOuterChainInstruction.java           |   3 +-
 .../instructions/mr/UnaryInstruction.java       |   3 +-
 .../instructions/mr/ZeroOutInstruction.java     |   3 +-
 .../spark/AggregateUnarySPInstruction.java      |   3 +-
 .../spark/ArithmeticBinarySPInstruction.java    |   3 +-
 .../spark/BinUaggChainSPInstruction.java        |   3 +-
 .../spark/CSVReblockSPInstruction.java          |   3 +-
 .../spark/CentralMomentSPInstruction.java       |   3 +-
 .../spark/CheckpointSPInstruction.java          |   3 +-
 .../spark/CovarianceSPInstruction.java          |   3 +-
 .../spark/CumulativeOffsetSPInstruction.java    |   3 +-
 .../spark/MatrixIndexingSPInstruction.java      |   3 +-
 .../spark/ReblockSPInstruction.java             |   3 +-
 .../spark/RelationalBinarySPInstruction.java    |   3 +-
 .../instructions/spark/ReorgSPInstruction.java  |   3 +-
 85 files changed, 360 insertions(+), 462 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
index d81e363..5b84ea2 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/CPInstructionParser.java
@@ -247,157 +247,150 @@ public class CPInstructionParser extends InstructionParser
 		String2CPFileInstructionType.put( "rmempty"	    , CPINSTRUCTION_TYPE.ParameterizedBuiltin);
 	}
 
-	public static CPInstruction parseSingleInstruction (String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static CPInstruction parseSingleInstruction (String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		if ( str == null || str.isEmpty() )
 			return null;
 
 		CPINSTRUCTION_TYPE cptype = InstructionUtils.getCPType(str); 
 		if ( cptype == null ) 
 			throw new DMLRuntimeException("Unable derive cptype for instruction: " + str);
-		CPInstruction cpinst = CPInstructionParser.parseSingleInstruction(cptype, str);
+		CPInstruction cpinst = parseSingleInstruction(cptype, str);
 		if ( cpinst == null )
 			throw new DMLRuntimeException("Unable to parse instruction: " + str);
 		return cpinst;
 	}
 	
-	public static CPInstruction parseSingleInstruction ( CPINSTRUCTION_TYPE cptype, String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static CPInstruction parseSingleInstruction ( CPINSTRUCTION_TYPE cptype, String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		ExecType execType = null; 
 		
 		if ( str == null || str.isEmpty() ) 
 			return null;
-		switch(cptype) {
-		case AggregateUnary:
-			return (CPInstruction) AggregateUnaryCPInstruction.parseInstruction(str);
 		
-		case AggregateBinary:
-			return (CPInstruction) AggregateBinaryCPInstruction.parseInstruction(str);
-
-		case AggregateTernary:
-			return (CPInstruction) AggregateTernaryCPInstruction.parseInstruction(str);
-			
-		case ArithmeticBinary:
-			return (CPInstruction) ArithmeticBinaryCPInstruction.parseInstruction(str);
-		
-		case Ternary:
-			return (CPInstruction) TernaryCPInstruction.parseInstruction(str);
-		
-		case Quaternary:
-			return (CPInstruction) QuaternaryCPInstruction.parseInstruction(str);
-		
-		case BooleanBinary:
-			return (CPInstruction) BooleanBinaryCPInstruction.parseInstruction(str);
-			
-		case BooleanUnary:
-			return (CPInstruction) BooleanUnaryCPInstruction.parseInstruction(str);
-			
-		case BuiltinBinary:
-			return (CPInstruction) BuiltinBinaryCPInstruction.parseInstruction(str);
-			
-		case BuiltinUnary:
-			return (CPInstruction) BuiltinUnaryCPInstruction.parseInstruction(str);
-			
-		case Reorg:
-			return (CPInstruction) ReorgCPInstruction.parseInstruction(str);
+		switch(cptype) 
+		{
+			case AggregateUnary:
+				return AggregateUnaryCPInstruction.parseInstruction(str);
 			
-		case UaggOuterChain:
-			return (CPInstruction) UaggOuterChainCPInstruction.parseInstruction(str);
+			case AggregateBinary:
+				return AggregateBinaryCPInstruction.parseInstruction(str);
+	
+			case AggregateTernary:
+				return AggregateTernaryCPInstruction.parseInstruction(str);
+				
+			case ArithmeticBinary:
+				return ArithmeticBinaryCPInstruction.parseInstruction(str);
 			
-		case MatrixReshape:
-			return (CPInstruction) MatrixReshapeCPInstruction.parseInstruction(str);	
-
-		case Append:
-			return (CPInstruction) AppendCPInstruction.parseInstruction(str);
+			case Ternary:
+				return TernaryCPInstruction.parseInstruction(str);
 			
-		case RelationalBinary:
-			return (CPInstruction) RelationalBinaryCPInstruction.parseInstruction(str);
+			case Quaternary:
+				return QuaternaryCPInstruction.parseInstruction(str);
 			
-		case File:
-			return (CPInstruction) FileCPInstruction.parseInstruction(str);
+			case BooleanBinary:
+				return BooleanBinaryCPInstruction.parseInstruction(str);
+				
+			case BooleanUnary:
+				return BooleanUnaryCPInstruction.parseInstruction(str);
+				
+			case BuiltinBinary:
+				return BuiltinBinaryCPInstruction.parseInstruction(str);
+				
+			case BuiltinUnary:
+				return BuiltinUnaryCPInstruction.parseInstruction(str);
+				
+			case Reorg:
+				return ReorgCPInstruction.parseInstruction(str);
+				
+			case UaggOuterChain:
+				return UaggOuterChainCPInstruction.parseInstruction(str);
+				
+			case MatrixReshape:
+				return MatrixReshapeCPInstruction.parseInstruction(str);	
+	
+			case Append:
+				return AppendCPInstruction.parseInstruction(str);
+				
+			case RelationalBinary:
+				return RelationalBinaryCPInstruction.parseInstruction(str);
+				
+			case File:
+				return FileCPInstruction.parseInstruction(str);
+				
+			case Variable:
+				return VariableCPInstruction.parseInstruction(str);
+				
+			case Rand:
+				return DataGenCPInstruction.parseInstruction(str);
+				
+			case StringInit:
+				return StringInitCPInstruction.parseInstruction(str);
+				
+			case External:
+				return FunctionCallCPInstruction.parseInstruction(str);
+				
+			case ParameterizedBuiltin: 
+				execType = ExecType.valueOf( str.split(Instruction.OPERAND_DELIM)[0] ); 
+				if( execType == ExecType.CP )
+					return ParameterizedBuiltinCPInstruction.parseInstruction(str);
+				else //exectype CP_FILE
+					return ParameterizedBuiltinCPFileInstruction.parseInstruction(str);
+	
+			case MultiReturnBuiltin:
+				return MultiReturnBuiltinCPInstruction.parseInstruction(str);
+				
+			case QSort: 
+				return QuantileSortCPInstruction.parseInstruction(str);
 			
-		case Variable:
-			return (CPInstruction) VariableCPInstruction.parseInstruction(str);
+			case QPick: 
+				return QuantilePickCPInstruction.parseInstruction(str);
 			
-		case Rand:
-			return (CPInstruction) DataGenCPInstruction.parseInstruction(str);
+			case MatrixIndexing: 
+				execType = ExecType.valueOf( str.split(Instruction.OPERAND_DELIM)[0] ); 
+				if( execType == ExecType.CP )
+					return MatrixIndexingCPInstruction.parseInstruction(str);
+				else //exectype CP_FILE
+					return MatrixIndexingCPFileInstruction.parseInstruction(str);
 			
-		case StringInit:
-			return (CPInstruction) StringInitCPInstruction.parseInstruction(str);
+			case Builtin: 
+				String []parts = InstructionUtils.getInstructionPartsWithValueType(str);
+				if ( parts[0].equals("log") || parts[0].equals("log_nz") ) {
+					if ( parts.length == 3 ) {
+						// B=log(A), y=log(x)
+						return BuiltinUnaryCPInstruction.parseInstruction(str);
+					} else if ( parts.length == 4 ) {
+						// B=log(A,10), y=log(x,10)
+						return BuiltinBinaryCPInstruction.parseInstruction(str);
+					}
+				}
+				else {
+					throw new DMLRuntimeException("Invalid Builtin Instruction: " + str );
+				}
+			case MMTSJ:
+				return MMTSJCPInstruction.parseInstruction(str);
 			
-		case External:
-			//return (CPInstruction) ExtBuiltinCPInstruction.parseInstruction(str);
-			return (CPInstruction) FunctionCallCPInstruction.parseInstruction(str);
+			case PMMJ:
+				return PMMJCPInstruction.parseInstruction(str);
 			
-		case ParameterizedBuiltin: 
-			execType = ExecType.valueOf( str.split(Instruction.OPERAND_DELIM)[0] ); 
-			if( execType == ExecType.CP )
-				return (CPInstruction) ParameterizedBuiltinCPInstruction.parseInstruction(str);
-			else //exectype CP_FILE
-				return (CPInstruction) ParameterizedBuiltinCPFileInstruction.parseInstruction(str);
-
-		case MultiReturnBuiltin:
-			return (CPInstruction) MultiReturnBuiltinCPInstruction.parseInstruction(str);
+			case MMChain:
+				return MMChainCPInstruction.parseInstruction(str);
 			
-		case QSort: 
-			return (CPInstruction) QuantileSortCPInstruction.parseInstruction(str);
-		
-		case QPick: 
-			return (CPInstruction) QuantilePickCPInstruction.parseInstruction(str);
-		
-		case MatrixIndexing: 
-			execType = ExecType.valueOf( str.split(Instruction.OPERAND_DELIM)[0] ); 
-			if( execType == ExecType.CP )
-				return (CPInstruction) MatrixIndexingCPInstruction.parseInstruction(str);
-			else //exectype CP_FILE
-				return (CPInstruction) MatrixIndexingCPFileInstruction.parseInstruction(str);
-		
-		case Builtin: 
-			String []parts = InstructionUtils.getInstructionPartsWithValueType(str);
-			if ( parts[0].equals("log") || parts[0].equals("log_nz") ) {
-				if ( parts.length == 3 ) {
-					// B=log(A), y=log(x)
-					return (CPInstruction) BuiltinUnaryCPInstruction.parseInstruction(str);
-				} else if ( parts.length == 4 ) {
-					// B=log(A,10), y=log(x,10)
-					return (CPInstruction) BuiltinBinaryCPInstruction.parseInstruction(str);
-				}
-			}
-			else {
-				throw new DMLRuntimeException("Invalid Builtin Instruction: " + str );
-			}
-		case MMTSJ:
-			return (CPInstruction) MMTSJCPInstruction.parseInstruction(str);
-		case PMMJ:
-			return (CPInstruction) PMMJCPInstruction.parseInstruction(str);
-		case MMChain:
-			return (CPInstruction) MMChainCPInstruction.parseInstruction(str);
-		
-		case Partition:
-			return (CPInstruction) DataPartitionCPInstruction.parseInstruction(str);	
-
-		case CentralMoment:
-			return (CPInstruction) CentralMomentCPInstruction.parseInstruction(str);
-
-		case Covariance:
-			return (CPInstruction) CovarianceCPInstruction.parseInstruction(str);
-			
-		case INVALID:
-		default: 
-			throw new DMLRuntimeException("Invalid CP Instruction Type: " + cptype );
-		}
-	}
+			case Partition:
+				return DataPartitionCPInstruction.parseInstruction(str);	
 	
-	public static CPInstruction[] parseMixedInstructions ( String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
-		if ( str == null || str.isEmpty() )
-			return null;
-		
-		Instruction[] inst = InstructionParser.parseMixedInstructions(str);
-		CPInstruction[] cpinst = new CPInstruction[inst.length];
-		for ( int i=0; i < inst.length; i++ ) {
-			cpinst[i] = (CPInstruction) inst[i];
+			case CentralMoment:
+				return CentralMomentCPInstruction.parseInstruction(str);
+	
+			case Covariance:
+				return CovarianceCPInstruction.parseInstruction(str);
+				
+			case INVALID:
+			
+			default: 
+				throw new DMLRuntimeException("Invalid CP Instruction Type: " + cptype );
 		}
-		
-		return cpinst;
 	}
-	
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/InstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/InstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/InstructionParser.java
index 040c77b..e00cde3 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/InstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/InstructionParser.java
@@ -28,10 +28,10 @@ import org.apache.sysml.runtime.instructions.spark.SPInstruction.SPINSTRUCTION_T
 
 
 public class InstructionParser 
-{
-		
-	public static Instruction parseSingleInstruction ( String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
-		
+{		
+	public static Instruction parseSingleInstruction ( String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{	
 		if ( str == null || str.isEmpty() )
 			return null;
 		
@@ -58,7 +58,9 @@ public class InstructionParser
 		}
 	}
 	
-	public static Instruction[] parseMixedInstructions ( String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static Instruction[] parseMixedInstructions ( String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		if ( str == null || str.isEmpty() )
 			return null;
 		
@@ -71,11 +73,4 @@ public class InstructionParser
 		
 		return inst;
 	}
-	
-	public static void printInstructions(Instruction[] instructions)
-	{
-		for(Instruction ins: instructions)
-			System.out.println(ins.toString());
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
index a28c00d..5aee907 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/MRInstructionParser.java
@@ -82,8 +82,7 @@ import org.apache.sysml.runtime.matrix.SortMR;
 
 
 public class MRInstructionParser extends InstructionParser 
-{
-	
+{	
 	static public HashMap<String, MRINSTRUCTION_TYPE> String2MRInstructionType;
 	static {
 		String2MRInstructionType = new HashMap<String, MRINSTRUCTION_TYPE>();
@@ -291,152 +290,158 @@ public class MRInstructionParser extends InstructionParser
 	}
 	
 	
-	public static MRInstruction parseSingleInstruction (String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static MRInstruction parseSingleInstruction (String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		if ( str == null || str.isEmpty() )
 			return null;
 		
 		MRINSTRUCTION_TYPE mrtype = InstructionUtils.getMRType(str); 
-		return MRInstructionParser.parseSingleInstruction(mrtype, str);
+		return parseSingleInstruction(mrtype, str);
 	}
 	
-	public static MRInstruction parseSingleInstruction (MRINSTRUCTION_TYPE mrtype, String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static MRInstruction parseSingleInstruction (MRINSTRUCTION_TYPE mrtype, String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		if ( str == null || str.isEmpty() )
 			return null;
 		
-		switch(mrtype) {
-		case Aggregate:
-			return (MRInstruction) AggregateInstruction.parseInstruction(str);
-			
-		case ArithmeticBinary: {
-			String opcode = InstructionUtils.getOpCode(str);
-			String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
-			// extract datatypes of first and second input operands
-			String dt1 = parts[1].split(Instruction.DATATYPE_PREFIX)[1].split(Instruction.VALUETYPE_PREFIX)[0];
-			String dt2 = parts[2].split(Instruction.DATATYPE_PREFIX)[1].split(Instruction.VALUETYPE_PREFIX)[0];
-			if ( dt1.equalsIgnoreCase("SCALAR") || dt2.equalsIgnoreCase("SCALAR") ) {
-				return (MRInstruction) ScalarInstruction.parseInstruction(str);
-			}
-			else {
-				if( BinaryM.isOpcode( opcode ) )
-					return (MRInstruction) BinaryMInstruction.parseInstruction(str);
-				else
-					return (MRInstruction) BinaryInstruction.parseInstruction(str);
+		switch(mrtype) 
+		{
+			case Aggregate:
+				return AggregateInstruction.parseInstruction(str);
+				
+			case ArithmeticBinary: {
+				String opcode = InstructionUtils.getOpCode(str);
+				String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
+				// extract datatypes of first and second input operands
+				String dt1 = parts[1].split(Instruction.DATATYPE_PREFIX)[1].split(Instruction.VALUETYPE_PREFIX)[0];
+				String dt2 = parts[2].split(Instruction.DATATYPE_PREFIX)[1].split(Instruction.VALUETYPE_PREFIX)[0];
+				if ( dt1.equalsIgnoreCase("SCALAR") || dt2.equalsIgnoreCase("SCALAR") ) {
+					return ScalarInstruction.parseInstruction(str);
+				}
+				else {
+					if( BinaryM.isOpcode( opcode ) )
+						return BinaryMInstruction.parseInstruction(str);
+					else
+						return BinaryInstruction.parseInstruction(str);
+				}
 			}
-		}
-		case AggregateBinary:
-			return (MRInstruction) AggregateBinaryInstruction.parseInstruction(str);
 			
-		case AggregateUnary:
-			return (MRInstruction) AggregateUnaryInstruction.parseInstruction(str);
+			case AggregateBinary:
+				return AggregateBinaryInstruction.parseInstruction(str);
+				
+			case AggregateUnary:
+				return AggregateUnaryInstruction.parseInstruction(str);
+				
+			case Ternary: 
+				return TernaryInstruction.parseInstruction(str);
 			
-		case Ternary: 
-			return (MRInstruction) TernaryInstruction.parseInstruction(str);
-		
-		case Quaternary: 
-			return (MRInstruction) QuaternaryInstruction.parseInstruction(str);
+			case Quaternary: 
+				return QuaternaryInstruction.parseInstruction(str);
+				
+			case Rand:
+				return RandInstruction.parseInstruction(str);
+				
+			case Seq:
+				return SeqInstruction.parseInstruction(str);
+				
+			case Reblock:
+				return ReblockInstruction.parseInstruction(str);
 			
-		case Rand:
-			return (MRInstruction) RandInstruction.parseInstruction(str);
+			case Append:
+				return AppendInstruction.parseInstruction(str);
+				
+			case Reorg:
+				return ReorgInstruction.parseInstruction(str);
+				
+			case Replicate:
+				return ReplicateInstruction.parseInstruction(str);
 			
-		case Seq:
-			return (MRInstruction) SeqInstruction.parseInstruction(str);
+			case Unary: {
+				String opcode = InstructionUtils.getOpCode(str);
+				String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
+				if( parts.length==4 && (opcode.equalsIgnoreCase("log") || opcode.equalsIgnoreCase("log_nz")) )
+					return ScalarInstruction.parseInstruction(str);
+				else //default case
+					return UnaryInstruction.parseInstruction(str);
+			}
+			case MMTSJ:
+				return MMTSJMRInstruction.parseInstruction(str);
 			
-		case Reblock:
-			return (MRInstruction) ReblockInstruction.parseInstruction(str);
-		
-		case Append:
-			return (MRInstruction) AppendInstruction.parseInstruction(str);
+			case PMMJ:
+				return PMMJMRInstruction.parseInstruction(str);
 			
-		case Reorg:
-			return (MRInstruction) ReorgInstruction.parseInstruction(str);
+			case MapMultChain:
+				return MapMultChainInstruction.parseInstruction(str);
 			
-		case Replicate:
-			return (MRInstruction) ReplicateInstruction.parseInstruction(str);
-		
-		case Unary: {
-			String opcode = InstructionUtils.getOpCode(str);
-			String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);
-			if( parts.length==4 && (opcode.equalsIgnoreCase("log") || opcode.equalsIgnoreCase("log_nz")) )
-				return (MRInstruction) ScalarInstruction.parseInstruction(str);
-			else //default case
-				return (MRInstruction) UnaryInstruction.parseInstruction(str);
-		}
-		case MMTSJ:
-			return (MRInstruction) MMTSJMRInstruction.parseInstruction(str);
-		
-		case PMMJ:
-			return (MRInstruction) PMMJMRInstruction.parseInstruction(str);
-		
-		case MapMultChain:
-			return (MRInstruction) MapMultChainInstruction.parseInstruction(str);
-		
-		case BinUaggChain:
-			return (MRInstruction) BinUaggChainInstruction.parseInstruction(str);
-		
-		case UaggOuterChain:
-			return (MRInstruction) UaggOuterChainInstruction.parseInstruction(str);
+			case BinUaggChain:
+				return BinUaggChainInstruction.parseInstruction(str);
 			
-		case CombineTernary:
-			return (MRInstruction) CombineTernaryInstruction.parseInstruction(str);
+			case UaggOuterChain:
+				return UaggOuterChainInstruction.parseInstruction(str);
+				
+			case CombineTernary:
+				return CombineTernaryInstruction.parseInstruction(str);
+				
+			case CombineBinary:
+				return CombineBinaryInstruction.parseInstruction(str);
+				
+			case CombineUnary:
+				return CombineUnaryInstruction.parseInstruction(str);
+				
+			case PickByCount:
+				return PickByCountInstruction.parseInstruction(str);
+				
+			case CM_N_COV:
+				return CM_N_COVInstruction.parseInstruction(str);
+		
+			case GroupedAggregate:
+				return GroupedAggregateInstruction.parseInstruction(str);
 			
-		case CombineBinary:
-			return (MRInstruction) CombineBinaryInstruction.parseInstruction(str);
+			case MapGroupedAggregate:
+				return GroupedAggregateMInstruction.parseInstruction(str);
 			
-		case CombineUnary:
-			return (MRInstruction) CombineUnaryInstruction.parseInstruction(str);
+			case RangeReIndex:
+				return RangeBasedReIndexInstruction.parseInstruction(str);
 			
-		case PickByCount:
-			return (MRInstruction) PickByCountInstruction.parseInstruction(str);
+			case ZeroOut:
+				return ZeroOutInstruction.parseInstruction(str);
 			
-		case CM_N_COV:
-			return (MRInstruction) CM_N_COVInstruction.parseInstruction(str);
-	
-		case GroupedAggregate:
-			return (MRInstruction) GroupedAggregateInstruction.parseInstruction(str);
-		
-		case MapGroupedAggregate:
-			return (MRInstruction) GroupedAggregateMInstruction.parseInstruction(str);
-		
-		case RangeReIndex:
-			return (MRInstruction) RangeBasedReIndexInstruction.parseInstruction(str);
-		
-		case ZeroOut:
-			return (MRInstruction) ZeroOutInstruction.parseInstruction(str);
-		
-		case MatrixReshape:
-			return (MRInstruction) MatrixReshapeMRInstruction.parseInstruction(str);	
-		
-		case Sort: //workaround for dummy MR sort instruction
-			return SortMR.parseSortInstruction(str);
-		
-		case CSVReblock:
-			return (MRInstruction)CSVReblockInstruction.parseInstruction(str);
+			case MatrixReshape:
+				return MatrixReshapeMRInstruction.parseInstruction(str);	
 			
-		case CSVWrite:
-			return (MRInstruction)CSVWriteInstruction.parseInstruction(str);
+			case Sort: //workaround for dummy MR sort instruction
+				return SortMR.parseSortInstruction(str);
 			
-		case ParameterizedBuiltin:
-			return (MRInstruction)ParameterizedBuiltinMRInstruction.parseInstruction(str);
-		
-		case RemoveEmpty:
-			return (MRInstruction)RemoveEmptyMRInstruction.parseInstruction(str);
+			case CSVReblock:
+				return CSVReblockInstruction.parseInstruction(str);
+				
+			case CSVWrite:
+				return CSVWriteInstruction.parseInstruction(str);
+				
+			case ParameterizedBuiltin:
+				return ParameterizedBuiltinMRInstruction.parseInstruction(str);
 			
-		case Partition:
-			return (MRInstruction)DataPartitionMRInstruction.parseInstruction(str);
+			case RemoveEmpty:
+				return RemoveEmptyMRInstruction.parseInstruction(str);
+				
+			case Partition:
+				return DataPartitionMRInstruction.parseInstruction(str);
+				
+			case CumsumAggregate:
+				return CumulativeAggregateInstruction.parseInstruction(str);
+				
+			case CumsumSplit:
+				return CumulativeSplitInstruction.parseInstruction(str);
 			
-		case CumsumAggregate:
-			return (MRInstruction)CumulativeAggregateInstruction.parseInstruction(str);
+			case CumsumOffset:
+				return CumulativeOffsetInstruction.parseInstruction(str);
 			
-		case CumsumSplit:
-			return (MRInstruction)CumulativeSplitInstruction.parseInstruction(str);
-		
-		case CumsumOffset:
-			return (MRInstruction)CumulativeOffsetInstruction.parseInstruction(str);
-		
-		case INVALID:
-		
-		default: 
-			throw new DMLRuntimeException("Invalid MR Instruction Type: " + mrtype );
+			case INVALID:
+			
+			default: 
+				throw new DMLRuntimeException("Invalid MR Instruction Type: " + mrtype );
 		}
 	}
 	
@@ -453,25 +458,6 @@ public class MRInstructionParser extends InstructionParser
 		return mrinst;
 	}
 	
-	// TODO: figure out if we need all the functions below 
-	
-	//unary operation contains scalar, transform, reorg, aggregate unary
-	public static UnaryInstruction[] parseUnaryInstructions(String str) throws DMLUnsupportedOperationException, DMLRuntimeException 
-	{
-		UnaryInstruction[] inst=null;
-		if(str!=null && !str.isEmpty())
-		{
-			String[] strlist = str.split(Instruction.INSTRUCTION_DELIM);
-			inst = new UnaryInstruction[strlist.length];
-			
-			for(int i=0; i < strlist.length; i++)
-			{
-				inst[i] = (UnaryInstruction) UnaryInstruction.parseInstruction( strlist[i] );
-			}
-		}
-		return inst;
-	}
-	
 	public static AggregateInstruction[] parseAggregateInstructions(String str) throws DMLUnsupportedOperationException, DMLRuntimeException 
 	{
 		AggregateInstruction[] inst=null;
@@ -621,5 +607,4 @@ public class MRInstructionParser extends InstructionParser
 		}
 		return inst;
 	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java b/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
index ea15019..1f5f961 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/SPInstructionParser.java
@@ -76,8 +76,8 @@ import org.apache.sysml.runtime.instructions.spark.WriteSPInstruction;
 import org.apache.sysml.runtime.instructions.spark.ZipmmSPInstruction;
 
 
-public class SPInstructionParser extends InstructionParser {
-	
+public class SPInstructionParser extends InstructionParser 
+{	
 	public static final HashMap<String, SPINSTRUCTION_TYPE> String2SPInstructionType;
 	static {
 		String2SPInstructionType = new HashMap<String, SPInstruction.SPINSTRUCTION_TYPE>();
@@ -253,7 +253,9 @@ public class SPInstructionParser extends InstructionParser {
 		String2SPInstructionType.put( "write"   , SPINSTRUCTION_TYPE.Write);
 	}
 
-	public static Instruction parseSingleInstruction (String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
+	public static SPInstruction parseSingleInstruction (String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{
 		if ( str == null || str.isEmpty() )
 			return null;
 
@@ -261,14 +263,15 @@ public class SPInstructionParser extends InstructionParser {
 		if ( cptype == null )
 			// return null;
 			throw new DMLUnsupportedOperationException("Invalid SP Instruction Type: " + str);
-		Instruction cpinst = SPInstructionParser.parseSingleInstruction(cptype, str);
-		if ( cpinst == null )
+		SPInstruction spinst = parseSingleInstruction(cptype, str);
+		if ( spinst == null )
 			throw new DMLRuntimeException("Unable to parse instruction: " + str);
-		return cpinst;
+		return spinst;
 	}
 	
-	public static Instruction parseSingleInstruction ( SPINSTRUCTION_TYPE sptype, String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
-		
+	public static SPInstruction parseSingleInstruction ( SPINSTRUCTION_TYPE sptype, String str ) 
+		throws DMLUnsupportedOperationException, DMLRuntimeException 
+	{	
 		if ( str == null || str.isEmpty() ) 
 			return null;
 		
@@ -305,10 +308,13 @@ public class SPInstructionParser extends InstructionParser {
 				
 			case MatrixIndexing:
 				return MatrixIndexingSPInstruction.parseInstruction(str);
+				
 			case Reorg:
 				return ReorgSPInstruction.parseInstruction(str);
+				
 			case ArithmeticBinary:
 				return ArithmeticBinarySPInstruction.parseInstruction(str);
+				
 			case RelationalBinary:
 				return RelationalBinarySPInstruction.parseInstruction(str);
 			
@@ -323,6 +329,7 @@ public class SPInstructionParser extends InstructionParser {
 			// Reblock instructions	
 			case Reblock:
 				return ReblockSPInstruction.parseInstruction(str);
+				
 			case CSVReblock:
 				return CSVReblockSPInstruction.parseInstruction(str);
 			
@@ -400,19 +407,4 @@ public class SPInstructionParser extends InstructionParser {
 				throw new DMLUnsupportedOperationException("Invalid SP Instruction Type: " + sptype );
 		}
 	}
-	
-	public static SPInstruction[] parseMixedInstructions ( String str ) throws DMLUnsupportedOperationException, DMLRuntimeException {
-		if ( str == null || str.isEmpty() )
-			return null;
-		
-		Instruction[] inst = InstructionParser.parseMixedInstructions(str);
-		SPInstruction[] cpinst = new SPInstruction[inst.length];
-		for ( int i=0; i < inst.length; i++ ) {
-			cpinst[i] = (SPInstruction) inst[i];
-		}
-		
-		return cpinst;
-	}
-	
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 41d89b8..4950df3 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
@@ -27,7 +27,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.Builtin;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -52,7 +51,7 @@ public class AggregateUnaryCPInstruction extends UnaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.AggregateUnary;		
 	}
 	
-	public static Instruction parseInstruction(String str)
+	public static AggregateUnaryCPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/AppendCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/AppendCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/AppendCPInstruction.java
index 58f0008..41cb68a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/AppendCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/AppendCPInstruction.java
@@ -24,7 +24,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.OffsetColumnIndex;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -50,7 +49,7 @@ public class AppendCPInstruction extends BinaryCPInstruction
 		_type = type;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static AppendCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/ArithmeticBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/ArithmeticBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/ArithmeticBinaryCPInstruction.java
index 525f6c5..c58b1f2 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/ArithmeticBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/ArithmeticBinaryCPInstruction.java
@@ -23,7 +23,6 @@ import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 
@@ -42,7 +41,9 @@ public abstract class ArithmeticBinaryCPInstruction extends BinaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.ArithmeticBinary;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException, DMLUnsupportedOperationException {
+	public static ArithmeticBinaryCPInstruction parseInstruction ( String str ) 
+		throws DMLRuntimeException, DMLUnsupportedOperationException 
+	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand in2 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand out = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanBinaryCPInstruction.java
index b85c653..ef28c10 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanBinaryCPInstruction.java
@@ -23,7 +23,6 @@ import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -38,7 +37,9 @@ public class BooleanBinaryCPInstruction extends BinaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.BooleanBinary;
 	}
 	
-	public static Instruction parseInstruction (String str) throws DMLRuntimeException {
+	public static BooleanBinaryCPInstruction parseInstruction (String str) 
+		throws DMLRuntimeException 
+	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand in2 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand out = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanUnaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanUnaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanUnaryCPInstruction.java
index 5d00203..2d7baaf 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanUnaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/BooleanUnaryCPInstruction.java
@@ -23,7 +23,6 @@ import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
 
@@ -37,7 +36,9 @@ public class BooleanUnaryCPInstruction extends UnaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.BooleanUnary;
 	}
 
-	public static Instruction parseInstruction (String str) throws DMLRuntimeException {
+	public static BooleanUnaryCPInstruction parseInstruction (String str) 
+		throws DMLRuntimeException 
+	{
 		CPOperand in = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand out = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		String opcode = parseUnaryInstruction(str, in, out);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinBinaryCPInstruction.java
index 1f6d716..89464ec 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinBinaryCPInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.Builtin;
 import org.apache.sysml.runtime.functionobjects.ValueFunction;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.RightScalarOperator;
@@ -47,7 +46,7 @@ public abstract class BuiltinBinaryCPInstruction extends BinaryCPInstruction
 		return arity;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static BuiltinBinaryCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException {
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand in2 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinUnaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinUnaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinUnaryCPInstruction.java
index b56db42..2980ef1 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinUnaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/BuiltinUnaryCPInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.Builtin;
 import org.apache.sysml.runtime.functionobjects.ValueFunction;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
@@ -48,7 +47,7 @@ public abstract class BuiltinUnaryCPInstruction extends UnaryCPInstruction
 		return arity;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static BuiltinUnaryCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		CPOperand in = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 55f1515..0b11d31 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
@@ -25,7 +25,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.CM;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.CMOperator;
@@ -39,7 +38,7 @@ public class CentralMomentCPInstruction extends AggregateUnaryCPInstruction
 		super(cm, in1, in2, in3, out, opcode, str);
 	}
 
-	public static Instruction parseInstruction(String str)
+	public static CentralMomentCPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 9ed2714..e7bb0cf 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
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.DataGen;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.LibMatrixDatagen;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -166,7 +165,8 @@ public class DataGenCPInstruction extends UnaryCPInstruction
 		this.sparsity = sparsity;
 	}
 
-	public static Instruction parseInstruction(String str) throws DMLRuntimeException 
+	public static DataGenCPInstruction parseInstruction(String str) 
+		throws DMLRuntimeException 
 	{
 		DataGenMethod method = DataGenMethod.INVALID;
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 650774d..16ee080 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
@@ -28,7 +28,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.ParForProgramBlock.PDataPartitionFormat;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.io.MatrixWriterFactory;
 import org.apache.sysml.runtime.io.WriterBinaryBlock;
@@ -61,7 +60,7 @@ public class DataPartitionCPInstruction extends UnaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static DataPartitionCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/FileCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/FileCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/FileCPInstruction.java
index cef69ac..43d5556 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/FileCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/FileCPInstruction.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.RemoveFile;
 import org.apache.sysml.runtime.functionobjects.RenameFile;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 import org.apache.sysml.runtime.matrix.operators.SimpleOperator;
@@ -67,7 +66,7 @@ public class FileCPInstruction extends CPInstruction
 			throw new DMLUnsupportedOperationException("Invalid function: " + str);
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static FileCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		String opcode = InstructionUtils.getOpCode(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
index f109540..7e8ac12 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/FunctionCallCPInstruction.java
@@ -81,8 +81,9 @@ public class FunctionCallCPInstruction extends CPInstruction
 	 * extFunct:::foo:::2:::1:::in1:::in2:::out1
 	 * 
 	 */
-	public static Instruction parseInstruction(String str) throws DMLRuntimeException, DMLUnsupportedOperationException {
-		
+	public static FunctionCallCPInstruction parseInstruction(String str) 
+		throws DMLRuntimeException, DMLUnsupportedOperationException 
+	{	
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
 		String namespace = parts[1];
 		String functionName = parts[2];

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 d2bf270..8d4600e 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
@@ -23,7 +23,6 @@ import org.apache.sysml.lops.MapMultChain.ChainType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -52,7 +51,7 @@ public class MMChainCPInstruction extends UnaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static MMChainCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		//parse instruction parts (without exec type)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 a73d7be..cc28230 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
@@ -23,7 +23,6 @@ import org.apache.sysml.lops.MMTSJ.MMTSJType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -52,7 +51,7 @@ public class MMTSJCPInstruction extends UnaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static MMTSJCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 d3a5d77..c03e7ae 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
@@ -25,7 +25,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -66,7 +65,7 @@ public class MatrixIndexingCPInstruction extends UnaryCPInstruction
 		colUpper = cu;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static MatrixIndexingCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException {
 		
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 e512153..8139c62 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
@@ -23,7 +23,6 @@ import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -56,7 +55,7 @@ public class MatrixReshapeCPInstruction extends UnaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static MatrixReshapeCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 59c3e02..e785375 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
@@ -20,13 +20,13 @@
 package org.apache.sysml.runtime.instructions.cp;
 
 import java.util.ArrayList;
+
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.LibCommonsMath;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -55,7 +55,7 @@ public class MultiReturnBuiltinCPInstruction extends ComputationCPInstruction
 		return _outputs.get(i);
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static MultiReturnBuiltinCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 e38d7fb..2961d21 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
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.cp;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -48,7 +47,7 @@ public class PMMJCPInstruction extends ComputationCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static PMMJCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 4dbc4a5..2245d4c 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
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.util.HashMap;
 
 import org.apache.wink.json4j.JSONException;
-
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.Statement;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -32,7 +31,6 @@ import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.ParameterizedBuiltin;
 import org.apache.sysml.runtime.functionobjects.ValueFunction;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.mr.GroupedAggregateInstruction;
 import org.apache.sysml.runtime.matrix.JobReturn;
@@ -74,7 +72,7 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 		return paramMap;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static ParameterizedBuiltinCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 bf25d99..ab48dd5 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
@@ -27,7 +27,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MetaData;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -60,7 +59,7 @@ public class QuantilePickCPInstruction extends BinaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static QuantilePickCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 a635645..3b748d6 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
@@ -25,7 +25,6 @@ import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -52,7 +51,7 @@ public class QuantileSortCPInstruction extends UnaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.QSort;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static QuantileSortCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
index faba42e..c749764 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cp/RelationalBinaryCPInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.cp;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 
@@ -36,7 +35,7 @@ public abstract class RelationalBinaryCPInstruction extends BinaryCPInstruction
 		_cptype = CPINSTRUCTION_TYPE.RelationalBinary;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static RelationalBinaryCPInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		InstructionUtils.checkNumFields (str, 3);
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);
 		CPOperand in2 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 9d72e2e..69ded28 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
@@ -27,7 +27,6 @@ import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.DiagIndex;
 import org.apache.sysml.runtime.functionobjects.SortIndex;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -75,7 +74,7 @@ public class ReorgCPInstruction extends UnaryCPInstruction
 		_ixret = ixret;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static ReorgCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		CPOperand in = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 791e65a..4d7c9e2 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
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.DataGen;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -62,7 +61,7 @@ public class StringInitCPInstruction extends UnaryCPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction(String str) 
+	public static StringInitCPInstruction parseInstruction(String str) 
 		throws DMLRuntimeException 
 	{
 		String opcode = InstructionUtils.getOpCode(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 94fc474..d03b817 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
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.functionobjects.ReduceAll;
 import org.apache.sysml.runtime.functionobjects.ReduceCol;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.LibMatrixOuterAgg;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -51,7 +50,7 @@ public class UaggOuterChainCPInstruction extends UnaryCPInstruction
 		instString = istr;
 	}
 
-	public static Instruction parseInstruction(String str)
+	public static UaggOuterChainCPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		String parts[] = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 9fa2ad0..5c1bbd7 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
@@ -236,7 +236,7 @@ public class VariableCPInstruction extends CPInstruction
 		}
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static VariableCPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{	
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 81d661d..352278c 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
@@ -24,7 +24,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.cp.MatrixIndexingCPInstruction;
@@ -57,7 +56,7 @@ public class MatrixIndexingCPFileInstruction extends MatrixIndexingCPInstruction
 		super( op, lhsInput, rhsInput, rl, ru, cl, cu, out, opcode, istr);
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static MatrixIndexingCPFileInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/cpfile/ParameterizedBuiltinCPFileInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/cpfile/ParameterizedBuiltinCPFileInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/cpfile/ParameterizedBuiltinCPFileInstruction.java
index 51b13de..e598e34 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/cpfile/ParameterizedBuiltinCPFileInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/cpfile/ParameterizedBuiltinCPFileInstruction.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.TextInputFormat;
-
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.parser.DMLTranslator;
 import org.apache.sysml.parser.Expression.DataType;
@@ -56,7 +55,6 @@ import org.apache.sysml.runtime.controlprogram.parfor.util.IDHandler;
 import org.apache.sysml.runtime.controlprogram.parfor.util.StagingFileUtils;
 import org.apache.sysml.runtime.functionobjects.ParameterizedBuiltin;
 import org.apache.sysml.runtime.functionobjects.ValueFunction;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.cp.ParameterizedBuiltinCPInstruction;
@@ -95,7 +93,7 @@ public class ParameterizedBuiltinCPFileInstruction extends ParameterizedBuiltinC
 	 * @throws DMLRuntimeException
 	 * @throws DMLUnsupportedOperationException
 	 */
-	public static Instruction parseInstruction( String str ) 
+	public static ParameterizedBuiltinCPFileInstruction parseInstruction( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateBinaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateBinaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateBinaryInstruction.java
index 703a188..d6c3d38 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateBinaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateBinaryInstruction.java
@@ -28,7 +28,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.Multiply;
 import org.apache.sysml.runtime.functionobjects.Plus;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -101,7 +100,7 @@ public class AggregateBinaryInstruction extends BinaryMRInstructionBase implemen
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static AggregateBinaryInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateInstruction.java
index 83f1617..06708b7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateInstruction.java
@@ -21,7 +21,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -40,7 +39,7 @@ public class AggregateInstruction extends UnaryMRInstructionBase
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static AggregateInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{	
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateUnaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateUnaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateUnaryInstruction.java
index 214c168..64ee42c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateUnaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AggregateUnaryInstruction.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.ReduceDiag;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
@@ -56,7 +55,7 @@ public class AggregateUnaryInstruction extends UnaryMRInstructionBase
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static AggregateUnaryInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 3 );
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendGInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendGInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendGInstruction.java
index a5ac898..e67291a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendGInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendGInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
@@ -47,7 +46,7 @@ public class AppendGInstruction extends AppendInstruction
 		_len = _offset + _offset2;
 	}
 
-	public static Instruction parseInstruction ( String str ) 
+	public static AppendGInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendInstruction.java
index 72a7f04..0efe595 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -60,7 +59,7 @@ public class AppendInstruction extends BinaryMRInstructionBase
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static AppendInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String opcode = InstructionUtils.getOpCode(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendMInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendMInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendMInstruction.java
index 10bd584..ad13fb6 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendMInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendMInstruction.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.sysml.lops.AppendM.CacheType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -46,7 +45,7 @@ public class AppendMInstruction extends AppendInstruction implements IDistribute
 		_offset = offset;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static AppendMInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendRInstruction.java
index dac4621..41cbb06 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/AppendRInstruction.java
@@ -21,7 +21,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -36,7 +35,7 @@ public class AppendRInstruction extends AppendInstruction
 		super(op, in1, in2, out, cbind, istr);
 	}
 
-	public static Instruction parseInstruction ( String str ) 
+	public static AppendRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/BinUaggChainInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinUaggChainInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinUaggChainInstruction.java
index 5af13b4..9f8bafc 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinUaggChainInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinUaggChainInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
@@ -77,7 +76,7 @@ public class BinUaggChainInstruction extends UnaryInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction( String str ) 
+	public static BinUaggChainInstruction parseInstruction( String str ) 
 		throws DMLRuntimeException 
 	{		
 		//check number of fields (2/3 inputs, output, type)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryInstruction.java
index e9a2410..d133a90 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryInstruction.java
@@ -21,7 +21,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -42,8 +41,9 @@ public class BinaryInstruction extends BinaryMRInstructionBase
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
-		
+	public static BinaryInstruction parseInstruction ( String str ) 
+		throws DMLRuntimeException 
+	{	
 		InstructionUtils.checkNumFields ( str, 3 );
 		
 		String[] parts = InstructionUtils.getInstructionParts ( str );



[4/4] incubator-systemml git commit: Improved naive bayes script (matrix aggregate, cleanup), incl new tests

Posted by mb...@apache.org.
Improved naive bayes script (matrix aggregate, cleanup), incl new tests

Results performance testsuite old/new naive bayes scripts (including
invocation overhead):
a) Hybrid Spark (20GB driver)
10k x 1k, dense: 19s -> 2s 
10k x 1k, sparse: 19s -> 1s
100k x 1k, dense: 22s -> 4s
100k x 1k, sparse: 23s -> 2s
1M x 1k, dense: 87s -> 42s
1M x 1k, sparse: 38s -> 4s
10M x 1k, dense: 226s -> 81s
10M x 1k, sparse: 369s -> 40s
 
b) Hybrid MapReduce (2GB client)
10k x 1k, dense: 3s -> 3s
10k x 1k, sparse: 1s -> 1s
100k x 1k, dense: 58s -> 23s
100k x 1k, sparse: 5s -> 3s
1M x 1k, dense: 98s -> 84s
1M x 1k, sparse: 44s -> 9s
10M x 1k, dense: 253s -> 139s
10M x 1k, sparse: X -> 80s
 





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

Branch: refs/heads/master
Commit: 8e7b6ed3df1c92eda2f0bf1390d1cdd8b59dc7e9
Parents: 5878e1d
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Jan 1 19:11:55 2016 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Fri Jan 1 19:11:55 2016 -0800

----------------------------------------------------------------------
 scripts/algorithms/naive-bayes.dml              |  13 +-
 .../algorithms/obsolete/naive-bayes-parfor.dml  | 100 +++++++++++++++
 .../applications/NaiveBayesParforTest.java      | 125 +++++++++++++++++++
 .../applications/NaiveBayesTest.java            |   2 +-
 .../dml/NaiveBayesParforDMLTest.java            |  40 ++++++
 .../pydml/NaiveBayesParforPyDMLTest.java        |  40 ++++++
 .../naive-bayes-parfor/naive-bayes.R            |  71 +++++++++++
 .../naive-bayes-parfor/naive-bayes.dml          |  78 ++++++++++++
 .../naive-bayes-parfor/naive-bayes.pydml        |  79 ++++++++++++
 .../applications/naive-bayes-parfor/readme.txt  |   1 +
 .../applications/naive-bayes/naive-bayes.dml    |  10 +-
 .../applications/naive-bayes/naive-bayes.pydml  |   9 +-
 .../integration/applications/ZPackageSuite.java |   2 +
 .../functions/aggregate/ZPackageSuite.java      |   1 +
 14 files changed, 548 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/scripts/algorithms/naive-bayes.dml
----------------------------------------------------------------------
diff --git a/scripts/algorithms/naive-bayes.dml b/scripts/algorithms/naive-bayes.dml
index c90e64c..f2ad2ed 100644
--- a/scripts/algorithms/naive-bayes.dml
+++ b/scripts/algorithms/naive-bayes.dml
@@ -58,11 +58,7 @@ numFeatures = ncol(D)
 # Compute conditionals
 
 # Compute the feature counts for each class
-classFeatureCounts = matrix(0, rows=numClasses, cols=numFeatures)
-parfor (i in 1:numFeatures) {
-  Col = D[,i]
-  classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=as.integer(numClasses))
-}
+classFeatureCounts = aggregate(target=D, groups=C, fn="sum", ngroups=as.integer(numClasses))
 
 # Compute the total feature count for each class 
 # and add the number of features to this sum
@@ -81,8 +77,8 @@ class_prior = class_counts / numRows;
 
 # Compute accuracy on training set
 ones = matrix(1, rows=numRows, cols=1)
-D_w_ones = append(D, ones)
-model = append(class_conditionals, class_prior)
+D_w_ones = cbind(D, ones)
+model = cbind(class_conditionals, class_prior)
 log_probs = D_w_ones %*% t(log(model))
 pred = rowIndexMax(log_probs)
 acc = sum(ppred(pred, C, "==")) / numRows * 100
@@ -91,8 +87,7 @@ acc_str = "Training Accuracy (%): " + acc
 print(acc_str)
 write(acc_str, $accuracy)
 
-extra_model_params = matrix(0, rows=1, cols=1)
-extra_model_params[1, 1] = numFeatures
+extra_model_params = as.matrix(numFeatures)
 class_prior = t(append(t(class_prior), extra_model_params))
 
 # write out the model

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/scripts/algorithms/obsolete/naive-bayes-parfor.dml
----------------------------------------------------------------------
diff --git a/scripts/algorithms/obsolete/naive-bayes-parfor.dml b/scripts/algorithms/obsolete/naive-bayes-parfor.dml
new file mode 100644
index 0000000..c90e64c
--- /dev/null
+++ b/scripts/algorithms/obsolete/naive-bayes-parfor.dml
@@ -0,0 +1,100 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+# Implements multinomial naive Bayes classifier with Laplace correction
+#
+# Example Usage:
+# hadoop jar SystemML.jar -f naive-bayes.dml -nvargs X=<Data> Y=<labels> laplace=<Laplace Correction> prior=<Model file1> conditionals=<Model file2> accuracy=<accuracy file> fmt="text"
+#
+
+# defaults
+cmdLine_laplace = ifdef($laplace, 1)
+cmdLine_fmt = ifdef($fmt, "text")
+
+# reading input args
+D = read($X)
+min_feature_val = min(D)
+if(min_feature_val < 0)
+	stop("Stopping due to invalid argument: Multinomial naive Bayes is meant for count-based feature values, minimum value in X is negative")
+numRows = nrow(D)
+if(numRows < 2)
+	stop("Stopping due to invalid inputs: Not possible to learn a classifier without at least 2 rows")
+
+C = read($Y)
+if(min(C) < 1)
+	stop("Stopping due to invalid argument: Label vector (Y) must be recoded")
+numClasses = max(C)
+if(numClasses == 1)
+	stop("Stopping due to invalid argument: Maximum label value is 1, need more than one class to learn a multi-class classifier")	
+mod1 = C %% 1
+mod1_should_be_nrow = sum(abs(ppred(mod1, 0, "==")))
+if(mod1_should_be_nrow != numRows)
+	stop("Stopping due to invalid argument: Please ensure that Y contains (positive) integral labels")
+	
+laplace_correction = cmdLine_laplace
+if(laplace_correction < 0)
+	stop("Stopping due to invalid argument: Laplacian correction (laplace) must be non-negative")
+
+numFeatures = ncol(D)
+
+# Compute conditionals
+
+# Compute the feature counts for each class
+classFeatureCounts = matrix(0, rows=numClasses, cols=numFeatures)
+parfor (i in 1:numFeatures) {
+  Col = D[,i]
+  classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=as.integer(numClasses))
+}
+
+# Compute the total feature count for each class 
+# and add the number of features to this sum
+# for subsequent regularization (Laplace's rule)
+classSums = rowSums(classFeatureCounts) + numFeatures*laplace_correction
+
+# Compute class conditional probabilities
+#ones = matrix(1, rows=1, cols=numFeatures)
+#repClassSums = classSums %*% ones
+#class_conditionals = (classFeatureCounts + laplace_correction) / repClassSums
+class_conditionals = (classFeatureCounts + laplace_correction) / classSums
+
+# Compute class priors
+class_counts = aggregate(target=C, groups=C, fn="count", ngroups=as.integer(numClasses))
+class_prior = class_counts / numRows;
+
+# Compute accuracy on training set
+ones = matrix(1, rows=numRows, cols=1)
+D_w_ones = append(D, ones)
+model = append(class_conditionals, class_prior)
+log_probs = D_w_ones %*% t(log(model))
+pred = rowIndexMax(log_probs)
+acc = sum(ppred(pred, C, "==")) / numRows * 100
+
+acc_str = "Training Accuracy (%): " + acc
+print(acc_str)
+write(acc_str, $accuracy)
+
+extra_model_params = matrix(0, rows=1, cols=1)
+extra_model_params[1, 1] = numFeatures
+class_prior = t(append(t(class_prior), extra_model_params))
+
+# write out the model
+write(class_prior, $prior, format=cmdLine_fmt);
+write(class_conditionals, $conditionals, format=cmdLine_fmt);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesParforTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesParforTest.java b/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesParforTest.java
new file mode 100644
index 0000000..7387e16
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesParforTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.applications;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+
+import org.junit.runners.Parameterized.Parameters;
+
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+
+public abstract class NaiveBayesParforTest  extends AutomatedTestBase{
+	
+	protected final static String TEST_DIR = "applications/naive-bayes-parfor/";
+	protected final static String TEST_NAME = "naive-bayes";
+	protected String TEST_CLASS_DIR = TEST_DIR + NaiveBayesParforTest.class.getSimpleName() + "/";
+
+	protected int numRecords, numFeatures, numClasses;
+    protected double sparsity;
+    
+    public NaiveBayesParforTest(int rows, int cols, int nc, double sp) {
+		numRecords = rows;
+		numFeatures = cols;
+		numClasses = nc;
+		sparsity = sp;
+	}
+    
+    @Parameters
+	 public static Collection<Object[]> data() {
+	   Object[][] data = new Object[][] { 
+			   //sparse tests (sparsity=0.01)
+			   {100, 50, 10, 0.01}, // example running time: 3.5s (dml: .3s)
+			   {1000, 500, 10, 0.01}, // example running time: 5s (dml: .8s)
+			   {10000, 750, 10, 0.01}, // example running time: 32s (dml: .7s)
+			   //{100000, 1000, 10, 0.01}, // example running time: 471s (dml: 3s)
+			   //dense tests (sparsity=0.7)
+			   {100, 50, 10, 0.7}, // example running time: 2s (dml: .2s)
+			   {1000, 500, 10, 0.7}, // example running time: 6s (dml: .7s)
+			   {10000, 750, 10, 0.7} // example running time: 61s (dml: 5.6s)
+			   };
+	   return Arrays.asList(data);
+	 }
+	 
+	 @Override
+	 public void setUp() {
+		 addTestConfiguration(TEST_CLASS_DIR, TEST_NAME);
+	 }
+	 
+	 protected void testNaiveBayes(ScriptType scriptType)
+	 {
+		 System.out.println("------------ BEGIN " + TEST_NAME + " " + scriptType + " TEST {" + numRecords + ", "
+					+ numFeatures + ", " + numClasses + ", " + sparsity + "} ------------");
+		 this.scriptType = scriptType;
+		 
+		 int rows = numRecords;
+		 int cols = numFeatures;
+		 int classes = numClasses;
+		 double sparsity = this.sparsity;
+		 double laplace_correction = 1;
+	        
+		 getAndLoadTestConfiguration(TEST_NAME);
+	     
+		 List<String> proArgs = new ArrayList<String>();
+		 if (scriptType == ScriptType.PYDML) {
+			 proArgs.add("-python");
+		 }
+		 proArgs.add("-stats");
+		 proArgs.add("-nvargs");
+		 proArgs.add("X=" + input("X"));
+		 proArgs.add("Y=" + input("Y"));
+		 proArgs.add("classes=" + classes);
+		 proArgs.add("laplace=" + laplace_correction);
+		 proArgs.add("prior=" + output("prior"));
+		 proArgs.add("conditionals=" + output("conditionals"));
+		 proArgs.add("accuracy=" + output("accuracy"));
+		 programArgs = proArgs.toArray(new String[proArgs.size()]);
+		
+		 fullDMLScriptName = getScript();
+
+		 rCmd = getRCmd(inputDir(), Integer.toString(classes), Double.toString(laplace_correction), expectedDir());
+		 
+		 double[][] X = getRandomMatrix(rows, cols, 0, 1, sparsity, -1);
+		 double[][] Y = getRandomMatrix(rows, 1, 0, 1, 1, -1);
+		 for(int i=0; i<rows; i++){
+			 Y[i][0] = (int)(Y[i][0]*classes) + 1;
+			 Y[i][0] = (Y[i][0] > classes) ? classes : Y[i][0];
+	     }	
+	        
+		 writeInputMatrixWithMTD("X", X, true);
+		 writeInputMatrixWithMTD("Y", Y, true);
+	        
+		 runTest(true, EXCEPTION_NOT_EXPECTED, null, -1);
+	        
+		 runRScript(true);
+	        
+		 HashMap<CellIndex, Double> priorR = readRMatrixFromFS("prior");
+		 HashMap<CellIndex, Double> priorSYSTEMML= readDMLMatrixFromHDFS("prior");
+		 HashMap<CellIndex, Double> conditionalsR = readRMatrixFromFS("conditionals");
+		 HashMap<CellIndex, Double> conditionalsSYSTEMML = readDMLMatrixFromHDFS("conditionals"); 
+		 TestUtils.compareMatrices(priorR, priorSYSTEMML, Math.pow(10, -12), "priorR", "priorSYSTEMML");
+		 TestUtils.compareMatrices(conditionalsR, conditionalsSYSTEMML, Math.pow(10.0, -12.0), "conditionalsR", "conditionalsSYSTEMML");
+	 }
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesTest.java b/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesTest.java
index c4bdd53..8030cb9 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/NaiveBayesTest.java
@@ -54,7 +54,7 @@ public abstract class NaiveBayesTest  extends AutomatedTestBase{
 			   {100, 50, 10, 0.01}, // example running time: 3.5s (dml: .3s)
 			   {1000, 500, 10, 0.01}, // example running time: 5s (dml: .8s)
 			   {10000, 750, 10, 0.01}, // example running time: 32s (dml: .7s)
-			   {100000, 1000, 10, 0.01}, // example running time: 471s (dml: 3s)
+			   //{100000, 1000, 10, 0.01}, // example running time: 471s (dml: 3s)
 			   //dense tests (sparsity=0.7)
 			   {100, 50, 10, 0.7}, // example running time: 2s (dml: .2s)
 			   {1000, 500, 10, 0.7}, // example running time: 6s (dml: .7s)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
new file mode 100644
index 0000000..8320867
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.applications.dml;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.apache.sysml.test.integration.applications.NaiveBayesParforTest;
+
+@RunWith(value = Parameterized.class)
+public class NaiveBayesParforDMLTest extends NaiveBayesParforTest {
+
+	public NaiveBayesParforDMLTest(int rows, int cols, int nc, double sp) {
+		super(rows, cols, nc, sp);
+		TEST_CLASS_DIR = TEST_DIR + NaiveBayesParforDMLTest.class.getSimpleName() + "/";
+	}
+
+	@Test
+	public void testNaiveBayesDml() {
+		testNaiveBayes(ScriptType.DML);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/java/org/apache/sysml/test/integration/applications/pydml/NaiveBayesParforPyDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/pydml/NaiveBayesParforPyDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/pydml/NaiveBayesParforPyDMLTest.java
new file mode 100644
index 0000000..cb6538f
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/applications/pydml/NaiveBayesParforPyDMLTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.applications.pydml;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.apache.sysml.test.integration.applications.NaiveBayesParforTest;
+
+@RunWith(value = Parameterized.class)
+public class NaiveBayesParforPyDMLTest extends NaiveBayesParforTest {
+
+	public NaiveBayesParforPyDMLTest(int rows, int cols, int nc, double sp) {
+		super(rows, cols, nc, sp);
+		TEST_CLASS_DIR = TEST_DIR + NaiveBayesParforPyDMLTest.class.getSimpleName() + "/";
+	}
+
+	@Test
+	public void testNaiveBayesPyDml() {
+		testNaiveBayes(ScriptType.PYDML);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.R b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.R
new file mode 100644
index 0000000..dc65b8a
--- /dev/null
+++ b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.R
@@ -0,0 +1,71 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+args <- commandArgs(TRUE)
+
+library("Matrix")
+
+D = as.matrix(readMM(paste(args[1], "X.mtx", sep="")))
+C = as.matrix(readMM(paste(args[1], "Y.mtx", sep="")))
+
+# reading input args
+numClasses = as.integer(args[2]);
+laplace_correction = as.double(args[3]);
+
+numRows = nrow(D)
+numFeatures = ncol(D)
+
+# Compute conditionals
+
+# Compute the feature counts for each class
+classFeatureCounts = matrix(0, numClasses, numFeatures)
+for (i in 1:numFeatures) {
+  Col = D[,i]
+  classFeatureCounts[,i] = aggregate(as.vector(Col), by=list(as.vector(C)), FUN=sum)[,2];
+}
+
+# Compute the total feature count for each class 
+# and add the number of features to this sum
+# for subsequent regularization (Laplace's rule)
+classSums = rowSums(classFeatureCounts) + numFeatures*laplace_correction
+
+# Compute class conditional probabilities
+ones = matrix(1, 1, numFeatures)
+repClassSums = classSums %*% ones;
+class_conditionals = (classFeatureCounts + laplace_correction) / repClassSums;
+
+# Compute class priors
+class_counts = aggregate(as.vector(C), by=list(as.vector(C)), FUN=length)[,2]
+class_prior = class_counts / numRows;
+
+# Compute accuracy on training set
+ones = matrix(1, numRows, 1)
+D_w_ones = cbind(D, ones)
+model = cbind(class_conditionals, class_prior)
+log_probs = D_w_ones %*% t(log(model))
+pred = max.col(log_probs,ties.method="last");
+acc = sum(pred == C) / numRows * 100
+
+print(paste("Training Accuracy (%): ", acc, sep=""))
+
+# write out the model
+writeMM(as(class_prior, "CsparseMatrix"), paste(args[4], "prior", sep=""));
+writeMM(as(class_conditionals, "CsparseMatrix"), paste(args[4], "conditionals", sep=""));

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.dml b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.dml
new file mode 100644
index 0000000..d0ce8d1
--- /dev/null
+++ b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.dml
@@ -0,0 +1,78 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+# Implements multinomial naive Bayes classifier with Laplace correction
+#
+# Example Usage:
+# hadoop jar SystemML.jar -f naive-bayes.dml -nvargs X=<Data> Y=<labels> classes=<Num Classes> laplace=<Laplace Correction> prior=<Model file1> conditionals=<Model file2> accuracy=<accuracy file> fmt="text"
+#
+
+# defaults
+# $laplace = 1
+fmt = ifdef($fmt, "text")
+
+# reading input args
+numClasses = $classes
+D = read($X)
+C = read($Y)
+laplace_correction = ifdef($laplace, 1)
+
+numRows = nrow(D)
+numFeatures = ncol(D)
+
+# Compute conditionals
+
+# Compute the feature counts for each class
+classFeatureCounts = matrix(0, rows=numClasses, cols=numFeatures)
+parfor (i in 1:numFeatures) {
+  Col = D[,i]
+  classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=as.integer(numClasses))
+}
+
+# Compute the total feature count for each class 
+# and add the number of features to this sum
+# for subsequent regularization (Laplace's rule)
+classSums = rowSums(classFeatureCounts) + numFeatures*laplace_correction
+
+# Compute class conditional probabilities
+ones = matrix(1, rows=1, cols=numFeatures)
+repClassSums = classSums %*% ones
+class_conditionals = (classFeatureCounts + laplace_correction) / repClassSums
+
+# Compute class priors
+class_counts = aggregate(target=C, groups=C, fn="count", ngroups=as.integer(numClasses))
+class_prior = class_counts / numRows;
+
+# Compute accuracy on training set
+ones = matrix(1, rows=numRows, cols=1)
+D_w_ones = append(D, ones)
+model = append(class_conditionals, class_prior)
+log_probs = D_w_ones %*% t(log(model))
+pred = rowIndexMax(log_probs)
+acc = sum(ppred(pred, C, "==")) / numRows * 100
+
+acc_str = "Training Accuracy (%): " + acc
+print(acc_str)
+write(acc_str, $accuracy)
+
+# write out the model
+write(class_prior, $prior, format=fmt);
+write(class_conditionals, $conditionals, format=fmt);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
new file mode 100644
index 0000000..5d84951
--- /dev/null
+++ b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
@@ -0,0 +1,79 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+# Implements multinomial naive Bayes classifier with Laplace correction
+#
+# Example Usage:
+# hadoop jar SystemML.jar -f naive-bayes.pydml -python -nvargs X=<Data> Y=<labels> classes=<Num Classes> laplace=<Laplace Correction> prior=<Model file1> conditionals=<Model file2> accuracy=<accuracy file> fmt="text"
+#
+
+# defaults
+# $laplace = 1
+fmt = ifdef($fmt, "text")
+
+# reading input args
+numClasses = $classes
+D = load($X)
+C = load($Y)
+laplace_correction = ifdef($laplace, 1)
+
+numRows = nrow(D)
+numFeatures = ncol(D)
+
+# Compute conditionals
+
+# Compute the feature counts for each class
+classFeatureCounts = full(0, rows=numClasses, cols=numFeatures)
+parfor (i in 1:numFeatures):
+    Col = D[,i]
+    classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=numClasses)
+
+# Compute the total feature count for each class 
+# and add the number of features to this sum
+# for subsequent regularization (Laplace's rule)
+classSums = rowSums(classFeatureCounts) + numFeatures*laplace_correction
+
+# Compute class conditional probabilities
+ones = full(1, rows=1, cols=numFeatures)
+repClassSums = dot(classSums, ones)
+class_conditionals = (classFeatureCounts + laplace_correction) / repClassSums
+
+# Compute class priors
+class_counts = aggregate(target=C, groups=C, fn="count", ngroups=numClasses)
+class_prior = class_counts / numRows
+
+# Compute accuracy on training set
+ones = full(1, rows=numRows, cols=1)
+D_w_ones = append(D, ones)
+model = append(class_conditionals, class_prior)
+log_model = log(model)
+transpose_log_model = log_model.transpose()
+log_probs = dot(D_w_ones, transpose_log_model)
+pred = rowIndexMax(log_probs)
+acc = sum(ppred(pred, C, "==")) / numRows * 100
+
+acc_str = "Training Accuracy (%): " + acc
+print(acc_str)
+save(acc_str, $accuracy)
+
+# write out the model
+save(class_prior, $prior, format=fmt)
+save(class_conditionals, $conditionals, format=fmt)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes-parfor/readme.txt
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes-parfor/readme.txt b/src/test/scripts/applications/naive-bayes-parfor/readme.txt
new file mode 100644
index 0000000..b68e0ce
--- /dev/null
+++ b/src/test/scripts/applications/naive-bayes-parfor/readme.txt
@@ -0,0 +1 @@
+This is the old naive bayes script, implemented via parfor. With the new grouped aggregate over matrices, this is not required anymore. However, for testing purposes we run both naive bayes scripts.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes/naive-bayes.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes/naive-bayes.dml b/src/test/scripts/applications/naive-bayes/naive-bayes.dml
index d0ce8d1..9b1558c 100644
--- a/src/test/scripts/applications/naive-bayes/naive-bayes.dml
+++ b/src/test/scripts/applications/naive-bayes/naive-bayes.dml
@@ -41,11 +41,7 @@ numFeatures = ncol(D)
 # Compute conditionals
 
 # Compute the feature counts for each class
-classFeatureCounts = matrix(0, rows=numClasses, cols=numFeatures)
-parfor (i in 1:numFeatures) {
-  Col = D[,i]
-  classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=as.integer(numClasses))
-}
+classFeatureCounts = aggregate(target=D, groups=C, fn="sum", ngroups=as.integer(numClasses));
 
 # Compute the total feature count for each class 
 # and add the number of features to this sum
@@ -63,8 +59,8 @@ class_prior = class_counts / numRows;
 
 # Compute accuracy on training set
 ones = matrix(1, rows=numRows, cols=1)
-D_w_ones = append(D, ones)
-model = append(class_conditionals, class_prior)
+D_w_ones = cbind(D, ones)
+model = cbind(class_conditionals, class_prior)
 log_probs = D_w_ones %*% t(log(model))
 pred = rowIndexMax(log_probs)
 acc = sum(ppred(pred, C, "==")) / numRows * 100

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test/scripts/applications/naive-bayes/naive-bayes.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes/naive-bayes.pydml b/src/test/scripts/applications/naive-bayes/naive-bayes.pydml
index 5d84951..073e1cb 100644
--- a/src/test/scripts/applications/naive-bayes/naive-bayes.pydml
+++ b/src/test/scripts/applications/naive-bayes/naive-bayes.pydml
@@ -41,10 +41,7 @@ numFeatures = ncol(D)
 # Compute conditionals
 
 # Compute the feature counts for each class
-classFeatureCounts = full(0, rows=numClasses, cols=numFeatures)
-parfor (i in 1:numFeatures):
-    Col = D[,i]
-    classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=numClasses)
+classFeatureCounts = aggregate(target=D, groups=C, fn="sum", ngroups=numClasses);
 
 # Compute the total feature count for each class 
 # and add the number of features to this sum
@@ -62,8 +59,8 @@ class_prior = class_counts / numRows
 
 # Compute accuracy on training set
 ones = full(1, rows=numRows, cols=1)
-D_w_ones = append(D, ones)
-model = append(class_conditionals, class_prior)
+D_w_ones = cbind(D, ones)
+model = cbind(class_conditionals, class_prior)
 log_model = log(model)
 transpose_log_model = log_model.transpose()
 log_probs = dot(D_w_ones, transpose_log_model)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test_suites/java/org/apache/sysml/test/integration/applications/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/applications/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/applications/ZPackageSuite.java
index 7dcb816..f9d1e91 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/applications/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/applications/ZPackageSuite.java
@@ -43,6 +43,7 @@ import org.junit.runners.Suite;
   org.apache.sysml.test.integration.applications.dml.MDABivariateStatsDMLTest.class,
   org.apache.sysml.test.integration.applications.dml.MultiClassSVMDMLTest.class,
   org.apache.sysml.test.integration.applications.dml.NaiveBayesDMLTest.class,
+  org.apache.sysml.test.integration.applications.dml.NaiveBayesParforDMLTest.class,
   org.apache.sysml.test.integration.applications.dml.PageRankDMLTest.class,
   org.apache.sysml.test.integration.applications.dml.WelchTDMLTest.class,
 
@@ -61,6 +62,7 @@ import org.junit.runners.Suite;
   org.apache.sysml.test.integration.applications.pydml.MDABivariateStatsPyDMLTest.class,
   org.apache.sysml.test.integration.applications.pydml.MultiClassSVMPyDMLTest.class,
   org.apache.sysml.test.integration.applications.pydml.NaiveBayesPyDMLTest.class,
+  org.apache.sysml.test.integration.applications.pydml.NaiveBayesParforPyDMLTest.class,
   org.apache.sysml.test.integration.applications.pydml.PageRankPyDMLTest.class,
   org.apache.sysml.test.integration.applications.pydml.WelchTPyDMLTest.class
   

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/8e7b6ed3/src/test_suites/java/org/apache/sysml/test/integration/functions/aggregate/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/aggregate/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/aggregate/ZPackageSuite.java
index bf284fd..39432c5 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/aggregate/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/aggregate/ZPackageSuite.java
@@ -44,6 +44,7 @@ import org.junit.runners.Suite;
 	FullAggregateTest.class,
 	FullColAggregateTest.class,
 	FullGroupedAggregateTest.class,
+	FullGroupedAggregateMatrixTest.class,
 	FullRowAggregateTest.class
 })
 


[2/4] incubator-systemml git commit: Cleanup instruction parsing (removed unnecessary casts, unused methods)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryMInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryMInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryMInstruction.java
index 2bdd9c2..b24f1c3 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryMInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/BinaryMInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.AppendM.CacheType;
 import org.apache.sysml.lops.BinaryM.VectorType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.OperationsOnMatrixValues;
@@ -56,7 +55,7 @@ public class BinaryMInstruction extends BinaryMRInstructionBase implements IDist
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static BinaryMInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{	
 		InstructionUtils.checkNumFields ( str, 5 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CM_N_COVInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CM_N_COVInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CM_N_COVInstruction.java
index 1eeb63a..6293f2f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CM_N_COVInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CM_N_COVInstruction.java
@@ -23,7 +23,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.CM;
 import org.apache.sysml.runtime.functionobjects.COV;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -44,8 +43,9 @@ public class CM_N_COVInstruction extends UnaryMRInstructionBase
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
-		
+	public static CM_N_COVInstruction parseInstruction ( String str ) 
+		throws DMLRuntimeException 
+	{	
 		String[] parts = InstructionUtils.getInstructionParts ( str );
 		
 		byte in, out;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVReblockInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVReblockInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVReblockInstruction.java
index 82b911c..b49b4be 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVReblockInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVReblockInstruction.java
@@ -59,7 +59,7 @@ public class CSVReblockInstruction extends ReblockInstruction
 		return parseInstruction(sb.toString());
 		
 	}
-	public static Instruction parseInstruction(String str) {
+	public static CSVReblockInstruction parseInstruction(String str) {
 		Operator op = null;
 		byte input, output;
 		String[] s=str.split(Instruction.OPERAND_DELIM);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVWriteInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVWriteInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVWriteInstruction.java
index 5f68771..8baf617 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVWriteInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CSVWriteInstruction.java
@@ -44,7 +44,7 @@ public class CSVWriteInstruction extends UnaryMRInstructionBase{
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction(String str) {
+	public static CSVWriteInstruction parseInstruction(String str) {
 		Operator op = null;
 		byte input, output;
 		String[] s=str.split(Instruction.OPERAND_DELIM);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineBinaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineBinaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineBinaryInstruction.java
index a1da5c2..d0f1b02 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineBinaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineBinaryInstruction.java
@@ -21,7 +21,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -43,7 +42,7 @@ public class CombineBinaryInstruction extends BinaryMRInstructionBase
 		instString = istr;
 	}
 
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static CombineBinaryInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 4 );
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineTernaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineTernaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineTernaryInstruction.java
index adeefdf..0593602 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineTernaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineTernaryInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.mr;
 import org.apache.sysml.lops.Ternary.OperationTypes;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -38,7 +37,7 @@ public class CombineTernaryInstruction extends TernaryInstruction
 		mrtype = MRINSTRUCTION_TYPE.CombineTernary;
 	}
 
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static CombineTernaryInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		// example instruction string - ctabletransform:::0:DOUBLE:::1:DOUBLE:::2:DOUBLE:::3:DOUBLE 
 		InstructionUtils.checkNumFields ( str, 4 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineUnaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineUnaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineUnaryInstruction.java
index 54cb3a9..925805c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineUnaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CombineUnaryInstruction.java
@@ -21,7 +21,6 @@ package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -41,7 +40,7 @@ public class CombineUnaryInstruction extends UnaryMRInstructionBase
 		instString = istr;
 	}
 
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static CombineUnaryInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 2 );
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeAggregateInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeAggregateInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeAggregateInstruction.java
index 63b9d5d..e1569be 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeAggregateInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeAggregateInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -51,7 +50,7 @@ public class CumulativeAggregateInstruction extends AggregateUnaryInstruction
 		_mcIn = mcIn;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static CumulativeAggregateInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 2 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeOffsetInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeOffsetInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeOffsetInstruction.java
index 8c10a3a..ae11ba0 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeOffsetInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeOffsetInstruction.java
@@ -24,7 +24,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.Builtin;
 import org.apache.sysml.runtime.functionobjects.Multiply;
 import org.apache.sysml.runtime.functionobjects.Plus;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -62,7 +61,7 @@ public class CumulativeOffsetInstruction extends BinaryInstruction
 		}
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static CumulativeOffsetInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 3 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeSplitInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeSplitInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeSplitInstruction.java
index 25c9f29..c3d533e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeSplitInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/CumulativeSplitInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -55,7 +54,7 @@ public class CumulativeSplitInstruction extends UnaryInstruction
 		_lastRowBlockIndex = (long)Math.ceil((double)_mcIn.getRows()/_mcIn.getRowsPerBlock());
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static CumulativeSplitInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 3 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/DataPartitionMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/DataPartitionMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/DataPartitionMRInstruction.java
index a591cc9..24a3018 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/DataPartitionMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/DataPartitionMRInstruction.java
@@ -20,7 +20,6 @@
 package org.apache.sysml.runtime.instructions.mr;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.operators.Operator;
 
@@ -36,7 +35,7 @@ public class DataPartitionMRInstruction extends UnaryInstruction
 		super(op, in, out, istr);
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static DataPartitionMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 3 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
index effafbd..24006e2 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateInstruction.java
@@ -24,7 +24,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.CM;
 import org.apache.sysml.runtime.functionobjects.KahanPlus;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -76,7 +75,7 @@ public class GroupedAggregateInstruction extends UnaryMRInstructionBase
 		
 	}
 
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static GroupedAggregateInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		String[] parts = InstructionUtils.getInstructionParts ( str );
 		if(parts.length<3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
index 2d1a238..4130fd9 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/GroupedAggregateMInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.PartialAggregate.CorrectionLocationType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.KahanPlus;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -55,7 +54,7 @@ public class GroupedAggregateMInstruction extends BinaryMRInstructionBase implem
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static GroupedAggregateMInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/MMTSJMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MMTSJMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MMTSJMRInstruction.java
index 9593030..de2ee5e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MMTSJMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MMTSJMRInstruction.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.sysml.lops.MMTSJ.MMTSJType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -66,7 +65,7 @@ public class MMTSJMRInstruction extends UnaryInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static MMTSJMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 3 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/MapMultChainInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MapMultChainInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MapMultChainInstruction.java
index cb55600..6fff0fb 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MapMultChainInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MapMultChainInstruction.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.sysml.lops.MapMultChain.ChainType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
@@ -115,7 +114,7 @@ public class MapMultChainInstruction extends MRInstruction implements IDistribut
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction( String str ) 
+	public static MapMultChainInstruction parseInstruction( String str ) 
 		throws DMLRuntimeException 
 	{		
 		//check number of fields (2/3 inputs, output, type)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/MatrixReshapeMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/MatrixReshapeMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/MatrixReshapeMRInstruction.java
index ce5dc1b..997ff7e 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/MatrixReshapeMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/MatrixReshapeMRInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
@@ -72,7 +71,7 @@ public class MatrixReshapeMRInstruction extends UnaryInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static MatrixReshapeMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 5 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/PMMJMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/PMMJMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/PMMJMRInstruction.java
index b2d1cf9..99e46ba 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/PMMJMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/PMMJMRInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.lops.PMMJ.CacheType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -73,7 +72,7 @@ public class PMMJMRInstruction extends BinaryMRInstructionBase implements IDistr
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static PMMJMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 6 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ParameterizedBuiltinMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ParameterizedBuiltinMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ParameterizedBuiltinMRInstruction.java
index e62f4c2..b58cf03 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ParameterizedBuiltinMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ParameterizedBuiltinMRInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
@@ -99,7 +98,7 @@ public class ParameterizedBuiltinMRInstruction extends UnaryInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static ParameterizedBuiltinMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionParts(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/PickByCountInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/PickByCountInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/PickByCountInstruction.java
index e7507c7..2aed2cb 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/PickByCountInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/PickByCountInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.mr;
 import org.apache.sysml.lops.PickByCount.OperationTypes;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -68,7 +67,7 @@ public class PickByCountInstruction extends MRInstruction
 		isValuePick=false;
 	}
 
-	public static Instruction parseInstruction ( String str ) 
+	public static PickByCountInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{	
 		InstructionUtils.checkNumFields ( str, 5 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/QuaternaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/QuaternaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/QuaternaryInstruction.java
index 1f51fe6..9d5782a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/QuaternaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/QuaternaryInstruction.java
@@ -34,7 +34,6 @@ import org.apache.sysml.lops.WeightedUnaryMMR;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
@@ -136,7 +135,7 @@ public class QuaternaryInstruction extends MRInstruction implements IDistributed
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction( String str ) 
+	public static QuaternaryInstruction parseInstruction( String str ) 
 		throws DMLRuntimeException 
 	{		
 		String opcode = InstructionUtils.getOpCode(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/RandInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/RandInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/RandInstruction.java
index 6d0eabb..136dd13 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/RandInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/RandInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.mr;
 import org.apache.sysml.hops.Hop.DataGenMethod;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -77,7 +76,7 @@ public class RandInstruction extends DataGenMRInstruction
 		return seed;
 	}
 
-	public static Instruction parseInstruction(String str) throws DMLRuntimeException 
+	public static RandInstruction parseInstruction(String str) throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 13 );
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/RangeBasedReIndexInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/RangeBasedReIndexInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/RangeBasedReIndexInstruction.java
index 8520229..0368e35 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/RangeBasedReIndexInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/RangeBasedReIndexInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.OperationsOnMatrixValues;
@@ -65,7 +64,7 @@ public class RangeBasedReIndexInstruction extends UnaryMRInstructionBase
 		return indexRange;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static RangeBasedReIndexInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 8 );
 		String[] parts = InstructionUtils.getInstructionParts ( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ReblockInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReblockInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReblockInstruction.java
index 0fbca84..1d65446 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReblockInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReblockInstruction.java
@@ -43,7 +43,7 @@ public class ReblockInstruction extends UnaryMRInstructionBase
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction(String str) 
+	public static ReblockInstruction parseInstruction(String str) 
 	{
 		Operator op = null;
 	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/RemoveEmptyMRInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/RemoveEmptyMRInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/RemoveEmptyMRInstruction.java
index ebf6e30..eaed828 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/RemoveEmptyMRInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/RemoveEmptyMRInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
@@ -70,7 +69,7 @@ public class RemoveEmptyMRInstruction extends BinaryInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static RemoveEmptyMRInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 5 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ReorgInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReorgInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReorgInstruction.java
index f1a103c..eb67564 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReorgInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReorgInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.DiagIndex;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -62,7 +61,7 @@ public class ReorgInstruction extends UnaryMRInstructionBase
 		_outputEmptyBlocks = flag; 
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static ReorgInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 2 );
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ReplicateInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReplicateInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReplicateInstruction.java
index 619e0e6..66dcf92 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ReplicateInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ReplicateInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
@@ -70,7 +69,7 @@ public class ReplicateInstruction extends UnaryMRInstructionBase
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static ReplicateInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		//check instruction format

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
index e849d8b..f896a32 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ScalarInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.Lop;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.OperationsOnMatrixValues;
@@ -53,7 +52,7 @@ public class ScalarInstruction extends UnaryMRInstructionBase
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str )
+	public static ScalarInstruction parseInstruction ( String str )
 		throws DMLRuntimeException 
 	{	
 		InstructionUtils.checkNumFields ( str, 3 );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/SeqInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/SeqInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/SeqInstruction.java
index a2ac446..3ff97d4 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/SeqInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/SeqInstruction.java
@@ -22,7 +22,6 @@ package org.apache.sysml.runtime.instructions.mr;
 import org.apache.sysml.hops.Hop.DataGenMethod;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.mapred.CachedValueMap;
@@ -47,7 +46,7 @@ public class SeqInstruction extends DataGenMRInstruction
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction(String str) throws DMLRuntimeException 
+	public static SeqInstruction parseInstruction(String str) throws DMLRuntimeException 
 	{
 		InstructionUtils.checkNumFields ( str, 10 );
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/TernaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/TernaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/TernaryInstruction.java
index 81d906f..c7b84c8 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/TernaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/TernaryInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.lops.Ternary;
 import org.apache.sysml.lops.Ternary.OperationTypes;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.CTableMap;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -157,7 +156,7 @@ public class TernaryInstruction extends MRInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static TernaryInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{		
 		// example instruction string 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/UaggOuterChainInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/UaggOuterChainInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/UaggOuterChainInstruction.java
index 1b2fb62..99123e7 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/UaggOuterChainInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/UaggOuterChainInstruction.java
@@ -28,7 +28,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.ReduceAll;
 import org.apache.sysml.runtime.functionobjects.ReduceCol;
 import org.apache.sysml.runtime.functionobjects.ReduceRow;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.LibMatrixOuterAgg;
@@ -91,7 +90,7 @@ public class UaggOuterChainInstruction extends BinaryInstruction implements IDis
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction( String str ) 
+	public static UaggOuterChainInstruction parseInstruction( String str ) 
 		throws DMLRuntimeException 
 	{		
 		//check number of fields (2/3 inputs, output, type)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/UnaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/UnaryInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/UnaryInstruction.java
index f1eb7b2..4c86751 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/UnaryInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/UnaryInstruction.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.functionobjects.Builtin;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.OperationsOnMatrixValues;
@@ -44,7 +43,7 @@ public class UnaryInstruction extends UnaryMRInstructionBase
 		instString = istr;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static UnaryInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		String opcode = InstructionUtils.getOpCode(str);
 	 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/mr/ZeroOutInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/mr/ZeroOutInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/mr/ZeroOutInstruction.java
index 6b4285a..145a089 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/mr/ZeroOutInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/mr/ZeroOutInstruction.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.matrix.data.MatrixValue;
 import org.apache.sysml.runtime.matrix.data.OperationsOnMatrixValues;
@@ -52,7 +51,7 @@ public class ZeroOutInstruction extends UnaryMRInstructionBase
 		indexRange=rng;
 	}
 	
-	public static Instruction parseInstruction ( String str ) throws DMLRuntimeException {
+	public static ZeroOutInstruction parseInstruction ( String str ) throws DMLRuntimeException {
 		
 		InstructionUtils.checkNumFields ( str, 6 );
 		

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 d855384..82fa6fd 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
@@ -34,7 +34,6 @@ import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.functionobjects.ReduceAll;
 import org.apache.sysml.runtime.functionobjects.ReduceCol;
 import org.apache.sysml.runtime.functionobjects.ReduceRow;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.spark.functions.AggregateDropCorrectionFunction;
@@ -68,7 +67,7 @@ public class AggregateUnarySPInstruction extends UnarySPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction(String str)
+	public static AggregateUnarySPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/ArithmeticBinarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ArithmeticBinarySPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ArithmeticBinarySPInstruction.java
index c952553..e4a8708 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ArithmeticBinarySPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ArithmeticBinarySPInstruction.java
@@ -25,7 +25,6 @@ import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -46,7 +45,7 @@ public abstract class ArithmeticBinarySPInstruction extends BinarySPInstruction
 	 * @throws DMLRuntimeException
 	 * @throws DMLUnsupportedOperationException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static ArithmeticBinarySPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/BinUaggChainSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/BinUaggChainSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/BinUaggChainSPInstruction.java
index b93a8e6..5c7ed71 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/BinUaggChainSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/BinUaggChainSPInstruction.java
@@ -26,7 +26,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -52,7 +51,7 @@ public class BinUaggChainSPInstruction extends UnarySPInstruction
 
 	}
 
-	public static Instruction parseInstruction ( String str ) 
+	public static BinUaggChainSPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		//parse instruction parts (without exec type)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/CSVReblockSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CSVReblockSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CSVReblockSPInstruction.java
index 4b9457c..19f8b5b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CSVReblockSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CSVReblockSPInstruction.java
@@ -29,7 +29,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils;
@@ -62,7 +61,7 @@ public class CSVReblockSPInstruction extends UnarySPInstruction
 		_missingValue = missingValue;
 	}
 
-	public static Instruction parseInstruction(String str)
+	public static CSVReblockSPInstruction parseInstruction(String str)
 			throws DMLRuntimeException 
 	{
 		String opcode = InstructionUtils.getOpCode(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/CentralMomentSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CentralMomentSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CentralMomentSPInstruction.java
index 4a01e2d..a64f825 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CentralMomentSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CentralMomentSPInstruction.java
@@ -32,7 +32,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.functionobjects.CM;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
@@ -61,7 +60,7 @@ public class CentralMomentSPInstruction extends UnarySPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction(String str)
+	public static CentralMomentSPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/CheckpointSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CheckpointSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CheckpointSPInstruction.java
index a262ba6..c5f69c1 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CheckpointSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CheckpointSPInstruction.java
@@ -29,7 +29,6 @@ import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.BooleanObject;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
@@ -53,7 +52,7 @@ public class CheckpointSPInstruction extends UnarySPInstruction
 		_level = level;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static CheckpointSPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/CovarianceSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CovarianceSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CovarianceSPInstruction.java
index d0dbbc4..ba75803 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CovarianceSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CovarianceSPInstruction.java
@@ -32,7 +32,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.functionobjects.COV;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
@@ -65,7 +64,7 @@ public class CovarianceSPInstruction extends BinarySPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction(String str)
+	public static CovarianceSPInstruction parseInstruction(String str)
 		throws DMLRuntimeException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/CumulativeOffsetSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/CumulativeOffsetSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/CumulativeOffsetSPInstruction.java
index db41c35..6022126 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/CumulativeOffsetSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/CumulativeOffsetSPInstruction.java
@@ -34,7 +34,6 @@ import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.functionobjects.Builtin;
 import org.apache.sysml.runtime.functionobjects.Multiply;
 import org.apache.sysml.runtime.functionobjects.Plus;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
@@ -83,7 +82,7 @@ public class CumulativeOffsetSPInstruction extends BinarySPInstruction
 	 * @return
 	 * @throws DMLRuntimeException
 	 */
-	public static Instruction parseInstruction ( String str ) 
+	public static CumulativeOffsetSPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType( str );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/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 2dfc4a6..89992b8 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
@@ -33,7 +33,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.spark.data.LazyIterableIterator;
@@ -92,7 +91,7 @@ public class MatrixIndexingSPInstruction  extends UnarySPInstruction
 		colUpper = cu;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static MatrixIndexingSPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{	
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType(str);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/ReblockSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ReblockSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ReblockSPInstruction.java
index 6e1e30c..c475517 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ReblockSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ReblockSPInstruction.java
@@ -29,7 +29,6 @@ import org.apache.sysml.runtime.DMLUnsupportedOperationException;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.spark.data.RDDProperties;
@@ -59,7 +58,7 @@ public class ReblockSPInstruction extends UnarySPInstruction
 		outputEmptyBlocks = emptyBlocks;
 	}
 	
-	public static Instruction parseInstruction(String str)  throws DMLRuntimeException 
+	public static ReblockSPInstruction parseInstruction(String str)  throws DMLRuntimeException 
 	{
 		String parts[] = InstructionUtils.getInstructionPartsWithValueType(str);
 		String opcode = parts[0];

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/RelationalBinarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/RelationalBinarySPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/RelationalBinarySPInstruction.java
index 8386a5e..2c6956c 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/RelationalBinarySPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/RelationalBinarySPInstruction.java
@@ -24,7 +24,6 @@ import org.apache.sysml.lops.BinaryM.VectorType;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.matrix.operators.Operator;
@@ -37,7 +36,7 @@ public abstract class RelationalBinarySPInstruction extends BinarySPInstruction
 		_sptype = SPINSTRUCTION_TYPE.RelationalBinary;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static RelationalBinarySPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		CPOperand in1 = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/5878e1db/src/main/java/org/apache/sysml/runtime/instructions/spark/ReorgSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/ReorgSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/ReorgSPInstruction.java
index 27b1157..aeea61b 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/ReorgSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/ReorgSPInstruction.java
@@ -36,7 +36,6 @@ import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysml.runtime.functionobjects.DiagIndex;
 import org.apache.sysml.runtime.functionobjects.SortIndex;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CPOperand;
 import org.apache.sysml.runtime.instructions.spark.functions.FilterDiagBlocksFunction;
@@ -73,7 +72,7 @@ public class ReorgSPInstruction extends UnarySPInstruction
 		_bSortIndInMem = bSortIndInMem;
 	}
 	
-	public static Instruction parseInstruction ( String str ) 
+	public static ReorgSPInstruction parseInstruction ( String str ) 
 		throws DMLRuntimeException 
 	{
 		CPOperand in = new CPOperand("", ValueType.UNKNOWN, DataType.UNKNOWN);