You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by re...@apache.org on 2017/11/03 18:02:51 UTC

[49/50] [abbrv] systemml git commit: [SYSTEMML-1983] New codegen cplan rewrite framework (micro optims)

[SYSTEMML-1983] New codegen cplan rewrite framework (micro optims)

This patch refactors the code generator by moving smaller micro
optimizations from the templates into a new cplan rewrite framework to
avoid redundancy and inconsistency across templates and to improve
debuggability. The goal is NOT to provide a fusion-aware rewrite
framework, but simply to apply smaller rewrites for better code quality.
An example rewrite is rowSums(X!=0) -> rowNnzs(X), which avoids an
unnecessary row intermediate and is realized as a meta data operation
for sparse input rows.


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

Branch: refs/heads/master
Commit: e1f5866a5c2e2100d68124b74a33a9022e89dd09
Parents: ee6060b
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Nov 2 00:15:11 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Nov 2 00:39:18 2017 -0700

----------------------------------------------------------------------
 .../sysml/hops/codegen/SpoofCompiler.java       |  43 +-----
 .../hops/codegen/template/CPlanOpRewriter.java  | 130 +++++++++++++++++++
 .../hops/codegen/template/TemplateCell.java     |  11 +-
 .../codegen/template/TemplateOuterProduct.java  |   9 +-
 .../hops/codegen/template/TemplateRow.java      |  10 --
 .../functions/codegen/RowAggTmplTest.java       |  20 ++-
 .../scripts/functions/codegen/rowAggPattern34.R |  32 +++++
 .../functions/codegen/rowAggPattern34.dml       |  29 +++++
 8 files changed, 219 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/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 4af8540..51cd0a2 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
@@ -39,7 +39,6 @@ import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.codegen.cplan.CNode;
-import org.apache.sysml.hops.codegen.cplan.CNodeBinary.BinType;
 import org.apache.sysml.hops.codegen.cplan.CNodeCell;
 import org.apache.sysml.hops.codegen.cplan.CNodeData;
 import org.apache.sysml.hops.codegen.cplan.CNodeMultiAgg;
@@ -53,7 +52,6 @@ import org.apache.sysml.hops.codegen.opt.PlanSelectionFuseCostBased;
 import org.apache.sysml.hops.codegen.opt.PlanSelectionFuseCostBasedV2;
 import org.apache.sysml.hops.codegen.opt.PlanSelectionFuseNoRedundancy;
 import org.apache.sysml.hops.codegen.cplan.CNodeTpl;
-import org.apache.sysml.hops.codegen.cplan.CNodeUnary.UnaryType;
 import org.apache.sysml.hops.codegen.template.TemplateBase;
 import org.apache.sysml.hops.codegen.template.TemplateBase.CloseType;
 import org.apache.sysml.hops.codegen.template.TemplateBase.TemplateType;
@@ -61,6 +59,7 @@ import org.apache.sysml.hops.codegen.template.CPlanCSERewriter;
 import org.apache.sysml.hops.codegen.template.CPlanMemoTable;
 import org.apache.sysml.hops.codegen.template.CPlanMemoTable.MemoTableEntry;
 import org.apache.sysml.hops.codegen.template.CPlanMemoTable.MemoTableEntrySet;
+import org.apache.sysml.hops.codegen.template.CPlanOpRewriter;
 import org.apache.sysml.hops.codegen.template.TemplateUtils;
 import org.apache.sysml.hops.recompile.RecompileStatus;
 import org.apache.sysml.hops.recompile.Recompiler;
@@ -68,7 +67,6 @@ import org.apache.sysml.hops.AggUnaryOp;
 import org.apache.sysml.hops.Hop;
 import org.apache.sysml.hops.Hop.OpOp1;
 import org.apache.sysml.hops.HopsException;
-import org.apache.sysml.hops.LiteralOp;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.hops.rewrite.HopRewriteUtils;
 import org.apache.sysml.hops.rewrite.ProgramRewriteStatus;
@@ -684,13 +682,15 @@ public class SpoofCompiler
 	private static HashMap<Long, Pair<Hop[],CNodeTpl>> cleanupCPlans(CPlanMemoTable memo, HashMap<Long, Pair<Hop[],CNodeTpl>> cplans) 
 	{
 		HashMap<Long, Pair<Hop[],CNodeTpl>> cplans2 = new HashMap<>();
+		CPlanOpRewriter rewriter = new CPlanOpRewriter();
 		CPlanCSERewriter cse = new CPlanCSERewriter();
 		
 		for( Entry<Long, Pair<Hop[],CNodeTpl>> e : cplans.entrySet() ) {
 			CNodeTpl tpl = e.getValue().getValue();
 			Hop[] inHops = e.getValue().getKey();
 			
-			//perform common subexpression elimination
+			//perform simplifications and cse rewrites
+			tpl = rewriter.simplifyCPlan(tpl);
 			tpl = cse.eliminateCommonSubexpressions(tpl);
 			
 			//update input hops (order-preserving)
@@ -727,10 +727,6 @@ public class SpoofCompiler
 			else
 				rFindAndRemoveLookup(tpl.getOutput(), in1, !(tpl instanceof CNodeRow));
 			
-			//remove unnecessary neq 0 on main input of outer template
-			if( tpl instanceof CNodeOuterProduct )
-				rFindAndRemoveBinaryMS(tpl.getOutput(), in1, BinType.NOTEQUAL, "0", "1");
-			
 			//remove invalid row templates (e.g., unsatisfied blocksize constraint)
 			if( tpl instanceof CNodeRow ) {
 				//check for invalid row cplan over column vector
@@ -810,37 +806,6 @@ public class SpoofCompiler
 		}
 	}
 	
-	@SuppressWarnings("unused")
-	private static void rFindAndRemoveUnary(CNode node, CNodeData mainInput, UnaryType type) {
-		for( int i=0; i<node.getInput().size(); i++ ) {
-			CNode tmp = node.getInput().get(i);
-			if( TemplateUtils.isUnary(tmp, type) && tmp.getInput().get(0) instanceof CNodeData
-				&& ((CNodeData)tmp.getInput().get(0)).getHopID()==mainInput.getHopID() )
-			{
-				node.getInput().set(i, tmp.getInput().get(0));
-			}
-			else
-				rFindAndRemoveUnary(tmp, mainInput, type);
-		}
-	}
-	
-	private static void rFindAndRemoveBinaryMS(CNode node, CNodeData mainInput, BinType type, String lit, String replace) {
-		for( int i=0; i<node.getInput().size(); i++ ) {
-			CNode tmp = node.getInput().get(i);
-			if( TemplateUtils.isBinary(tmp, type) && tmp.getInput().get(1).isLiteral()
-				&& tmp.getInput().get(1).getVarname().equals(lit)
-				&& tmp.getInput().get(0) instanceof CNodeData
-				&& ((CNodeData)tmp.getInput().get(0)).getHopID()==mainInput.getHopID() )
-			{
-				CNodeData cnode = new CNodeData(new LiteralOp(replace));
-				cnode.setLiteral(true);
-				node.getInput().set(i, cnode);
-			}
-			else
-				rFindAndRemoveBinaryMS(tmp, mainInput, type, lit, replace);
-		}
-	}
-	
 	private static boolean rHasLookupRC1(CNode node, CNodeData mainInput, boolean includeRC1) {
 		boolean ret = false;
 		for( int i=0; i<node.getInput().size() && !ret; i++ ) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/src/main/java/org/apache/sysml/hops/codegen/template/CPlanOpRewriter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/template/CPlanOpRewriter.java b/src/main/java/org/apache/sysml/hops/codegen/template/CPlanOpRewriter.java
new file mode 100644
index 0000000..8ec750c
--- /dev/null
+++ b/src/main/java/org/apache/sysml/hops/codegen/template/CPlanOpRewriter.java
@@ -0,0 +1,130 @@
+/*
+ * 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.hops.codegen.template;
+
+import java.util.ArrayList;
+
+import org.apache.sysml.hops.LiteralOp;
+import org.apache.sysml.hops.codegen.cplan.CNode;
+import org.apache.sysml.hops.codegen.cplan.CNodeData;
+import org.apache.sysml.hops.codegen.cplan.CNodeMultiAgg;
+import org.apache.sysml.hops.codegen.cplan.CNodeOuterProduct;
+import org.apache.sysml.hops.codegen.cplan.CNodeTpl;
+import org.apache.sysml.hops.codegen.cplan.CNodeUnary;
+import org.apache.sysml.hops.codegen.cplan.CNodeBinary.BinType;
+import org.apache.sysml.hops.codegen.cplan.CNodeUnary.UnaryType;
+
+/**
+ * This cplan rewriter is meant to be the central place for any cplan 
+ * enhancements before code generation. These rewrites do not aim to
+ * handle reorderings or other algebraic simplifications but rather
+ * focus on low-level simplifications to produce better code, while
+ * keeping the cplan construction of the individual templates clean
+ * and without unnecessary redundancy.
+ * 
+ * Assumption: This rewriter should be called before CSE as these
+ * rewrites potentially destroy common subexpressions.
+ */
+public class CPlanOpRewriter 
+{
+	public CNodeTpl simplifyCPlan(CNodeTpl tpl) {
+		//apply template specific rewrites
+		tpl = rewriteRemoveOuterNeq0(tpl); // Outer(a!=0) -> Outer(1)
+		
+		//apply operation specific rewrites
+		if( tpl instanceof CNodeMultiAgg ) {
+			ArrayList<CNode> outputs = ((CNodeMultiAgg)tpl).getOutputs();
+			for( int i=0; i< outputs.size(); i++ )
+				outputs.set(i, rSimplifyCNode(outputs.get(i)));
+		}
+		else {
+			tpl.setOutput(rSimplifyCNode(tpl.getOutput()));
+		}
+		
+		return tpl;
+	}
+	
+	private static CNode rSimplifyCNode(CNode node) {
+		//process children recursively
+		for(int i=0; i<node.getInput().size(); i++)
+			node.getInput().set(i, rSimplifyCNode(node.getInput().get(i)));
+		
+		//apply all node-local simplification rewrites
+		node = rewriteRowCountNnz(node); //rowSums(X!=0) -> rowNnz(X)
+		node = rewriteRowSumSq(node);    //rowSums(X^2) -> rowSumSqs(X)
+		node = rewriteBinaryPow2(node);  //x^2 -> x*x
+		node = rewriteBinaryMult2(node); //x*2 -> x+x;
+		return node;
+	}
+	
+	private static CNode rewriteRowCountNnz(CNode node) {
+		return (TemplateUtils.isUnary(node, UnaryType.ROW_SUMS)
+			&& TemplateUtils.isBinary(node.getInput().get(0), BinType.VECT_NOTEQUAL_SCALAR)
+			&& node.getInput().get(0).getInput().get(1).isLiteral()
+			&& node.getInput().get(0).getInput().get(1).getVarname().equals("0")) ?
+			new CNodeUnary(node.getInput().get(0).getInput().get(0), UnaryType.ROW_COUNTNNZS) : node;
+	}
+	
+	private static CNode rewriteRowSumSq(CNode node) {
+		return (TemplateUtils.isUnary(node, UnaryType.ROW_SUMS)
+			&& TemplateUtils.isBinary(node.getInput().get(0), BinType.VECT_POW_SCALAR)
+			&& node.getInput().get(0).getInput().get(1).isLiteral()
+			&& node.getInput().get(0).getInput().get(1).getVarname().equals("2")) ?
+			new CNodeUnary(node.getInput().get(0).getInput().get(0), UnaryType.ROW_SUMSQS) : node;
+	}
+
+	private static CNode rewriteBinaryPow2(CNode node) {
+		return (TemplateUtils.isBinary(node, BinType.POW) 
+			&& node.getInput().get(1).isLiteral()
+			&& node.getInput().get(1).getVarname().equals("2")) ?
+			new CNodeUnary(node.getInput().get(0), UnaryType.POW2) : node;
+	}
+	
+	private static CNode rewriteBinaryMult2(CNode node) {
+		return (TemplateUtils.isBinary(node, BinType.MULT) 
+			&& node.getInput().get(1).isLiteral()
+			&& node.getInput().get(1).getVarname().equals("2")) ?
+			new CNodeUnary(node.getInput().get(0), UnaryType.MULT2) : node;
+	}
+	
+	private static CNodeTpl rewriteRemoveOuterNeq0(CNodeTpl tpl) {
+		if( tpl instanceof CNodeOuterProduct )
+			rFindAndRemoveBinaryMS(tpl.getOutput(), (CNodeData)
+				tpl.getInput().get(0), BinType.NOTEQUAL, "0", "1");
+		return tpl;
+	}
+	
+	private static void rFindAndRemoveBinaryMS(CNode node, CNodeData mainInput, BinType type, String lit, String replace) {
+		for( int i=0; i<node.getInput().size(); i++ ) {
+			CNode tmp = node.getInput().get(i);
+			if( TemplateUtils.isBinary(tmp, type) && tmp.getInput().get(1).isLiteral()
+				&& tmp.getInput().get(1).getVarname().equals(lit)
+				&& tmp.getInput().get(0) instanceof CNodeData
+				&& ((CNodeData)tmp.getInput().get(0)).getHopID()==mainInput.getHopID() )
+			{
+				CNodeData cnode = new CNodeData(new LiteralOp(replace));
+				cnode.setLiteral(true);
+				node.getInput().set(i, cnode);
+			}
+			else
+				rFindAndRemoveBinaryMS(tmp, mainInput, type, lit, replace);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/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 4f3d4f4..fe5a1e7 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
@@ -195,12 +195,9 @@ public class TemplateCell extends TemplateBase
 			cdata1 = TemplateUtils.wrapLookupIfNecessary(cdata1, hop.getInput().get(0));
 			cdata2 = TemplateUtils.wrapLookupIfNecessary(cdata2, hop.getInput().get(1));
 			
-			if( bop.getOp()==OpOp2.POW && cdata2.isLiteral() && cdata2.getVarname().equals("2") )
-				out = new CNodeUnary(cdata1, UnaryType.POW2);
-			else if( bop.getOp()==OpOp2.MULT && cdata2.isLiteral() && cdata2.getVarname().equals("2") )
-				out = new CNodeUnary(cdata1, UnaryType.MULT2);
-			else //default binary	
-				out = new CNodeBinary(cdata1, cdata2, BinType.valueOf(primitiveOpName));
+			//construct binary cnode
+			out = new CNodeBinary(cdata1, cdata2, 
+				BinType.valueOf(primitiveOpName));
 		}
 		else if(hop instanceof TernaryOp) 
 		{
@@ -215,7 +212,7 @@ public class TemplateCell extends TemplateBase
 			
 			//construct ternary cnode, primitive operation derived from OpOp3
 			out = new CNodeTernary(cdata1, cdata2, cdata3, 
-					TernaryType.valueOf(top.getOp().name()));
+				TernaryType.valueOf(top.getOp().name()));
 		}
 		else if( hop instanceof ParameterizedBuiltinOp ) 
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateOuterProduct.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateOuterProduct.java b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateOuterProduct.java
index 256f540..188bac2 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/template/TemplateOuterProduct.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/template/TemplateOuterProduct.java
@@ -174,7 +174,6 @@ public class TemplateOuterProduct extends TemplateBase {
 		}
 		else if(hop instanceof BinaryOp)
 		{
-			BinaryOp bop = (BinaryOp) hop;
 			CNode cdata1 = tmp.get(hop.getInput().get(0).getHopID());
 			CNode cdata2 = tmp.get(hop.getInput().get(1).getHopID());
 			String primitiveOpName = ((BinaryOp)hop).getOp().toString();
@@ -187,12 +186,8 @@ public class TemplateOuterProduct extends TemplateBase {
 			//add lookups if required
 			cdata1 = TemplateUtils.wrapLookupIfNecessary(cdata1, hop.getInput().get(0));
 			cdata2 = TemplateUtils.wrapLookupIfNecessary(cdata2, hop.getInput().get(1));
-			if( bop.getOp()==OpOp2.POW && cdata2.isLiteral() && cdata2.getVarname().equals("2") )
-				out = new CNodeUnary(cdata1, UnaryType.POW2);
-			else if( bop.getOp()==OpOp2.MULT && cdata2.isLiteral() && cdata2.getVarname().equals("2") )
-				out = new CNodeUnary(cdata1, UnaryType.MULT2);
-			else
-				out = new CNodeBinary(cdata1, cdata2, BinType.valueOf(primitiveOpName));
+			
+			out = new CNodeBinary(cdata1, cdata2, BinType.valueOf(primitiveOpName));
 		}
 		else if(hop instanceof AggBinaryOp)
 		{

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/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 b862abf..dc08dbf 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
@@ -257,16 +257,6 @@ public class TemplateRow extends TemplateBase
 			if( ((AggUnaryOp)hop).getDirection() == Direction.Row && HopRewriteUtils.isAggUnaryOp(hop, SUPPORTED_ROW_AGG) ) {
 				if(hop.getInput().get(0).getDim2()==1)
 					out = (cdata1.getDataType()==DataType.SCALAR) ? cdata1 : new CNodeUnary(cdata1,UnaryType.LOOKUP_R);
-				else if( HopRewriteUtils.isAggUnaryOp(hop, AggOp.SUM)
-					&& HopRewriteUtils.isBinaryMatrixScalar(hop.getInput().get(0), OpOp2.NOTEQUAL, 0)
-					&& cdata1 instanceof CNodeBinary ) {
-					out = new CNodeUnary(cdata1.getInput().get(0), UnaryType.ROW_COUNTNNZS);
-				}
-				else if( HopRewriteUtils.isAggUnaryOp(hop, AggOp.SUM)
-					&& HopRewriteUtils.isBinaryMatrixScalar(hop.getInput().get(0), OpOp2.POW, 2)
-					&& cdata1 instanceof CNodeBinary ) {
-					out = new CNodeUnary(cdata1.getInput().get(0), UnaryType.ROW_SUMSQS);
-				}
 				else {
 					String opcode = "ROW_"+((AggUnaryOp)hop).getOp().name().toUpperCase()+"S";
 					out = new CNodeUnary(cdata1, UnaryType.valueOf(opcode));

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/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 5d2015f..b5426ae 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
@@ -70,6 +70,7 @@ public class RowAggTmplTest extends AutomatedTestBase
 	private static final String TEST_NAME31 = TEST_NAME+"31"; //MLogreg - matrix-vector cbind 0s generalized
 	private static final String TEST_NAME32 = TEST_NAME+"32"; //X[, 1] - rowSums(X)
 	private static final String TEST_NAME33 = TEST_NAME+"33"; //Kmeans, inner loop
+	private static final String TEST_NAME34 = TEST_NAME+"34"; //X / rowSums(X!=0)
 	
 	private static final String TEST_DIR = "functions/codegen/";
 	private static final String TEST_CLASS_DIR = TEST_DIR + RowAggTmplTest.class.getSimpleName() + "/";
@@ -81,7 +82,7 @@ public class RowAggTmplTest extends AutomatedTestBase
 	@Override
 	public void setUp() {
 		TestUtils.clearAssertionInformation();
-		for(int i=1; i<=33; i++)
+		for(int i=1; i<=34; i++)
 			addTestConfiguration( TEST_NAME+i, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME+i, new String[] { String.valueOf(i) }) );
 	}
 	
@@ -580,6 +581,21 @@ public class RowAggTmplTest extends AutomatedTestBase
 		testCodegenIntegration( TEST_NAME33, false, ExecType.SPARK );
 	}
 	
+	@Test
+	public void testCodegenRowAggRewrite34CP() {
+		testCodegenIntegration( TEST_NAME34, true, ExecType.CP );
+	}
+	
+	@Test
+	public void testCodegenRowAgg34CP() {
+		testCodegenIntegration( TEST_NAME34, false, ExecType.CP );
+	}
+	
+	@Test
+	public void testCodegenRowAgg34SP() {
+		testCodegenIntegration( TEST_NAME34, false, ExecType.SPARK );
+	}
+	
 	private void testCodegenIntegration( String testname, boolean rewrites, ExecType instType )
 	{	
 		boolean oldFlag = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;
@@ -601,7 +617,7 @@ public class RowAggTmplTest extends AutomatedTestBase
 			
 			String HOME = SCRIPT_DIR + TEST_DIR;
 			fullDMLScriptName = HOME + testname + ".dml";
-			programArgs = new String[]{"-explain", "recompile_hops", "-stats", "-args", output("S") };
+			programArgs = new String[]{"-explain", "recompile_runtime", "-stats", "-args", output("S") };
 			
 			fullRScriptName = HOME + testname + ".R";
 			rCmd = getRCmd(inputDir(), expectedDir());			

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/src/test/scripts/functions/codegen/rowAggPattern34.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/codegen/rowAggPattern34.R b/src/test/scripts/functions/codegen/rowAggPattern34.R
new file mode 100644
index 0000000..2deea5d
--- /dev/null
+++ b/src/test/scripts/functions/codegen/rowAggPattern34.R
@@ -0,0 +1,32 @@
+#-------------------------------------------------------------
+#
+# 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)
+options(digits=22)
+library("Matrix")
+library("matrixStats")
+
+X = matrix(seq(1,6000)/6000, 300, 20, byrow=TRUE);
+X[,6:20] = matrix(0, 300, 15);
+
+R = X / rowSums(X!=0);
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "S", sep="")); 

http://git-wip-us.apache.org/repos/asf/systemml/blob/e1f5866a/src/test/scripts/functions/codegen/rowAggPattern34.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/codegen/rowAggPattern34.dml b/src/test/scripts/functions/codegen/rowAggPattern34.dml
new file mode 100644
index 0000000..12d9b7f
--- /dev/null
+++ b/src/test/scripts/functions/codegen/rowAggPattern34.dml
@@ -0,0 +1,29 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+X = matrix(seq(1,6000)/6000, 300, 20);
+X[,6:20] = matrix(0, 300, 15);
+while(FALSE){}
+
+R = X / rowSums(X!=0);
+
+write(R, $1)