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 2018/07/13 00:46:11 UTC

[1/3] systemml git commit: [MINOR] Simplify and cleanup GPU-specific rewrites (rewrite utils)

Repository: systemml
Updated Branches:
  refs/heads/master 58ab12761 -> f1bf97baf


[MINOR] Simplify and cleanup GPU-specific rewrites (rewrite utils)

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

Branch: refs/heads/master
Commit: b429551dbd9917746f0001c74c16afbdb8231592
Parents: 58ab127
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Jul 12 15:54:29 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Jul 12 15:54:29 2018 -0700

----------------------------------------------------------------------
 .../hops/rewrite/RewriteGPUSpecificOps.java     | 38 ++++++++------------
 1 file changed, 15 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/b429551d/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
index 987d9cd..1c00c6f 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
@@ -24,7 +24,6 @@ import java.util.HashMap;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.hops.AggUnaryOp;
-import org.apache.sysml.hops.BinaryOp;
 import org.apache.sysml.hops.Hop;
 import org.apache.sysml.hops.Hop.AggOp;
 import org.apache.sysml.hops.Hop.Direction;
@@ -35,8 +34,6 @@ import org.apache.sysml.hops.Hop.ReOrgOp;
 import org.apache.sysml.hops.LiteralOp;
 import org.apache.sysml.hops.DnnOp;
 import org.apache.sysml.hops.OptimizerUtils;
-import org.apache.sysml.hops.ReorgOp;
-import org.apache.sysml.hops.UnaryOp;
 import org.apache.sysml.runtime.instructions.gpu.context.GPUContextPool;
 
 /*
@@ -97,8 +94,7 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 			return;
 		
 		//recursively process children
-		for( int i=0; i<hop.getInput().size(); i++)
-		{
+		for( int i=0; i<hop.getInput().size(); i++) {
 			Hop hi = hop.getInput().get(i);
 			
 			//process childs recursively first (to allow roll-up)
@@ -116,11 +112,11 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 	}
 	
 	private static boolean isBiasAdd(Hop h) {
-		return h instanceof DnnOp && ((DnnOp) h).getOp() == OpOpDnn.BIASADD;
+		return HopRewriteUtils.isDnn(h, OpOpDnn.BIASADD);
 	}
 	
 	private static boolean isBiasMultiply(Hop h) {
-		return h instanceof DnnOp && ((DnnOp) h).getOp() == OpOpDnn.BIASMULT;
+		return HopRewriteUtils.isDnn(h, OpOpDnn.BIASMULT);
 	}
 	
 	private static boolean fitsOnGPU(Hop h, double multiplier) {
@@ -168,24 +164,22 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 	}
 	
 	private static boolean isUnaryMinus(Hop h) {
-		return h instanceof BinaryOp && ((BinaryOp)h).getOp() == OpOp2.MINUS 
-				&& Hop.computeSizeInformation(h.getInput().get(0)) == 0;
+		return HopRewriteUtils.isBinary(h, OpOp2.MINUS)
+			&& HopRewriteUtils.isLiteralOfValue(h.getInput().get(0), 0);
 	}
 	
 	private static boolean isOneDivideBySqrt(Hop h) {
-		return h instanceof BinaryOp && ((BinaryOp)h).getOp() == OpOp2.DIV 
-				&& h.getInput().get(1) instanceof UnaryOp
-				&& ((UnaryOp)h.getInput().get(1)).getOp() == OpOp1.SQRT
-				&& Hop.computeSizeInformation(h.getInput().get(0)) == 1;
+		return HopRewriteUtils.isBinary(h, OpOp2.DIV)
+			&& HopRewriteUtils.isUnary(h.getInput().get(1), OpOp1.SQRT)
+			&& HopRewriteUtils.isLiteralOfValue(h.getInput().get(0), 1);
 	}
 	
-	private static Hop channelSums(Hop parent, Hop hi, int pos) 
-	{
+	private static Hop channelSums(Hop parent, Hop hi, int pos) {
 		if(hi instanceof AggUnaryOp) {
 			AggUnaryOp hop = (AggUnaryOp) hi;
 			// output = rowSums(matrix(colSums(x), rows=numChannels, cols=imgSize*imgSize))
-			if(hop.getOp() == AggOp.SUM && hop.getDirection() == Direction.Row
-				&& hop.getInput().get(0) instanceof ReorgOp && ((ReorgOp)hop.getInput().get(0)).getOp() == ReOrgOp.RESHAPE) {
+			if( hop.getOp() == AggOp.SUM && hop.getDirection() == Direction.Row
+				&& HopRewriteUtils.isReorg(hop.getInput().get(0), ReOrgOp.RESHAPE) ) {
 				Hop colSumsInput = hop.getInput().get(0).getInput().get(0);
 				if(colSumsInput instanceof AggUnaryOp && ((AggUnaryOp)colSumsInput).getOp() == AggOp.SUM && ((AggUnaryOp)colSumsInput).getDirection() == Direction.Col) {
 					ArrayList<Hop> inHops = new ArrayList<Hop>();
@@ -206,19 +200,18 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 		return hi;
 	}
 	
-	private static Hop batchNormTest(Hop parent, Hop hi, int pos) 
-	{		
+	private static Hop batchNormTest(Hop parent, Hop hi, int pos) {
 		// norm = bias_multiply(bias_add(X, -mean), 1/sqrt(var+eps))
 		// hi = bias_add(bias_multiply(norm, gamma), beta)
 		// 2x for input and output and 1x for overhead
-		if( isBiasAdd(hi) && isBiasMultiply(getFirstInput(hi)) && fitsOnGPU(hi, 3) ) {	
+		if( isBiasAdd(hi) && isBiasMultiply(getFirstInput(hi)) && fitsOnGPU(hi, 3) ) {
 			Hop norm = getFirstInput(getFirstInput(hi));
 			if(isBiasMultiply(norm) && isBiasAdd(getFirstInput(norm)) 
 					&& isUnaryMinus(getSecondInput(getFirstInput(norm)))
 					&& isOneDivideBySqrt(getSecondInput(norm))) {
 				double eps = 0;
 				Hop var = getFirstInput(getSecondInput(getSecondInput(norm)));
-				if(var instanceof BinaryOp && ((BinaryOp) var).getOp() == OpOp2.PLUS &&
+				if( HopRewriteUtils.isBinary(var, OpOp2.PLUS) &&
 					(getFirstInput(var) instanceof LiteralOp || getSecondInput(var) instanceof LiteralOp)) {
 					// eps + ema_var
 					if(getFirstInput(var) instanceof LiteralOp) {
@@ -248,10 +241,9 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 							OpOpDnn.BATCH_NORM2D_TEST, inHops);
 					return HopRewriteUtils.rewireAllParentChildReferences(hi, newHop);
 				}
-			}			
+			}
 		}
 		
 		return hi;
 	}
-
 }


[3/3] systemml git commit: [SYSTEMML-2431] Fix codegen multi-agg compilation w/ interleaved MMs

Posted by mb...@apache.org.
[SYSTEMML-2431] Fix codegen multi-agg compilation w/ interleaved MMs

This patch fixes special cases of compiling code generation plans of
multi-aggregates with interleaved matrix multiplications (e.g.,
t(X)%*%X, t(X)%*Y, t(Y)%*%Y) over transient reads which require
dedicated handling of data operators.


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

Branch: refs/heads/master
Commit: f1bf97baf342035764c676b50d361e36e2bbae62
Parents: d065c3d
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Jul 12 17:46:30 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Jul 12 17:46:30 2018 -0700

----------------------------------------------------------------------
 .../sysml/hops/codegen/template/TemplateCell.java     | 14 ++++++++++----
 1 file changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/f1bf97ba/src/main/java/org/apache/sysml/hops/codegen/template/TemplateCell.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateCell.java b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateCell.java
index 3789b3f..d4cb8fc 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateCell.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateCell.java
@@ -188,11 +188,17 @@ public class TemplateCell extends TemplateBase
 				&& (me.type!=TemplateType.MAGG || memo.contains(c.getHopID(), TemplateType.CELL)))
 				rConstructCplan(c, memo, tmp, inHops, compileLiterals);
 			else if( me!=null && (me.type==TemplateType.MAGG || me.type==TemplateType.CELL) 
-					&& HopRewriteUtils.isMatrixMultiply(hop) && i==0 ) //skip transpose
-				rConstructCplan(c.getInput().get(0), memo, tmp, inHops, compileLiterals);
+					&& HopRewriteUtils.isMatrixMultiply(hop) && i==0 ) { //skip transpose
+				if( c.getInput().get(0) instanceof DataOp ) {
+					tmp.put(c.getInput().get(0).getHopID(),
+						TemplateUtils.createCNodeData(c.getInput().get(0), compileLiterals));
+					inHops.add(c.getInput().get(0));
+				}
+				else
+					rConstructCplan(c.getInput().get(0), memo, tmp, inHops, compileLiterals);
+			}
 			else {
-				CNodeData cdata = TemplateUtils.createCNodeData(c, compileLiterals);
-				tmp.put(c.getHopID(), cdata);
+				tmp.put(c.getHopID(), TemplateUtils.createCNodeData(c, compileLiterals));
 				inHops.add(c);
 			}
 		}


[2/3] systemml git commit: [SYSTEMML-2430] Codegen support for rowMeans in row templates

Posted by mb...@apache.org.
[SYSTEMML-2430] Codegen support for rowMeans in row templates

This patch adds codegen support for unary aggregate rowMeans to the
codegen row templates. In detail, this includes extended compiler
support for the necessary dense/sparse vector primitives. Furthermore,
this also cleans up some convenience methods for full aggregates on
matrix blocks.


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

Branch: refs/heads/master
Commit: d065c3d13c15f8a8bb4c5e882856e6b0d648675d
Parents: b429551
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Jul 12 17:11:14 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Jul 12 17:11:14 2018 -0700

----------------------------------------------------------------------
 .../sysml/hops/codegen/SpoofCompiler.java       |  2 +-
 .../sysml/hops/codegen/cplan/CNodeUnary.java    |  6 +-
 .../hops/codegen/template/TemplateRow.java      |  2 +-
 .../runtime/codegen/LibSpoofPrimitives.java     |  8 ++
 .../sysml/runtime/matrix/data/MatrixBlock.java  | 27 ++++--
 .../codegen/CPlanVectorPrimitivesTest.java      | 20 +++-
 .../functions/codegen/RowAggTmplTest.java       | 18 +++-
 .../scripts/functions/codegen/rowAggPattern44.R | 99 ++++++++++++++++++++
 .../functions/codegen/rowAggPattern44.dml       | 35 +++++++
 9 files changed, 199 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java b/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
index fd012ec..8302eca 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
@@ -102,7 +102,7 @@ public class SpoofCompiler
 	private static final Log LOG = LogFactory.getLog(SpoofCompiler.class.getName());
 	
 	//internal configuration flags
-	public static final boolean LDEBUG                 = false;
+	public static final boolean LDEBUG                 = true;
 	public static CompilerType JAVA_COMPILER           = CompilerType.JANINO; 
 	public static PlanSelector PLAN_SEL_POLICY         = PlanSelector.FUSE_COST_BASED_V2; 
 	public static final IntegrationType INTEGRATION    = IntegrationType.RUNTIME;

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/main/java/org/apache/sysml/hops/codegen/cplan/CNodeUnary.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/cplan/CNodeUnary.java b/src/main/java/org/apache/sysml/hops/codegen/cplan/CNodeUnary.java
index b269139..ba41fad 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/cplan/CNodeUnary.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/cplan/CNodeUnary.java
@@ -32,7 +32,8 @@ public class CNodeUnary extends CNode
 {
 	public enum UnaryType {
 		LOOKUP_R, LOOKUP_C, LOOKUP_RC, LOOKUP0, //codegen specific
-		ROW_SUMS, ROW_SUMSQS, ROW_MINS, ROW_MAXS, ROW_COUNTNNZS, //codegen specific
+		ROW_SUMS, ROW_SUMSQS, ROW_COUNTNNZS, //codegen specific
+		ROW_MEANS, ROW_MINS, ROW_MAXS,
 		VECT_EXP, VECT_POW2, VECT_MULT2, VECT_SQRT, VECT_LOG,
 		VECT_ABS, VECT_ROUND, VECT_CEIL, VECT_FLOOR, VECT_SIGN, 
 		VECT_SIN, VECT_COS, VECT_TAN, VECT_ASIN, VECT_ACOS, VECT_ATAN, 
@@ -54,6 +55,7 @@ public class CNodeUnary extends CNode
 				case ROW_SUMSQS:
 				case ROW_MINS:
 				case ROW_MAXS:
+				case ROW_MEANS:
 				case ROW_COUNTNNZS: {
 					String vectName = StringUtils.capitalize(name().substring(4, name().length()-1).toLowerCase());
 					return sparse ? "    double %TMP% = LibSpoofPrimitives.vect"+vectName+"(%IN1v%, %IN1i%, %POS1%, alen, len);\n": 
@@ -249,6 +251,7 @@ public class CNodeUnary extends CNode
 			case ROW_SUMSQS: return "u(Rsq+)";
 			case ROW_MINS:   return "u(Rmin)";
 			case ROW_MAXS:   return "u(Rmax)";
+			case ROW_MEANS:  return "u(Rmean)";
 			case ROW_COUNTNNZS: return "u(Rnnz)";
 			case VECT_EXP:
 			case VECT_POW2:
@@ -319,6 +322,7 @@ public class CNodeUnary extends CNode
 			case ROW_SUMSQS:
 			case ROW_MINS:
 			case ROW_MAXS:
+			case ROW_MEANS:
 			case ROW_COUNTNNZS:
 			case EXP:
 			case LOOKUP_R:

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/main/java/org/apache/sysml/hops/codegen/template/TemplateRow.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateRow.java b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateRow.java
index 15bec59..9df67d0 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateRow.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateRow.java
@@ -67,7 +67,7 @@ import org.apache.sysml.runtime.matrix.data.Pair;
 
 public class TemplateRow extends TemplateBase 
 {
-	private static final Hop.AggOp[] SUPPORTED_ROW_AGG = new AggOp[]{AggOp.SUM, AggOp.MIN, AggOp.MAX};
+	private static final Hop.AggOp[] SUPPORTED_ROW_AGG = new AggOp[]{AggOp.SUM, AggOp.MIN, AggOp.MAX, AggOp.MEAN};
 	private static final Hop.OpOp1[] SUPPORTED_VECT_UNARY = new OpOp1[]{
 			OpOp1.EXP, OpOp1.SQRT, OpOp1.LOG, OpOp1.ABS, OpOp1.ROUND, OpOp1.CEIL, OpOp1.FLOOR, OpOp1.SIGN,
 			OpOp1.SIN, OpOp1.COS, OpOp1.TAN, OpOp1.ASIN, OpOp1.ACOS, OpOp1.ATAN, OpOp1.SINH, OpOp1.COSH, OpOp1.TANH,

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/main/java/org/apache/sysml/runtime/codegen/LibSpoofPrimitives.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/codegen/LibSpoofPrimitives.java b/src/main/java/org/apache/sysml/runtime/codegen/LibSpoofPrimitives.java
index 42d61d3..fc0c1d2 100644
--- a/src/main/java/org/apache/sysml/runtime/codegen/LibSpoofPrimitives.java
+++ b/src/main/java/org/apache/sysml/runtime/codegen/LibSpoofPrimitives.java
@@ -340,6 +340,14 @@ public class LibSpoofPrimitives
 		return alen;
 	}
 	
+	public static double vectMean(double[] a, int ai, int len) {
+		return vectSum(a, ai, len) / len;
+	} 
+	
+	public static double vectMean(double[] avals, int[] aix, int ai, int alen, int len) {
+		return vectSum(avals, aix, ai, alen, len) / len;
+	}
+	
 	//custom vector div
 	
 	public static void vectDivAdd(double[] a, double bval, double[] c, int ai, int ci, int len) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/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 4bb0c5c..5c50326 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
@@ -64,6 +64,7 @@ import org.apache.sysml.runtime.functionobjects.RevIndex;
 import org.apache.sysml.runtime.functionobjects.SortIndex;
 import org.apache.sysml.runtime.functionobjects.SwapIndex;
 import org.apache.sysml.runtime.functionobjects.TernaryValueFunction.ValueFunctionWithConstant;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
 import org.apache.sysml.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysml.runtime.instructions.cp.KahanObject;
 import org.apache.sysml.runtime.instructions.cp.ScalarObject;
@@ -793,17 +794,26 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	}
 	
 	/**
+	 * Wrapper method for reduceall-mean of a matrix.
+	 * 
+	 * @return ?
+	 */
+	public double mean() {
+		MatrixBlock out = new MatrixBlock(1, 3, false);
+		LibMatrixAgg.aggregateUnaryMatrix(this, out,
+			InstructionUtils.parseBasicAggregateUnaryOperator("uamean", 1));
+		return out.quickGetValue(0, 0);
+	}
+	
+	/**
 	 * Wrapper method for reduceall-min of a matrix.
 	 * 
 	 * @return ?
 	 */
 	public double min() {
-		//construct operator
-		AggregateOperator aop = new AggregateOperator(Double.POSITIVE_INFINITY, Builtin.getBuiltinFnObject("min"));
-		AggregateUnaryOperator auop = new AggregateUnaryOperator( aop, ReduceAll.getReduceAllFnObject());
-		//execute operation
 		MatrixBlock out = new MatrixBlock(1, 1, false);
-		LibMatrixAgg.aggregateUnaryMatrix(this, out, auop);
+		LibMatrixAgg.aggregateUnaryMatrix(this, out,
+			InstructionUtils.parseBasicAggregateUnaryOperator("uamin", 1));
 		return out.quickGetValue(0, 0);
 	}
 	
@@ -813,12 +823,9 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	 * @return ?
 	 */
 	public double max() {
-		//construct operator
-		AggregateOperator aop = new AggregateOperator(Double.NEGATIVE_INFINITY, Builtin.getBuiltinFnObject("max"));
-		AggregateUnaryOperator auop = new AggregateUnaryOperator( aop, ReduceAll.getReduceAllFnObject());
-		//execute operation
 		MatrixBlock out = new MatrixBlock(1, 1, false);
-		LibMatrixAgg.aggregateUnaryMatrix(this, out, auop);
+		LibMatrixAgg.aggregateUnaryMatrix(this, out,
+			InstructionUtils.parseBasicAggregateUnaryOperator("uamax", 1));
 		return out.quickGetValue(0, 0);
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/test/java/org/apache/sysml/test/integration/functions/codegen/CPlanVectorPrimitivesTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/codegen/CPlanVectorPrimitivesTest.java b/src/test/java/org/apache/sysml/test/integration/functions/codegen/CPlanVectorPrimitivesTest.java
index 23dd6fb..562005e 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/codegen/CPlanVectorPrimitivesTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/codegen/CPlanVectorPrimitivesTest.java
@@ -89,6 +89,16 @@ public class CPlanVectorPrimitivesTest extends AutomatedTestBase
 		testVectorAggPrimitive(UnaryType.ROW_MAXS, InputType.VECTOR_SPARSE);
 	}
 	
+	@Test
+	public void testVectorMeansDense() {
+		testVectorAggPrimitive(UnaryType.ROW_MEANS, InputType.VECTOR_DENSE);
+	}
+	
+	@Test
+	public void testVectorMeansSparse() {
+		testVectorAggPrimitive(UnaryType.ROW_MEANS, InputType.VECTOR_SPARSE);
+	}
+	
 	//support unary vector primitives (pow2/mult2 current excluded because not unary)
 	
 	@Test
@@ -716,7 +726,8 @@ public class CPlanVectorPrimitivesTest extends AutomatedTestBase
 			MatrixBlock in = MatrixBlock.randOperations(m, n, sparsity, -1, 1, "uniform", 7);
 			
 			//get vector primitive via reflection
-			String meName = "vect"+StringUtils.camelize(aggtype.name().split("_")[1].substring(0, 3));
+			String tmp = StringUtils.camelize(aggtype.name().split("_")[1]);
+			String meName = "vect"+tmp.substring(0, tmp.length()-1);
 			Method me = (type1 == InputType.VECTOR_DENSE) ? 
 				LibSpoofPrimitives.class.getMethod(meName, new Class[]{double[].class, int.class, int.class}) : 
 				LibSpoofPrimitives.class.getMethod(meName, new Class[]{double[].class, int[].class, int.class, int.class, int.class});
@@ -732,9 +743,10 @@ public class CPlanVectorPrimitivesTest extends AutomatedTestBase
 				MatrixBlock in2 = in.slice(i, i, 0, n-1, new MatrixBlock());
 				Double ret2 = -1d;
 				switch( aggtype ) {
-					case ROW_SUMS: ret2 = in2.sum(); break;
-					case ROW_MAXS: ret2 = in2.max(); break;
-					case ROW_MINS: ret2 = in2.min(); break;	
+					case ROW_SUMS:  ret2 = in2.sum(); break;
+					case ROW_MAXS:  ret2 = in2.max(); break;
+					case ROW_MINS:  ret2 = in2.min(); break;
+					case ROW_MEANS: ret2 = in2.mean(); break;
 				}
 				
 				//compare results

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/test/java/org/apache/sysml/test/integration/functions/codegen/RowAggTmplTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/codegen/RowAggTmplTest.java b/src/test/java/org/apache/sysml/test/integration/functions/codegen/RowAggTmplTest.java
index bec9be2..04891d0 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/codegen/RowAggTmplTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/codegen/RowAggTmplTest.java
@@ -80,6 +80,7 @@ public class RowAggTmplTest extends AutomatedTestBase
 	private static final String TEST_NAME41 = TEST_NAME+"41"; //X*rowSums(X/seq(1,N)+t(seq(M,1)))
 	private static final String TEST_NAME42 = TEST_NAME+"42"; //X/rowSums(min(X, Y, Z))
 	private static final String TEST_NAME43 = TEST_NAME+"43"; //bias_add(X,B) + bias_mult(X,B)
+	private static final String TEST_NAME44 = TEST_NAME+"44"; //maxpool(X - mean(X));
 	
 	private static final String TEST_DIR = "functions/codegen/";
 	private static final String TEST_CLASS_DIR = TEST_DIR + RowAggTmplTest.class.getSimpleName() + "/";
@@ -91,7 +92,7 @@ public class RowAggTmplTest extends AutomatedTestBase
 	@Override
 	public void setUp() {
 		TestUtils.clearAssertionInformation();
-		for(int i=1; i<=43; i++)
+		for(int i=1; i<=44; i++)
 			addTestConfiguration( TEST_NAME+i, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME+i, new String[] { String.valueOf(i) }) );
 	}
 	
@@ -739,6 +740,21 @@ public class RowAggTmplTest extends AutomatedTestBase
 	public void testCodegenRowAgg43SP() {
 		testCodegenIntegration( TEST_NAME43, false, ExecType.SPARK );
 	}
+	
+	@Test
+	public void testCodegenRowAggRewrite44CP() {
+		testCodegenIntegration( TEST_NAME44, true, ExecType.CP );
+	}
+
+	@Test
+	public void testCodegenRowAgg44CP() {
+		testCodegenIntegration( TEST_NAME44, false, ExecType.CP );
+	}
+
+	@Test
+	public void testCodegenRowAgg44SP() {
+		testCodegenIntegration( TEST_NAME44, false, ExecType.SPARK );
+	}
 
 	private void testCodegenIntegration( String testname, boolean rewrites, ExecType instType )
 	{

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/test/scripts/functions/codegen/rowAggPattern44.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/codegen/rowAggPattern44.R b/src/test/scripts/functions/codegen/rowAggPattern44.R
new file mode 100644
index 0000000..99ba0b0
--- /dev/null
+++ b/src/test/scripts/functions/codegen/rowAggPattern44.R
@@ -0,0 +1,99 @@
+#-------------------------------------------------------------
+#
+# 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")
+library("matrixStats") 
+
+imgSize=8
+numImg=16
+numChannels=4
+poolSize1=imgSize*imgSize
+poolSize2=1
+stride=1
+pad=0
+
+X = matrix(seq(1, numImg*numChannels*imgSize*imgSize), numImg, numChannels*imgSize*imgSize, byrow=TRUE)
+X = X - rowMeans(X)
+
+pad_image <- function(img, Hin, Win, padh, padw){
+  C = nrow(img)
+  img_padded = matrix(0, C, (Hin+2*padh)*(Win+2*padw))  # zeros
+  for (c in 1:C) {
+    img_slice = matrix(img[c,], Hin, Win, byrow=TRUE)  # depth slice C reshaped
+    img_padded_slice = matrix(0, Hin+2*padh, Win+2*padw)
+    img_padded_slice[(padh+1):(padh+Hin), (padw+1):(padw+Win)] = img_slice
+    img_padded[c,] = matrix(t(img_padded_slice), 1, (Hin+2*padh)*(Win+2*padw))  # reshape
+  }
+  img_padded
+}
+
+im2col <- function(img, Hin, Win, Hf, Wf, strideh, stridew) {
+  C = nrow(img)
+  Hout = as.integer((Hin - Hf) / strideh + 1)
+  Wout = as.integer((Win - Wf) / stridew + 1)
+
+  img_cols = matrix(0, C*Hf*Wf, Hout*Wout, byrow=TRUE)  # zeros
+  for (hout in 1:Hout) {  # all output rows
+    hin = (hout-1) * strideh + 1
+    for (wout in 1:Wout) {  # all output columns
+      win = (wout-1) * stridew + 1
+      # Extract a local patch of the input image corresponding spatially to the filter sizes.
+      img_patch = matrix(0, C, Hf*Wf, byrow=TRUE)  # zeros
+      for (c in 1:C) {  # all channels
+        img_slice = matrix(img[c,], Hin, Win, byrow=TRUE)  # reshape
+        img_patch[c,] = matrix(t(img_slice[hin:(hin+Hf-1), win:(win+Wf-1)]), 1, Hf*Wf)
+      }
+      img_cols[,(hout-1)*Wout + wout] = matrix(t(img_patch), C*Hf*Wf, 1)  # reshape
+    }
+  }
+  img_cols
+}
+
+max_pool <- function(X, N, C, Hin, Win, Hf, Wf,
+                   strideh, stridew) {
+  Hout = as.integer((Hin - Hf) / strideh + 1)
+  Wout = as.integer((Win - Wf) / stridew + 1)
+
+  # Create output volume
+  out = matrix(0, N, C*Hout*Wout, byrow=TRUE)
+
+  # Max pooling - im2col implementation
+  for (n in 1:N) {  # all examples
+    img = matrix(X[n,], C, Hin*Win, byrow=TRUE)  # reshape
+    img_maxes = matrix(0, C, Hout*Wout, byrow=TRUE)  # zeros
+
+    for (c in 1:C) {  # all channels
+      # Extract local image slice patches into columns with im2col, of shape (Hf*Wf, Hout*Wout)
+      img_slice_cols = im2col(matrix(t(img[c,]), 1, Hin*Win) , Hin, Win, Hf, Wf, strideh, stridew)
+
+      # Max pooling on patches
+      img_maxes[c,] = colMaxs(img_slice_cols)
+    }
+
+    out[n,] = matrix(t(img_maxes), 1, C*Hout*Wout)
+  }
+  
+  out
+}
+
+R = max_pool(X, numImg, numChannels, imgSize*imgSize, 1, poolSize1, poolSize2, stride, stride)
+
+writeMM(as(R,"CsparseMatrix"), paste(args[2], "S", sep=""))

http://git-wip-us.apache.org/repos/asf/systemml/blob/d065c3d1/src/test/scripts/functions/codegen/rowAggPattern44.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/codegen/rowAggPattern44.dml b/src/test/scripts/functions/codegen/rowAggPattern44.dml
new file mode 100644
index 0000000..f236451
--- /dev/null
+++ b/src/test/scripts/functions/codegen/rowAggPattern44.dml
@@ -0,0 +1,35 @@
+#-------------------------------------------------------------
+#
+# 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.
+# 
+#-------------------------------------------------------------
+imgSize=8
+numImg=16
+numChannels=4
+poolSize1=imgSize*imgSize
+poolSize2=1
+stride=1
+pad=0
+
+X = matrix(seq(1, numImg*numChannels*imgSize*imgSize), rows=numImg, cols=numChannels*imgSize*imgSize);
+while(FALSE){}
+
+X = X - rowMeans(X);
+R = max_pool(X, stride=[stride, stride], padding=[pad, pad], input_shape=[numImg, numChannels, imgSize*imgSize, 1], pool_size=[poolSize1, poolSize2]);
+
+write(R, $1, format="text");