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/11/02 07:38:14 UTC

[1/3] systemml git commit: [SYSTEMML-1981] Fix graceful value type casts on function invocations

Repository: systemml
Updated Branches:
  refs/heads/master fc4789165 -> e1f5866a5


[SYSTEMML-1981] Fix graceful value type casts on function invocations

The existing graceful value type casts - on function invocations with
wrong value type - incorrectly took the input parameter type instead of
the function parameter type for comparison. This patch fixes this
casting issue, which avoids unnecessary warnings on function invocations
and recompilation exceptions for boolean parameters.


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

Branch: refs/heads/master
Commit: a2f0598c606db16e75790cdbc3dbe37dc32d89a0
Parents: fc47891
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Nov 1 21:37:49 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Nov 2 00:39:16 2017 -0700

----------------------------------------------------------------------
 .../cp/FunctionCallCPInstruction.java           | 29 ++++----
 .../functions/misc/FunctionReturnTest.java      | 78 ++++++++++++++++++++
 .../functions/misc/FunctionReturnBoolean.dml    | 34 +++++++++
 .../functions/misc/FunctionReturnInt.dml        | 34 +++++++++
 .../functions/misc/ZPackageSuite.java           |  1 +
 5 files changed, 160 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/a2f0598c/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 402d4a5..b901dfc 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
@@ -60,7 +60,6 @@ public class FunctionCallCPInstruction extends CPInstruction {
 	private FunctionCallCPInstruction(String namespace, String functName, ArrayList<CPOperand> boundInParamOperands,
 			ArrayList<String> boundInParamNames, ArrayList<String> boundOutParamNames, String istr) {
 		super(null, functName, istr);
-
 		_cptype = CPINSTRUCTION_TYPE.External;
 		_functionName = functName;
 		_namespace = namespace;
@@ -72,7 +71,7 @@ public class FunctionCallCPInstruction extends CPInstruction {
 
 	public static FunctionCallCPInstruction parseInstruction(String str) 
 		throws DMLRuntimeException 
-	{	
+	{
 		//schema: extfunct, fname, num inputs, num outputs, inputs, outputs
 		String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
 		String namespace = parts[1];
@@ -94,8 +93,7 @@ public class FunctionCallCPInstruction extends CPInstruction {
 		return new FunctionCallCPInstruction ( namespace,functionName, 
 				boundInParamOperands, boundInParamNames, boundOutParamNames, str );
 	}
-
-		
+	
 	@Override
 	public Instruction preprocessInstruction(ExecutionContext ec)
 		throws DMLRuntimeException 
@@ -114,7 +112,7 @@ public class FunctionCallCPInstruction extends CPInstruction {
 	@Override
 	public void processInstruction(ExecutionContext ec) 
 		throws DMLRuntimeException 
-	{		
+	{
 		if( LOG.isTraceEnabled() ){
 			LOG.trace("Executing instruction : " + this.toString());
 		}
@@ -130,19 +128,19 @@ public class FunctionCallCPInstruction extends CPInstruction {
 		
 		// create bindings to formal parameters for given function call
 		// These are the bindings passed to the FunctionProgramBlock for function execution 
-		LocalVariableMap functionVariables = new LocalVariableMap();		
+		LocalVariableMap functionVariables = new LocalVariableMap();
 		for( int i=0; i<fpb.getInputParams().size(); i++) 
-		{				
+		{
 			DataIdentifier currFormalParam = fpb.getInputParams().get(i);
 			String currFormalParamName = currFormalParam.getName();
 			Data currFormalParamValue = null; 
-				
+			
 			CPOperand operand = _boundInputParamOperands.get(i);
 			String varname = operand.getName();
 			//error handling non-existing variables
 			if( !operand.isLiteral() && !ec.containsVariable(varname) ) {
 				throw new DMLRuntimeException("Input variable '"+varname+"' not existing on call of " + 
-						DMLProgram.constructFunctionKey(_namespace, _functionName) + " (line "+getLineNum()+").");
+					DMLProgram.constructFunctionKey(_namespace, _functionName) + " (line "+getLineNum()+").");
 			}
 			//get input matrix/frame/scalar
 			currFormalParamValue = (operand.getDataType()!=DataType.SCALAR) ? ec.getVariable(varname) : 
@@ -150,19 +148,18 @@ public class FunctionCallCPInstruction extends CPInstruction {
 			
 			//graceful value type conversion for scalar inputs with wrong type
 			if( currFormalParamValue.getDataType() == DataType.SCALAR
-				&& currFormalParamValue.getValueType() != operand.getValueType() )
+				&& currFormalParamValue.getValueType() != currFormalParam.getValueType() ) 
 			{
-				ScalarObject so = (ScalarObject) currFormalParamValue;
-				currFormalParamValue = ScalarObjectFactory
-					.createScalarObject(operand.getValueType(), so);
+				currFormalParamValue = ScalarObjectFactory.createScalarObject(
+					currFormalParam.getValueType(), (ScalarObject) currFormalParamValue);
 			}
 			
-			functionVariables.put(currFormalParamName, currFormalParamValue);						
+			functionVariables.put(currFormalParamName, currFormalParamValue);
 		}
 		
 		// Pin the input variables so that they do not get deleted 
 		// from pb's symbol table at the end of execution of function
-	    HashMap<String,Boolean> pinStatus = ec.pinVariables(_boundInputParamNames);
+		HashMap<String,Boolean> pinStatus = ec.pinVariables(_boundInputParamNames);
 		
 		// Create a symbol table under a new execution context for the function invocation,
 		// and copy the function arguments into the created table. 
@@ -185,7 +182,7 @@ public class FunctionCallCPInstruction extends CPInstruction {
 			String fname = DMLProgram.constructFunctionKey(_namespace, _functionName);
 			throw new DMLRuntimeException("error executing function " + fname, e);
 		}
-		LocalVariableMap retVars = fn_ec.getVariables();  
+		LocalVariableMap retVars = fn_ec.getVariables();
 		
 		// cleanup all returned variables w/o binding 
 		Collection<String> retVarnames = new LinkedList<>(retVars.keySet());

http://git-wip-us.apache.org/repos/asf/systemml/blob/a2f0598c/src/test/java/org/apache/sysml/test/integration/functions/misc/FunctionReturnTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/FunctionReturnTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/FunctionReturnTest.java
new file mode 100644
index 0000000..b83ac39
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/FunctionReturnTest.java
@@ -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.
+ */
+
+package org.apache.sysml.test.integration.functions.misc;
+
+import org.junit.Test;
+
+import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+
+public class FunctionReturnTest extends AutomatedTestBase 
+{
+	private final static String TEST_DIR = "functions/misc/";
+	private final static String TEST_NAME1 = "FunctionReturnInt";
+	private final static String TEST_NAME2 = "FunctionReturnBoolean";
+	private final static String TEST_CLASS_DIR = TEST_DIR + FunctionReturnTest.class.getSimpleName() + "/";
+	
+	@Override
+	public void setUp() {
+		addTestConfiguration( TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] { "Rout" }) );
+		addTestConfiguration( TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2, new String[] { "Rout" }) );
+	}
+
+	@Test
+	public void testFunctionReturnInt() {
+		runFunctionReturnTest(TEST_NAME1, false);
+	}
+	
+	@Test
+	public void testFunctionReturnBool() {
+		runFunctionReturnTest(TEST_NAME2, false);
+	}
+	
+	@Test
+	public void testFunctionReturnIntIPA() {
+		runFunctionReturnTest(TEST_NAME1, true);
+	}
+	
+	@Test
+	public void testFunctionReturnBoolIPA() {
+		runFunctionReturnTest(TEST_NAME2, true);
+	}
+
+	private void runFunctionReturnTest( String testname, boolean IPA ) {
+		boolean oldIPA = OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS;
+		OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS = IPA;
+		try {
+			TestConfiguration config = getTestConfiguration(testname);
+			loadTestConfiguration(config);
+			
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = HOME + testname + ".dml";
+			programArgs = new String[]{"-explain"};
+	
+			runTest(true, false, null, -1); 
+		}
+		finally {
+			OptimizerUtils.ALLOW_INTER_PROCEDURAL_ANALYSIS = oldIPA;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/a2f0598c/src/test/scripts/functions/misc/FunctionReturnBoolean.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/FunctionReturnBoolean.dml b/src/test/scripts/functions/misc/FunctionReturnBoolean.dml
new file mode 100644
index 0000000..afa563b
--- /dev/null
+++ b/src/test/scripts/functions/misc/FunctionReturnBoolean.dml
@@ -0,0 +1,34 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+foo = function(Matrix[double] X, boolean input) return (boolean out) {
+  out = input;
+  for(i in 1:2) {
+    tmp = sum(X);
+    out = input & as.logical(tmp);
+  }
+}
+
+X = seq(1,100);
+xmax = max(X);
+y = foo(X, xmax);
+print(y);

http://git-wip-us.apache.org/repos/asf/systemml/blob/a2f0598c/src/test/scripts/functions/misc/FunctionReturnInt.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/FunctionReturnInt.dml b/src/test/scripts/functions/misc/FunctionReturnInt.dml
new file mode 100644
index 0000000..ba916a4
--- /dev/null
+++ b/src/test/scripts/functions/misc/FunctionReturnInt.dml
@@ -0,0 +1,34 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+foo = function(Matrix[double] X, int input) return (int out) {
+  out = input;
+  for(i in 1:2) {
+    tmp = sum(X);
+    out = input + as.integer(tmp);
+  }
+}
+
+X = seq(1,100);
+xmax = max(X);
+y = foo(X, xmax);
+print(y);

http://git-wip-us.apache.org/repos/asf/systemml/blob/a2f0598c/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
index cac39e1..a453cbd 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/misc/ZPackageSuite.java
@@ -31,6 +31,7 @@ import org.junit.runners.Suite;
 	DataTypeChangeTest.class,
 	FunctionInliningTest.class,
 	FunctionNamespaceTest.class,
+	FunctionReturnTest.class,
 	IfTest.class,
 	InvalidBuiltinFunctionCallTest.class,
 	InvalidFunctionAssignmentTest.class,


[2/3] systemml git commit: [SYSTEMML-1982] Improved IPA pass for recompile_once functions

Posted by mb...@apache.org.
[SYSTEMML-1982] Improved IPA pass for recompile_once functions

This patch improved the existing inter-procedural-analysis pass for
recompile_once functions that are recompiled on function entry. So far
all functions with loops were marked for recompile_once. The problem is
that recompilation information is not available during IPA. Hence, we
correct these flags now after generating the runtime program and making
the decisions on recompilation.


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

Branch: refs/heads/master
Commit: ee6060bfc1576cf777dfa99c48126a5a7b35db3a
Parents: a2f0598
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Nov 1 22:51:46 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Nov 2 00:39:17 2017 -0700

----------------------------------------------------------------------
 .../org/apache/sysml/parser/DMLTranslator.java  | 57 +++++++++-----------
 .../apache/sysml/parser/ForStatementBlock.java  | 15 ++----
 .../apache/sysml/parser/IfStatementBlock.java   | 12 ++---
 .../org/apache/sysml/parser/StatementBlock.java |  7 +--
 .../sysml/parser/WhileStatementBlock.java       | 12 ++---
 5 files changed, 45 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/ee6060bf/src/main/java/org/apache/sysml/parser/DMLTranslator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLTranslator.java b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
index 75103d1..fb8404f 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -317,25 +317,21 @@ public class DMLTranslator
 	}
 	
 	public void constructLops(DMLProgram dmlp) throws ParseException, LanguageException, HopsException, LopsException {
-
 		// for each namespace, handle function program blocks handle function 
-		for (String namespaceKey : dmlp.getNamespaces().keySet()){
-			for (String fname: dmlp.getFunctionStatementBlocks(namespaceKey).keySet()) {
-				FunctionStatementBlock current = dmlp.getFunctionStatementBlock(namespaceKey, fname);
-				constructLops(current);
-			}
-		}
+		for( String namespaceKey : dmlp.getNamespaces().keySet() )
+			for( FunctionStatementBlock fsb : dmlp.getFunctionStatementBlocks(namespaceKey).values() )
+				constructLops(fsb);
 		
 		// handle regular program blocks
-		for (int i = 0; i < dmlp.getNumStatementBlocks(); i++) {
-			StatementBlock current = dmlp.getStatementBlock(i);
-			constructLops(current);
-		}
+		for( StatementBlock sb : dmlp.getStatementBlocks() )
+			constructLops(sb);
 	}
 
-	public void constructLops(StatementBlock sb) 
+	public boolean constructLops(StatementBlock sb) 
 		throws HopsException, LopsException 
-	{	
+	{
+		boolean ret = false;
+		
 		if (sb instanceof WhileStatementBlock)
 		{
 			WhileStatementBlock wsb = (WhileStatementBlock)sb;
@@ -348,13 +344,13 @@ public class DMLTranslator
 			}
 			// step through stmt blocks in while stmt body
 			for (StatementBlock stmtBlock : body){
-				constructLops(stmtBlock);
+				ret |= constructLops(stmtBlock);
 			}
 			
 			// handle while stmt predicate
 			Lop l = wsb.getPredicateHops().constructLops();
 			wsb.set_predicateLops(l);	
-			wsb.updatePredicateRecompilationFlag();
+			ret |= wsb.updatePredicateRecompilationFlag();
 		}
 		
 		else if (sb instanceof IfStatementBlock)
@@ -370,16 +366,16 @@ public class DMLTranslator
 			}
 			// step through stmt blocks in if stmt ifBody
 			for (StatementBlock stmtBlock : ifBody)
-				constructLops(stmtBlock);
+				ret |= constructLops(stmtBlock);
 			
 			// step through stmt blocks in if stmt elseBody
 			for (StatementBlock stmtBlock : elseBody)
-				constructLops(stmtBlock);
+				ret |= constructLops(stmtBlock);
 			
 			// handle if stmt predicate
 			Lop l = isb.getPredicateHops().constructLops();
 			isb.set_predicateLops(l);
-			isb.updatePredicateRecompilationFlag();
+			ret |= isb.updatePredicateRecompilationFlag();
 		}
 		
 		else if (sb instanceof ForStatementBlock) //NOTE: applies to ForStatementBlock and ParForStatementBlock
@@ -394,7 +390,7 @@ public class DMLTranslator
 			}
 			// step through stmt blocks in FOR stmt body
 			for (StatementBlock stmtBlock : body)
-				constructLops(stmtBlock);
+				ret |= constructLops(stmtBlock);
 			
 			// handle for stmt predicate
 			if (fsb.getFromHops() != null){
@@ -409,37 +405,36 @@ public class DMLTranslator
 				Lop llobs = fsb.getIncrementHops().constructLops();
 				fsb.setIncrementLops(llobs);
 			}
-			fsb.updatePredicateRecompilationFlags();
+			ret |= fsb.updatePredicateRecompilationFlags();
 		}
-		else if (sb instanceof FunctionStatementBlock){
+		else if (sb instanceof FunctionStatementBlock) {
+			FunctionStatementBlock fsb = (FunctionStatementBlock) sb;
 			FunctionStatement functStmt = (FunctionStatement)sb.getStatement(0);
 			ArrayList<StatementBlock> body = functStmt.getBody();
-			
 			if (sb.get_hops() != null && !sb.get_hops().isEmpty()) {
 				LOG.error(sb.printBlockErrorLocation() + "FunctionStatementBlock should not have hops");
 				throw new HopsException(sb.printBlockErrorLocation() + "FunctionStatementBlock should not have hops");
 			}
 			// step through stmt blocks in while stmt body
-			for (StatementBlock stmtBlock : body){
-				constructLops(stmtBlock);
-			}
+			for( StatementBlock stmtBlock : body )
+				ret |= constructLops(stmtBlock);
+			if( fsb.isRecompileOnce() )
+				fsb.setRecompileOnce(ret);
 		}
 		
 		// handle default case for regular StatementBlock
 		else {
-			
 			if (sb.get_hops() == null)
 				sb.set_hops(new ArrayList<Hop>());
-			
 			ArrayList<Lop> lops = new ArrayList<>();
-			for (Hop hop : sb.get_hops()) {
+			for (Hop hop : sb.get_hops())
 				lops.add(hop.constructLops());
-			}
 			sb.setLops(lops);
-			sb.updateRecompilationFlag(); 
+			ret |= sb.updateRecompilationFlag(); 
 		}
 		
-	} // end method
+		return ret;
+	}
 	
 	
 	public Program getRuntimeProgram(DMLProgram prog, DMLConfig config) 

http://git-wip-us.apache.org/repos/asf/systemml/blob/ee6060bf/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ForStatementBlock.java b/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
index 856e151..686ce7a 100644
--- a/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
@@ -410,29 +410,24 @@ public class ForStatementBlock extends StatementBlock
 	// materialized hops recompilation flags
 	////
 	
-	public void updatePredicateRecompilationFlags() 
-		throws HopsException
-	{
+	public boolean updatePredicateRecompilationFlags() throws HopsException {
 		if( ConfigurationManager.isDynamicRecompilation() ) {
 			_requiresFromRecompile = Recompiler.requiresRecompilation(getFromHops());
 			_requiresToRecompile = Recompiler.requiresRecompilation(getToHops());
 			_requiresIncrementRecompile = Recompiler.requiresRecompilation(getIncrementHops());
 		}
+		return (_requiresFromRecompile || _requiresToRecompile || _requiresIncrementRecompile);
 	}
 	
-	public boolean requiresFromRecompilation()
-	{
+	public boolean requiresFromRecompilation() {
 		return _requiresFromRecompile;
 	}
 	
-	public boolean requiresToRecompilation()
-	{
+	public boolean requiresToRecompilation() {
 		return _requiresToRecompile;
 	}
 	
-	public boolean requiresIncrementRecompilation()
-	{
+	public boolean requiresIncrementRecompilation() {
 		return _requiresIncrementRecompile;
 	}
-	
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/systemml/blob/ee6060bf/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/IfStatementBlock.java b/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
index 2a66857..6803eb2 100644
--- a/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
@@ -524,15 +524,13 @@ public class IfStatementBlock extends StatementBlock
 	// materialized hops recompilation flags
 	////
 	
-	public void updatePredicateRecompilationFlag() 
-		throws HopsException
-	{
-		_requiresPredicateRecompile =  ConfigurationManager.isDynamicRecompilation() 	
-			                           && Recompiler.requiresRecompilation(getPredicateHops());
+	public boolean updatePredicateRecompilationFlag() throws HopsException {
+		return (_requiresPredicateRecompile =
+			ConfigurationManager.isDynamicRecompilation()
+			&& Recompiler.requiresRecompilation(getPredicateHops()));
 	}
 	
-	public boolean requiresPredicateRecompilation()
-	{
+	public boolean requiresPredicateRecompilation() {
 		return _requiresPredicateRecompile;
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/ee6060bf/src/main/java/org/apache/sysml/parser/StatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/StatementBlock.java b/src/main/java/org/apache/sysml/parser/StatementBlock.java
index 4a24675..c2a3e01 100644
--- a/src/main/java/org/apache/sysml/parser/StatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/StatementBlock.java
@@ -1044,9 +1044,10 @@ public class StatementBlock extends LiveVariableAnalysis implements ParseInfo
 	// materialized hops recompilation / updateinplace flags
 	////
 
-	public void updateRecompilationFlag() throws HopsException {
-		_requiresRecompile = ConfigurationManager.isDynamicRecompilation()
-			                 && Recompiler.requiresRecompilation(get_hops());
+	public boolean updateRecompilationFlag() throws HopsException {
+		return (_requiresRecompile =
+			ConfigurationManager.isDynamicRecompilation()
+			&& Recompiler.requiresRecompilation(get_hops()));
 	}
 
 	public boolean requiresRecompilation() {

http://git-wip-us.apache.org/repos/asf/systemml/blob/ee6060bf/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java b/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
index f3f613e..05e2c2c 100644
--- a/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
@@ -324,15 +324,13 @@ public class WhileStatementBlock extends StatementBlock
 	// materialized hops recompilation flags
 	////
 	
-	public void updatePredicateRecompilationFlag() 
-		throws HopsException
-	{
-		_requiresPredicateRecompile =  ConfigurationManager.isDynamicRecompilation() 
-			                           && Recompiler.requiresRecompilation(getPredicateHops());
+	public boolean updatePredicateRecompilationFlag() throws HopsException {
+		return (_requiresPredicateRecompile = 
+			ConfigurationManager.isDynamicRecompilation() 
+			&& Recompiler.requiresRecompilation(getPredicateHops()));
 	}
 	
-	public boolean requiresPredicateRecompilation()
-	{
+	public boolean requiresPredicateRecompilation() {
 		return _requiresPredicateRecompile;
 	}
 }
\ No newline at end of file


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

Posted by mb...@apache.org.
[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)