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

systemml git commit: [SYSTEMML-1818] Fix ctable size propagation w/ two sequence inputs

Repository: systemml
Updated Branches:
  refs/heads/master ebd7ec6f3 -> dbe7427e9


[SYSTEMML-1818] Fix ctable size propagation w/ two sequence inputs

For ctable, we exploit the knowledge of sequence inputs for better size
propagation. This patch fixes an issue of incorrect size propagation,
where the sequence rewrite applies to the right, but we mistakenly
leverage the size of the left input if it is a sequence as well. An
example for this problematic pattern is (from Kmeans train):

R = table(seq(i, NC*(NR-1)+i, NC), seq(1, NR, 1));

Furthermore, this patch also fixes the unnecessary eager spark context
creation in forced singlenode execution mode and globally renames any
remaining occurrences of tertiary operators to ternary operators.


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

Branch: refs/heads/master
Commit: dbe7427e9f11a3180ce80503eff34a995570b202
Parents: ebd7ec6
Author: Matthias Boehm <mb...@gmail.com>
Authored: Sat Jul 29 15:45:22 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Sat Jul 29 15:45:22 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/hops/BinaryOp.java    |   2 +-
 .../java/org/apache/sysml/hops/TernaryOp.java   | 176 ++++++++-----------
 .../hops/cost/CostEstimatorStaticRuntime.java   |   2 +-
 .../RewriteAlgebraicSimplificationDynamic.java  |   6 +-
 .../java/org/apache/sysml/lops/Ternary.java     |   2 +-
 .../context/SparkExecutionContext.java          |   3 +-
 .../sysml/runtime/matrix/data/MatrixBlock.java  |   6 +-
 .../sysml/runtime/matrix/data/MatrixValue.java  |  15 +-
 .../matrix/data/OperationsOnMatrixValues.java   |  15 +-
 9 files changed, 99 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/hops/BinaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/BinaryOp.java b/src/main/java/org/apache/sysml/hops/BinaryOp.java
index d072afd..4a450ad 100644
--- a/src/main/java/org/apache/sysml/hops/BinaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/BinaryOp.java
@@ -1185,7 +1185,7 @@ public class BinaryOp extends Hop
 	}
 	
 	/**
-	 * Special case tertiary append. Here, we also compile a MR_RAPPEND or MR_GAPPEND
+	 * Special case ternary append. Here, we also compile a MR_RAPPEND or MR_GAPPEND
 	 * 
 	 * @param left ?
 	 * @param right1 ?

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/hops/TernaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/TernaryOp.java b/src/main/java/org/apache/sysml/hops/TernaryOp.java
index 22895e5..98c8ad3 100644
--- a/src/main/java/org/apache/sysml/hops/TernaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/TernaryOp.java
@@ -394,7 +394,7 @@ public class TernaryOp extends Hop
 		DataType dt1 = getInput().get(0).getDataType(); 
 		DataType dt2 = getInput().get(1).getDataType(); 
 		DataType dt3 = getInput().get(2).getDataType(); 
-		Ternary.OperationTypes tertiaryOpOrig = Ternary.findCtableOperationByInputDataTypes(dt1, dt2, dt3);
+		Ternary.OperationTypes ternaryOpOrig = Ternary.findCtableOperationByInputDataTypes(dt1, dt2, dt3);
  		
 		// Compute lops for all inputs
 		Lop[] inputLops = new Lop[getInput().size()];
@@ -410,7 +410,8 @@ public class TernaryOp extends Hop
 		if ( et == ExecType.CP  || et == ExecType.SPARK) 
 		{	
 			//for CP we support only ctable expand left
-			Ternary.OperationTypes tertiaryOp = isSequenceRewriteApplicable(true) ? Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT : tertiaryOpOrig;
+			Ternary.OperationTypes ternaryOp = isSequenceRewriteApplicable(true) ? 
+				Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT : ternaryOpOrig;
 			boolean ignoreZeros = false;
 			
 			if( isMatrixIgnoreZeroRewriteApplicable() ) { 
@@ -419,39 +420,39 @@ public class TernaryOp extends Hop
 				inputLops[1] = ((ParameterizedBuiltinOp)getInput().get(1)).getTargetHop().getInput().get(0).constructLops();
 			}
 			
-			Ternary tertiary = new Ternary(inputLops, tertiaryOp, getDataType(), getValueType(), ignoreZeros, et);
+			Ternary ternary = new Ternary(inputLops, ternaryOp, getDataType(), getValueType(), ignoreZeros, et);
 			
-			tertiary.getOutputParameters().setDimensions(_dim1, _dim2, getRowsInBlock(), getColsInBlock(), -1);
-			tertiary.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+			ternary.getOutputParameters().setDimensions(_dim1, _dim2, getRowsInBlock(), getColsInBlock(), -1);
+			setLineNumbers(ternary);
 			
 			//force blocked output in CP (see below), otherwise binarycell
 			if ( et == ExecType.SPARK ) {
-				tertiary.getOutputParameters().setDimensions(_dim1, _dim2, -1, -1, -1);
+				ternary.getOutputParameters().setDimensions(_dim1, _dim2, -1, -1, -1);
 				setRequiresReblock( true );
 			}
 			else
-				tertiary.getOutputParameters().setDimensions(_dim1, _dim2, getRowsInBlock(), getColsInBlock(), -1);
+				ternary.getOutputParameters().setDimensions(_dim1, _dim2, getRowsInBlock(), getColsInBlock(), -1);
 			
-			//tertiary opt, w/o reblock in CP
-			setLops(tertiary);
+			//ternary opt, w/o reblock in CP
+			setLops(ternary);
 			
 			
 		}
 		else //MR
 		{
 			//for MR we support both ctable expand left and right
-			Ternary.OperationTypes tertiaryOp = isSequenceRewriteApplicable() ? Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT : tertiaryOpOrig;
+			Ternary.OperationTypes ternaryOp = isSequenceRewriteApplicable() ? 
+				Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT : ternaryOpOrig;
 			
 			Group group1 = null, group2 = null, group3 = null, group4 = null;
 			group1 = new Group(inputLops[0], Group.OperationTypes.Sort, getDataType(), getValueType());
 			group1.getOutputParameters().setDimensions(getDim1(),
 					getDim2(), getRowsInBlock(), getColsInBlock(), getNnz());
+			setLineNumbers(group1);
 			
-			group1.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
-
-			Ternary tertiary = null;
+			Ternary ternary = null;
 			// create "group" lops for MATRIX inputs
-			switch (tertiaryOp) 
+			switch (ternaryOp) 
 			{
 				case CTABLE_TRANSFORM:
 					// F = ctable(A,B,W)
@@ -462,7 +463,7 @@ public class TernaryOp extends Hop
 					group2.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group2.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					setLineNumbers(group2);
 					
 					group3 = new Group(
 							inputLops[2],
@@ -471,19 +472,17 @@ public class TernaryOp extends Hop
 					group3.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group3.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					setLineNumbers(group3);
 					
 					if ( inputLops.length == 3 )
-						tertiary = new Ternary(
-								new Lop[] {group1, group2, group3},
-								tertiaryOp,
-								getDataType(), getValueType(), et);	
+						ternary = new Ternary(
+							new Lop[] {group1, group2, group3}, ternaryOp,
+							getDataType(), getValueType(), et);	
 					else 
 						// output dimensions are given
-						tertiary = new Ternary(
-								new Lop[] {group1, group2, group3, inputLops[3], inputLops[4]},
-								tertiaryOp,
-								getDataType(), getValueType(), et);	
+						ternary = new Ternary(
+							new Lop[] {group1, group2, group3, inputLops[3], inputLops[4]},
+							ternaryOp, getDataType(), getValueType(), et);	
 					break;
 	
 				case CTABLE_TRANSFORM_SCALAR_WEIGHT:
@@ -495,18 +494,16 @@ public class TernaryOp extends Hop
 					group2.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group2.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					setLineNumbers(group2);
 					
 					if ( inputLops.length == 3)
-						tertiary = new Ternary(
+						ternary = new Ternary(
 								new Lop[] {group1,group2,inputLops[2]},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+								ternaryOp, getDataType(), getValueType(), et);
 					else
-						tertiary = new Ternary(
+						ternary = new Ternary(
 								new Lop[] {group1,group2,inputLops[2], inputLops[3], inputLops[4]},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+								ternaryOp, getDataType(), getValueType(), et);
 						
 					break;
 			
@@ -524,50 +521,40 @@ public class TernaryOp extends Hop
 					//TODO remove group, whenever we push it into the map task
 					
 					if (inputLops.length == 3)
-						tertiary = new Ternary(
-								new Lop[] {					
-										group, //matrix
-										getInput().get(2).constructLops(), //weight
-										new LiteralOp(left).constructLops() //left
+						ternary = new Ternary(
+								new Lop[] {group, //matrix
+									getInput().get(2).constructLops(), //weight
+									new LiteralOp(left).constructLops() //left
 								},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+								ternaryOp, getDataType(), getValueType(), et);
 					else
-						tertiary = new Ternary(
-								new Lop[] {					
-										group,//getInput().get(1).constructLops(), //matrix
-										getInput().get(2).constructLops(), //weight
-										new LiteralOp(left).constructLops(), //left
-										inputLops[3],
-										inputLops[4]
+						ternary = new Ternary(
+								new Lop[] {group, //matrix
+									getInput().get(2).constructLops(), //weight
+									new LiteralOp(left).constructLops(), //left
+									inputLops[3], inputLops[4]
 								},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
-					
+								ternaryOp, getDataType(), getValueType(), et);
 					break;
 					
 				case CTABLE_TRANSFORM_HISTOGRAM:
 					// F=ctable(A,1) or F = ctable(A,1,1)
 					if ( inputLops.length == 3 )
-						tertiary = new Ternary(
-								new Lop[] {
-										group1, 
-										getInput().get(1).constructLops(),
-										getInput().get(2).constructLops()
+						ternary = new Ternary(
+								new Lop[] {group1, 
+									getInput().get(1).constructLops(),
+									getInput().get(2).constructLops()
 								},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+								ternaryOp, getDataType(), getValueType(), et);
 					else
-						tertiary = new Ternary(
-								new Lop[] {
-										group1, 
-										getInput().get(1).constructLops(),
-										getInput().get(2).constructLops(),
-										inputLops[3],
-										inputLops[4]
+						ternary = new Ternary(
+								new Lop[] {group1, 
+									getInput().get(1).constructLops(),
+									getInput().get(2).constructLops(),
+									inputLops[3],
+									inputLops[4]
 								},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+								ternaryOp, getDataType(), getValueType(), et);
 						
 					break;
 				case CTABLE_TRANSFORM_WEIGHTED_HISTOGRAM:
@@ -579,24 +566,20 @@ public class TernaryOp extends Hop
 					group3.getOutputParameters().setDimensions(getDim1(),
 							getDim2(), getRowsInBlock(),
 							getColsInBlock(), getNnz());
-					group3.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
+					setLineNumbers(group3);
 					
 					if ( inputLops.length == 3)
-						tertiary = new Ternary(
-								new Lop[] {
-										group1,
-										getInput().get(1).constructLops(),
-										group3},
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+						ternary = new Ternary(
+								new Lop[] {group1,
+									getInput().get(1).constructLops(),
+									group3},
+								ternaryOp, getDataType(), getValueType(), et);
 					else
-						tertiary = new Ternary(
-								new Lop[] {
-										group1,
-										getInput().get(1).constructLops(),
-										group3, inputLops[3], inputLops[4] },
-								tertiaryOp,
-								getDataType(), getValueType(), et);
+						ternary = new Ternary(
+								new Lop[] {group1,
+									getInput().get(1).constructLops(),
+									group3, inputLops[3], inputLops[4] },
+								ternaryOp, getDataType(), getValueType(), et);
 						
 					break;
 				
@@ -605,28 +588,27 @@ public class TernaryOp extends Hop
 			}
 
 			// output dimensions are not known at compilation time
-			tertiary.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
-			setLineNumbers(tertiary);
+			ternary.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
+			setLineNumbers(ternary);
 			
-			Lop lctable = tertiary;
+			Lop lctable = ternary;
 			
-			if( !(_disjointInputs || tertiaryOp == Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT) ) 
+			if( !(_disjointInputs || ternaryOp == Ternary.OperationTypes.CTABLE_EXPAND_SCALAR_WEIGHT) ) 
 			{ 
 				//no need for aggregation if (1) input indexed disjoint	or one side is sequence	w/ 1 increment
 				
 				group4 = new Group(
-						tertiary, Group.OperationTypes.Sort, getDataType(),
+						ternary, Group.OperationTypes.Sort, getDataType(),
 						getValueType());
 				group4.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
-				group4.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
-	
+				setLineNumbers(group4);
+				
 				Aggregate agg1 = new Aggregate(
 						group4, HopsAgg2Lops.get(AggOp.SUM), getDataType(),
 						getValueType(), ExecType.MR);
 				agg1.getOutputParameters().setDimensions(_dim1, _dim2, ( _dimInputsPresent ? getRowsInBlock() : -1), ( _dimInputsPresent ? getColsInBlock() : -1), -1);
-	
-				agg1.setAllPositions(this.getFilename(), this.getBeginLine(), this.getBeginColumn(), this.getEndLine(), this.getEndColumn());
-
+				setLineNumbers(agg1);
+				
 				// kahamSum is used for aggregation but inputs do not have
 				// correction values
 				agg1.setupCorrectionLocation(CorrectionLocationType.NONE);
@@ -884,20 +866,16 @@ public class TernaryOp extends Hop
 					Hop input2 = getInput().get(1);
 					Hop input3 = getInput().get(2);
 					
-					
 					if ( _dim1 == -1 || _dim2 == -1 ) { 
 						//for ctable_expand at least one dimension is known
-						if( isSequenceRewriteApplicable() )
-						{
-							if( input1 instanceof DataGenOp && ((DataGenOp)input1).getOp()==DataGenMethod.SEQ )
-								setDim1( input1._dim1 );
-							else //if( input2 instanceof DataGenOp && ((DataGenOp)input2).getDataGenMethod()==DataGenMethod.SEQ )
-								setDim2( input2._dim1 );
-						}
+						if( isSequenceRewriteApplicable(true) )
+							setDim1( input1._dim1 );
+						else if( isSequenceRewriteApplicable(false) )
+							setDim2( input2._dim1 );
 						//for ctable_histogram also one dimension is known
-						Ternary.OperationTypes tertiaryOp = Ternary.findCtableOperationByInputDataTypes(
+						Ternary.OperationTypes ternaryOp = Ternary.findCtableOperationByInputDataTypes(
 																input1.getDataType(), input2.getDataType(), input3.getDataType());
-						if(  tertiaryOp==Ternary.OperationTypes.CTABLE_TRANSFORM_HISTOGRAM
+						if(  ternaryOp==Ternary.OperationTypes.CTABLE_TRANSFORM_HISTOGRAM
 							&& input2 instanceof LiteralOp )
 						{
 							setDim2( HopRewriteUtils.getIntValueSafe((LiteralOp)input2) );

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java b/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
index b868d0d..0ca29ea 100644
--- a/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
+++ b/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
@@ -1146,7 +1146,7 @@ public class CostEstimatorStaticRuntime extends CostEstimator
 					return   d1m * d1n * d1s
 					       + d2m * d2n * d2s;
 					
-				case CombineTernary: //opcodes: combinetertiary
+				case CombineTernary: //opcodes: combineternary
 					return   d1m * d1n * d1s
 				           + d2m * d2n * d2s
 				           + d3m * d3n * d3s;

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 09b66de..8513d27 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -2525,15 +2525,17 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 				Hop hnew = new LiteralOp(hi.getInput().get(0).getDim1());
 				HopRewriteUtils.replaceChildReference(parent, hi, hnew, pos, false);
 				HopRewriteUtils.cleanupUnreferenced(hi);
+				LOG.debug("Applied simplifyNrowComputation nrow("+hi.getHopID()+") -> "
+					+ hnew.getName()+" (line "+hi.getBeginLine()+").");
 				hi = hnew;
-				LOG.debug("Applied simplifyNrowComputation.");
 			}
 			else if( ((UnaryOp)hi).getOp()==OpOp1.NCOL && hi.getInput().get(0).getDim2()>0 ) {
 				Hop hnew = new LiteralOp(hi.getInput().get(0).getDim2());
 				HopRewriteUtils.replaceChildReference(parent, hi, hnew, pos, false);
 				HopRewriteUtils.cleanupUnreferenced(hi);
+				LOG.debug("Applied simplifyNcolComputation ncol("+hi.getHopID()+") -> "
+					+ hnew.getName()+" (line "+hi.getBeginLine()+").");
 				hi = hnew;
-				LOG.debug("Applied simplifyNcolComputation.");	
 			}
 		}
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/lops/Ternary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/lops/Ternary.java b/src/main/java/org/apache/sysml/lops/Ternary.java
index a2d674b..eee9574 100644
--- a/src/main/java/org/apache/sysml/lops/Ternary.java
+++ b/src/main/java/org/apache/sysml/lops/Ternary.java
@@ -26,7 +26,7 @@ import org.apache.sysml.parser.Expression.*;
 
 
 /**
- * Lop to perform tertiary operation. All inputs must be matrices or vectors. 
+ * Lop to perform ternary operation. All inputs must be matrices or vectors. 
  * For example, this lop is used in evaluating A = ctable(B,C,W)
  * 
  * Currently, this lop is used only in case of CTABLE functionality.

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
index c67cdd8..6f2f766 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
@@ -38,6 +38,7 @@ import org.apache.spark.storage.RDDInfo;
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.util.LongAccumulator;
 import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.api.mlcontext.MLContext;
 import org.apache.sysml.api.mlcontext.MLContextUtil;
 import org.apache.sysml.conf.ConfigurationManager;
@@ -118,7 +119,7 @@ public class SparkExecutionContext extends ExecutionContext
 		super( allocateVars, prog );
 
 		//spark context creation via internal initializer
-		if( !(LAZY_SPARKCTX_CREATION && OptimizerUtils.isHybridExecutionMode()) ) {
+		if( !LAZY_SPARKCTX_CREATION || DMLScript.rtplatform==RUNTIME_PLATFORM.SPARK ) {
 			initSparkContext();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
index f3c9f7b..93f48c7 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
@@ -440,7 +440,7 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	}
 	
 	/**
-	 * NOTE: setNumRows() and setNumColumns() are used only in tertiaryInstruction (for contingency tables)
+	 * NOTE: setNumRows() and setNumColumns() are used only in ternaryInstruction (for contingency tables)
 	 * and pmm for meta corrections.
 	 * 
 	 * @param r number of rows
@@ -4961,9 +4961,9 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	{
 		//check input dimensions and operators
 		if( m1.rlen!=m2.rlen || m1.clen!=m2.clen || (m3!=null && (m2.rlen!=m3.rlen || m2.clen!=m3.clen)) )
-			throw new DMLRuntimeException("Invalid dimensions for aggregate tertiary ("+m1.rlen+"x"+m1.clen+", "+m2.rlen+"x"+m2.clen+", "+m3.rlen+"x"+m3.clen+").");
+			throw new DMLRuntimeException("Invalid dimensions for aggregate ternary ("+m1.rlen+"x"+m1.clen+", "+m2.rlen+"x"+m2.clen+", "+m3.rlen+"x"+m3.clen+").");
 		if( !( op.aggOp.increOp.fn instanceof KahanPlus && op.binaryFn instanceof Multiply) )
-			throw new DMLRuntimeException("Unsupported operator for aggregate tertiary operations.");
+			throw new DMLRuntimeException("Unsupported operator for aggregate ternary operations.");
 		
 		//create output matrix block w/ corrections
 		int rl = (op.indexFn instanceof ReduceRow) ? 2 : 1;

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixValue.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixValue.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixValue.java
index bb95af2..6e9986a 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixValue.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixValue.java
@@ -115,25 +115,20 @@ public abstract class MatrixValue implements WritableComparable
 			int startRow, int startColumn, int length)
 	throws DMLRuntimeException;
 	
-	// tertiary where all three inputs are matrices
 	public abstract void ternaryOperations(Operator op, MatrixValue that, MatrixValue that2, CTableMap resultMap, MatrixBlock resultBlock)
-	throws DMLRuntimeException;
+		throws DMLRuntimeException;
 	
-	// tertiary where first two inputs are matrices, and third input is a scalar (double)
 	public abstract void ternaryOperations(Operator op, MatrixValue that, double scalar_that2, boolean ignoreZeros, CTableMap resultMap, MatrixBlock resultBlock)
-	throws DMLRuntimeException;
+		throws DMLRuntimeException;
 	
-	// tertiary where first input is a matrix, and second and third inputs are scalars (double)
 	public abstract void ternaryOperations(Operator op, double scalar_that, double scalar_that2, CTableMap resultMap, MatrixBlock resultBlock)
-	throws DMLRuntimeException;
+		throws DMLRuntimeException;
 	
-	// tertiary where first input is a matrix, and second and third inputs are scalars (double)
 	public abstract void ternaryOperations(Operator op, MatrixIndexes ix1, double scalar_that, boolean left, int brlen, CTableMap resultMap, MatrixBlock resultBlock)
-	throws DMLRuntimeException;
+		throws DMLRuntimeException;
 
-	// tertiary where first and third inputs are matrices and second is a scalar
 	public abstract void ternaryOperations(Operator op, double scalarThat, MatrixValue that2, CTableMap ctableResult, MatrixBlock ctableResultBlock)
-	throws DMLRuntimeException;
+		throws DMLRuntimeException;
 
 	public abstract MatrixValue quaternaryOperations(QuaternaryOperator qop, MatrixValue um, MatrixValue vm, MatrixValue wm, MatrixValue out)
 		throws DMLRuntimeException;

http://git-wip-us.apache.org/repos/asf/systemml/blob/dbe7427e/src/main/java/org/apache/sysml/runtime/matrix/data/OperationsOnMatrixValues.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/OperationsOnMatrixValues.java b/src/main/java/org/apache/sysml/runtime/matrix/data/OperationsOnMatrixValues.java
index 27734e1..73a35be 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/OperationsOnMatrixValues.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/OperationsOnMatrixValues.java
@@ -86,46 +86,41 @@ public class OperationsOnMatrixValues
 	}
 	
 	// ------------- Ternary Operations -------------
-	// tertiary where all three inputs are matrices
 	public static void performTernary(MatrixIndexes indexesIn1, MatrixValue valueIn1, MatrixIndexes indexesIn2, MatrixValue valueIn2, 
 			MatrixIndexes indexesIn3, MatrixValue valueIn3, CTableMap resultMap, MatrixBlock resultBlock, Operator op ) 
-	throws DMLRuntimeException
+		throws DMLRuntimeException
 	{
 		//operation on the cells inside the value
 		valueIn1.ternaryOperations(op, valueIn2, valueIn3, resultMap, resultBlock);
 	}
 	
-	// tertiary where first two inputs are matrices, and third input is a scalar (double)
 	public static void performTernary(MatrixIndexes indexesIn1, MatrixValue valueIn1, MatrixIndexes indexesIn2, MatrixValue valueIn2, 
 			double scalarIn3, CTableMap resultMap, MatrixBlock resultBlock, Operator op) 
-	throws DMLRuntimeException
+		throws DMLRuntimeException
 	{
 		//operation on the cells inside the value
 		valueIn1.ternaryOperations(op, valueIn2, scalarIn3, false, resultMap, resultBlock);
 	}
 	
-	// tertiary where first input is a matrix, and second and third inputs are scalars (double)
 	public static void performTernary(MatrixIndexes indexesIn1, MatrixValue valueIn1, double scalarIn2, 
 			double scalarIn3, CTableMap resultMap, MatrixBlock resultBlock, Operator op ) 
-	throws DMLRuntimeException
+		throws DMLRuntimeException
 	{
 		//operation on the cells inside the value
 		valueIn1.ternaryOperations(op, scalarIn2, scalarIn3, resultMap, resultBlock);
 	}
 	
-	// tertiary where first input is a matrix, and second is scalars (double)
 	public static void performTernary(MatrixIndexes indexesIn1, MatrixValue valueIn1, double scalarIn2, boolean left,
 			int brlen, CTableMap resultMap, MatrixBlock resultBlock, Operator op ) 
-	throws DMLRuntimeException
+		throws DMLRuntimeException
 	{
 		//operation on the cells inside the value
 		valueIn1.ternaryOperations(op, indexesIn1, scalarIn2, left, brlen, resultMap, resultBlock);
 	}
 	
-	// tertiary where first and third inputs are matrices, and second is a scalars (double)
 	public static void performTernary(MatrixIndexes indexesIn1, MatrixValue valueIn1, double scalarIn2, 
 			MatrixIndexes indexesIn3, MatrixValue valueIn3, CTableMap resultMap, MatrixBlock resultBlock, Operator op ) 
-	throws DMLRuntimeException
+		throws DMLRuntimeException
 	{
 		//operation on the cells inside the value
 		valueIn1.ternaryOperations(op, scalarIn2, valueIn3, resultMap, resultBlock);