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/05/29 03:54:11 UTC

[4/4] incubator-systemml git commit: [SYSTEMML-1641] Fix rewrite 'matrix multiplication chain optimization'

[SYSTEMML-1641] Fix rewrite 'matrix multiplication chain optimization'

A recent change, that simplified the hop visit status handling,
introduced a severe bug into the rewrite for matrix multiplication chain
optimization, which affectively disabled this rewrite. This issue has
huge performance implications for scripts that do not provide a good
initial matrix multiplication order. This patch simply fixes the issue,
and adds tests that explicitly check for the correct matrix product
order to avoid such an issue in the future.


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

Branch: refs/heads/master
Commit: 28c92b93f6f69dad2f0bd13483f13bd097696c5c
Parents: 52a0a7f
Author: Matthias Boehm <mb...@gmail.com>
Authored: Sun May 28 19:56:17 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Sun May 28 19:56:17 2017 -0700

----------------------------------------------------------------------
 .../RewriteMatrixMultChainOptimization.java     | 127 +++++++++----------
 .../misc/RewriteMatrixMultChainOptTest.java     | 126 ++++++++++++++++++
 .../functions/misc/RewriteMatrixMultChainOp.R   |  33 +++++
 .../functions/misc/RewriteMatrixMultChainOp.dml |  28 ++++
 .../functions/codegen/ZPackageSuite.java        |   1 +
 .../functions/misc/ZPackageSuite.java           |   1 +
 6 files changed, 249 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/src/main/java/org/apache/sysml/hops/rewrite/RewriteMatrixMultChainOptimization.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMatrixMultChainOptimization.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMatrixMultChainOptimization.java
index d43cb81..396dafc 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMatrixMultChainOptimization.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMatrixMultChainOptimization.java
@@ -22,6 +22,7 @@ package org.apache.sysml.hops.rewrite;
 import java.util.ArrayList;
 import java.util.Arrays;
 
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Level;
@@ -41,12 +42,10 @@ import org.apache.sysml.utils.Explain;
  */
 public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 {
-
 	private static final Log LOG = LogFactory.getLog(RewriteMatrixMultChainOptimization.class.getName());
 	private static final boolean LDEBUG = false;
 	
-	static
-	{
+	static {
 		// for internal debugging only
 		if( LDEBUG ) {
 			Logger.getLogger("org.apache.sysml.hops.rewrite.RewriteMatrixMultChainOptimization")
@@ -61,11 +60,9 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		if( roots == null )
 			return null;
 
+		// Find the optimal order for the chain whose result is the current HOP
 		for( Hop h : roots ) 
-		{
-			// Find the optimal order for the chain whose result is the current HOP
 			rule_OptimizeMMChains(h);
-		}		
 		
 		return roots;
 	}
@@ -93,18 +90,18 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 	private void rule_OptimizeMMChains(Hop hop) 
 		throws HopsException 
 	{
-		if(hop.isVisited())
-				return;
+		if( hop.isVisited() )
+			return;
 		
-		if (  HopRewriteUtils.isMatrixMultiply(hop)
-			  && !((AggBinaryOp)hop).hasLeftPMInput() && !hop.isVisited() ) 
+		if( HopRewriteUtils.isMatrixMultiply(hop)
+			&& !((AggBinaryOp)hop).hasLeftPMInput() && !hop.isVisited() ) 
 		{
 			// Try to find and optimize the chain in which current Hop is the
 			// last operator
 			optimizeMMChain(hop);
 		}
 		
-		for (Hop hi : hop.getInput())
+		for( Hop hi : hop.getInput() )
 			rule_OptimizeMMChains(hi);
 
 		hop.setVisited();
@@ -124,8 +121,8 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 	private void optimizeMMChain( Hop hop ) throws HopsException 
 	{
 		if( LOG.isTraceEnabled() ) {
-			LOG.trace("MM Chain Optimization for HOP: (" + " " + hop.getClass().getSimpleName() + ", " + hop.getHopID() + ", "
-						+ hop.getName() + ")");
+			LOG.trace("MM Chain Optimization for HOP: (" + hop.getClass().getSimpleName()
+				+ ", " + hop.getHopID() + ", " + hop.getName() + ")");
 		}
 		
 		ArrayList<Hop> mmChain = new ArrayList<Hop>();
@@ -135,17 +132,16 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		// Step 1: Identify the chain (mmChain) & clear all links among the Hops
 		// that are involved in mmChain.
 
-		mmOperators.add(hop);
 		// Initialize mmChain with my inputs
-		for (Hop hi : hop.getInput()) {
+		mmOperators.add(hop);
+		for( Hop hi : hop.getInput() )
 			mmChain.add(hi);
-		}
 
 		// expand each Hop in mmChain to find the entire matrix multiplication
 		// chain
 		int i = 0;
-		while (i < mmChain.size()) {
-
+		while( i < mmChain.size() )
+		{
 			boolean expandable = false;
 
 			Hop h = mmChain.get(i);
@@ -157,27 +153,29 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 			 * 3) Its output should not be used in multiple places
 			 *    (either within chain or outside the chain)
 			 */
-
+			
 			if (    HopRewriteUtils.isMatrixMultiply(h)
-			     && !((AggBinaryOp)hop).hasLeftPMInput() && h.isVisited() ) 
+			     && !((AggBinaryOp)hop).hasLeftPMInput() && !h.isVisited() ) 
 			{
 				// check if the output of "h" is used at multiple places. If yes, it can
 				// not be expanded.
-				if (h.getParent().size() > 1 || inputCount( (Hop) ((h.getParent().toArray())[0]), h) > 1 ) {
+				if( h.getParent().size() > 1 || inputCount(h.getParent().get(0), h) > 1 ) {
 					expandable = false;
 					break;
 				}
-				else 
+				else {
 					expandable = true;
+				}
 			}
 
 			h.setVisited();
 
-			if ( !expandable ) {
+			if( !expandable ) {
 				i = i + 1;
-			} else {
+			}
+			else {
 				tempList = mmChain.get(i).getInput();
-				if (tempList.size() != 2) {
+				if( tempList.size() != 2 ) {
 					throw new HopsException(hop.printErrorLocation() + "Hops::rule_OptimizeMMChain(): AggBinary must have exactly two inputs.");
 				}
 
@@ -191,12 +189,12 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		// print the MMChain
 		if( LOG.isTraceEnabled() ) {
 			LOG.trace("Identified MM Chain: ");
-			for (Hop h : mmChain) {
+			for( Hop h : mmChain ) {
 				logTraceHop(h, 1);
 			}
 		}
 
-		if (mmChain.size() == 2) {
+		if( mmChain.size() == 2 ) {
 			// If the chain size is 2, then there is nothing to optimize.
 			return;
 		} 
@@ -236,25 +234,25 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		int[][] split = new int[size][size]; //min cost index table
 
 		//init minimum costs for chains of length 1
-		for (int i = 0; i < size; i++) {
+		for( int i = 0; i < size; i++ ) {
 			Arrays.fill(dpMatrix[i], 0);
 			Arrays.fill(split[i], -1);
 		}
 
 		//compute cost-optimal chains for increasing chain sizes 
-		for (int l = 2; l <= size; l++) { // chain length
-			for (int i = 0; i < size - l + 1; i++) {
+		for( int l = 2; l <= size; l++ ) { // chain length
+			for( int i = 0; i < size - l + 1; i++ ) {
 				int j = i + l - 1;
 				// find cost of (i,j)
 				dpMatrix[i][j] = Double.MAX_VALUE;
-				for (int k = i; k <= j - 1; k++) 
+				for( int k = i; k <= j - 1; k++ ) 
 				{
 					//recursive cost computation
 					double cost = dpMatrix[i][k] + dpMatrix[k + 1][j] 
 							  + (dimArray[i] * dimArray[k + 1] * dimArray[j + 1]);
 					
 					//prune suboptimal
-					if (cost < dpMatrix[i][j]) {
+					if( cost < dpMatrix[i][j] ) {
 						dpMatrix[i][j] = cost;
 						split[i][j] = k;
 					}
@@ -282,7 +280,7 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 			int opIndex, int[][] split, int level) 
 	{
 		//single matrix - end of recursion
-		if (i == j) {
+		if( i == j ) {
 			logTraceHop(h, level);
 			return;
 		}
@@ -293,20 +291,22 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		}
 		
 		// Set Input1 for current Hop h
-		if (i == split[i][j]) {
+		if( i == split[i][j] ) {
 			h.getInput().add(mmChain.get(i));
 			mmChain.get(i).getParent().add(h);
-		} else {
+		}
+		else {
 			h.getInput().add(mmOperators.get(opIndex));
 			mmOperators.get(opIndex).getParent().add(h);
 			opIndex = opIndex + 1;
 		}
 
 		// Set Input2 for current Hop h
-		if (split[i][j] + 1 == j) {
+		if( split[i][j] + 1 == j ) {
 			h.getInput().add(mmChain.get(j));
 			mmChain.get(j).getParent().add(h);
-		} else {
+		} 
+		else {
 			h.getInput().add(mmOperators.get(opIndex));
 			mmOperators.get(opIndex).getParent().add(h);
 			opIndex = opIndex + 1;
@@ -325,18 +325,17 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		}
 	}
 
-	private void clearLinksWithinChain ( Hop hop, ArrayList<Hop> operators ) 
+	private void clearLinksWithinChain( Hop hop, ArrayList<Hop> operators ) 
 		throws HopsException 
 	{
-		Hop op, input1, input2;
-		
-		for ( int i=0; i < operators.size(); i++ ) {
-			op = operators.get(i);
-			if ( op.getInput().size() != 2 || (i != 0 && op.getParent().size() > 1 ) ) {
-				throw new HopsException(hop.printErrorLocation() + "Unexpected error while applying optimization on matrix-mult chain. \n");
+		for( int i=0; i < operators.size(); i++ ) {
+			Hop op = operators.get(i);
+			if( op.getInput().size() != 2 || (i != 0 && op.getParent().size() > 1 ) ) {
+				throw new HopsException(hop.printErrorLocation() + 
+					"Unexpected error while applying optimization on matrix-mult chain. \n");
 			}
-			input1 = op.getInput().get(0);
-			input2 = op.getInput().get(1);
+			Hop input1 = op.getInput().get(0);
+			Hop input2 = op.getInput().get(1);
 			
 			op.getInput().clear();
 			input1.getParent().remove(op);
@@ -362,28 +361,27 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		
 		// Build the array containing dimensions from all matrices in the chain		
 		// check the dimensions in the matrix chain to insure all dimensions are known
-		for (int i=0; i< chain.size(); i++){
-			if (chain.get(i).getDim1() <= 0 || chain.get(i).getDim2() <= 0)
+		for( int i=0; i< chain.size(); i++ )
+			if( chain.get(i).getDim1() <= 0 || chain.get(i).getDim2() <= 0 )
 				dimsKnown = false;
-		}
 		
 		if( dimsKnown ) { //populate dims array if all dims known
-			for (int i = 0; i < chain.size(); i++) 
-			{
+			for( int i = 0; i < chain.size(); i++ ) {
 				if (i == 0) {
 					dimsArray[i] = chain.get(i).getDim1();
 					if (dimsArray[i] <= 0) {
 						throw new HopsException(hop.printErrorLocation() + 
 								"Hops::optimizeMMChain() : Invalid Matrix Dimension: "+ dimsArray[i]);
 					}
-				} else {
-					if (chain.get(i - 1).getDim2() != chain.get(i).getDim1()) {
-						throw new HopsException(hop.printErrorLocation() +
-								"Hops::optimizeMMChain() : Matrix Dimension Mismatch: "+chain.get(i - 1).getDim2()+" != "+chain.get(i).getDim1());
-					}
 				}
+				else if (chain.get(i - 1).getDim2() != chain.get(i).getDim1()) {
+					throw new HopsException(hop.printErrorLocation() +
+						"Hops::optimizeMMChain() : Matrix Dimension Mismatch: " + 
+						chain.get(i - 1).getDim2()+" != "+chain.get(i).getDim1());
+				}
+				
 				dimsArray[i + 1] = chain.get(i).getDim2();
-				if (dimsArray[i + 1] <= 0) {
+				if( dimsArray[i + 1] <= 0 ) {
 					throw new HopsException(hop.printErrorLocation() + 
 							"Hops::optimizeMMChain() : Invalid Matrix Dimension: " + dimsArray[i + 1]);
 				}
@@ -393,20 +391,15 @@ public class RewriteMatrixMultChainOptimization extends HopRewriteRule
 		return dimsKnown;
 	}
 
-	private int inputCount ( Hop p, Hop h ) {
-		int count = 0;
-		for ( int i=0; i < p.getInput().size(); i++ )
-			if ( p.getInput().get(i).equals(h) )
-				count++;
-		return count;
+	private int inputCount( Hop p, Hop h ) {
+		return CollectionUtils.cardinality(h, p.getInput());
 	}
 	
-	private void logTraceHop( Hop hop, int level )
-	{
+	private void logTraceHop( Hop hop, int level ) {
 		if( LOG.isTraceEnabled() ) {
 			String offset = Explain.getIdentation(level);
-			LOG.trace(offset+ "Hop " + hop.getName() + "(" + hop.getClass().getSimpleName() + ", " + hop.getHopID() + ")" + " "
-					+ hop.getDim1() + "x" + hop.getDim2());
+			LOG.trace(offset+ "Hop " + hop.getName() + "(" + hop.getClass().getSimpleName() 
+				+ ", " + hop.getHopID() + ")" + " " + hop.getDim1() + "x" + hop.getDim2());
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/src/test/java/org/apache/sysml/test/integration/functions/misc/RewriteMatrixMultChainOptTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/misc/RewriteMatrixMultChainOptTest.java b/src/test/java/org/apache/sysml/test/integration/functions/misc/RewriteMatrixMultChainOptTest.java
new file mode 100644
index 0000000..152ce52
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/misc/RewriteMatrixMultChainOptTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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 java.util.HashMap;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.lops.LopProperties.ExecType;
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.apache.sysml.test.utils.TestUtils;
+
+public class RewriteMatrixMultChainOptTest extends AutomatedTestBase 
+{
+	private static final String TEST_NAME1 = "RewriteMatrixMultChainOp";
+	private static final String TEST_DIR = "functions/misc/";
+	private static final String TEST_CLASS_DIR = TEST_DIR + RewriteMatrixMultChainOptTest.class.getSimpleName() + "/";
+	
+	private static final int rows = 1234;
+	private static final int cols = 321;
+	private static final double eps = Math.pow(10, -10);
+	
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration( TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] { "R" }) );
+	}
+
+	@Test
+	public void testMatrixMultChainOptNoRewritesCP() {
+		testRewriteMatrixMultChainOp(TEST_NAME1, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testMatrixMultChainOptNoRewritesSP() {
+		testRewriteMatrixMultChainOp(TEST_NAME1, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testMatrixMultChainOptRewritesCP() {
+		testRewriteMatrixMultChainOp(TEST_NAME1, true, ExecType.CP);
+	}
+	
+	@Test
+	public void testMatrixMultChainOptRewritesSP() {
+		testRewriteMatrixMultChainOp(TEST_NAME1, true, ExecType.SPARK);
+	}
+
+	private void testRewriteMatrixMultChainOp(String testname, boolean rewrites, ExecType et)
+	{	
+		RUNTIME_PLATFORM platformOld = rtplatform;
+		switch( et ){
+			case MR: rtplatform = RUNTIME_PLATFORM.HADOOP; break;
+			case SPARK: rtplatform = RUNTIME_PLATFORM.SPARK; break;
+			default: rtplatform = RUNTIME_PLATFORM.HYBRID_SPARK; break;
+		}
+		
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		if( rtplatform == RUNTIME_PLATFORM.SPARK )
+			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+		
+		boolean rewritesOld = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;
+		OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewrites;
+		
+		try
+		{
+			TestConfiguration config = getTestConfiguration(testname);
+			loadTestConfiguration(config);
+			
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = HOME + testname + ".dml";
+			programArgs = new String[]{ "-explain", "hops", "-stats", 
+				"-args", input("X"), input("Y"), output("R") };
+			fullRScriptName = HOME + testname + ".R";
+			rCmd = getRCmd(inputDir(), expectedDir());			
+
+			double[][] X = getRandomMatrix(rows, cols, -1, 1, 0.97d, 7);
+			double[][] Y = getRandomMatrix(cols, 1, -1, 1, 0.9d, 3);
+			writeInputMatrixWithMTD("X", X, true);
+			writeInputMatrixWithMTD("Y", Y, true);
+			
+			//execute tests
+			runTest(true, false, null, -1); 
+			runRScript(true); 
+			
+			//compare matrices 
+			HashMap<CellIndex, Double> dmlfile = readDMLMatrixFromHDFS("R");
+			HashMap<CellIndex, Double> rfile  = readRMatrixFromFS("R");
+			TestUtils.compareMatrices(dmlfile, rfile, eps, "Stat-DML", "Stat-R");
+			
+			//check for correct matrix multiplication order, which also allows
+			//the compilation of mmchain operators
+			if( rewrites ) {
+				Assert.assertTrue(heavyHittersContainsSubString("mmchain")
+					|| heavyHittersContainsSubString("sp_mapmmchain"));
+			}
+		}
+		finally {
+			OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewritesOld;
+			rtplatform = platformOld;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+		}
+	}	
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.R b/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.R
new file mode 100644
index 0000000..3d27844
--- /dev/null
+++ b/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.R
@@ -0,0 +1,33 @@
+#-------------------------------------------------------------
+#
+# 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 = as.matrix(readMM(paste(args[1], "X.mtx", sep="")))
+Y = as.matrix(readMM(paste(args[1], "Y.mtx", sep="")))
+
+R = t(X) %*% X %*% Y;
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "R", sep="")); 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.dml b/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.dml
new file mode 100644
index 0000000..7d43efb
--- /dev/null
+++ b/src/test/scripts/functions/misc/RewriteMatrixMultChainOp.dml
@@ -0,0 +1,28 @@
+#-------------------------------------------------------------
+#
+# 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 = read($1);
+Y = read($2);
+
+R = t(X) %*% X %*% Y;
+
+write(R, $3);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
index 6b0b5be..bc5c21f 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
@@ -34,6 +34,7 @@ import org.junit.runners.Suite;
 	AlgorithmPNMF.class,
 	CellwiseTmplTest.class,
 	CompressedCellwiseTest.class,
+	CompressedMultiAggregateTest.class,
 	DAGCellwiseTmplTest.class,
 	MultiAggTmplTest.class,
 	OuterProdTmplTest.class,

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/28c92b93/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 cba9120..8a06322 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
@@ -51,6 +51,7 @@ import org.junit.runners.Suite;
 	RewriteCTableToRExpandTest.class,
 	RewriteFusedRandTest.class,
 	RewriteLoopVectorization.class,
+	RewriteMatrixMultChainOptTest.class,
 	RewritePushdownSumBinaryMult.class,
 	RewritePushdownSumOnBinaryTest.class,
 	RewritePushdownUaggTest.class,