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 2015/11/23 04:52:56 UTC

[1/8] incubator-systemml git commit: New guarded rdd parallelize (export if parallelize exceeds mem budget)

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 9f29c0199 -> a2f78e74e


New guarded rdd parallelize (export if parallelize exceeds mem budget)

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

Branch: refs/heads/master
Commit: aa9253a653bd1bc41bbac891c0c8ddf9ffdbbe06
Parents: 9f29c01
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Nov 20 12:26:53 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:37:02 2015 -0800

----------------------------------------------------------------------
 .../com/ibm/bi/dml/hops/OptimizerUtils.java     | 16 ++++++++++++++
 .../context/SparkExecutionContext.java          | 22 +++++++++++++++-----
 2 files changed, 33 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/aa9253a6/src/main/java/com/ibm/bi/dml/hops/OptimizerUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/OptimizerUtils.java b/src/main/java/com/ibm/bi/dml/hops/OptimizerUtils.java
index 0c76425..136abd0 100644
--- a/src/main/java/com/ibm/bi/dml/hops/OptimizerUtils.java
+++ b/src/main/java/com/ibm/bi/dml/hops/OptimizerUtils.java
@@ -468,6 +468,22 @@ public class OptimizerUtils
 	
 	/**
 	 * 
+	 * @param mc
+	 * @param memPinned
+	 * @return
+	 */
+	public static boolean checkSparkCollectMemoryBudget( MatrixCharacteristics mc, long memPinned )
+	{
+		return checkSparkCollectMemoryBudget(
+				mc.getRows(), 
+				mc.getCols(),
+				mc.getRowsPerBlock(),
+				mc.getColsPerBlock(),
+				mc.getNonZeros(), memPinned);
+	}
+	
+	/**
+	 * 
 	 * @param rlen
 	 * @param clen
 	 * @param brlen

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/aa9253a6/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
index 9c72cdb..23a1927 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
@@ -257,8 +257,7 @@ public class SparkExecutionContext extends ExecutionContext
 	@SuppressWarnings("unchecked")
 	public JavaPairRDD<?,?> getRDDHandleForMatrixObject( MatrixObject mo, InputInfo inputInfo ) 
 		throws DMLRuntimeException, DMLUnsupportedOperationException
-	{
-		
+	{		
 		//NOTE: MB this logic should be integrated into MatrixObject
 		//However, for now we cannot assume that spark libraries are 
 		//always available and hence only store generic references in 
@@ -278,12 +277,25 @@ public class SparkExecutionContext extends ExecutionContext
 		else if( mo.isDirty() || mo.isCached(false) )
 		{
 			//get in-memory matrix block and parallelize it
-			MatrixBlock mb = mo.acquireRead(); //pin matrix in memory
-			rdd = toJavaPairRDD(getSparkContext(), mb, (int)mo.getNumRowsPerBlock(), (int)mo.getNumColumnsPerBlock());
-			mo.release(); //unpin matrix
+			//w/ guarded parallelize (fallback to export, rdd from file if too large)
+			boolean fromFile = false;
+			if( !OptimizerUtils.checkSparkCollectMemoryBudget(mo.getMatrixCharacteristics(), 0) ) {
+				if( mo.isDirty() ) { //write only if necessary
+					mo.exportData();
+				}
+				rdd = getSparkContext().hadoopFile( mo.getFileName(), inputInfo.inputFormatClass, inputInfo.inputKeyClass, inputInfo.inputValueClass);
+				rdd = ((JavaPairRDD<MatrixIndexes, MatrixBlock>)rdd).mapToPair( new CopyBlockPairFunction() ); //cp is workaround for read bug			
+				fromFile = true;
+			}
+			else { //default case
+				MatrixBlock mb = mo.acquireRead(); //pin matrix in memory
+				rdd = toJavaPairRDD(getSparkContext(), mb, (int)mo.getNumRowsPerBlock(), (int)mo.getNumColumnsPerBlock());
+				mo.release(); //unpin matrix
+			}
 			
 			//keep rdd handle for future operations on it
 			RDDObject rddhandle = new RDDObject(rdd, mo.getVarName());
+			rddhandle.setHDFSFile(fromFile);
 			mo.setRDDHandle(rddhandle);
 		}
 		//CASE 3: non-dirty (file exists on HDFS)


[6/8] incubator-systemml git commit: New wumm quaternary op (rewrite, cp/mr/sp compiler/runtime, tests, docs)

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/java/com/ibm/bi/dml/test/integration/functions/quaternary/WeightedUnaryMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/com/ibm/bi/dml/test/integration/functions/quaternary/WeightedUnaryMatrixMultTest.java b/src/test/java/com/ibm/bi/dml/test/integration/functions/quaternary/WeightedUnaryMatrixMultTest.java
new file mode 100644
index 0000000..078cbd6
--- /dev/null
+++ b/src/test/java/com/ibm/bi/dml/test/integration/functions/quaternary/WeightedUnaryMatrixMultTest.java
@@ -0,0 +1,284 @@
+/**
+ * (C) Copyright IBM Corp. 2010, 2015
+ *
+ * Licensed 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 com.ibm.bi.dml.test.integration.functions.quaternary;
+
+import java.util.HashMap;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.ibm.bi.dml.api.DMLScript;
+import com.ibm.bi.dml.api.DMLScript.RUNTIME_PLATFORM;
+import com.ibm.bi.dml.hops.OptimizerUtils;
+import com.ibm.bi.dml.hops.QuaternaryOp;
+import com.ibm.bi.dml.lops.LopProperties.ExecType;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
+import com.ibm.bi.dml.runtime.instructions.Instruction;
+import com.ibm.bi.dml.runtime.matrix.MatrixCharacteristics;
+import com.ibm.bi.dml.runtime.matrix.data.MatrixValue.CellIndex;
+import com.ibm.bi.dml.test.integration.AutomatedTestBase;
+import com.ibm.bi.dml.test.integration.TestConfiguration;
+import com.ibm.bi.dml.test.utils.TestUtils;
+import com.ibm.bi.dml.utils.Statistics;
+
+/**
+ * 
+ * 
+ */
+public class WeightedUnaryMatrixMultTest extends AutomatedTestBase 
+{
+	private final static String TEST_NAME1 = "WeightedUnaryMMExpMult";
+	private final static String TEST_NAME2 = "WeightedUnaryMMExpDiv";	
+	private final static String TEST_NAME3 = "WeightedUnaryMMPow2";
+	private final static String TEST_NAME4 = "WeightedUnaryMMMult2";
+	private final static String TEST_DIR = "functions/quaternary/";
+	private final static String TEST_CLASS_DIR = TEST_DIR + WeightedUnaryMatrixMultTest.class.getSimpleName() + "/";
+	
+	private final static double eps = 1e-6;
+	
+	private final static int rows = 1201;
+	private final static int cols = 1103;
+	private final static int rank = 10;
+	private final static double spSparse = 0.001;
+	private final static double spDense = 0.6;
+	
+	@Override
+	public void setUp() 
+	{
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_NAME1,new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1,new String[]{"R"}));
+		addTestConfiguration(TEST_NAME2,new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2,new String[]{"R"}));
+		addTestConfiguration(TEST_NAME3,new TestConfiguration(TEST_CLASS_DIR, TEST_NAME3,new String[]{"R"}));
+		addTestConfiguration(TEST_NAME4,new TestConfiguration(TEST_CLASS_DIR, TEST_NAME4,new String[]{"R"}));
+	}
+
+	//cp testcases
+	
+	@Test
+	public void testWeightedUnaryMMExpMultDenseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME1, false, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivDenseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME2, false, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2DenseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME3, false, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2DenseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME4, false, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpMultSparseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME1, true, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivSparseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME2, true, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2SparseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME3, true, true, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2SparseCP()  {
+		runWeightedUnaryMMTest(TEST_NAME4, true, true, false, ExecType.CP);
+	}
+	
+	//sp testcases
+	
+	@Test
+	public void testWeightedUnaryMMExpMultDenseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME1, false, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivDenseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME2, false, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2DenseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME3, false, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2DenseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME4, false, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpMultSparseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME1, true, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivSparseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME2, true, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2SparseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME3, true, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2SparseSP()  {
+		runWeightedUnaryMMTest(TEST_NAME4, true, true, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpMultDenseRepSP()  {
+		runWeightedUnaryMMTest(TEST_NAME1, false, true, true, ExecType.SPARK);
+	}
+
+	//mr testcases
+	
+	@Test
+	public void testWeightedUnaryMMExpMultDenseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME1, false, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivDenseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME2, false, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2DenseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME3, false, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2DenseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME4, false, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpMultSparseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME1, true, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMExpDivSparseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME2, true, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMPow2SparseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME3, true, true, false, ExecType.MR);
+	}
+	
+	@Test
+	public void testWeightedUnaryMMMult2SparseMR()  {
+		runWeightedUnaryMMTest(TEST_NAME4, true, true, false, ExecType.MR);
+	}
+
+	@Test
+	public void testWeightedUnaryMMExpMultDenseRepMR()  {
+		runWeightedUnaryMMTest(TEST_NAME1, false, true, true, ExecType.MR);
+	}
+	
+	
+	/**
+	 * 
+	 * @param sparseM1
+	 * @param sparseM2
+	 * @param instType
+	 */
+	private void runWeightedUnaryMMTest( String testname, boolean sparse, boolean rewrites, boolean rep, ExecType instType)
+	{		
+		RUNTIME_PLATFORM platformOld = rtplatform;
+		switch( instType ){
+			case MR: rtplatform = RUNTIME_PLATFORM.HADOOP; break;
+			case SPARK: rtplatform = RUNTIME_PLATFORM.SPARK; break;
+			default: rtplatform = RUNTIME_PLATFORM.HYBRID; 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;
+		boolean forceOld = QuaternaryOp.FORCE_REPLICATION;
+
+		OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewrites;
+		QuaternaryOp.FORCE_REPLICATION = rep;
+	    
+		try
+		{
+			double sparsity = (sparse) ? spSparse : spDense;
+			String TEST_NAME = testname;
+			
+			TestConfiguration config = getTestConfiguration(TEST_NAME);
+			loadTestConfiguration(config);
+			
+			// This is for running the junit test the new way, i.e., construct the arguments directly
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = HOME + TEST_NAME + ".dml";
+			programArgs = new String[]{"-stats", "-explain", "runtime", "-args",
+				input("W"), input("U"), input("V"), output("R") };
+			
+			fullRScriptName = HOME + TEST_NAME + ".R";
+			rCmd = "Rscript" + " " + fullRScriptName + " " + inputDir() + " " + expectedDir();
+	
+			//generate actual dataset 
+			double[][] W = getRandomMatrix(rows, cols, 0, 1, sparsity, 7); 
+			writeInputMatrixWithMTD("W", W, true);
+			double[][] U = getRandomMatrix(rows, rank, 0, 1, 1.0, 713); 
+			writeInputMatrixWithMTD("U", U, true);
+			double[][] V = getRandomMatrix(cols, rank, 0, 1, 1.0, 812); 
+			writeInputMatrixWithMTD("V", V, true);
+			
+			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");
+			checkDMLMetaDataFile("R", new MatrixCharacteristics(rows, cols, 1, 1));
+
+			//check statistics for right operator in cp and spark
+			if( instType == ExecType.CP && rewrites ) {
+				Assert.assertTrue("Missing opcode wumm", Statistics.getCPHeavyHitterOpCodes().contains(WeightedUnaryMM.OPCODE_CP));
+			}
+			else if( instType == ExecType.SPARK && rewrites ) {
+				String opcode = Instruction.SP_INST_PREFIX + ((rep)?WeightedUnaryMMR.OPCODE:WeightedUnaryMM.OPCODE);
+				Assert.assertTrue("Missing opcode sp_wumm", Statistics.getCPHeavyHitterOpCodes().contains(opcode) );
+			}
+		}
+		finally
+		{
+			rtplatform = platformOld;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+			OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewritesOld;
+			QuaternaryOp.FORCE_REPLICATION = forceOld;
+		}
+	}	
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.R b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.R
new file mode 100644
index 0000000..e0c1e7f
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.R
@@ -0,0 +1,33 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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")
+
+W = as.matrix(readMM(paste(args[1], "W.mtx", sep="")))
+U = as.matrix(readMM(paste(args[1], "U.mtx", sep="")))
+V = as.matrix(readMM(paste(args[1], "V.mtx", sep="")))
+
+R = W/exp(U%*%t(V));
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "R", sep="")); 
+
+

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.dml b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.dml
new file mode 100644
index 0000000..8b9ac1b
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpDiv.dml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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.
+#
+#-------------------------------------------------------------
+
+
+
+W = read($1);
+U = read($2);
+V = read($3);
+
+R = W/exp(U%*%t(V));
+
+write(R, $4);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.R b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.R
new file mode 100644
index 0000000..e3caa5f
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.R
@@ -0,0 +1,33 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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")
+
+W = as.matrix(readMM(paste(args[1], "W.mtx", sep="")))
+U = as.matrix(readMM(paste(args[1], "U.mtx", sep="")))
+V = as.matrix(readMM(paste(args[1], "V.mtx", sep="")))
+
+R = W*exp(U%*%t(V));
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "R", sep="")); 
+
+

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.dml b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.dml
new file mode 100644
index 0000000..31a371b
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMExpMult.dml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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.
+#
+#-------------------------------------------------------------
+
+
+
+W = read($1);
+U = read($2);
+V = read($3);
+
+R = W*exp(U%*%t(V));
+
+write(R, $4);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.R b/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.R
new file mode 100644
index 0000000..63b08a5
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.R
@@ -0,0 +1,33 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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")
+
+W = as.matrix(readMM(paste(args[1], "W.mtx", sep="")))
+U = as.matrix(readMM(paste(args[1], "U.mtx", sep="")))
+V = as.matrix(readMM(paste(args[1], "V.mtx", sep="")))
+
+R = W*(2*(U%*%t(V)));
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "R", sep="")); 
+
+

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.dml b/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.dml
new file mode 100644
index 0000000..3a00f59
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMMult2.dml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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.
+#
+#-------------------------------------------------------------
+
+
+
+W = read($1);
+U = read($2);
+V = read($3);
+
+R = W*(2*(U%*%t(V)));
+
+write(R, $4);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.R b/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.R
new file mode 100644
index 0000000..84178df
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.R
@@ -0,0 +1,33 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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")
+
+W = as.matrix(readMM(paste(args[1], "W.mtx", sep="")))
+U = as.matrix(readMM(paste(args[1], "U.mtx", sep="")))
+V = as.matrix(readMM(paste(args[1], "V.mtx", sep="")))
+
+R = W/(U%*%t(V))^2;
+
+writeMM(as(R, "CsparseMatrix"), paste(args[2], "R", sep="")); 
+
+

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.dml b/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.dml
new file mode 100644
index 0000000..4c50bec
--- /dev/null
+++ b/src/test/scripts/functions/quaternary/WeightedUnaryMMPow2.dml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# (C) Copyright IBM Corp. 2010, 2015
+#
+# Licensed 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.
+#
+#-------------------------------------------------------------
+
+
+
+W = read($1);
+U = read($2);
+V = read($3);
+
+R = W/(U%*%t(V))^2;
+
+write(R, $4);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/test_suites/java/com/ibm/bi/dml/test/integration/functions/quaternary/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/com/ibm/bi/dml/test/integration/functions/quaternary/ZPackageSuite.java b/src/test_suites/java/com/ibm/bi/dml/test/integration/functions/quaternary/ZPackageSuite.java
index 5b7a32f..1d5782e 100644
--- a/src/test_suites/java/com/ibm/bi/dml/test/integration/functions/quaternary/ZPackageSuite.java
+++ b/src/test_suites/java/com/ibm/bi/dml/test/integration/functions/quaternary/ZPackageSuite.java
@@ -28,7 +28,8 @@ import org.junit.runners.Suite;
 	WeightedCrossEntropyTest.class,
 	WeightedDivMatrixMultTest.class,
 	WeightedSquaredLossTest.class,
-	WeightedSigmoidTest.class
+	WeightedSigmoidTest.class,
+	WeightedUnaryMatrixMultTest.class
 })
 
 


[5/8] incubator-systemml git commit: Fix rewrite 'fuse sum_sq' (after wsloss rewrite), for kmeans_predict

Posted by mb...@apache.org.
Fix rewrite 'fuse sum_sq' (after wsloss rewrite), for kmeans_predict 

The new rewrite sum(X^2) -> sum_sq(X) was mistakenly applied before
sum((X-L%*%R)^2) -> wsloss(X,L,R) and hence led to a performance
regression due to shuffle for the join between X and L%*%R. This change
moved all quaternary rewrites to the category of 'dynamic' rewrites
because they anyway check for various size parameters. By applying the
quaternary rewrites before the 'fuse sum_sq', both rewrites trigger for
the intended patterns. On a kmeans 80GB use case, this change reduced
the end-to-end runtime from 590s to 114s.


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

Branch: refs/heads/master
Commit: e52e0c0a6b5a39e375383b046e3ce0465ff8d662
Parents: cc4aae7
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sat Nov 21 00:51:27 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:38:23 2015 -0800

----------------------------------------------------------------------
 .../RewriteAlgebraicSimplificationDynamic.java  | 616 ++++++++++++++++++-
 .../RewriteAlgebraicSimplificationStatic.java   |  11 +-
 2 files changed, 616 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/e52e0c0a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 5c7a0fb..10dbedf 100644
--- a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -28,10 +28,12 @@ import com.ibm.bi.dml.hops.AggUnaryOp;
 import com.ibm.bi.dml.hops.BinaryOp;
 import com.ibm.bi.dml.hops.DataGenOp;
 import com.ibm.bi.dml.hops.Hop;
+import com.ibm.bi.dml.hops.QuaternaryOp;
 import com.ibm.bi.dml.hops.Hop.AggOp;
 import com.ibm.bi.dml.hops.Hop.DataGenMethod;
 import com.ibm.bi.dml.hops.Hop.Direction;
 import com.ibm.bi.dml.hops.Hop.OpOp1;
+import com.ibm.bi.dml.hops.Hop.OpOp4;
 import com.ibm.bi.dml.hops.Hop.ReOrgOp;
 import com.ibm.bi.dml.hops.HopsException;
 import com.ibm.bi.dml.hops.IndexingOp;
@@ -40,6 +42,7 @@ import com.ibm.bi.dml.hops.LiteralOp;
 import com.ibm.bi.dml.hops.Hop.OpOp2;
 import com.ibm.bi.dml.hops.ReorgOp;
 import com.ibm.bi.dml.hops.UnaryOp;
+import com.ibm.bi.dml.lops.MapMultChain.ChainType;
 import com.ibm.bi.dml.parser.DMLTranslator;
 import com.ibm.bi.dml.parser.DataExpression;
 import com.ibm.bi.dml.parser.Expression.DataType;
@@ -56,10 +59,8 @@ import com.ibm.bi.dml.parser.Expression.ValueType;
  */
 public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 {
-	
 	private static final Log LOG = LogFactory.getLog(RewriteAlgebraicSimplificationDynamic.class.getName());
 	
-	
 	//valid aggregation operation types for rowOp to Op conversions (not all operations apply)
 	private static AggOp[] LOOKUP_VALID_ROW_COL_AGGREGATE = new AggOp[]{AggOp.SUM, AggOp.SUM_SQ, AggOp.MIN, AggOp.MAX, AggOp.MEAN};
 	
@@ -70,6 +71,8 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 	//valid unary operation types for empty (sparse-safe) operations (not all operations apply)
 	private static OpOp1[] LOOKUP_VALID_EMPTY_UNARY = new OpOp1[]{OpOp1.ABS, OpOp1.SIN, OpOp1.TAN, OpOp1.SQRT, OpOp1.ROUND, OpOp1.CUMSUM}; 
 	
+	//valid pseudo-sparse-safe binary operators for wdivmm 
+	private static OpOp2[] LOOKUP_VALID_WDIVMM_BINARY = new OpOp2[]{OpOp2.MULT, OpOp2.DIV}; 
 	
 	
 	@Override
@@ -159,10 +162,14 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 			hi = simplifyDiagMatrixMult(hop, hi, i);          //e.g., diag(X%*%Y)->rowSums(X*t(Y)); if col vector
 			hi = simplifySumDiagToTrace(hi);                  //e.g., sum(diag(X)) -> trace(X); if col vector
 			hi = pushdownBinaryOperationOnDiag(hop, hi, i);   //e.g., diag(X)*7 -> diag(X*7); if col vector
+			hi = simplifyWeightedSquaredLoss(hop, hi, i);     //e.g., sum(W * (X - U %*% t(V)) ^ 2) -> wsl(X, U, t(V), W, true), 
+			hi = simplifyWeightedSigmoidMMChains(hop, hi, i); //e.g., W * sigmoid(Y%*%t(X)) -> wsigmoid(W, Y, t(X), type)
+			hi = simplifyWeightedDivMM(hop, hi, i);           //e.g., t(U) %*% (X/(U%*%t(V))) -> wdivmm(X, U, t(V), left)
+			hi = simplifyWeightedCrossEntropy(hop, hi, i);    //e.g., sum(X*log(U%*%t(V))) -> wcemm(X, U, t(V))
 			hi = simplifyDotProductSum(hop, hi, i);           //e.g., sum(v^2) -> t(v)%*%v if ncol(v)==1 
 			hi = fuseSumSquared(hop, hi, i);                  //e.g., sum(X^2) -> sumSq(X), if ncol(X)>1
 			hi = reorderMinusMatrixMult(hop, hi, i);          //e.g., (-t(X))%*%y->-(t(X)%*%y), TODO size
-			hi = simplifySumMatrixMult(hop, hi, i);           //e.g., sum(A%*%B) -> sum(t(colSums(A))*rowSums(B)), if not dot product
+			hi = simplifySumMatrixMult(hop, hi, i);           //e.g., sum(A%*%B) -> sum(t(colSums(A))*rowSums(B)), if not dot product / wsloss
 			hi = simplifyEmptyBinaryOperation(hop, hi, i);    //e.g., X*Y -> matrix(0,nrow(X), ncol(X)) / X+Y->X / X-Y -> X
 			hi = simplifyScalarMVBinaryOperation(hi); 		  //e.g., X*y -> X*as.scalar(y), if y is a 1-1 matrix
 			hi = simplifyNnzComputation(hop, hi, i);          //e.g., sum(ppred(X,0,"!=")) -> literal(nnz(X)), if nnz known
@@ -1335,6 +1342,609 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 		return hi;
 	}
 	
+
+	/**
+	 * Searches for weighted squared loss expressions and replaces them with a quaternary operator. 
+	 * Currently, this search includes the following three patterns:
+	 * 1) sum (W * (X - U %*% t(V)) ^ 2) (post weighting)
+	 * 2) sum ((X - W * (U %*% t(V))) ^ 2) (pre weighting)
+	 * 3) sum ((X - (U %*% t(V))) ^ 2) (no weighting)
+	 * 
+	 * NOTE: We include transpose into the pattern because during runtime we need to compute
+	 * U%*% t(V) pointwise; having V and not t(V) at hand allows for a cache-friendly implementation
+	 * without additional memory requirements for internal transpose.
+	 * 
+	 * This rewrite is conceptually a static rewrite; however, the current MR runtime only supports
+	 * U/V factors of rank up to the blocksize (1000). We enforce this contraint here during the general
+	 * rewrite because this is an uncommon case. Also, the intention is to remove this constaint as soon
+	 * as we generalized the runtime or hop/lop compilation. 
+	 * 
+	 * @param parent
+	 * @param hi
+	 * @param pos
+	 * @return
+	 * @throws HopsException 
+	 */
+	private Hop simplifyWeightedSquaredLoss(Hop parent, Hop hi, int pos) 
+		throws HopsException
+	{
+		//NOTE: there might be also a general simplification without custom operator
+		//via (X-UVt)^2 -> X^2 - 2X*UVt + UVt^2
+		Hop hnew = null;
+		
+		if( hi instanceof AggUnaryOp && ((AggUnaryOp)hi).getDirection()==Direction.RowCol
+			&& ((AggUnaryOp)hi).getOp() == AggOp.SUM     //all patterns rooted by sum()
+			&& hi.getInput().get(0) instanceof BinaryOp  //all patterns subrooted by binary op
+			&& hi.getInput().get(0).getDim2() > 1  )     //not applied for vector-vector mult
+		{
+			BinaryOp bop = (BinaryOp) hi.getInput().get(0);
+			boolean appliedPattern = false;
+			
+			//Pattern 1) sum (W * (X - U %*% t(V)) ^ 2) (post weighting)
+			//alternative pattern: sum (W * (U %*% t(V) - X) ^ 2)
+			if( bop.getOp()==OpOp2.MULT && bop.getInput().get(1) instanceof BinaryOp	
+				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
+				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0), bop.getInput().get(1)) //prevent mv
+				&& ((BinaryOp)bop.getInput().get(1)).getOp()==OpOp2.POW 
+				&& bop.getInput().get(1).getInput().get(1) instanceof LiteralOp
+				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1).getInput().get(1))==2)
+			{
+				Hop W = bop.getInput().get(0);
+				Hop tmp = bop.getInput().get(1).getInput().get(0); //(X - U %*% t(V))
+				
+				if( tmp instanceof BinaryOp && ((BinaryOp)tmp).getOp()==OpOp2.MINUS
+					&& HopRewriteUtils.isEqualSize(tmp.getInput().get(0), tmp.getInput().get(1)) //prevent mv	
+					&& tmp.getInput().get(0).getDataType() == DataType.MATRIX )
+				{
+					//a) sum (W * (X - U %*% t(V)) ^ 2)
+					int uvIndex = -1;
+					if( tmp.getInput().get(1) instanceof AggBinaryOp  //ba gurantees matrices
+							&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(1).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
+					{
+						uvIndex = 1;   
+					}
+					//b) sum (W * (U %*% t(V) - X) ^ 2)
+					else if(tmp.getInput().get(0) instanceof AggBinaryOp  //ba gurantees matrices
+						&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(0).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
+					{
+						uvIndex = 0;
+					}   
+				 
+					if( uvIndex >= 0 ) //rewrite match
+					{
+						Hop X = tmp.getInput().get((uvIndex==0)?1:0); 
+						Hop U = tmp.getInput().get(uvIndex).getInput().get(0);
+						Hop V = tmp.getInput().get(uvIndex).getInput().get(1);
+	                    
+						if( !HopRewriteUtils.isTransposeOperation(V) ) {
+							V = HopRewriteUtils.createTranspose(V);
+						}
+						else{
+							V = V.getInput().get(0);
+						}
+	                    
+						//handle special case of post_nz
+						if( HopRewriteUtils.isNonZeroIndicator(W, X) ){
+							W = new LiteralOp(1);
+						}
+						
+						//construct quaternary hop
+						hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
+								OpOp4.WSLOSS, X, U, V, W, true);
+						HopRewriteUtils.setOutputParametersForScalar(hnew);
+	
+						appliedPattern = true;
+						LOG.debug("Applied simplifyWeightedSquaredLoss1"+uvIndex+" (line "+hi.getBeginLine()+")");  
+					}
+				}
+			}
+			
+			//Pattern 2) sum ((X - W * (U %*% t(V))) ^ 2) (pre weighting)
+			//alternative pattern: sum ((W * (U %*% t(V)) - X) ^ 2)
+			if( !appliedPattern
+				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
+				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1))==2
+				&& bop.getInput().get(0) instanceof BinaryOp	
+				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
+				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS
+				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0).getInput().get(0), bop.getInput().get(0).getInput().get(1)) //prevent mv
+				&& bop.getInput().get(0).getInput().get(0).getDataType()==DataType.MATRIX)
+			{
+			    Hop lleft = bop.getInput().get(0).getInput().get(0); 
+			    Hop lright = bop.getInput().get(0).getInput().get(1); 
+                
+			    //a) sum ((X - W * (U %*% t(V))) ^ 2)
+			    int wuvIndex = -1;
+			    if( lright instanceof BinaryOp && lright.getInput().get(1) instanceof AggBinaryOp ){
+			    	wuvIndex = 1;
+			    }
+			    //b) sum ((W * (U %*% t(V)) - X) ^ 2)
+			    else if( lleft instanceof BinaryOp && lleft.getInput().get(1) instanceof AggBinaryOp ){
+			    	wuvIndex = 0;
+			    }
+			    
+			    if( wuvIndex >= 0 ) //rewrite match
+			    {
+			    	Hop X = bop.getInput().get(0).getInput().get((wuvIndex==0)?1:0);
+			    	Hop tmp = bop.getInput().get(0).getInput().get(wuvIndex); //(W * (U %*% t(V)))
+    				
+    				if( ((BinaryOp)tmp).getOp()==OpOp2.MULT
+    					&& tmp.getInput().get(0).getDataType() == DataType.MATRIX	
+    					&& HopRewriteUtils.isEqualSize(tmp.getInput().get(0), tmp.getInput().get(1)) //prevent mv
+    					&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(1).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
+    				{
+    					Hop W = tmp.getInput().get(0); 
+    					Hop U = tmp.getInput().get(1).getInput().get(0);
+    					Hop V = tmp.getInput().get(1).getInput().get(1);
+    					
+    					if( !HopRewriteUtils.isTransposeOperation(V) ) { 
+    						V = HopRewriteUtils.createTranspose(V);
+    					}
+    					else {
+    						V = V.getInput().get(0);
+    					}
+    					
+    					hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
+    							  OpOp4.WSLOSS, X, U, V, W, false);
+    					HopRewriteUtils.setOutputParametersForScalar(hnew);
+    
+    					appliedPattern = true;
+    					LOG.debug("Applied simplifyWeightedSquaredLoss2"+wuvIndex+" (line "+hi.getBeginLine()+")");	
+    				}
+			    }
+			}
+			
+			//Pattern 3) sum ((X - (U %*% t(V))) ^ 2) (no weighting)
+			//alternative pattern: sum (((U %*% t(V)) - X) ^ 2)
+			if( !appliedPattern
+				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
+				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1))==2
+				&& bop.getInput().get(0) instanceof BinaryOp	
+				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
+				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS
+				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0).getInput().get(0), bop.getInput().get(0).getInput().get(1)) //prevent mv
+				&& bop.getInput().get(0).getInput().get(0).getDataType()==DataType.MATRIX)
+			{
+				Hop lleft = bop.getInput().get(0).getInput().get(0);
+				Hop lright = bop.getInput().get(0).getInput().get(1);
+                
+				//a) sum ((X - (U %*% t(V))) ^ 2)
+				int uvIndex = -1;
+				if( lright instanceof AggBinaryOp //ba gurantees matrices
+					&& HopRewriteUtils.isSingleBlock(lright.getInput().get(0),true) )  //BLOCKSIZE CONSTRAINT
+				{
+					uvIndex = 1;
+				}
+				//b) sum (((U %*% t(V)) - X) ^ 2)
+				else if( lleft instanceof AggBinaryOp //ba gurantees matrices
+						&& HopRewriteUtils.isSingleBlock(lleft.getInput().get(0),true) )  //BLOCKSIZE CONSTRAINT
+				{
+					uvIndex = 0;
+				}
+			    
+				if( uvIndex >= 0 ) //rewrite match
+				{
+					Hop X = bop.getInput().get(0).getInput().get((uvIndex==0)?1:0);
+					Hop tmp = bop.getInput().get(0).getInput().get(uvIndex); //(U %*% t(V))
+					Hop W = new LiteralOp(1); //no weighting 
+					Hop U = tmp.getInput().get(0);
+					Hop V = tmp.getInput().get(1);
+	
+					if( !HopRewriteUtils.isTransposeOperation(V) ) { 
+						V = HopRewriteUtils.createTranspose(V);
+					}
+					else {
+						V = V.getInput().get(0);
+					}
+					
+					hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
+							  OpOp4.WSLOSS, X, U, V, W, false);
+					HopRewriteUtils.setOutputParametersForScalar(hnew);
+
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedSquaredLoss3"+uvIndex+" (line "+hi.getBeginLine()+")");	
+				}
+			}			
+		}
+		
+		//relink new hop into original position
+		if( hnew != null ) {
+			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
+			HopRewriteUtils.addChildReference(parent, hnew, pos);
+			hi = hnew;
+		}
+		
+		return hi;
+	}
+	
+	/**
+	 * 
+	 * @param parent
+	 * @param hi
+	 * @param pos
+	 * @return
+	 * @throws HopsException
+	 */
+	private Hop simplifyWeightedSigmoidMMChains(Hop parent, Hop hi, int pos) 
+		throws HopsException
+	{
+		Hop hnew = null;
+		
+		if(    hi instanceof BinaryOp //all patterns subrooted by W *
+			&& ((BinaryOp) hi).getOp()==OpOp2.MULT
+			&& hi.getDim2() > 1       //not applied for vector-vector mult
+			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
+			&& hi.getInput().get(0).getDataType()==DataType.MATRIX 
+			&& hi.getInput().get(1) instanceof UnaryOp ) //sigmoid/log
+		{
+			UnaryOp uop = (UnaryOp) hi.getInput().get(1);
+			boolean appliedPattern = false;
+			
+			//Pattern 1) W * sigmoid(Y%*%t(X)) (basic)
+			if(    uop.getOp() == OpOp1.SIGMOID 
+				&& uop.getInput().get(0) instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(0),true) )
+			{
+				Hop W = hi.getInput().get(0); 
+				Hop Y = uop.getInput().get(0).getInput().get(0);
+				Hop tX = uop.getInput().get(0).getInput().get(1);
+				
+				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
+					tX = HopRewriteUtils.createTranspose(tX);
+				}
+				else 
+					tX = tX.getInput().get(0);
+				
+				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+						  OpOp4.WSIGMOID, W, Y, tX, false, false);
+				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+				appliedPattern = true;
+				LOG.debug("Applied simplifyWeightedSigmoid1 (line "+hi.getBeginLine()+")");	
+			}
+			
+			//Pattern 2) W * sigmoid(-(Y%*%t(X))) (minus)
+			if(    !appliedPattern 
+				&& uop.getOp() == OpOp1.SIGMOID 
+				&& uop.getInput().get(0) instanceof BinaryOp
+				&& ((BinaryOp)uop.getInput().get(0)).getOp()==OpOp2.MINUS
+				&& uop.getInput().get(0).getInput().get(0) instanceof LiteralOp
+				&& HopRewriteUtils.getDoubleValueSafe(
+				   (LiteralOp)uop.getInput().get(0).getInput().get(0))==0
+				&& uop.getInput().get(0).getInput().get(1) instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(1).getInput().get(0),true))
+			{
+				Hop W = hi.getInput().get(0); 
+				Hop Y = uop.getInput().get(0).getInput().get(1).getInput().get(0);
+				Hop tX = uop.getInput().get(0).getInput().get(1).getInput().get(1);
+				
+				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
+					tX = HopRewriteUtils.createTranspose(tX);
+				}
+				else 
+					tX = tX.getInput().get(0);
+				
+				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+						  OpOp4.WSIGMOID, W, Y, tX, false, true);
+				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+				appliedPattern = true;
+				LOG.debug("Applied simplifyWeightedSigmoid2 (line "+hi.getBeginLine()+")");	
+			}
+			
+			//Pattern 3) W * log(sigmoid(Y%*%t(X))) (log)			
+			if(    !appliedPattern 
+				&& uop.getOp() == OpOp1.LOG
+				&& uop.getInput().get(0) instanceof UnaryOp
+				&& ((UnaryOp)uop.getInput().get(0)).getOp() == OpOp1.SIGMOID 
+				&& uop.getInput().get(0).getInput().get(0) instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(0).getInput().get(0),true) )
+			{
+				Hop W = hi.getInput().get(0); 
+				Hop Y = uop.getInput().get(0).getInput().get(0).getInput().get(0);
+				Hop tX = uop.getInput().get(0).getInput().get(0).getInput().get(1);
+				
+				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
+					tX = HopRewriteUtils.createTranspose(tX);
+				}
+				else 
+					tX = tX.getInput().get(0);
+				
+				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+						  OpOp4.WSIGMOID, W, Y, tX, true, false);
+				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+				appliedPattern = true;
+				LOG.debug("Applied simplifyWeightedSigmoid3 (line "+hi.getBeginLine()+")");	
+			}			
+			
+			//Pattern 4) W * log(sigmoid(-(Y%*%t(X)))) (log_minus)
+			if(    !appliedPattern 
+				&& uop.getOp() == OpOp1.LOG
+				&& uop.getInput().get(0) instanceof UnaryOp
+				&& ((UnaryOp)uop.getInput().get(0)).getOp() == OpOp1.SIGMOID 
+				&& uop.getInput().get(0).getInput().get(0) instanceof BinaryOp )
+			{
+				BinaryOp bop = (BinaryOp) uop.getInput().get(0).getInput().get(0);
+				
+				if(    bop.getOp() == OpOp2.MINUS 
+					&& bop.getInput().get(0) instanceof LiteralOp
+					&& HopRewriteUtils.getDoubleValueSafe((LiteralOp)bop.getInput().get(0))==0
+					&& bop.getInput().get(1) instanceof AggBinaryOp
+					&& HopRewriteUtils.isSingleBlock(bop.getInput().get(1).getInput().get(0),true))
+				{
+					Hop W = hi.getInput().get(0); 
+					Hop Y = bop.getInput().get(1).getInput().get(0);
+					Hop tX = bop.getInput().get(1).getInput().get(1);
+					
+					if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
+						tX = HopRewriteUtils.createTranspose(tX);
+					}
+					else 
+						tX = tX.getInput().get(0);
+					
+					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+							  OpOp4.WSIGMOID, W, Y, tX, true, true);
+					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+	
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedSigmoid4 (line "+hi.getBeginLine()+")");	
+				}
+			}
+		}
+		
+		//relink new hop into original position
+		if( hnew != null ) {
+			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
+			HopRewriteUtils.addChildReference(parent, hnew, pos);
+			hi = hnew;
+		}
+		
+		return hi;
+	}
+
+	/**
+	 * 
+	 * @param parent
+	 * @param hi
+	 * @param pos
+	 * @return
+	 * @throws HopsException
+	 */
+	private Hop simplifyWeightedDivMM(Hop parent, Hop hi, int pos) 
+		throws HopsException
+	{
+		Hop hnew = null;
+		boolean appliedPattern = false;
+		
+		//left/right patterns rooted by 'ab - b(div)' or 'ab - b(mult)'
+		//note: we do not rewrite t(X)%*%(w*(X%*%v)) where w and v are vectors (see mmchain ops) 
+		if( hi instanceof AggBinaryOp && ((AggBinaryOp)hi).isMatrixMultiply()  
+			&& (hi.getInput().get(0) instanceof BinaryOp
+			&& HopRewriteUtils.isValidOp(((BinaryOp)hi.getInput().get(0)).getOp(), LOOKUP_VALID_WDIVMM_BINARY)
+			|| hi.getInput().get(1) instanceof BinaryOp 
+			&& hi.getDim2() > 1 //not applied for vector-vector mult
+			&& HopRewriteUtils.isValidOp(((BinaryOp)hi.getInput().get(1)).getOp(), LOOKUP_VALID_WDIVMM_BINARY)) ) 
+		{
+			Hop left = hi.getInput().get(0);
+			Hop right = hi.getInput().get(1);
+			
+			//Pattern 1) t(U) %*% (W/(U%*%t(V)))
+			//alternative pattern: t(U) %*% (W*(U%*%t(V)))
+			if( right instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)right).getOp(),LOOKUP_VALID_WDIVMM_BINARY)	
+				&& HopRewriteUtils.isEqualSize(right.getInput().get(0), right.getInput().get(1)) //prevent mv
+				&& right.getInput().get(1) instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(right.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
+			{
+				Hop W = right.getInput().get(0); 
+				Hop U = right.getInput().get(1).getInput().get(0);
+				Hop V = right.getInput().get(1).getInput().get(1);
+				
+				if( HopRewriteUtils.isTransposeOfItself(left, U) ) 
+				{
+					if( !HopRewriteUtils.isTransposeOperation(V) )
+						V = HopRewriteUtils.createTranspose(V);
+					else 
+						V = V.getInput().get(0);
+					
+					boolean mult = ((BinaryOp)right).getOp() == OpOp2.MULT;
+					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+							  OpOp4.WDIVMM, W, U, V, 1, mult, false);
+					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+					
+					//add output transpose for efficient target indexing (redundant t() removed by other rewrites)
+					hnew = HopRewriteUtils.createTranspose(hnew);
+					
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedDivMM1 (line "+hi.getBeginLine()+")");					
+				}
+			}	
+			
+			//Pattern 2) (W/(U%*%t(V))) %*% V
+			//alternative pattern: (W*(U%*%t(V))) %*% V
+			if( !appliedPattern
+				&& left instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)left).getOp(), LOOKUP_VALID_WDIVMM_BINARY)	
+				&& HopRewriteUtils.isEqualSize(left.getInput().get(0), left.getInput().get(1)) //prevent mv
+				&& left.getInput().get(1) instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(left.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
+			{
+				Hop W = left.getInput().get(0); 
+				Hop U = left.getInput().get(1).getInput().get(0);
+				Hop V = left.getInput().get(1).getInput().get(1);
+				
+				if( HopRewriteUtils.isTransposeOfItself(right, V) ) 
+				{
+					if( !HopRewriteUtils.isTransposeOperation(V) )
+						V = right;
+					else 
+						V = V.getInput().get(0);
+					
+					boolean mult = ((BinaryOp)left).getOp() == OpOp2.MULT;
+					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+							  OpOp4.WDIVMM, W, U, V, 2, mult, false);
+					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedDivMM2 (line "+hi.getBeginLine()+")");	
+				}
+			}
+			
+			//Pattern 3) t(U) %*% ((X!=0)*(U%*%t(V)-X))
+			if( right instanceof BinaryOp && ((BinaryOp)right).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT
+				&& right.getInput().get(1) instanceof BinaryOp && ((BinaryOp)right.getInput().get(1)).getOp()==OpOp2.MINUS	
+				&& right.getInput().get(1).getInput().get(0) instanceof AggBinaryOp
+                && right.getInput().get(1).getInput().get(1).getDataType() == DataType.MATRIX
+				&& HopRewriteUtils.isSingleBlock(right.getInput().get(1).getInput().get(0).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
+			{
+				Hop W = right.getInput().get(0); 
+				Hop U = right.getInput().get(1).getInput().get(0).getInput().get(0);
+				Hop V = right.getInput().get(1).getInput().get(0).getInput().get(1);
+				Hop X = right.getInput().get(1).getInput().get(1);
+				
+				if(    HopRewriteUtils.isNonZeroIndicator(W, X)        //W-X constraint
+				    && HopRewriteUtils.isTransposeOfItself(left, U) )  //t(U)-U constraint
+				{
+					if( !HopRewriteUtils.isTransposeOperation(V) )
+						V = HopRewriteUtils.createTranspose(V);
+					else 
+						V = V.getInput().get(0);
+					
+					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+							  OpOp4.WDIVMM, X, U, V, 1, true, true);
+					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+					
+					//add output transpose for efficient target indexing (redundant t() removed by other rewrites)
+					hnew = HopRewriteUtils.createTranspose(hnew);
+					
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedDivMM3 (line "+hi.getBeginLine()+")");					
+				}
+			}	
+			
+			//Pattern 4) ((X!=0)*(U%*%t(V)-X)) %*% V
+			if( !appliedPattern
+				&& left instanceof BinaryOp && ((BinaryOp)left).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT	
+				&& left.getInput().get(1) instanceof BinaryOp && ((BinaryOp)left.getInput().get(1)).getOp()==OpOp2.MINUS	
+				&& left.getInput().get(1).getInput().get(0) instanceof AggBinaryOp
+                && left.getInput().get(1).getInput().get(1).getDataType() == DataType.MATRIX
+				&& HopRewriteUtils.isSingleBlock(left.getInput().get(1).getInput().get(0).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
+			{
+				Hop W = left.getInput().get(0); 
+				Hop U = left.getInput().get(1).getInput().get(0).getInput().get(0);
+				Hop V = left.getInput().get(1).getInput().get(0).getInput().get(1);
+				Hop X = left.getInput().get(1).getInput().get(1);
+				
+				if(    HopRewriteUtils.isNonZeroIndicator(W, X)        //W-X constraint
+					&& HopRewriteUtils.isTransposeOfItself(right, V) )  //V-t(V) constraint
+				{
+					if( !HopRewriteUtils.isTransposeOperation(V) )
+						V = right;
+					else 
+						V = V.getInput().get(0);
+					
+					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+							  OpOp4.WDIVMM, X, U, V, 2, true, true);
+					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+					appliedPattern = true;
+					LOG.debug("Applied simplifyWeightedDivMM4 (line "+hi.getBeginLine()+")");	
+				}
+			}
+		}
+		
+		//Pattern 5) (W*(U%*%t(V)))
+		if( !appliedPattern
+			&& hi instanceof BinaryOp && ((BinaryOp)hi).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT	
+			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
+			&& hi.getDim2() > 1 //not applied for vector-vector mult
+			&& hi.getInput().get(0).getDataType() == DataType.MATRIX 
+			&& hi.getInput().get(0).getDim2() > hi.getInput().get(0).getColsInBlock()
+			&& hi.getInput().get(1) instanceof AggBinaryOp
+			&& (((AggBinaryOp) hi.getInput().get(1)).checkMapMultChain() == ChainType.NONE || hi.getInput().get(1).getInput().get(1).getDim2() > 1) //no mmchain
+			&& HopRewriteUtils.isSingleBlock(hi.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
+		{
+			Hop W = hi.getInput().get(0); 
+			Hop U = hi.getInput().get(1).getInput().get(0);
+			Hop V = hi.getInput().get(1).getInput().get(1);
+			
+			if( !HopRewriteUtils.isTransposeOperation(V) )
+				V = HopRewriteUtils.createTranspose(V);
+			else 
+				V = V.getInput().get(0);
+				
+			hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+					  OpOp4.WDIVMM, W, U, V, 0, true, false);
+			HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+			appliedPattern = true;
+			LOG.debug("Applied simplifyWeightedDivMM5 (line "+hi.getBeginLine()+")");	
+		}
+		
+		//relink new hop into original position
+		if( hnew != null ) {
+			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
+			HopRewriteUtils.addChildReference(parent, hnew, pos);
+			hi = hnew;
+		}
+		
+		return hi;
+	}
+
+	/**
+	 * 
+	 * @param parent
+	 * @param hi
+	 * @param pos
+	 * @return
+	 * @throws HopsException
+	 */
+	private Hop simplifyWeightedCrossEntropy(Hop parent, Hop hi, int pos) 
+		throws HopsException
+	{
+		Hop hnew = null;
+		
+		//Pattern 1) sum( X * log(U %*% t(V)))
+		if( hi instanceof AggUnaryOp && ((AggUnaryOp)hi).getDirection()==Direction.RowCol
+			&& ((AggUnaryOp)hi).getOp() == AggOp.SUM     //pattern rooted by sum()
+			&& hi.getInput().get(0) instanceof BinaryOp  //pattern subrooted by binary op
+			&& hi.getInput().get(0).getDim2() > 1   )    //not applied for vector-vector mult
+		{
+			BinaryOp bop = (BinaryOp) hi.getInput().get(0);
+			Hop left = bop.getInput().get(0);
+			Hop right = bop.getInput().get(1);
+			
+			if( bop.getOp()==OpOp2.MULT && left.getDataType()==DataType.MATRIX		
+				&& HopRewriteUtils.isEqualSize(left, right)  //prevent mb
+				&& right instanceof UnaryOp	&& ((UnaryOp)right).getOp()==OpOp1.LOG
+				&& right.getInput().get(0) instanceof AggBinaryOp  //ba gurantees matrices
+				&& HopRewriteUtils.isSingleBlock(right.getInput().get(0).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
+			{
+				Hop X = left; 
+				Hop U = right.getInput().get(0).getInput().get(0);
+				Hop V = right.getInput().get(0).getInput().get(1);
+				
+				if( !HopRewriteUtils.isTransposeOperation(V) )
+					V = HopRewriteUtils.createTranspose(V);
+				else 
+					V = V.getInput().get(0);
+					
+				hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, OpOp4.WCEMM, X, U, V);
+				HopRewriteUtils.setOutputBlocksizes(hnew, X.getRowsInBlock(), X.getColsInBlock());
+					
+				LOG.debug("Applied simplifyWeightedCEMM (line "+hi.getBeginLine()+")");					
+			}
+		}
+		
+		//relink new hop into original position
+		if( hnew != null ) {
+			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
+			HopRewriteUtils.addChildReference(parent, hnew, pos);
+			hi = hnew;
+		}
+		
+		return hi;
+	}
+	
 	/**
 	 * NOTE: dot-product-sum could be also applied to sum(a*b). However, we 
 	 * restrict ourselfs to sum(a^2) and transitively sum(a*a) since a general mm

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/e52e0c0a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
index 4fc23eb..d2887a2 100644
--- a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
+++ b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
@@ -29,9 +29,7 @@ import com.ibm.bi.dml.hops.BinaryOp;
 import com.ibm.bi.dml.hops.DataGenOp;
 import com.ibm.bi.dml.hops.Hop;
 import com.ibm.bi.dml.hops.Hop.OpOp1;
-import com.ibm.bi.dml.hops.Hop.OpOp4;
 import com.ibm.bi.dml.hops.IndexingOp;
-import com.ibm.bi.dml.hops.QuaternaryOp;
 import com.ibm.bi.dml.hops.TernaryOp;
 import com.ibm.bi.dml.hops.UnaryOp;
 import com.ibm.bi.dml.hops.Hop.AggOp;
@@ -44,7 +42,6 @@ import com.ibm.bi.dml.hops.LiteralOp;
 import com.ibm.bi.dml.hops.Hop.OpOp2;
 import com.ibm.bi.dml.hops.ParameterizedBuiltinOp;
 import com.ibm.bi.dml.hops.ReorgOp;
-import com.ibm.bi.dml.lops.MapMultChain.ChainType;
 import com.ibm.bi.dml.parser.DataExpression;
 import com.ibm.bi.dml.parser.Statement;
 import com.ibm.bi.dml.parser.Expression.DataType;
@@ -66,7 +63,6 @@ public class RewriteAlgebraicSimplificationStatic extends HopRewriteRule
 	
 	private static OpOp2[] LOOKUP_VALID_DISTRIBUTIVE_BINARY = new OpOp2[]{OpOp2.PLUS, OpOp2.MINUS}; 
 	private static OpOp2[] LOOKUP_VALID_ASSOCIATIVE_BINARY = new OpOp2[]{OpOp2.PLUS, OpOp2.MULT}; 
-	private static OpOp2[] LOOKUP_VALID_WDIVMM_BINARY = new OpOp2[]{OpOp2.MULT, OpOp2.DIV}; 
 	
 	@Override
 	public ArrayList<Hop> rewriteHopDAGs(ArrayList<Hop> roots, ProgramRewriteStatus state) 
@@ -149,10 +145,6 @@ public class RewriteAlgebraicSimplificationStatic extends HopRewriteRule
 			hi = removeUnnecessaryTranspose(hop, hi, i);         //e.g., t(t(X))->X; potentially introduced by diag/trace_MM
 			hi = removeUnnecessaryMinus(hop, hi, i);             //e.g., -(-X)->X; potentially introduced by simplfiy binary or dyn rewrites
 			hi = simplifyGroupedAggregate(hi);          	     //e.g., aggregate(target=X,groups=y,fn="count") -> aggregate(target=y,groups=y,fn="count")
-			hi = simplifyWeightedSquaredLoss(hop, hi, i);        //e.g., sum(W * (X - U %*% t(V)) ^ 2) -> wsl(X, U, t(V), W, true)
-			hi = simplifyWeightedSigmoidMMChains(hop, hi, i);    //e.g., W * sigmoid(Y%*%t(X)) -> wsigmoid(W, Y, t(X), type)
-			hi = simplifyWeightedDivMM(hop, hi, i);              //e.g., t(U) %*% (X/(U%*%t(V))) -> wdivmm(X, U, t(V), left)
-			hi = simplifyWeightedCrossEntropy(hop, hi, i);       //e.g., sum(X*log(U%*%t(V))) -> wcemm(X, U, t(V))
 			hi = fuseMinusNzBinaryOperation(hop, hi, i);         //e.g., X-mean*ppred(X,0,!=) -> X -nz mean
 			hi = fuseLogNzBinaryOperation(hop, hi, i);           //e.g., ppred(X,0,"!=")*log(X,0.5) -> log_nz(X,0.5)
 			hi = simplifyOuterSeqExpand(hop, hi, i);             //e.g., outer(v, seq(1,m), "==") -> rexpand(v, max=m, dir=row, ignore=true, cast=false)
@@ -1298,6 +1290,7 @@ public class RewriteAlgebraicSimplificationStatic extends HopRewriteRule
 					LOG.debug("Applied simplifyGroupedAggregateCount");	
 				}
 			}
+<<<<<<< Upstream, based on branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-systemml.git
 		}
 		
 		return hi;
@@ -1900,6 +1893,8 @@ public class RewriteAlgebraicSimplificationStatic extends HopRewriteRule
 			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
 			HopRewriteUtils.addChildReference(parent, hnew, pos);
 			hi = hnew;
+=======
+>>>>>>> 04aa86c Fix rewrite 'fuse sum_sq' (after wsloss rewrite), for kmeans_predict 
 		}
 		
 		return hi;


[2/8] incubator-systemml git commit: Fix hdfs file cleanup w/ pending rdd operations (deferred file removal)

Posted by mb...@apache.org.
Fix hdfs file cleanup w/ pending rdd operations (deferred file removal)

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

Branch: refs/heads/master
Commit: abcebc6dae18b3349c5758bb1f552b64894e86da
Parents: aa9253a
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Nov 20 17:54:42 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:37:07 2015 -0800

----------------------------------------------------------------------
 .../context/SparkExecutionContext.java          | 33 ++++++++++++++------
 .../instructions/spark/data/RDDObject.java      | 15 ++++++---
 .../ibm/bi/dml/runtime/util/MapReduceTool.java  |  5 +++
 3 files changed, 40 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/abcebc6d/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
index 23a1927..8f817f2 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/controlprogram/context/SparkExecutionContext.java
@@ -17,6 +17,7 @@
 
 package com.ibm.bi.dml.runtime.controlprogram.context;
 
+import java.io.IOException;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -813,12 +814,14 @@ public class SparkExecutionContext extends ExecutionContext
 					//clean cached data	
 					mo.clearData(); 
 					
-					//clean hdfs data
-					if( mo.isFileExists() ) {
-						String fpath = mo.getFileName();
-						if (fpath != null) {
-							MapReduceTool.deleteFileIfExistOnHDFS(fpath);
-							MapReduceTool.deleteFileIfExistOnHDFS(fpath + ".mtd");
+					//clean hdfs data if no pending rdd operations on it
+					if( mo.isFileExists() && mo.getFileName()!=null ) {
+						if( mo.getRDDHandle()==null ) {
+							MapReduceTool.deleteFileWithMTDIfExistOnHDFS(mo.getFileName());
+						}
+						else { //deferred file removal
+							RDDObject rdd = mo.getRDDHandle();
+							rdd.setHDFSFilename(mo.getFileName());
 						}
 					}
 					
@@ -839,7 +842,13 @@ public class SparkExecutionContext extends ExecutionContext
 		}
 	}
 	
-	private void rCleanupLineageObject(LineageObject lob)
+	/**
+	 * 
+	 * @param lob
+	 * @throws IOException
+	 */
+	private void rCleanupLineageObject(LineageObject lob) 
+		throws IOException
 	{		
 		//abort recursive cleanup if still consumers
 		if( lob.getNumReferences() > 0 )
@@ -851,8 +860,14 @@ public class SparkExecutionContext extends ExecutionContext
 			return;
 		
 		//cleanup current lineage object (from driver/executors)
-		if( lob instanceof RDDObject )
-			cleanupRDDVariable(((RDDObject)lob).getRDD());
+		//incl deferred hdfs file removal (only if metadata set by cleanup call)
+		if( lob instanceof RDDObject ) {
+			RDDObject rdd = (RDDObject)lob;
+			cleanupRDDVariable(rdd.getRDD());
+			if( rdd.getHDFSFilename()!=null ) { //deferred file removal
+				MapReduceTool.deleteFileWithMTDIfExistOnHDFS(rdd.getHDFSFilename());
+			}
+		}
 		else if( lob instanceof BroadcastObject ) {
 			PartitionedBroadcastMatrix pbm = ((BroadcastObject)lob).getBroadcast();
 			for( Broadcast<PartitionedMatrixBlock> bc : pbm.getBroadcasts() )

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/abcebc6d/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/data/RDDObject.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/data/RDDObject.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/data/RDDObject.java
index cba93f0..84e426f 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/data/RDDObject.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/data/RDDObject.java
@@ -27,6 +27,7 @@ public class RDDObject extends LineageObject
 	//meta data on origin of given rdd handle
 	private boolean _checkpointed = false; //created via checkpoint instruction
 	private boolean _hdfsfile = false;     //created from hdfs file
+	private String  _hdfsFname = null;     //hdfs filename, if created from hdfs.  
 	
 	public RDDObject( JavaPairRDD<?,?> rddvar, String varName)
 	{
@@ -53,16 +54,22 @@ public class RDDObject extends LineageObject
 		return _checkpointed;
 	}
 	
-	public void setHDFSFile( boolean flag )
-	{
+	public void setHDFSFile( boolean flag ) {
 		_hdfsfile = flag;
 	}
 	
-	public boolean isHDFSFile()
-	{
+	public void setHDFSFilename( String fname ) {
+		_hdfsFname = fname;
+	}
+	
+	public boolean isHDFSFile() {
 		return _hdfsfile;
 	}
 	
+	public String getHDFSFilename() {
+		return _hdfsFname;
+	}
+	
 
 	/**
 	 * Indicates if rdd is an hdfs file or a checkpoint over an hdfs file;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/abcebc6d/src/main/java/com/ibm/bi/dml/runtime/util/MapReduceTool.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/util/MapReduceTool.java b/src/main/java/com/ibm/bi/dml/runtime/util/MapReduceTool.java
index d8ba652..63c58a7 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/util/MapReduceTool.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/util/MapReduceTool.java
@@ -137,6 +137,11 @@ public class MapReduceTool
 		}
 	}
 
+	public static void deleteFileWithMTDIfExistOnHDFS(String fname)  throws IOException {
+		deleteFileIfExistOnHDFS(fname);
+		deleteFileIfExistOnHDFS(fname + ".mtd");
+	}
+	
 	public static void deleteFileIfExistOnHDFS(String dir) throws IOException {
 		Path outpath = new Path(dir);
 		FileSystem fs = FileSystem.get(_rJob);


[7/8] incubator-systemml git commit: New wumm quaternary op (rewrite, cp/mr/sp compiler/runtime, tests, docs)

Posted by mb...@apache.org.
New wumm quaternary op (rewrite, cp/mr/sp compiler/runtime, tests, docs)

This change adds a new quaternary operation 'wumm' for the pattern
X*uop(L%*%t(R)), where uop is an arbitrary unary operator (with few
exceptions) or matrix-scalar/scalar-matrix operation that is internally
mapped to an unary operator (e.g., X^2, 2*X).  

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

Branch: refs/heads/master
Commit: d70c4524726386ab6dec80b21914e60f80e52af1
Parents: e52e0c0
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sat Nov 21 22:00:15 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:38:28 2015 -0800

----------------------------------------------------------------------
 docs/devdocs/MatrixMultiplicationOperators.txt  |  18 +-
 src/main/java/com/ibm/bi/dml/hops/Hop.java      |   1 +
 .../java/com/ibm/bi/dml/hops/QuaternaryOp.java  | 265 +++++++++++++-
 .../RewriteAlgebraicSimplificationDynamic.java  | 112 ++++++
 src/main/java/com/ibm/bi/dml/lops/Lop.java      |   2 +-
 src/main/java/com/ibm/bi/dml/lops/Unary.java    |  27 +-
 .../ibm/bi/dml/lops/WeightedCrossEntropy.java   |  30 +-
 .../ibm/bi/dml/lops/WeightedCrossEntropyR.java  |  36 +-
 .../com/ibm/bi/dml/lops/WeightedSigmoid.java    |  30 +-
 .../com/ibm/bi/dml/lops/WeightedSigmoidR.java   |  36 +-
 .../com/ibm/bi/dml/lops/WeightedUnaryMM.java    | 165 +++++++++
 .../com/ibm/bi/dml/lops/WeightedUnaryMMR.java   | 162 +++++++++
 .../dml/runtime/functionobjects/Multiply2.java  |   6 +-
 .../bi/dml/runtime/functionobjects/Power2.java  |   5 +
 .../runtime/functionobjects/ValueFunction.java  |   1 -
 .../instructions/CPInstructionParser.java       |   1 +
 .../runtime/instructions/InstructionUtils.java  |   8 +-
 .../instructions/MRInstructionParser.java       |   4 +
 .../instructions/SPInstructionParser.java       |   4 +
 .../cp/QuaternaryCPInstruction.java             |  16 +-
 .../instructions/mr/QuaternaryInstruction.java  |  34 +-
 .../spark/QuaternarySPInstruction.java          |  34 +-
 .../dml/runtime/matrix/data/LibMatrixMult.java  | 341 +++++++++++++++++++
 .../bi/dml/runtime/matrix/data/MatrixBlock.java |   8 +-
 .../matrix/operators/QuaternaryOperator.java    |  25 +-
 .../quaternary/WeightedUnaryMatrixMultTest.java | 284 +++++++++++++++
 .../quaternary/WeightedUnaryMMExpDiv.R          |  33 ++
 .../quaternary/WeightedUnaryMMExpDiv.dml        |  27 ++
 .../quaternary/WeightedUnaryMMExpMult.R         |  33 ++
 .../quaternary/WeightedUnaryMMExpMult.dml       |  27 ++
 .../functions/quaternary/WeightedUnaryMMMult2.R |  33 ++
 .../quaternary/WeightedUnaryMMMult2.dml         |  27 ++
 .../functions/quaternary/WeightedUnaryMMPow2.R  |  33 ++
 .../quaternary/WeightedUnaryMMPow2.dml          |  27 ++
 .../functions/quaternary/ZPackageSuite.java     |   3 +-
 35 files changed, 1764 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/docs/devdocs/MatrixMultiplicationOperators.txt
----------------------------------------------------------------------
diff --git a/docs/devdocs/MatrixMultiplicationOperators.txt b/docs/devdocs/MatrixMultiplicationOperators.txt
index 54cb5ec..7bc8a9c 100644
--- a/docs/devdocs/MatrixMultiplicationOperators.txt
+++ b/docs/devdocs/MatrixMultiplicationOperators.txt
@@ -1,6 +1,6 @@
 #####################################################################
 # TITLE: An Overview of Matrix Multiplication Operators in SystemML #
-# DATE MODIFIED: 09/26/2015                                         #
+# DATE MODIFIED: 11/21/2015                                         #
 #####################################################################
 
 In the following, we give an overview of backend-specific physical matrix multiplication operators in SystemML as well as their internally used matrix multiplication block operations.
@@ -48,6 +48,7 @@ A QuaternaryOp hop can be compiled into the following physical operators. Note t
   - WSigmoid (weighted sigmoid)                          --> wsigmoid
   - WDivMM (weighted divide matrix multiplication)       --> wdivmm
   - WCeMM (weighted cross entropy matrix multiplication) --> wcemm
+  - WuMM (weighted unary op matrix multiplication)       --> wumm
 
 * 2) Physical Operator in MR (distributed, mapreduce)
   - MapWSLoss (map-side weighted squared loss)           --> wsloss
@@ -58,6 +59,8 @@ A QuaternaryOp hop can be compiled into the following physical operators. Note t
   - RedWDivMM (reduce-side weighted divide matrix mult)  --> wdivmm
   - MapWCeMM (map-side weighted cross entr. matrix mult) --> wcemm
   - RedWCeMM (reduce-side w. cross entr. matrix mult)    --> wcemm
+  - MapWuMM (map-side weighted unary op matrix mult)     --> wumm
+  - RedWuMM (reduce-side weighted unary op matrix mult)  --> wumm
 
 * 3) Physical Operators in SPARK (distributed, spark)
   - MapWSLoss (see MR, mappartitions + reduce)           --> wsloss           
@@ -70,8 +73,11 @@ A QuaternaryOp hop can be compiled into the following physical operators. Note t
   - RedWDivMM (see MR, 1/2x flatmaptopair + 1/2x join +  --> wdivmm 
     maptopair + reducebykey)  
   - MapWCeMM (see MR, mappartitions + reduce)            --> wcemm           
-  - RedWDivMM (see MR, 1/2x flatmaptopair + 1/2x join +  --> wcemm 
+  - RedWCeMM (see MR, 1/2x flatmaptopair + 1/2x join +   --> wcemm 
     maptopair + reduce)  
+  - MapWuMM (see MR, mappartitions)                      --> wumm
+  - RedWuMM (see MR, 1/2x flatmaptopair +                --> wumm
+    1/2x join + maptopair)          
   
   
 C) CORE MATRIX MULT PRIMITIVES LibMatrixMult (incl related script patterns)
@@ -112,9 +118,11 @@ C) CORE MATRIX MULT PRIMITIVES LibMatrixMult (incl related script patterns)
   - sequential / multi-threaded (same block ops, par over rows in X)                 
   - all dense, sparse-dense factors, sparse/dense-* x 7 patterns
 
-* 8) wcemm   (sum(X*log(U%*%t(V))))  
+* 8) wcemm    (sum(X*log(U%*%t(V))))  
   - sequential / multi-threaded (same block ops, par over rows in X)                 
   - all dense, sparse-dense factors, sparse/dense-*, 1 pattern
 
-
-  
\ No newline at end of file
+* 9) wumm     ((a) X*uop(U%*%t(V)), (b) X/uop(U%*%t(V)))
+  - any unary operator, e.g., X*exp(U%*%t(V)) or X*(U%*%t(V))^2  
+  - sequential / multi-threaded (same block ops, par over rows in X)                 
+  - all dense, sparse-dense factors, sparse/dense-*, 2 pattern

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/hops/Hop.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/Hop.java b/src/main/java/com/ibm/bi/dml/hops/Hop.java
index ab328a8..b968952 100644
--- a/src/main/java/com/ibm/bi/dml/hops/Hop.java
+++ b/src/main/java/com/ibm/bi/dml/hops/Hop.java
@@ -1054,6 +1054,7 @@ public abstract class Hop
 		WSIGMOID, //weighted sigmoid mm
 		WDIVMM, //weighted divide mm
 		WCEMM, //weighted cross entropy mm
+		WUMM, //weighted unary mm 
 		INVALID 
 	};
 	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/hops/QuaternaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/QuaternaryOp.java b/src/main/java/com/ibm/bi/dml/hops/QuaternaryOp.java
index db0df09..fd329f0 100644
--- a/src/main/java/com/ibm/bi/dml/hops/QuaternaryOp.java
+++ b/src/main/java/com/ibm/bi/dml/hops/QuaternaryOp.java
@@ -25,6 +25,7 @@ import com.ibm.bi.dml.lops.Lop;
 import com.ibm.bi.dml.lops.LopsException;
 import com.ibm.bi.dml.lops.RepMat;
 import com.ibm.bi.dml.lops.Transform;
+import com.ibm.bi.dml.lops.Unary;
 import com.ibm.bi.dml.lops.UnaryCP;
 import com.ibm.bi.dml.lops.LopProperties.ExecType;
 import com.ibm.bi.dml.lops.PartialAggregate.CorrectionLocationType;
@@ -40,6 +41,9 @@ import com.ibm.bi.dml.lops.WeightedSigmoidR;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
 import com.ibm.bi.dml.lops.WeightedSquaredLossR;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.parser.Expression.DataType;
 import com.ibm.bi.dml.parser.Expression.ValueType;
 import com.ibm.bi.dml.runtime.controlprogram.ParForProgramBlock.PDataPartitionFormat;
@@ -72,6 +76,11 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 	private boolean _mult = false;
 	private boolean _minus = false;
 	
+	//wumm-specific attributes
+	private boolean _umult = false;
+	private OpOp1 _uop = null;
+	private OpOp2 _sop = null;
+	
 	private QuaternaryOp() {
 		//default constructor for clone
 	}
@@ -131,6 +140,16 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 		_minus = flag2;
 	}
 	
+	public QuaternaryOp(String l, DataType dt, ValueType vt, Hop.OpOp4 o,
+			Hop inW, Hop inU, Hop inV, boolean umult, OpOp1 uop, OpOp2 sop) 
+	{
+		this(l, dt, vt, o, inW, inU, inV);
+		
+		_umult = umult;
+		_uop = uop;
+		_sop = sop;
+	}
+	
 	/**
 	 * 
 	 * @param l
@@ -235,6 +254,20 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 					break;
 				}
 				
+				case WUMM:{
+					WUMMType wtype = _umult ? WUMMType.MULT : WUMMType.DIV;
+					
+					if( et == ExecType.CP )
+						constructCPLopsWeightedUMM(wtype);
+					else if( et == ExecType.MR )
+						constructMRLopsWeightedUMM(wtype);
+					else if( et == ExecType.SPARK )
+						constructSparkLopsWeightedUMM(wtype);
+					else
+						throw new HopsException("Unsupported quaternaryop-wumm exec type: "+et);
+					break;
+				}
+				
 				default:
 					throw new HopsException(this.printErrorLocation() + "Unknown QuaternaryOp (" + _op + ") while constructing Lops");
 			}
@@ -1165,6 +1198,225 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 			setLops(wcemm);
 		}
 	}
+
+	/**
+	 * 
+	 * @param wtype 
+	 * @throws HopsException
+	 * @throws LopsException
+	 */
+	private void constructCPLopsWeightedUMM(WUMMType wtype) 
+		throws HopsException, LopsException
+	{
+		Unary.OperationTypes uop = _uop!=null ? 
+				HopsOpOp1LopsU.get(_uop) : _sop==OpOp2.POW ? 
+				Unary.OperationTypes.POW2 : Unary.OperationTypes.MULTIPLY2;	
+		
+		WeightedUnaryMM wsig = new WeightedUnaryMM(
+				getInput().get(0).constructLops(),
+				getInput().get(1).constructLops(),
+				getInput().get(2).constructLops(),
+				getDataType(), getValueType(), wtype, uop, ExecType.CP);
+		
+		//set degree of parallelism
+		int k = OptimizerUtils.getConstrainedNumThreads(_maxNumThreads);
+		wsig.setNumThreads(k);
+		
+		setOutputDimensions( wsig );
+		setLineNumbers( wsig );
+		setLops( wsig );
+	}
+	
+	/**
+	 * 
+	 * @param wtype
+	 * @throws HopsException
+	 * @throws LopsException
+	 */
+	private void constructMRLopsWeightedUMM( WUMMType wtype ) 
+		throws HopsException, LopsException
+	{
+		//NOTE: the common case for wsigmoid are factors U/V with a rank of 10s to 100s; the current runtime only
+		//supports single block outer products (U/V rank <= blocksize, i.e., 1000 by default); we enforce this
+		//by applying the hop rewrite for Weighted Squared Loss only if this constraint holds. 
+		
+		Unary.OperationTypes uop = _uop!=null ? 
+				HopsOpOp1LopsU.get(_uop) : _sop==OpOp2.POW ? 
+				Unary.OperationTypes.POW2 : Unary.OperationTypes.MULTIPLY2;	
+		
+		Hop X = getInput().get(0);
+		Hop U = getInput().get(1);
+		Hop V = getInput().get(2);
+		
+		//MR operator selection, part1
+		double m1Size = OptimizerUtils.estimateSize(U.getDim1(), U.getDim2()); //size U
+		double m2Size = OptimizerUtils.estimateSize(V.getDim1(), V.getDim2()); //size V
+		boolean isMapWsloss = (m1Size+m2Size < OptimizerUtils.getRemoteMemBudgetMap(true)); 
+		
+		if( !FORCE_REPLICATION && isMapWsloss ) //broadcast
+		{
+			//partitioning of U
+			boolean needPartU = !U.dimsKnown() || U.getDim1() * U.getDim2() > DistributedCacheInput.PARTITION_SIZE;
+			Lop lU = U.constructLops();
+			if( needPartU ){ //requires partitioning
+				lU = new DataPartition(lU, DataType.MATRIX, ValueType.DOUBLE, (m1Size>OptimizerUtils.getLocalMemBudget())?ExecType.MR:ExecType.CP, PDataPartitionFormat.ROW_BLOCK_WISE_N);
+				lU.getOutputParameters().setDimensions(U.getDim1(), U.getDim2(), getRowsInBlock(), getColsInBlock(), U.getNnz());
+				setLineNumbers(lU);	
+			}
+			
+			//partitioning of V
+			boolean needPartV = !V.dimsKnown() || V.getDim1() * V.getDim2() > DistributedCacheInput.PARTITION_SIZE;
+			Lop lV = V.constructLops();
+			if( needPartV ){ //requires partitioning
+				lV = new DataPartition(lV, DataType.MATRIX, ValueType.DOUBLE, (m2Size>OptimizerUtils.getLocalMemBudget())?ExecType.MR:ExecType.CP, PDataPartitionFormat.ROW_BLOCK_WISE_N);
+				lV.getOutputParameters().setDimensions(V.getDim1(), V.getDim2(), getRowsInBlock(), getColsInBlock(), V.getNnz());
+				setLineNumbers(lV);	
+			}
+			
+			//map-side wsloss always with broadcast
+			Lop wumm = new WeightedUnaryMM( X.constructLops(), lU, lV,  
+					DataType.MATRIX, ValueType.DOUBLE, wtype, uop, ExecType.MR);
+			setOutputDimensions(wumm);
+			setLineNumbers(wumm);
+			setLops( wumm );
+			
+			//in contrast to wsloss no aggregation required 
+		}
+		else //general case
+		{
+			//MR operator selection part 2
+			boolean cacheU = !FORCE_REPLICATION && (m1Size < OptimizerUtils.getRemoteMemBudgetReduce());
+			boolean cacheV = !FORCE_REPLICATION && ((!cacheU && m2Size < OptimizerUtils.getRemoteMemBudgetReduce()) 
+					        || (cacheU && m1Size+m2Size < OptimizerUtils.getRemoteMemBudgetReduce()));
+			
+			Group grpX = new Group(X.constructLops(), Group.OperationTypes.Sort, DataType.MATRIX, ValueType.DOUBLE);
+			grpX.getOutputParameters().setDimensions(X.getDim1(), X.getDim2(), X.getRowsInBlock(), X.getColsInBlock(), X.getNnz());
+			setLineNumbers(grpX);
+			
+			Lop lU = null;
+			if( cacheU ) {
+				//partitioning of U for read through distributed cache
+				boolean needPartU = !U.dimsKnown() || U.getDim1() * U.getDim2() > DistributedCacheInput.PARTITION_SIZE;
+				lU = U.constructLops();
+				if( needPartU ){ //requires partitioning
+					lU = new DataPartition(lU, DataType.MATRIX, ValueType.DOUBLE, (m1Size>OptimizerUtils.getLocalMemBudget())?ExecType.MR:ExecType.CP, PDataPartitionFormat.ROW_BLOCK_WISE_N);
+					lU.getOutputParameters().setDimensions(U.getDim1(), U.getDim2(), getRowsInBlock(), getColsInBlock(), U.getNnz());
+					setLineNumbers(lU);	
+				}
+			}
+			else {
+				//replication of U for shuffle to target block
+				Lop offset = createOffsetLop(V, false); //ncol of t(V) -> nrow of V determines num replicates
+				lU = new RepMat(U.constructLops(), offset, true, V.getDataType(), V.getValueType());
+				lU.getOutputParameters().setDimensions(U.getDim1(), U.getDim2(), 
+						U.getRowsInBlock(), U.getColsInBlock(), U.getNnz());
+				setLineNumbers(lU);
+				
+				Group grpU = new Group(lU, Group.OperationTypes.Sort, DataType.MATRIX, ValueType.DOUBLE);
+				grpU.getOutputParameters().setDimensions(U.getDim1(), U.getDim2(), U.getRowsInBlock(), U.getColsInBlock(), -1);
+				setLineNumbers(grpU);
+				lU = grpU;
+			}
+			
+			Lop lV = null;
+			if( cacheV ) {
+				//partitioning of V for read through distributed cache
+				boolean needPartV = !V.dimsKnown() || V.getDim1() * V.getDim2() > DistributedCacheInput.PARTITION_SIZE;
+				lV = V.constructLops();
+				if( needPartV ){ //requires partitioning
+					lV = new DataPartition(lV, DataType.MATRIX, ValueType.DOUBLE, (m2Size>OptimizerUtils.getLocalMemBudget())?ExecType.MR:ExecType.CP, PDataPartitionFormat.ROW_BLOCK_WISE_N);
+					lV.getOutputParameters().setDimensions(V.getDim1(), V.getDim2(), getRowsInBlock(), getColsInBlock(), V.getNnz());
+					setLineNumbers(lV);	
+				}
+			}
+			else {
+				//replication of t(V) for shuffle to target block
+				Transform ltV = new Transform( V.constructLops(), HopsTransf2Lops.get(ReOrgOp.TRANSPOSE), getDataType(), getValueType(), ExecType.MR);
+				ltV.getOutputParameters().setDimensions(V.getDim2(), V.getDim1(), 
+						V.getColsInBlock(), V.getRowsInBlock(), V.getNnz());
+				setLineNumbers(ltV);
+				
+				Lop offset = createOffsetLop(U, false); //nrow of U determines num replicates
+				lV = new RepMat(ltV, offset, false, V.getDataType(), V.getValueType());
+				lV.getOutputParameters().setDimensions(V.getDim2(), V.getDim1(), 
+						V.getColsInBlock(), V.getRowsInBlock(), V.getNnz());
+				setLineNumbers(lV);
+				
+				Group grpV = new Group(lV, Group.OperationTypes.Sort, DataType.MATRIX, ValueType.DOUBLE);
+				grpV.getOutputParameters().setDimensions(V.getDim2(), V.getDim1(), V.getColsInBlock(), V.getRowsInBlock(), -1);
+				setLineNumbers(grpV);
+				lV = grpV;
+			}
+			
+			//reduce-side wsloss w/ or without broadcast
+			Lop wumm = new WeightedUnaryMMR( 
+					grpX, lU, lV, DataType.MATRIX, ValueType.DOUBLE, wtype, uop, cacheU, cacheV, ExecType.MR);
+			setOutputDimensions(wumm);
+			setLineNumbers(wumm);
+			setLops(wumm);
+			
+			//in contrast to wsloss no aggregation required 	
+		}
+	}
+	
+	/**
+	 * 
+	 * @param wtype
+	 * @throws HopsException
+	 * @throws LopsException
+	 */
+	private void constructSparkLopsWeightedUMM( WUMMType wtype ) 
+		throws HopsException, LopsException
+	{
+		//NOTE: the common case for wsigmoid are factors U/V with a rank of 10s to 100s; the current runtime only
+		//supports single block outer products (U/V rank <= blocksize, i.e., 1000 by default); we enforce this
+		//by applying the hop rewrite for Weighted Squared Loss only if this constraint holds. 
+
+		Unary.OperationTypes uop = _uop!=null ? 
+				HopsOpOp1LopsU.get(_uop) : _sop==OpOp2.POW ? 
+				Unary.OperationTypes.POW2 : Unary.OperationTypes.MULTIPLY2;	
+		
+		//Notes: Any broadcast needs to fit twice in local memory because we partition the input in cp,
+		//and needs to fit once in executor broadcast memory. The 2GB broadcast constraint is no longer
+		//required because the max_int byte buffer constraint has been fixed in Spark 1.4 
+		double memBudgetExec = SparkExecutionContext.getBroadcastMemoryBudget();
+		double memBudgetLocal = OptimizerUtils.getLocalMemBudget();
+
+		Hop X = getInput().get(0);
+		Hop U = getInput().get(1);
+		Hop V = getInput().get(2);
+		
+		//MR operator selection, part1
+		double m1Size = OptimizerUtils.estimateSize(U.getDim1(), U.getDim2()); //size U
+		double m2Size = OptimizerUtils.estimateSize(V.getDim1(), V.getDim2()); //size V
+		boolean isMapWsloss = (m1Size+m2Size < memBudgetExec
+				&& 2*m1Size<memBudgetLocal && 2*m2Size<memBudgetLocal); 
+		
+		if( !FORCE_REPLICATION && isMapWsloss ) //broadcast
+		{
+			//map-side wsloss always with broadcast
+			Lop wsigmoid = new WeightedUnaryMM( X.constructLops(), U.constructLops(), V.constructLops(),  
+					DataType.MATRIX, ValueType.DOUBLE, wtype, uop, ExecType.SPARK);
+			setOutputDimensions(wsigmoid);
+			setLineNumbers(wsigmoid);
+			setLops( wsigmoid );
+		}
+		else //general case
+		{
+			//MR operator selection part 2
+			boolean cacheU = !FORCE_REPLICATION && (m1Size < memBudgetExec && 2*m1Size < memBudgetLocal);
+			boolean cacheV = !FORCE_REPLICATION && ((!cacheU && m2Size < memBudgetExec ) 
+					        || (cacheU && m1Size+m2Size < memBudgetExec)) && 2*m2Size < memBudgetLocal;
+			
+			//reduce-side wsloss w/ or without broadcast
+			Lop wsigmoid = new WeightedUnaryMMR( 
+					X.constructLops(), U.constructLops(), V.constructLops(), 
+					DataType.MATRIX, ValueType.DOUBLE, wtype, uop, cacheU, cacheV, ExecType.SPARK);
+			setOutputDimensions(wsigmoid);
+			setLineNumbers(wsigmoid);
+			setLops(wsigmoid);
+		}
+	}
 	
 	/**
 	 * 
@@ -1238,6 +1490,7 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 				
 			case WSIGMOID: 
 			case WDIVMM: 
+			case WUMM:	
 				double sp = OptimizerUtils.getSparsity(dim1, dim2, nnz);
 				return OptimizerUtils.estimateSizeExactSparsity(dim1, dim2, sp);
 				
@@ -1263,7 +1516,8 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 				ret = null;
 				break;
 			
-			case WSIGMOID: {
+			case WSIGMOID: 
+			case WUMM: {
 				MatrixCharacteristics mcW = memo.getAllInputStats(getInput().get(0));
 				ret = new long[]{mcW.getRows(), mcW.getCols(), mcW.getNonZeros()};
 				break;
@@ -1336,7 +1590,8 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 				//do nothing: always scalar
 				break;
 				
-			case WSIGMOID: {
+			case WSIGMOID:
+			case WUMM: {
 				Hop inW = getInput().get(0);
 				setDim1( inW.getDim1() );
 				setDim2( inW.getDim2() );
@@ -1385,6 +1640,9 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 		ret._baseType      = _baseType;
 		ret._mult          = _mult;
 		ret._minus         = _minus;
+		ret._umult         = _umult;
+		ret._uop           = _uop;
+		ret._sop           = _sop;		
 		ret._maxNumThreads = _maxNumThreads;
 		
 		return ret;
@@ -1416,6 +1674,9 @@ public class QuaternaryOp extends Hop implements MultiThreadedHop
 		ret &= _baseType      == that2._baseType;
 		ret &= _mult          == that2._mult;
 		ret &= _minus         == that2._minus;
+		ret &= _umult         == that2._umult;
+		ret &= _uop           == that2._uop;
+		ret &= _sop           == that2._sop;
 		ret &= _maxNumThreads == that2._maxNumThreads;
 		
 		return ret;

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 10dbedf..1a0710f 100644
--- a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -74,6 +74,9 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 	//valid pseudo-sparse-safe binary operators for wdivmm 
 	private static OpOp2[] LOOKUP_VALID_WDIVMM_BINARY = new OpOp2[]{OpOp2.MULT, OpOp2.DIV}; 
 	
+	//valid unary and binary operators for wumm
+	private static OpOp1[] LOOKUP_VALID_WUMM_UNARY = new OpOp1[]{OpOp1.ABS, OpOp1.ROUND, OpOp1.CEIL, OpOp1.FLOOR, OpOp1.EXP, OpOp1.LOG, OpOp1.SQRT,  OpOp1.SIGMOID, OpOp1.SPROP}; 
+	private static OpOp2[] LOOKUP_VALID_WUMM_BINARY = new OpOp2[]{OpOp2.MULT, OpOp2.POW}; 
 	
 	@Override
 	public ArrayList<Hop> rewriteHopDAGs(ArrayList<Hop> roots, ProgramRewriteStatus state) 
@@ -166,6 +169,7 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 			hi = simplifyWeightedSigmoidMMChains(hop, hi, i); //e.g., W * sigmoid(Y%*%t(X)) -> wsigmoid(W, Y, t(X), type)
 			hi = simplifyWeightedDivMM(hop, hi, i);           //e.g., t(U) %*% (X/(U%*%t(V))) -> wdivmm(X, U, t(V), left)
 			hi = simplifyWeightedCrossEntropy(hop, hi, i);    //e.g., sum(X*log(U%*%t(V))) -> wcemm(X, U, t(V))
+			hi = simplifyWeightedUnaryMM(hop, hi, i);         //e.g., X*exp(U%*%t(V)) -> wumm(X, U, t(V), exp)
 			hi = simplifyDotProductSum(hop, hi, i);           //e.g., sum(v^2) -> t(v)%*%v if ncol(v)==1 
 			hi = fuseSumSquared(hop, hi, i);                  //e.g., sum(X^2) -> sumSq(X), if ncol(X)>1
 			hi = reorderMinusMatrixMult(hop, hi, i);          //e.g., (-t(X))%*%y->-(t(X)%*%y), TODO size
@@ -1946,6 +1950,114 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 	}
 	
 	/**
+	 * 
+	 * @param parent
+	 * @param hi
+	 * @param pos
+	 * @return
+	 * @throws HopsException
+	 */
+	private Hop simplifyWeightedUnaryMM(Hop parent, Hop hi, int pos) 
+		throws HopsException
+	{
+		Hop hnew = null;
+		boolean appliedPattern = false;
+		
+		//Pattern 1) (W*uop(U%*%t(V)))
+		if( hi instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)hi).getOp(),LOOKUP_VALID_WDIVMM_BINARY)	
+			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
+			&& hi.getDim2() > 1 //not applied for vector-vector mult
+			&& hi.getInput().get(0).getDataType() == DataType.MATRIX 
+			&& hi.getInput().get(0).getDim2() > hi.getInput().get(0).getColsInBlock()
+			&& hi.getInput().get(1) instanceof UnaryOp
+			&& HopRewriteUtils.isValidOp(((UnaryOp)hi.getInput().get(1)).getOp(), LOOKUP_VALID_WUMM_UNARY) 
+			&& hi.getInput().get(1).getInput().get(0) instanceof AggBinaryOp
+			&& HopRewriteUtils.isSingleBlock(hi.getInput().get(1).getInput().get(0).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT			
+		{
+			Hop W = hi.getInput().get(0); 
+			Hop U = hi.getInput().get(1).getInput().get(0).getInput().get(0);
+			Hop V = hi.getInput().get(1).getInput().get(0).getInput().get(1);
+			boolean mult = ((BinaryOp)hi).getOp()==OpOp2.MULT;
+			OpOp1 op = ((UnaryOp)hi.getInput().get(1)).getOp();
+			
+			if( !HopRewriteUtils.isTransposeOperation(V) )
+				V = HopRewriteUtils.createTranspose(V);
+			else 
+				V = V.getInput().get(0);
+				
+			hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+					  OpOp4.WUMM, W, U, V, mult, op, null);
+			HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+
+			appliedPattern = true;
+			LOG.debug("Applied simplifyWeightedUnaryMM1 (line "+hi.getBeginLine()+")");	
+		}
+		
+		//Pattern 2) (W*sop(U%*%t(V),c)) for known sop translating to unary ops
+		if( !appliedPattern
+			&& hi instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)hi).getOp(),LOOKUP_VALID_WDIVMM_BINARY)
+			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
+			&& hi.getDim2() > 1 //not applied for vector-vector mult
+			&& hi.getInput().get(0).getDataType() == DataType.MATRIX
+			&& hi.getInput().get(0).getDim2() > hi.getInput().get(0).getColsInBlock()
+			&& hi.getInput().get(1) instanceof BinaryOp
+			&& HopRewriteUtils.isValidOp(((BinaryOp)hi.getInput().get(1)).getOp(), LOOKUP_VALID_WUMM_BINARY) )
+		{
+			Hop left = hi.getInput().get(1).getInput().get(0);
+			Hop right = hi.getInput().get(1).getInput().get(1);
+			Hop abop = null;
+			
+			//pattern 2a) matrix-scalar operations
+			if( right.getDataType()==DataType.SCALAR && right instanceof LiteralOp
+				&& HopRewriteUtils.getDoubleValue((LiteralOp)right)==2 //pow2, mult2
+				&& left instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(left.getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT			
+			{
+				abop = left;
+			}
+			//pattern 2b) scalar-matrix operations
+			else if( left.getDataType()==DataType.SCALAR && left instanceof LiteralOp 
+				&& HopRewriteUtils.getDoubleValue((LiteralOp)left)==2 //mult2
+				&& ((BinaryOp)hi.getInput().get(1)).getOp() == OpOp2.MULT
+				&& right instanceof AggBinaryOp
+				&& HopRewriteUtils.isSingleBlock(right.getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT			
+			{
+				abop = right;
+			}
+			
+			if( abop != null ) {
+				Hop W = hi.getInput().get(0); 
+				Hop U = abop.getInput().get(0);
+				Hop V = abop.getInput().get(1);
+				boolean mult = ((BinaryOp)hi).getOp()==OpOp2.MULT;
+				OpOp2 op = ((BinaryOp)hi.getInput().get(1)).getOp();
+				
+				if( !HopRewriteUtils.isTransposeOperation(V) )
+					V = HopRewriteUtils.createTranspose(V);
+				else 
+					V = V.getInput().get(0);
+					
+				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
+						  OpOp4.WUMM, W, U, V, mult, null, op);
+				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
+	
+				appliedPattern = true;
+				LOG.debug("Applied simplifyWeightedUnaryMM2 (line "+hi.getBeginLine()+")");	
+			}
+		}
+		
+		
+		//relink new hop into original position
+		if( hnew != null ) {
+			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
+			HopRewriteUtils.addChildReference(parent, hnew, pos);
+			hi = hnew;
+		}
+		
+		return hi;
+	}
+	
+	/**
 	 * NOTE: dot-product-sum could be also applied to sum(a*b). However, we 
 	 * restrict ourselfs to sum(a^2) and transitively sum(a*a) since a general mm
 	 * a%*%b on MR can be also counter-productive (e.g., MMCJ) while tsmm is always 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/Lop.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/Lop.java b/src/main/java/com/ibm/bi/dml/lops/Lop.java
index 1de8f60..751ca1f 100644
--- a/src/main/java/com/ibm/bi/dml/lops/Lop.java
+++ b/src/main/java/com/ibm/bi/dml/lops/Lop.java
@@ -53,7 +53,7 @@ public abstract class Lop
 		ParameterizedBuiltin,                               //CP/MR parameterized ops (name/value)
 		FunctionCallCP, 									//CP function calls 
 		CumulativePartialAggregate, CumulativeSplitAggregate, CumulativeOffsetBinary, //MR cumsum/cumprod/cummin/cummax
-		WeightedSquaredLoss, WeightedSigmoid, WeightedDivMM, WeightedCeMM,
+		WeightedSquaredLoss, WeightedSigmoid, WeightedDivMM, WeightedCeMM, WeightedUMM,
 		SortKeys, PickValues,
 		Checkpoint, 										//Spark persist into storage level
 	};

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/Unary.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/Unary.java b/src/main/java/com/ibm/bi/dml/lops/Unary.java
index 4f5cd0e..b29e822 100644
--- a/src/main/java/com/ibm/bi/dml/lops/Unary.java
+++ b/src/main/java/com/ibm/bi/dml/lops/Unary.java
@@ -160,8 +160,27 @@ public class Unary extends Lop
 			return "Operation: " + operation + " " + "Label: N/A";
 	}
 
-	private String getOpcode() throws LopsException {
-		switch (operation) {
+	/**
+	 * 
+	 * @return
+	 * @throws LopsException
+	 */
+	private String getOpcode() 
+		throws LopsException 
+	{
+		return getOpcode(operation);
+	}
+	
+	/**
+	 * 
+	 * @param op
+	 * @return
+	 * @throws LopsException
+	 */
+	public static String getOpcode(OperationTypes op) 
+		throws LopsException 
+	{
+		switch (op) {
 		case NOT:
 			return "!";
 		case ABS:
@@ -289,8 +308,8 @@ public class Unary extends Lop
 			return "sel+";
 		
 		default:
-			throw new LopsException(this.printErrorLocation() + 
-					"Instruction not defined for Unary operation: " + operation);
+			throw new LopsException(
+					"Instruction not defined for Unary operation: " + op);
 		}
 	}
 	public String getInstructions(String input1, String output) 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropy.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropy.java b/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropy.java
index ea8a160..df131c9 100644
--- a/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropy.java
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropy.java
@@ -86,31 +86,13 @@ public class WeightedCrossEntropy extends Lop
 	}
 	
 	@Override
-	public String getInstructions(int input_index1, int input_index2, int input_index3, int output_index)
+	public String getInstructions(int input1, int input2, int input3, int output)
 	{
-		StringBuilder sb = new StringBuilder();
-		
-		sb.append(getExecType());
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(OPCODE);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(0).prepInputOperand(input_index1));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(1).prepInputOperand(input_index2));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(2).prepInputOperand(input_index3));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( prepOutputOperand(output_index));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_wcemmType);
-		
-		return sb.toString();
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropyR.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropyR.java b/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropyR.java
index 5b40628..58f3cc7 100644
--- a/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropyR.java
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedCrossEntropyR.java
@@ -89,37 +89,13 @@ public class WeightedCrossEntropyR extends Lop
 	}
 	
 	@Override
-	public String getInstructions(int input_index1, int input_index2, int input_index3, int output_index)
+	public String getInstructions(int input1, int input2, int input3, int output)
 	{
-		StringBuilder sb = new StringBuilder();
-		
-		sb.append(getExecType());
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(OPCODE);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(0).prepInputOperand(input_index1));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(1).prepInputOperand(input_index2));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(2).prepInputOperand(input_index3));
-	
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( prepOutputOperand(output_index));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_wcemmType);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_cacheU);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_cacheV);
-		
-		return sb.toString();
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoid.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoid.java b/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoid.java
index 2f187b9..310ae37 100644
--- a/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoid.java
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoid.java
@@ -90,31 +90,13 @@ public class WeightedSigmoid extends Lop
 	}
 	
 	@Override
-	public String getInstructions(int input_index1, int input_index2, int input_index3, int output_index)
+	public String getInstructions(int input1, int input2, int input3, int output)
 	{
-		StringBuilder sb = new StringBuilder();
-		
-		sb.append(getExecType());
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(OPCODE);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(0).prepInputOperand(input_index1));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(1).prepInputOperand(input_index2));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(2).prepInputOperand(input_index3));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( prepOutputOperand(output_index));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_wsigmoidType);
-		
-		return sb.toString();
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoidR.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoidR.java b/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoidR.java
index 6aff6d2..de99bfd 100644
--- a/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoidR.java
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedSigmoidR.java
@@ -89,37 +89,13 @@ public class WeightedSigmoidR extends Lop
 	}
 	
 	@Override
-	public String getInstructions(int input_index1, int input_index2, int input_index3, int output_index)
+	public String getInstructions(int input1, int input2, int input3, int output)
 	{
-		StringBuilder sb = new StringBuilder();
-		
-		sb.append(getExecType());
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(OPCODE);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(0).prepInputOperand(input_index1));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(1).prepInputOperand(input_index2));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( getInputs().get(2).prepInputOperand(input_index3));
-	
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append( prepOutputOperand(output_index));
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_wsType);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_cacheU);
-		
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(_cacheV);
-		
-		return sb.toString();
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMM.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMM.java b/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMM.java
new file mode 100644
index 0000000..48269df
--- /dev/null
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMM.java
@@ -0,0 +1,165 @@
+/**
+ * (C) Copyright IBM Corp. 2010, 2015
+ *
+ * Licensed 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 com.ibm.bi.dml.lops;
+
+import com.ibm.bi.dml.lops.LopProperties.ExecLocation;
+import com.ibm.bi.dml.lops.LopProperties.ExecType;
+import com.ibm.bi.dml.lops.Unary.OperationTypes;
+import com.ibm.bi.dml.lops.compile.JobType;
+import com.ibm.bi.dml.parser.Expression.DataType;
+import com.ibm.bi.dml.parser.Expression.ValueType;
+
+/**
+ * 
+ */
+public class WeightedUnaryMM extends Lop 
+{
+	public static final String OPCODE = "mapwumm";
+	public static final String OPCODE_CP = "wumm";
+
+	public enum WUMMType {
+		MULT,
+		DIV,
+	}
+	
+	private WUMMType _wummType = null;
+	private OperationTypes _uop = null;
+	private int _numThreads = 1;
+	
+	public WeightedUnaryMM(Lop input1, Lop input2, Lop input3, DataType dt, ValueType vt, WUMMType wt, OperationTypes op, ExecType et) 
+		throws LopsException 
+	{
+		super(Lop.Type.WeightedUMM, dt, vt);		
+		addInput(input1); //X
+		addInput(input2); //U
+		addInput(input3); //V
+		input1.addOutput(this); 
+		input2.addOutput(this);
+		input3.addOutput(this);
+		
+		//setup mapmult parameters
+		_wummType = wt;
+		_uop = op;
+		setupLopProperties(et);
+	}
+	
+	/**
+	 * 
+	 * @param et
+	 */
+	private void setupLopProperties( ExecType et )
+	{
+		if( et == ExecType.MR )
+		{
+			//setup MR parameters 
+			boolean breaksAlignment = true;
+			boolean aligner = false;
+			boolean definesMRJob = false;
+			lps.addCompatibility(JobType.GMR);
+			lps.addCompatibility(JobType.DATAGEN);
+			lps.setProperties( inputs, ExecType.MR, ExecLocation.Map, breaksAlignment, aligner, definesMRJob );
+		}
+		else //Spark/CP
+		{
+			//setup Spark parameters 
+			boolean breaksAlignment = false;
+			boolean aligner = false;
+			boolean definesMRJob = false;
+			lps.addCompatibility(JobType.INVALID);
+			lps.setProperties( inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob );
+		}
+	}
+
+	public String toString() {
+		return "Operation = WeightedUMM";
+	}
+	
+	@Override
+	public String getInstructions(int input1, int input2, int input3, int output) 
+		throws LopsException
+	{
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
+	}
+
+	@Override
+	public String getInstructions(String input1, String input2, String input3, String output) 
+		throws LopsException
+	{
+		StringBuilder sb = new StringBuilder();
+		
+		sb.append(getExecType());
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		if( getExecType() == ExecType.CP )
+			sb.append(OPCODE_CP);
+		else
+			sb.append(OPCODE);
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(Unary.getOpcode(_uop));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(0).prepInputOperand(input1));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(1).prepInputOperand(input2));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(2).prepInputOperand(input3));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( prepOutputOperand(output));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(_wummType);
+		
+		//append degree of parallelism
+		if( getExecType()==ExecType.CP ) {
+			sb.append( OPERAND_DELIMITOR );
+			sb.append( _numThreads );
+		}
+		
+		return sb.toString();
+	}
+	
+	@Override
+	public boolean usesDistributedCache() 
+	{
+		if( getExecType()==ExecType.MR )
+			return true;
+		else
+			return false;
+	}
+	
+	@Override
+	public int[] distributedCacheInputIndex() 
+	{
+		if( getExecType()==ExecType.MR )
+			return new int[]{2,3};
+		else
+			return new int[]{-1};
+	}
+	
+	public void setNumThreads(int k) {
+		_numThreads = k;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMMR.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMMR.java b/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMMR.java
new file mode 100644
index 0000000..362cb4d
--- /dev/null
+++ b/src/main/java/com/ibm/bi/dml/lops/WeightedUnaryMMR.java
@@ -0,0 +1,162 @@
+/**
+ * (C) Copyright IBM Corp. 2010, 2015
+ *
+ * Licensed 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 com.ibm.bi.dml.lops;
+
+import com.ibm.bi.dml.lops.LopProperties.ExecLocation;
+import com.ibm.bi.dml.lops.LopProperties.ExecType;
+import com.ibm.bi.dml.lops.Unary.OperationTypes;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
+import com.ibm.bi.dml.lops.compile.JobType;
+import com.ibm.bi.dml.parser.Expression.DataType;
+import com.ibm.bi.dml.parser.Expression.ValueType;
+
+/**
+ * 
+ */
+public class WeightedUnaryMMR extends Lop 
+{
+	public static final String OPCODE = "redwumm";
+	
+	private WUMMType _wummType = null;
+	private OperationTypes _uop = null;
+	private boolean _cacheU = false;
+	private boolean _cacheV = false;
+	
+	public WeightedUnaryMMR(Lop input1, Lop input2, Lop input3, DataType dt, ValueType vt, WUMMType wt, OperationTypes op, boolean cacheU, boolean cacheV, ExecType et) 
+		throws LopsException 
+	{
+		super(Lop.Type.WeightedUMM, dt, vt);		
+		addInput(input1); //X
+		addInput(input2); //U
+		addInput(input3); //V
+		input1.addOutput(this); 
+		input2.addOutput(this);
+		input3.addOutput(this);
+		
+		//setup mapmult parameters
+		_wummType = wt;
+		_uop = op;
+		_cacheU = cacheU;
+		_cacheV = cacheV;
+		setupLopProperties(et);
+	}
+	
+	/**
+	 * 
+	 * @param et
+	 * @throws LopsException 
+	 */
+	private void setupLopProperties( ExecType et ) 
+		throws LopsException
+	{
+		if( et == ExecType.MR )
+		{
+			//setup MR parameters 
+			boolean breaksAlignment = true;
+			boolean aligner = false;
+			boolean definesMRJob = false;
+			lps.addCompatibility(JobType.GMR);
+			lps.addCompatibility(JobType.DATAGEN);
+			lps.setProperties( inputs, ExecType.MR, ExecLocation.Reduce, breaksAlignment, aligner, definesMRJob );
+		}
+		else //Spark/CP
+		{
+			//setup Spark parameters 
+			boolean breaksAlignment = false;
+			boolean aligner = false;
+			boolean definesMRJob = false;
+			lps.addCompatibility(JobType.INVALID);
+			lps.setProperties( inputs, et, ExecLocation.ControlProgram, breaksAlignment, aligner, definesMRJob );
+		}
+	}
+
+	public String toString() {
+		return "Operation = WeightedUMMR";
+	}
+	
+	@Override
+	public String getInstructions(int input1, int input2, int input3, int output)
+		throws LopsException
+	{
+		return getInstructions(
+				String.valueOf(input1),
+				String.valueOf(input2),
+				String.valueOf(input3),
+				String.valueOf(output));
+	}
+
+	@Override
+	public String getInstructions(String input1, String input2, String input3, String output)
+		throws LopsException
+	{
+		StringBuilder sb = new StringBuilder();
+		
+		sb.append(getExecType());
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(OPCODE);
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(Unary.getOpcode(_uop));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(0).prepInputOperand(input1));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(1).prepInputOperand(input2));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( getInputs().get(2).prepInputOperand(input3));
+	
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append( prepOutputOperand(output));
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(_wummType);
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(_cacheU);
+		
+		sb.append(Lop.OPERAND_DELIMITOR);
+		sb.append(_cacheV);
+		
+		return sb.toString();
+	}
+	
+	@Override
+	public boolean usesDistributedCache() 
+	{
+		if( _cacheU || _cacheV )
+			return true;
+		else
+			return false;
+	}
+	
+	@Override
+	public int[] distributedCacheInputIndex() 
+	{
+		if( !_cacheU && !_cacheV )
+			return new int[]{-1};
+		else if( _cacheU && !_cacheV )
+			return new int[]{2};
+		else if( !_cacheU && _cacheV )
+			return new int[]{3};
+		else
+			return new int[]{2,3};
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Multiply2.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Multiply2.java b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Multiply2.java
index af07c0b..01af490 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Multiply2.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Multiply2.java
@@ -40,6 +40,11 @@ public class Multiply2 extends ValueFunction
 	}
 	
 	@Override
+	public double execute(double in1) {
+		return in1 + in1; //ignore in2 because always 2; 
+	}
+	
+	@Override
 	public double execute(double in1, double in2) {
 		return in1 + in1; //ignore in2 because always 2; 
 	}
@@ -63,5 +68,4 @@ public class Multiply2 extends ValueFunction
 		
 		return in1 + in1; //ignore in2 because always 2; 
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Power2.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Power2.java b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Power2.java
index 1b13923..87d91b8 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Power2.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/Power2.java
@@ -42,6 +42,11 @@ public class Power2 extends ValueFunction
 	}
 	
 	@Override
+	public double execute(double in1) {
+		return in1*in1; //ignore in2 because always 2; 
+	}
+	
+	@Override
 	public double execute(double in1, double in2) {
 		return in1*in1; //ignore in2 because always 2; 
 	}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/functionobjects/ValueFunction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/ValueFunction.java b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/ValueFunction.java
index fd6455c..af42955 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/functionobjects/ValueFunction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/functionobjects/ValueFunction.java
@@ -21,6 +21,5 @@ import java.io.Serializable;
 
 public class ValueFunction extends FunctionObject implements Serializable
 {
-	
 	private static final long serialVersionUID = -4985988545393861058L;
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/CPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/CPInstructionParser.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/CPInstructionParser.java
index cf3dcaf..c9744a9 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/CPInstructionParser.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/CPInstructionParser.java
@@ -203,6 +203,7 @@ public class CPInstructionParser extends InstructionParser
 		String2CPInstructionType.put( "wsigmoid", CPINSTRUCTION_TYPE.Quaternary);
 		String2CPInstructionType.put( "wdivmm"  , CPINSTRUCTION_TYPE.Quaternary);
 		String2CPInstructionType.put( "wcemm"   , CPINSTRUCTION_TYPE.Quaternary);
+		String2CPInstructionType.put( "wumm"    , CPINSTRUCTION_TYPE.Quaternary);
 		
 		// User-defined function Opcodes
 		String2CPInstructionType.put( "extfunct"   	, CPINSTRUCTION_TYPE.External);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/InstructionUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/InstructionUtils.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/InstructionUtils.java
index 4f0fc1c..d32ca61 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/InstructionUtils.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/InstructionUtils.java
@@ -34,6 +34,8 @@ import com.ibm.bi.dml.lops.WeightedSigmoid;
 import com.ibm.bi.dml.lops.WeightedSigmoidR;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss;
 import com.ibm.bi.dml.lops.WeightedSquaredLossR;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.functionobjects.And;
@@ -799,7 +801,9 @@ public class InstructionUtils
 			|| WeightedSigmoidR.OPCODE.equalsIgnoreCase(opcode)        //redwsigmoid
 			|| WeightedDivMM.OPCODE.equalsIgnoreCase(opcode)           //mapwdivmm
 			|| WeightedDivMMR.OPCODE.equalsIgnoreCase(opcode)          //redwdivmm
-			|| WeightedCrossEntropy.OPCODE.equalsIgnoreCase(opcode)    //mapwdcemm
-			|| WeightedCrossEntropyR.OPCODE.equalsIgnoreCase(opcode);  //redwdcemm
+			|| WeightedCrossEntropy.OPCODE.equalsIgnoreCase(opcode)    //mapwcemm
+			|| WeightedCrossEntropyR.OPCODE.equalsIgnoreCase(opcode)   //redwcemm
+			|| WeightedUnaryMM.OPCODE.equalsIgnoreCase(opcode)         //mapwumm
+			|| WeightedUnaryMMR.OPCODE.equalsIgnoreCase(opcode);       //redwumm
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/MRInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/MRInstructionParser.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/MRInstructionParser.java
index 51e5c84..932cb08 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/MRInstructionParser.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/MRInstructionParser.java
@@ -31,6 +31,8 @@ import com.ibm.bi.dml.lops.WeightedSigmoid;
 import com.ibm.bi.dml.lops.WeightedSigmoidR;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss;
 import com.ibm.bi.dml.lops.WeightedSquaredLossR;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.instructions.mr.AggregateBinaryInstruction;
@@ -224,6 +226,8 @@ public class MRInstructionParser extends InstructionParser
 		String2MRInstructionType.put( WeightedDivMMR.OPCODE,       MRINSTRUCTION_TYPE.Quaternary);
 		String2MRInstructionType.put( WeightedCrossEntropy.OPCODE, MRINSTRUCTION_TYPE.Quaternary);
 		String2MRInstructionType.put( WeightedCrossEntropyR.OPCODE,MRINSTRUCTION_TYPE.Quaternary);
+		String2MRInstructionType.put( WeightedUnaryMM.OPCODE,      MRINSTRUCTION_TYPE.Quaternary);
+		String2MRInstructionType.put( WeightedUnaryMMR.OPCODE,     MRINSTRUCTION_TYPE.Quaternary);
 		
 		// Combine Instruction Opcodes
 		String2MRInstructionType.put( "combinebinary" , MRINSTRUCTION_TYPE.CombineBinary);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/SPInstructionParser.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/SPInstructionParser.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/SPInstructionParser.java
index d6ba5ec..95102be 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/SPInstructionParser.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/SPInstructionParser.java
@@ -29,6 +29,8 @@ import com.ibm.bi.dml.lops.WeightedSigmoid;
 import com.ibm.bi.dml.lops.WeightedSigmoidR;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss;
 import com.ibm.bi.dml.lops.WeightedSquaredLossR;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.instructions.spark.AggregateTernarySPInstruction;
@@ -224,6 +226,8 @@ public class SPInstructionParser extends InstructionParser {
 		String2SPInstructionType.put( WeightedDivMMR.OPCODE,       SPINSTRUCTION_TYPE.Quaternary);
 		String2SPInstructionType.put( WeightedCrossEntropy.OPCODE, SPINSTRUCTION_TYPE.Quaternary);
 		String2SPInstructionType.put( WeightedCrossEntropyR.OPCODE,SPINSTRUCTION_TYPE.Quaternary);
+		String2SPInstructionType.put( WeightedUnaryMM.OPCODE,      SPINSTRUCTION_TYPE.Quaternary);
+		String2SPInstructionType.put( WeightedUnaryMMR.OPCODE,     SPINSTRUCTION_TYPE.Quaternary);
 		
 		//cumsum/cumprod/cummin/cummax
 		String2SPInstructionType.put( "ucumack+"  , SPINSTRUCTION_TYPE.CumsumAggregate);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/QuaternaryCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/QuaternaryCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/QuaternaryCPInstruction.java
index 4188235..42889fc 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/QuaternaryCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/QuaternaryCPInstruction.java
@@ -21,6 +21,7 @@ import com.ibm.bi.dml.lops.WeightedDivMM.WDivMMType;
 import com.ibm.bi.dml.lops.WeightedSigmoid.WSigmoidType;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
 import com.ibm.bi.dml.lops.WeightedCrossEntropy.WCeMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.controlprogram.context.ExecutionContext;
@@ -93,6 +94,19 @@ public class QuaternaryCPInstruction extends ComputationCPInstruction
 			else if( opcode.equalsIgnoreCase("wcemm") ) 		
 				return new QuaternaryCPInstruction(new QuaternaryOperator(WCeMMType.valueOf(parts[5])), in1, in2, in3, null, out, k, opcode, inst);
 		}
+		else if( opcode.equalsIgnoreCase("wumm") )
+		{
+			InstructionUtils.checkNumFields ( parts, 7 );
+			
+			String uopcode = parts[1];
+			CPOperand in1 = new CPOperand(parts[2]);
+			CPOperand in2 = new CPOperand(parts[3]);
+			CPOperand in3 = new CPOperand(parts[4]);
+			CPOperand out = new CPOperand(parts[5]);
+			int k = Integer.parseInt(parts[7]);
+			
+			return new QuaternaryCPInstruction(new QuaternaryOperator(WUMMType.valueOf(parts[6]),uopcode), in1, in2, in3, null, out, k, opcode, inst);
+		}
 		
 		throw new DMLRuntimeException("Unexpected opcode in QuaternaryCPInstruction: " + inst);
 	}
@@ -124,7 +138,7 @@ public class QuaternaryCPInstruction extends ComputationCPInstruction
 				ec.releaseMatrixInput(input4.getName());
 			ec.setVariable(output.getName(), new DoubleObject(out.getValue(0, 0)));
 		}
-		else { //wsigmoid / wdivmm
+		else { //wsigmoid / wdivmm / wumm
 			ec.setMatrixOutput(output.getName(), (MatrixBlock)out);
 		}
 	}	

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/mr/QuaternaryInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/mr/QuaternaryInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/mr/QuaternaryInstruction.java
index 814150f..087f554 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/mr/QuaternaryInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/mr/QuaternaryInstruction.java
@@ -26,6 +26,9 @@ import com.ibm.bi.dml.lops.WeightedSigmoid.WSigmoidType;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
 import com.ibm.bi.dml.lops.WeightedSquaredLossR;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.functionobjects.SwapIndex;
@@ -111,7 +114,7 @@ public class QuaternaryInstruction extends MRInstruction implements IDistributed
 			//output size independent of chain type (scalar)
 			dimOut.set(1, 1, mc1.getRowsPerBlock(), mc1.getColsPerBlock());
 		}
-		else if( qop.wtype2 != null ) { //wsigmoid
+		else if( qop.wtype2 != null || qop.wtype5 != null ) { //wsigmoid/wumm
 			//output size determined by main input
 			dimOut.set(mc1.getRows(), mc1.getCols(), mc1.getRowsPerBlock(), mc1.getColsPerBlock());
 		}
@@ -169,6 +172,33 @@ public class QuaternaryInstruction extends MRInstruction implements IDistributed
 			
 			return new QuaternaryInstruction(new QuaternaryOperator(wtype), in1, in2, in3, in4, out, cacheU, cacheV, str);	
 		}
+		else if(    WeightedUnaryMM.OPCODE.equalsIgnoreCase(opcode)    //wumm
+				|| WeightedUnaryMMR.OPCODE.equalsIgnoreCase(opcode) )
+		{
+			boolean isRed = WeightedUnaryMMR.OPCODE.equalsIgnoreCase(opcode);
+			
+			//check number of fields (4 inputs, output, type)
+			if( isRed )
+				InstructionUtils.checkNumFields ( str, 8 );
+			else
+				InstructionUtils.checkNumFields ( str, 6 );
+				
+			//parse instruction parts (without exec type)
+			String[] parts = InstructionUtils.getInstructionParts(str);
+			
+			String uopcode = parts[1];
+			byte in1 = Byte.parseByte(parts[2]);
+			byte in2 = Byte.parseByte(parts[3]);
+			byte in3 = Byte.parseByte(parts[4]);
+			byte out = Byte.parseByte(parts[5]);
+			WUMMType wtype = WUMMType.valueOf(parts[6]);
+			
+			//in mappers always through distcache, in reducers through distcache/shuffle
+			boolean cacheU = isRed ? Boolean.parseBoolean(parts[7]) : true;
+			boolean cacheV = isRed ? Boolean.parseBoolean(parts[8]) : true;
+			
+			return new QuaternaryInstruction(new QuaternaryOperator(wtype,uopcode), in1, in2, in3, (byte)-1, out, cacheU, cacheV, str);	
+		}
 		else //wsigmoid / wdivmm / wcemm
 		{
 			boolean isRed = opcode.startsWith("red");
@@ -303,7 +333,7 @@ public class QuaternaryInstruction extends MRInstruction implements IDistributed
 				
 				if( qop.wtype1 != null || qop.wtype4 != null) 
 					outIx.setIndexes(1, 1); //wsloss
-				else if ( qop.wtype2 != null || qop.wtype3!=null && qop.wtype3.isBasic() ) 
+				else if ( qop.wtype2 != null || qop.wtype5 != null || qop.wtype3!=null && qop.wtype3.isBasic() ) 
 					outIx.setIndexes(inIx); //wsigmoid/wdivmm-basic
 				else { //wdivmm
 					boolean left = qop.wtype3.isLeft();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/QuaternarySPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/QuaternarySPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/QuaternarySPInstruction.java
index e7effbb..ac8dce8 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/QuaternarySPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/spark/QuaternarySPInstruction.java
@@ -39,6 +39,9 @@ import com.ibm.bi.dml.lops.WeightedSquaredLossR;
 import com.ibm.bi.dml.lops.WeightedSigmoid.WSigmoidType;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
 import com.ibm.bi.dml.lops.WeightedCrossEntropy.WCeMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMM;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMMR;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.controlprogram.context.ExecutionContext;
@@ -117,6 +120,30 @@ public class QuaternarySPInstruction extends ComputationSPInstruction
 			
 			return new QuaternarySPInstruction(new QuaternaryOperator(wtype), in1, in2, in3, in4, out, cacheU, cacheV, opcode, str);	
 		}
+		else if(    WeightedUnaryMM.OPCODE.equalsIgnoreCase(opcode)    //wumm
+			|| WeightedUnaryMMR.OPCODE.equalsIgnoreCase(opcode) )
+		{
+			boolean isRed = WeightedUnaryMMR.OPCODE.equalsIgnoreCase(opcode);
+			
+			//check number of fields (4 inputs, output, type)
+			if( isRed )
+				InstructionUtils.checkNumFields ( parts, 8 );
+			else
+				InstructionUtils.checkNumFields ( parts, 6 );
+			
+			String uopcode = parts[1];
+			CPOperand in1 = new CPOperand(parts[2]);
+			CPOperand in2 = new CPOperand(parts[3]);
+			CPOperand in3 = new CPOperand(parts[4]);
+			CPOperand out = new CPOperand(parts[5]);
+			WUMMType wtype = WUMMType.valueOf(parts[6]);
+			
+			//in mappers always through distcache, in reducers through distcache/shuffle
+			boolean cacheU = isRed ? Boolean.parseBoolean(parts[7]) : true;
+			boolean cacheV = isRed ? Boolean.parseBoolean(parts[8]) : true;
+			
+			return new QuaternarySPInstruction(new QuaternaryOperator(wtype, uopcode), in1, in2, in3, null, out, cacheU, cacheV, opcode, str);	
+		}
 		else //map/redwsigmoid, map/redwdivmm, map/redwcemm
 		{
 			boolean isRed = opcode.startsWith("red");
@@ -171,7 +198,8 @@ public class QuaternarySPInstruction extends ComputationSPInstruction
 		if(    WeightedSquaredLoss.OPCODE.equalsIgnoreCase(getOpcode())  
 			|| WeightedSigmoid.OPCODE.equalsIgnoreCase(getOpcode())
 			|| WeightedDivMM.OPCODE.equalsIgnoreCase(getOpcode()) 
-			|| WeightedCrossEntropy.OPCODE.equalsIgnoreCase(getOpcode()) ) 
+			|| WeightedCrossEntropy.OPCODE.equalsIgnoreCase(getOpcode())
+			|| WeightedUnaryMM.OPCODE.equalsIgnoreCase(getOpcode())) 
 		{
 			PartitionedBroadcastMatrix bc1 = sec.getBroadcastForVariable( input2.getName() );
 			PartitionedBroadcastMatrix bc2 = sec.getBroadcastForVariable( input3.getName() );
@@ -242,7 +270,7 @@ public class QuaternarySPInstruction extends ComputationSPInstruction
 			DoubleObject ret = new DoubleObject(tmp.getValue(0, 0));
 			sec.setVariable(output.getName(), ret);
 		}
-		else //map/redwsigmoid, map/redwdivmm 
+		else //map/redwsigmoid, map/redwdivmm, map/redwumm 
 		{
 			//aggregation if required (map/redwdivmm)
 			if( qop.wtype3 != null && !qop.wtype3.isBasic() )
@@ -275,7 +303,7 @@ public class QuaternarySPInstruction extends ComputationSPInstruction
 		MatrixCharacteristics mcIn3 = sec.getMatrixCharacteristics(input3.getName());
 		MatrixCharacteristics mcOut = sec.getMatrixCharacteristics(output.getName());
 		
-		if( qop.wtype2 != null ) {
+		if( qop.wtype2 != null || qop.wtype5 != null ) {
 			//output size determined by main input
 			mcOut.set(mcIn1.getRows(), mcIn1.getCols(), mcIn1.getRowsPerBlock(), mcIn1.getColsPerBlock());
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/matrix/data/LibMatrixMult.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/matrix/data/LibMatrixMult.java b/src/main/java/com/ibm/bi/dml/runtime/matrix/data/LibMatrixMult.java
index ebab2c6..6bae2f8 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/matrix/data/LibMatrixMult.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/matrix/data/LibMatrixMult.java
@@ -31,9 +31,11 @@ import com.ibm.bi.dml.lops.WeightedCrossEntropy.WCeMMType;
 import com.ibm.bi.dml.lops.WeightedDivMM.WDivMMType;
 import com.ibm.bi.dml.lops.WeightedSigmoid.WSigmoidType;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
 import com.ibm.bi.dml.runtime.DMLRuntimeException;
 import com.ibm.bi.dml.runtime.DMLUnsupportedOperationException;
 import com.ibm.bi.dml.runtime.functionobjects.SwapIndex;
+import com.ibm.bi.dml.runtime.functionobjects.ValueFunction;
 import com.ibm.bi.dml.runtime.matrix.operators.ReorgOperator;
 import com.ibm.bi.dml.runtime.util.UtilFunctions;
 
@@ -887,6 +889,101 @@ public class LibMatrixMult
 		//System.out.println("MMWCe "+wt.toString()+" k="+k+" ("+mW.isInSparseFormat()+","+mW.getNumRows()+","+mW.getNumColumns()+","+mW.getNonZeros()+")x" +
 		//                 "("+mV.isInSparseFormat()+","+mV.getNumRows()+","+mV.getNumColumns()+","+mV.getNonZeros()+") in "+time.stop());
 	}
+
+	/**
+	 * 
+	 * @param mW
+	 * @param mU
+	 * @param mV
+	 * @param ret
+	 * @param wt
+	 * @throws DMLRuntimeException
+	 */
+	public static void matrixMultWuMM(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn) 
+		throws DMLRuntimeException 
+	{
+		//check for empty result
+		if( mW.isEmptyBlock(false) ) {
+			ret.examSparsity(); //turn empty dense into sparse
+			return; 
+		}
+
+		//Timing time = new Timing(true);
+
+		//pre-processing
+		ret.sparse = mW.sparse;
+		ret.allocateDenseOrSparseBlock();
+		
+		//core weighted square sum mm computation
+		if( !mW.sparse && !mU.sparse && !mV.sparse && !mU.isEmptyBlock() && !mV.isEmptyBlock() )
+			matrixMultWuMMDense(mW, mU, mV, ret, wt, fn, 0, mW.rlen);
+		else if( mW.sparse && !mU.sparse && !mV.sparse && !mU.isEmptyBlock() && !mV.isEmptyBlock())
+			matrixMultWuMMSparseDense(mW, mU, mV, ret, wt, fn, 0, mW.rlen);
+		else
+			matrixMultWuMMGeneric(mW, mU, mV, ret, wt, fn, 0, mW.rlen);
+		
+		//post-processing
+		ret.recomputeNonZeros();
+		ret.examSparsity();
+		
+		//System.out.println("MMWu "+wt.toString()+" ("+mW.isInSparseFormat()+","+mW.getNumRows()+","+mW.getNumColumns()+","+mW.getNonZeros()+")x" +
+		//                 "("+mV.isInSparseFormat()+","+mV.getNumRows()+","+mV.getNumColumns()+","+mV.getNonZeros()+") in "+time.stop());
+	}
+
+	/**
+	 * 
+	 * @param mW
+	 * @param mU
+	 * @param mV
+	 * @param ret
+	 * @param wt
+	 * @param k
+	 * @throws DMLRuntimeException
+	 */
+	public static void matrixMultWuMM(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int k) 
+		throws DMLRuntimeException 
+	{
+		//check for empty result
+		if( mW.isEmptyBlock(false) ) {
+			ret.examSparsity(); //turn empty dense into sparse
+			return; 
+		}
+
+		//check no parallelization benefit (fallback to sequential)
+		if (mW.rlen == 1) {
+			matrixMultWuMM(mW, mU, mV, ret, wt, fn);
+			return;
+		}
+		
+		//Timing time = new Timing(true);
+
+		//pre-processing
+		ret.sparse = mW.sparse;
+		ret.allocateDenseOrSparseBlock();
+		
+		try 
+		{			
+			ExecutorService pool = Executors.newFixedThreadPool(k);
+			ArrayList<MatrixMultWuTask> tasks = new ArrayList<MatrixMultWuTask>();
+			int blklen = (int)(Math.ceil((double)mW.rlen/k));
+			for( int i=0; i<k & i*blklen<mW.rlen; i++ )
+				tasks.add(new MatrixMultWuTask(mW, mU, mV, ret, wt, fn, i*blklen, Math.min((i+1)*blklen, mW.rlen)));
+			pool.invokeAll(tasks);
+			pool.shutdown();
+			ret.nonZeros = 0; //reset after execute
+			for( MatrixMultWuTask task : tasks )
+				ret.nonZeros += task.getPartialNnz();
+		} 
+		catch (InterruptedException e) {
+			throw new DMLRuntimeException(e);
+		}
+
+		//post-processing (nnz maintained in parallel)
+		ret.examSparsity();
+
+		//System.out.println("MMWu "+wt.toString()+" k="+k+" ("+mW.isInSparseFormat()+","+mW.getNumRows()+","+mW.getNumColumns()+","+mW.getNonZeros()+")x" +
+		//                   "("+mV.isInSparseFormat()+","+mV.getNumRows()+","+mV.getNumColumns()+","+mV.getNonZeros()+") in "+time.stop() + ".");
+	}
 	
 	//////////////////////////////////////////
 	// optimized matrix mult implementation //
@@ -2704,7 +2801,149 @@ public class LibMatrixMult
 
 		ret.quickSetValue(0, 0, wceval);
 	}
+
+	/**
+	 * 
+	 * @param mW
+	 * @param mU
+	 * @param mV
+	 * @param ret
+	 * @param wt
+	 * @param rl
+	 * @param ru
+	 * @throws DMLRuntimeException
+	 */
+	private static void matrixMultWuMMDense(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int rl, int ru) 
+		throws DMLRuntimeException 
+	{	
+		double[] w = mW.denseBlock;
+		double[] c = ret.denseBlock;
+		double[] u = mU.denseBlock;
+		double[] v = mV.denseBlock;
+		final int n = mW.clen;
+		final int cd = mU.clen;
+		
+		//note: cannot compute U %*% t(V) in-place of result w/ regular mm because
+		//t(V) comes in transformed form and hence would require additional memory
 	
+		boolean flagmult = (wt==WUMMType.MULT); 
+		
+		//approach: iterate over non-zeros of w, selective mm computation
+		//cache-conscious blocking: due to blocksize constraint (default 1000),
+		//a blocksize of 16 allows to fit blocks of UV into L2 cache (256KB) 
+		
+		final int blocksizeIJ = 16; //u/v block (max at typical L2 size) 
+		
+		//blocked execution
+		for( int bi = rl; bi < ru; bi+=blocksizeIJ )
+			for( int bj = 0, bimin = Math.min(ru, bi+blocksizeIJ); bj < n; bj+=blocksizeIJ ) 
+			{
+				int bjmin = Math.min(n, bj+blocksizeIJ);
+						
+				//core wsigmoid computation
+				for( int i=bi, ix=bi*n, uix=bi*cd; i<bimin; i++, ix+=n, uix+=cd )
+					for( int j=bj, vix=bj*cd; j<bjmin; j++, vix+=cd) {
+						double wij = w[ix+j];
+						if( wij != 0 )
+							c[ix+j] = wumm(wij, u, v, uix, vix, flagmult, fn, cd);
+					}
+			}
+	}
+	
+	/**
+	 * 
+	 * @param mX
+	 * @param mU
+	 * @param mV
+	 * @param mW
+	 * @param ret
+	 * @param wt
+	 * @param rl
+	 * @param ru
+	 * @throws DMLRuntimeException 
+	 */
+	private static void matrixMultWuMMSparseDense(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int rl, int ru) 
+		throws DMLRuntimeException
+	{
+		SparseRow[] w = mW.sparseRows;
+		SparseRow[] c = ret.sparseRows;
+		double[] u = mU.denseBlock;
+		double[] v = mV.denseBlock;
+		final int n = mW.clen; 
+		final int cd = mU.clen;
+		
+		boolean flagmult = (wt==WUMMType.MULT); 
+	
+		//approach: iterate over non-zeros of w, selective mm computation
+		for( int i=rl, uix=rl*cd; i<ru; i++, uix+=cd )
+			if( w[i] != null && !w[i].isEmpty() ) {
+				int wlen = w[i].size();
+				int[] wix = w[i].getIndexContainer();
+				double[] wval = w[i].getValueContainer();
+				c[i] = new SparseRow(wlen, n);
+				
+				for( int k=0; k<wlen; k++ ) {
+					double cval = wumm(wval[k], u, v, uix, wix[k]*cd, flagmult, fn, cd);
+					c[i].append(wix[k], cval);
+				}
+			}
+	}
+
+	/**
+	 * 
+	 * @param mX
+	 * @param mU
+	 * @param mV
+	 * @param mW
+	 * @param ret
+	 * @param wt
+	 * @param rl
+	 * @param ru
+	 * @throws DMLRuntimeException 
+	 */
+	private static void matrixMultWuMMGeneric (MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int rl, int ru) 
+		throws DMLRuntimeException
+	{
+		final int n = mW.clen; 
+		final int cd = mU.clen;
+	
+		boolean flagmult = (wt==WUMMType.MULT); 
+		
+		//approach: iterate over non-zeros of w, selective mm computation
+		if( mW.sparse ) //SPARSE
+		{
+			//w and c always in same representation
+			SparseRow[] w = mW.sparseRows;
+			SparseRow[] c = ret.sparseRows;
+			
+			for( int i=rl; i<ru; i++ )
+				if( w[i] != null && !w[i].isEmpty() ) {
+					int wlen = w[i].size();
+					int[] wix = w[i].getIndexContainer();
+					double[] wval = w[i].getValueContainer();
+					c[i] = new SparseRow(wlen, n);
+					
+					for( int k=0; k<wlen; k++ ) {
+						double cval = wumm(wval[k], mU, mV, i, wix[k], flagmult, fn, cd);
+						c[i].append(wix[k], cval);
+					}
+				}	
+		}
+		else //DENSE
+		{
+			//w and c always in same representation
+			double[] w = mW.denseBlock;
+			double[] c = ret.denseBlock;
+		
+			for( int i=rl, ix=rl*n; i<ru; i++ )
+				for( int j=0; j<n; j++, ix++) {
+					double wij = w[ix];
+					if( wij != 0 ) {
+						c[ix] = wumm(wij, mU, mV, i, j, flagmult, fn, cd);
+					}
+				}
+		}
+	}
 	
 	////////////////////////////////////////////
 	// performance-relevant utility functions //
@@ -3312,6 +3551,58 @@ public class LibMatrixMult
 	
 	/**
 	 * 
+	 * @param wij
+	 * @param u
+	 * @param v
+	 * @param uix
+	 * @param vix
+	 * @param flagmult
+	 * @param fn
+	 * @param len
+	 * @return
+	 * @throws DMLRuntimeException 
+	 */
+	private static double wumm( final double wij, double[] u, double[] v, final int uix, final int vix, final boolean flagmult, ValueFunction fn, final int len ) 
+		throws DMLRuntimeException
+	{
+		//compute dot product over ui vj 
+		double uvij = dotProduct(u, v, uix, vix, len);
+		
+		//compute unary operations
+		double cval = fn.execute(uvij);
+		
+		//compute weighted output
+		return flagmult ? wij * cval : wij / cval;
+	}
+	
+	/**
+	 * 
+	 * @param wij
+	 * @param u
+	 * @param v
+	 * @param uix
+	 * @param vix
+	 * @param flagminus
+	 * @param flaglog
+	 * @param len
+	 * @return
+	 * @throws DMLRuntimeException 
+	 */
+	private static double wumm( final double wij, MatrixBlock u, MatrixBlock v, final int uix, final int vix, final boolean flagmult, ValueFunction fn, final int len ) 
+		throws DMLRuntimeException
+	{
+		//compute dot product over ui vj 
+		double uvij = dotProductGeneric(u, v, uix, vix, len);
+
+		//compute unary operations
+		double cval = fn.execute(uvij);
+		
+		//compute weighted output
+		return flagmult ? wij * cval : wij / cval;
+	}
+	
+	/**
+	 * 
 	 * @param a
 	 * @param b
 	 * @param ai
@@ -3910,4 +4201,54 @@ public class LibMatrixMult
 			return _ret.quickGetValue(0, 0);
 		}
 	}
+	
+	/**
+	 * 
+	 */
+	private static class MatrixMultWuTask implements Callable<Object> 
+	{
+		private MatrixBlock _mW = null;
+		private MatrixBlock _mU = null;
+		private MatrixBlock _mV = null;
+		private MatrixBlock _ret = null;
+		private WUMMType _wt = null;
+		private ValueFunction _fn = null;
+		private int _rl = -1;
+		private int _ru = -1;
+		private long _nnz = -1;
+		
+		protected MatrixMultWuTask(MatrixBlock mW, MatrixBlock mU, MatrixBlock mV, MatrixBlock ret, WUMMType wt, ValueFunction fn, int rl, int ru) 
+			throws DMLRuntimeException
+		{
+			_mW = mW;
+			_mU = mU;
+			_mV = mV;
+			_ret = ret;
+			_wt = wt;
+			_fn = fn;
+			_rl = rl;
+			_ru = ru;
+		}
+		
+		@Override
+		public Object call() throws DMLRuntimeException
+		{
+			//core weighted square sum mm computation
+			if( !_mW.sparse && !_mU.sparse && !_mV.sparse && !_mU.isEmptyBlock() && !_mV.isEmptyBlock() )
+				matrixMultWuMMDense(_mW, _mU, _mV, _ret, _wt, _fn, _rl, _ru);
+			else if( _mW.sparse && !_mU.sparse && !_mV.sparse && !_mU.isEmptyBlock() && !_mV.isEmptyBlock())
+				matrixMultWuMMSparseDense(_mW, _mU, _mV, _ret, _wt, _fn, _rl, _ru);
+			else
+				matrixMultWuMMGeneric(_mW, _mU, _mV, _ret, _wt, _fn, _rl, _ru);
+			
+			//maintain block nnz (upper bounds inclusive)
+			_nnz = _ret.recomputeNonZeros(_rl, _ru-1, 0, _ret.getNumColumns()-1);
+			
+			return null;
+		}
+		
+		public long getPartialNnz() {
+			return _nnz;
+		}
+	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/matrix/data/MatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/matrix/data/MatrixBlock.java b/src/main/java/com/ibm/bi/dml/runtime/matrix/data/MatrixBlock.java
index 38ab456..18fce81 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/matrix/data/MatrixBlock.java
@@ -5914,7 +5914,7 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 		//prepare intermediates and output
 		if( qop.wtype1 != null || qop.wtype4 != null )
 			R.reset(1, 1, false);
-		else if( qop.wtype2 != null )
+		else if( qop.wtype2 != null || qop.wtype5 != null )
 			R.reset(rlen, clen, sparse);
 		else if( qop.wtype3 != null ) {
 			MatrixCharacteristics mc = qop.wtype3.computeOutputCharacteristics(X.rlen, X.clen, U.clen);
@@ -5948,6 +5948,12 @@ public class MatrixBlock extends MatrixValue implements Externalizable
 			else
 				LibMatrixMult.matrixMultWCeMM(X, U, V, R, qop.wtype4);	
 		}
+		else if( qop.wtype5 != null ){ //wumm
+			if( k > 1 )
+				LibMatrixMult.matrixMultWuMM(X, U, V, R, qop.wtype5, qop.fn, k);
+			else
+				LibMatrixMult.matrixMultWuMM(X, U, V, R, qop.wtype5, qop.fn);	
+		}
 		
 		return R;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/d70c4524/src/main/java/com/ibm/bi/dml/runtime/matrix/operators/QuaternaryOperator.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/matrix/operators/QuaternaryOperator.java b/src/main/java/com/ibm/bi/dml/runtime/matrix/operators/QuaternaryOperator.java
index ada6f7d..c0b1412 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/matrix/operators/QuaternaryOperator.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/matrix/operators/QuaternaryOperator.java
@@ -22,8 +22,11 @@ import com.ibm.bi.dml.lops.WeightedDivMM.WDivMMType;
 import com.ibm.bi.dml.lops.WeightedSigmoid.WSigmoidType;
 import com.ibm.bi.dml.lops.WeightedSquaredLoss.WeightsType;
 import com.ibm.bi.dml.lops.WeightedCrossEntropy.WCeMMType;
+import com.ibm.bi.dml.lops.WeightedUnaryMM.WUMMType;
 import com.ibm.bi.dml.runtime.functionobjects.Builtin;
-import com.ibm.bi.dml.runtime.functionobjects.FunctionObject;
+import com.ibm.bi.dml.runtime.functionobjects.Multiply2;
+import com.ibm.bi.dml.runtime.functionobjects.Power2;
+import com.ibm.bi.dml.runtime.functionobjects.ValueFunction;
 
 public class QuaternaryOperator extends Operator 
 {
@@ -34,8 +37,9 @@ public class QuaternaryOperator extends Operator
 	public WSigmoidType wtype2 = null;
 	public WDivMMType wtype3 = null;
 	public WCeMMType wtype4 = null;
+	public WUMMType wtype5 = null;
 	
-	public FunctionObject fn;
+	public ValueFunction fn;
 	
 	/**
 	 * wsloss
@@ -73,4 +77,21 @@ public class QuaternaryOperator extends Operator
 	public QuaternaryOperator( WCeMMType wt ) {
 		wtype4 = wt;
 	}
+	
+	/**
+	 * wumm
+	 * 
+	 * @param wt
+	 * @param op
+	 */
+	public QuaternaryOperator( WUMMType wt, String op ) {
+		wtype5 = wt;
+		
+		if( op.equals("^2") )
+			fn = Power2.getPower2FnObject();
+		else if( op.equals("*2") )
+			fn = Multiply2.getMultiply2FnObject();
+		else
+			fn = Builtin.getBuiltinFnObject(op);
+	}
 }


[4/8] incubator-systemml git commit: New cp unary/binary guarded dense/sparse output conversion, for kmeans

Posted by mb...@apache.org.
New cp unary/binary guarded dense/sparse output conversion, for kmeans

By default we estimate the output sparsity on each operation, allocate
the right output, execute the operation, and only check for explicit
dense/sparse conversion if the optimizer reflects this in the memory
estimates used for operator selection. This leads to issues with certain
binary/unary operations, resulting in wrong output format (e.g.,
mistakenly dense) and thus affects performance of subsequent operations.
For example, P=ppred(D, minD, "<="); is estimated as dense although it
should be sparse for sufficiently many clusters. This change adds a
guarded sparse/dense format check (incl conversion if necessary) to all
binary and unary cp instructions. The additional memory requirements of
a potential format change are guarded by the released memory of all
inputs (if sufficiently large).

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

Branch: refs/heads/master
Commit: cc4aae71f082e727f29f8753588f8b77757ae854
Parents: 3adca72
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sat Nov 21 00:21:23 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:37:17 2015 -0800

----------------------------------------------------------------------
 .../cp/ComputationCPInstruction.java            | 27 ++++++++++++++++++++
 .../cp/MatrixBuiltinCPInstruction.java          | 19 +++++++++-----
 .../cp/MatrixMatrixArithmeticCPInstruction.java | 13 +++++++---
 .../cp/MatrixMatrixBuiltinCPInstruction.java    | 22 +++++++++-------
 .../cp/MatrixMatrixRelationalCPInstruction.java | 19 +++++++++-----
 .../cp/MatrixScalarBuiltinCPInstruction.java    | 14 +++++++---
 .../cp/ScalarMatrixArithmeticCPInstruction.java | 14 +++++++---
 .../cp/ScalarMatrixRelationalCPInstruction.java | 14 +++++++---
 8 files changed, 104 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ComputationCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ComputationCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ComputationCPInstruction.java
index 0f6b225..ddd49ac 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ComputationCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ComputationCPInstruction.java
@@ -17,6 +17,8 @@
 
 package com.ibm.bi.dml.runtime.instructions.cp;
 
+import com.ibm.bi.dml.hops.OptimizerUtils;
+import com.ibm.bi.dml.runtime.matrix.data.MatrixBlock;
 import com.ibm.bi.dml.runtime.matrix.operators.Operator;
 
 public abstract class ComputationCPInstruction extends CPInstruction 
@@ -45,4 +47,29 @@ public abstract class ComputationCPInstruction extends CPInstruction
 	public String getOutputVariableName() {
 		return output.getName();
 	}
+	
+	/**
+	 * 
+	 * @param in1
+	 * @param out
+	 * @return
+	 */
+	protected boolean checkGuardedRepresentationChange( MatrixBlock in1, MatrixBlock out ) {
+		return checkGuardedRepresentationChange(in1, null, out);
+	}
+	
+	/**
+	 * 
+	 * @param in1
+	 * @param in2
+	 * @param out
+	 * @return
+	 */
+	protected boolean checkGuardedRepresentationChange( MatrixBlock in1, MatrixBlock in2, MatrixBlock out )
+	{
+		double memDense = OptimizerUtils.estimateSize(out.getNumRows(), out.getNumColumns());
+		double memIn1 = (in1 != null) ? in1.getSizeInMemory() : 0;
+		double memIn2 = (in2 != null) ? in2.getSizeInMemory() : 0;
+		return ( memDense < memIn1 + memIn2 );	
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
index 2b95254..face2db 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixBuiltinCPInstruction.java
@@ -39,19 +39,24 @@ public class MatrixBuiltinCPInstruction extends BuiltinUnaryCPInstruction
 	{	
 		UnaryOperator u_op = (UnaryOperator) _optr;
 		String output_name = output.getName();
-		MatrixBlock resultBlock = null;
 		
 		String opcode = getOpcode();
 		if(LibCommonsMath.isSupportedUnaryOperation(opcode)) {
-			resultBlock = LibCommonsMath.unaryOperations((MatrixObject)ec.getVariable(input1.getName()),getOpcode());
-			ec.setMatrixOutput(output_name, resultBlock);
+			MatrixBlock retBlock = LibCommonsMath.unaryOperations((MatrixObject)ec.getVariable(input1.getName()),getOpcode());
+			ec.setMatrixOutput(output_name, retBlock);
 		}
 		else {
-			MatrixBlock matBlock = ec.getMatrixInput(input1.getName());
-			resultBlock = (MatrixBlock) (matBlock.unaryOperations(u_op, new MatrixBlock()));
-			
-			ec.setMatrixOutput(output_name, resultBlock);
+			MatrixBlock inBlock = ec.getMatrixInput(input1.getName());
+			MatrixBlock retBlock = (MatrixBlock) (inBlock.unaryOperations(u_op, new MatrixBlock()));
+		
 			ec.releaseMatrixInput(input1.getName());
+			
+			// Ensure right dense/sparse output representation (guarded by released input memory)
+			if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
+	 			retBlock.examSparsity();
+	 		}
+			
+			ec.setMatrixOutput(output_name, retBlock);
 		}		
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
index ba9c845..e262dff 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixArithmeticCPInstruction.java
@@ -42,18 +42,23 @@ public class MatrixMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		throws DMLRuntimeException, DMLUnsupportedOperationException
 	{
 		// Read input matrices
-        MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
+        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
 		
 		// Perform computation using input matrices, and produce the result matrix
 		BinaryOperator bop = (BinaryOperator) _optr;
-		MatrixBlock soresBlock = (MatrixBlock) (matBlock1.binaryOperations (bop, matBlock2, new MatrixBlock()));
+		MatrixBlock retBlock = (MatrixBlock) (inBlock1.binaryOperations (bop, inBlock2, new MatrixBlock()));
 		
 		// Release the memory occupied by input matrices
 		ec.releaseMatrixInput(input1.getName());
 		ec.releaseMatrixInput(input2.getName());
 		
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
+			retBlock.examSparsity();
+		}
+		
 		// Attach result matrix with MatrixObject associated with output_name
-		ec.setMatrixOutput(output.getName(), soresBlock);
+		ec.setMatrixOutput(output.getName(), retBlock);
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
index 374867d..2e452f5 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixBuiltinCPInstruction.java
@@ -41,8 +41,8 @@ public class MatrixMatrixBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 	
 	@Override
 	public void processInstruction(ExecutionContext ec) 
-		throws DMLRuntimeException, DMLUnsupportedOperationException{
-
+		throws DMLRuntimeException, DMLUnsupportedOperationException
+	{
 		String opcode = getOpcode();
         
         if ( LibCommonsMath.isSupportedMatrixMatrixOperation(opcode) ) {
@@ -55,15 +55,19 @@ public class MatrixMatrixBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		String output_name = output.getName();
 		BinaryOperator bop = (BinaryOperator) _optr;
 		
-        MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
-		
-        MatrixBlock resultBlock = (MatrixBlock) matBlock1.binaryOperations(bop, matBlock2, new MatrixBlock());
-		
-		ec.setMatrixOutput(output_name, resultBlock);
+		MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
+		MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
 		
+		MatrixBlock retBlock = (MatrixBlock) inBlock1.binaryOperations(bop, inBlock2, new MatrixBlock());
+	
 		ec.releaseMatrixInput(input1.getName());
 		ec.releaseMatrixInput(input2.getName());
+		
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
+ 			retBlock.examSparsity();
+ 		}
+        
+		ec.setMatrixOutput(output_name, retBlock);
 	}
-	
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
index 1e4efaa..f19437f 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixMatrixRelationalCPInstruction.java
@@ -39,17 +39,24 @@ public class MatrixMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 
 	@Override
 	public void processInstruction(ExecutionContext ec) 
-		throws DMLRuntimeException, DMLUnsupportedOperationException{
-        MatrixBlock matBlock1 = ec.getMatrixInput(input1.getName());
-        MatrixBlock matBlock2 = ec.getMatrixInput(input2.getName());
+		throws DMLRuntimeException, DMLUnsupportedOperationException
+	{
+        MatrixBlock inBlock1 = ec.getMatrixInput(input1.getName());
+        MatrixBlock inBlock2 = ec.getMatrixInput(input2.getName());
 		
 		String output_name = output.getName();
 		BinaryOperator bop = (BinaryOperator) _optr;
 		
-		MatrixBlock resultBlock = (MatrixBlock) matBlock1.binaryOperations(bop, matBlock2, new MatrixBlock());
-		
-		ec.setMatrixOutput(output_name, resultBlock);
+		MatrixBlock retBlock = (MatrixBlock) inBlock1.binaryOperations(bop, inBlock2, new MatrixBlock());
+
 		ec.releaseMatrixInput(input1.getName());
 		ec.releaseMatrixInput(input2.getName());
+		
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock1, inBlock2, retBlock) ) {
+ 			retBlock.examSparsity();
+ 		}
+		
+		ec.setMatrixOutput(output_name, retBlock);
 	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
index 02831e9..648df9c 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/MatrixScalarBuiltinCPInstruction.java
@@ -45,15 +45,21 @@ public class MatrixScalarBuiltinCPInstruction extends BuiltinBinaryCPInstruction
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock matBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator)	_optr;
 		sc_op.setConstant(constant.getDoubleValue());
 		
-		MatrixBlock resultBlock = (MatrixBlock) matBlock.scalarOperations(sc_op, new MatrixBlock());
-		
+		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
+
 		ec.releaseMatrixInput(mat.getName());
-		ec.setMatrixOutput(output.getName(), resultBlock);
+		
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
+ 			retBlock.examSparsity();
+ 		}
+		
+		ec.setMatrixOutput(output.getName(), retBlock);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
index f0ef4fb..fbe5bcf 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixArithmeticCPInstruction.java
@@ -25,7 +25,7 @@ import com.ibm.bi.dml.runtime.matrix.data.MatrixBlock;
 import com.ibm.bi.dml.runtime.matrix.operators.Operator;
 import com.ibm.bi.dml.runtime.matrix.operators.ScalarOperator;
 
-
+// TODO rename to MatrixScalar...
 public class ScalarMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstruction
 {
 	
@@ -45,15 +45,21 @@ public class ScalarMatrixArithmeticCPInstruction extends ArithmeticBinaryCPInstr
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock matBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 
 		ScalarOperator sc_op = (ScalarOperator) _optr;
 		sc_op.setConstant(constant.getDoubleValue());
 		
-		MatrixBlock resultBlock = (MatrixBlock) matBlock.scalarOperations(sc_op, new MatrixBlock());
+		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		
 		ec.releaseMatrixInput(mat.getName());
-		ec.setMatrixOutput(output.getName(), resultBlock);
+		
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
+ 			retBlock.examSparsity();
+ 		}
+		
+		ec.setMatrixOutput(output.getName(), retBlock);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/cc4aae71/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
index e6cc354..2f19f10 100644
--- a/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
+++ b/src/main/java/com/ibm/bi/dml/runtime/instructions/cp/ScalarMatrixRelationalCPInstruction.java
@@ -25,7 +25,7 @@ import com.ibm.bi.dml.runtime.matrix.data.MatrixBlock;
 import com.ibm.bi.dml.runtime.matrix.operators.Operator;
 import com.ibm.bi.dml.runtime.matrix.operators.ScalarOperator;
 
-
+//TODO rename to MatrixScalar...
 public class ScalarMatrixRelationalCPInstruction extends RelationalBinaryCPInstruction
 {
 	
@@ -45,15 +45,21 @@ public class ScalarMatrixRelationalCPInstruction extends RelationalBinaryCPInstr
 		CPOperand mat = ( input1.getDataType() == DataType.MATRIX ) ? input1 : input2;
 		CPOperand scalar = ( input1.getDataType() == DataType.MATRIX ) ? input2 : input1;
 		
-		MatrixBlock matBlock = ec.getMatrixInput(mat.getName());
+		MatrixBlock inBlock = ec.getMatrixInput(mat.getName());
 		ScalarObject constant = (ScalarObject) ec.getScalarInput(scalar.getName(), scalar.getValueType(), scalar.isLiteral());
 		
 		ScalarOperator sc_op = (ScalarOperator) _optr;
 		sc_op.setConstant(constant.getDoubleValue());
 		
-		MatrixBlock resultBlock = (MatrixBlock) matBlock.scalarOperations(sc_op, new MatrixBlock());
+		MatrixBlock retBlock = (MatrixBlock) inBlock.scalarOperations(sc_op, new MatrixBlock());
 		
 		ec.releaseMatrixInput(mat.getName());
-		ec.setMatrixOutput(output.getName(), resultBlock);
+
+		// Ensure right dense/sparse output representation (guarded by released input memory)
+		if( checkGuardedRepresentationChange(inBlock, retBlock) ) {
+ 			retBlock.examSparsity();
+ 		}
+		
+		ec.setMatrixOutput(output.getName(), retBlock);
 	}
 }


[3/8] incubator-systemml git commit: Improved right indexing hop size inference (block indexing), for kmeans

Posted by mb...@apache.org.
Improved right indexing hop size inference (block indexing), for kmeans

The parfor optimizer injects repartition operations if there are zipmm
or cpmm (unknown) matrix mult instructions in the body in order to avoid
shuffle per iteration. For kmeans, we encountered false positives, where
we injected repartition operations but executed mapmm in the body, which
rendered the repartition overhead unnecessary. This change improves the
size inference logic of right indexing hops such that, for kmeans, all
sizes of matrix multiplications can be inferred and thus avoids
unnecessary repartitioning.

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

Branch: refs/heads/master
Commit: 3adca729d5039b646c8820cc77b23ccd4c467fa3
Parents: abcebc6
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Fri Nov 20 20:56:29 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:37:12 2015 -0800

----------------------------------------------------------------------
 src/main/java/com/ibm/bi/dml/hops/IndexingOp.java | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/3adca729/src/main/java/com/ibm/bi/dml/hops/IndexingOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/IndexingOp.java b/src/main/java/com/ibm/bi/dml/hops/IndexingOp.java
index a46781a..b351d61 100644
--- a/src/main/java/com/ibm/bi/dml/hops/IndexingOp.java
+++ b/src/main/java/com/ibm/bi/dml/hops/IndexingOp.java
@@ -33,7 +33,6 @@ import com.ibm.bi.dml.runtime.matrix.MatrixCharacteristics;
 //for now only works for range based indexing op
 public class IndexingOp extends Hop 
 {
-	
 	public static String OPSTRING = "rix"; //"Indexing";
 	
 	private boolean _rowLowerEqualsUpper = false;
@@ -417,6 +416,10 @@ public class IndexingOp extends Hop
 			setDim1( HopRewriteUtils.getIntValueSafe((LiteralOp)input3)
 					-HopRewriteUtils.getIntValueSafe((LiteralOp)input2)+1 );
 		}
+		else if( isBlockIndexingExpression(input2, input3) ) {
+			setDim1(getBlockIndexingExpressionSize(input2, input3));
+		}
+		
 		if( _colLowerEqualsUpper ) //COLS
 			setDim2(1);
 		else if( allCols ) 
@@ -424,7 +427,10 @@ public class IndexingOp extends Hop
 		else if( constColRange ){
 			setDim2( HopRewriteUtils.getIntValueSafe((LiteralOp)input5)
 					-HopRewriteUtils.getIntValueSafe((LiteralOp)input4)+1 );
-		} 
+		}
+		else if( isBlockIndexingExpression(input4, input5) ) {
+			setDim2(getBlockIndexingExpressionSize(input4, input5));
+		}
 	}
 	
 	@Override


[8/8] incubator-systemml git commit: Merged simplification rewrites (quaternary operators)

Posted by mb...@apache.org.
Merged simplification rewrites (quaternary operators)

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

Branch: refs/heads/master
Commit: a2f78e74e51047144268205b0cbbe8f40cddf275
Parents: d70c452
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sun Nov 22 19:50:30 2015 -0800
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sun Nov 22 19:50:30 2015 -0800

----------------------------------------------------------------------
 .../RewriteAlgebraicSimplificationDynamic.java  |   6 +-
 .../RewriteAlgebraicSimplificationStatic.java   | 605 -------------------
 2 files changed, 3 insertions(+), 608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/a2f78e74/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 1a0710f..5b1111a 100644
--- a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -1391,7 +1391,7 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0), bop.getInput().get(1)) //prevent mv
 				&& ((BinaryOp)bop.getInput().get(1)).getOp()==OpOp2.POW 
 				&& bop.getInput().get(1).getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1).getInput().get(1))==2)
+				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1).getInput().get(1))==2)
 			{
 				Hop W = bop.getInput().get(0);
 				Hop tmp = bop.getInput().get(1).getInput().get(0); //(X - U %*% t(V))
@@ -1447,7 +1447,7 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 			//alternative pattern: sum ((W * (U %*% t(V)) - X) ^ 2)
 			if( !appliedPattern
 				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1))==2
+				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1))==2
 				&& bop.getInput().get(0) instanceof BinaryOp	
 				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
 				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS
@@ -1502,7 +1502,7 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 			//alternative pattern: sum (((U %*% t(V)) - X) ^ 2)
 			if( !appliedPattern
 				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getIntValue((LiteralOp)bop.getInput().get(1))==2
+				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1))==2
 				&& bop.getInput().get(0) instanceof BinaryOp	
 				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
 				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/a2f78e74/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
----------------------------------------------------------------------
diff --git a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
index d2887a2..4a7b2d4 100644
--- a/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
+++ b/src/main/java/com/ibm/bi/dml/hops/rewrite/RewriteAlgebraicSimplificationStatic.java
@@ -1290,616 +1290,11 @@ public class RewriteAlgebraicSimplificationStatic extends HopRewriteRule
 					LOG.debug("Applied simplifyGroupedAggregateCount");	
 				}
 			}
-<<<<<<< Upstream, based on branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-systemml.git
 		}
 		
 		return hi;
 	}
-	
-	/**
-	 * Searches for weighted squared loss expressions and replaces them with a quaternary operator. 
-	 * Currently, this search includes the following three patterns:
-	 * 1) sum (W * (X - U %*% t(V)) ^ 2) (post weighting)
-	 * 2) sum ((X - W * (U %*% t(V))) ^ 2) (pre weighting)
-	 * 3) sum ((X - (U %*% t(V))) ^ 2) (no weighting)
-	 * 
-	 * NOTE: We include transpose into the pattern because during runtime we need to compute
-	 * U%*% t(V) pointwise; having V and not t(V) at hand allows for a cache-friendly implementation
-	 * without additional memory requirements for internal transpose.
-	 * 
-	 * This rewrite is conceptually a static rewrite; however, the current MR runtime only supports
-	 * U/V factors of rank up to the blocksize (1000). We enforce this contraint here during the general
-	 * rewrite because this is an uncommon case. Also, the intention is to remove this constaint as soon
-	 * as we generalized the runtime or hop/lop compilation. 
-	 * 
-	 * @param parent
-	 * @param hi
-	 * @param pos
-	 * @return
-	 * @throws HopsException 
-	 */
-	private Hop simplifyWeightedSquaredLoss(Hop parent, Hop hi, int pos) 
-		throws HopsException
-	{
-		//NOTE: there might be also a general simplification without custom operator
-		//via (X-UVt)^2 -> X^2 - 2X*UVt + UVt^2
-		Hop hnew = null;
-		
-		if( hi instanceof AggUnaryOp && ((AggUnaryOp)hi).getDirection()==Direction.RowCol
-			&& ((AggUnaryOp)hi).getOp() == AggOp.SUM     //all patterns rooted by sum()
-			&& hi.getInput().get(0) instanceof BinaryOp  //all patterns subrooted by binary op
-			&& hi.getInput().get(0).getDim2() > 1  )     //not applied for vector-vector mult
-		{
-			BinaryOp bop = (BinaryOp) hi.getInput().get(0);
-			boolean appliedPattern = false;
-			
-			//Pattern 1) sum (W * (X - U %*% t(V)) ^ 2) (post weighting)
-			//alternative pattern: sum (W * (U %*% t(V) - X) ^ 2)
-			if( bop.getOp()==OpOp2.MULT && bop.getInput().get(1) instanceof BinaryOp	
-				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
-				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0), bop.getInput().get(1)) //prevent mv
-				&& ((BinaryOp)bop.getInput().get(1)).getOp()==OpOp2.POW 
-				&& bop.getInput().get(1).getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1).getInput().get(1))==2)
-			{
-				Hop W = bop.getInput().get(0);
-				Hop tmp = bop.getInput().get(1).getInput().get(0); //(X - U %*% t(V))
-				
-				if( tmp instanceof BinaryOp && ((BinaryOp)tmp).getOp()==OpOp2.MINUS
-					&& HopRewriteUtils.isEqualSize(tmp.getInput().get(0), tmp.getInput().get(1)) //prevent mv	
-					&& tmp.getInput().get(0).getDataType() == DataType.MATRIX )
-				{
-					//a) sum (W * (X - U %*% t(V)) ^ 2)
-					int uvIndex = -1;
-					if( tmp.getInput().get(1) instanceof AggBinaryOp  //ba gurantees matrices
-							&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(1).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
-					{
-						uvIndex = 1;   
-					}
-					//b) sum (W * (U %*% t(V) - X) ^ 2)
-					else if(tmp.getInput().get(0) instanceof AggBinaryOp  //ba gurantees matrices
-						&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(0).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
-					{
-						uvIndex = 0;
-					}   
-				 
-					if( uvIndex >= 0 ) //rewrite match
-					{
-						Hop X = tmp.getInput().get((uvIndex==0)?1:0); 
-						Hop U = tmp.getInput().get(uvIndex).getInput().get(0);
-						Hop V = tmp.getInput().get(uvIndex).getInput().get(1);
-	                    
-						if( !HopRewriteUtils.isTransposeOperation(V) ) {
-							V = HopRewriteUtils.createTranspose(V);
-						}
-						else{
-							V = V.getInput().get(0);
-						}
-	                    
-						//handle special case of post_nz
-						if( HopRewriteUtils.isNonZeroIndicator(W, X) ){
-							W = new LiteralOp(1);
-						}
-						
-						//construct quaternary hop
-						hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
-								OpOp4.WSLOSS, X, U, V, W, true);
-						HopRewriteUtils.setOutputParametersForScalar(hnew);
-	
-						appliedPattern = true;
-						LOG.debug("Applied simplifyWeightedSquaredLoss1"+uvIndex+" (line "+hi.getBeginLine()+")");  
-					}
-				}
-			}
-			
-			//Pattern 2) sum ((X - W * (U %*% t(V))) ^ 2) (pre weighting)
-			//alternative pattern: sum ((W * (U %*% t(V)) - X) ^ 2)
-			if( !appliedPattern
-				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1))==2
-				&& bop.getInput().get(0) instanceof BinaryOp	
-				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
-				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS
-				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0).getInput().get(0), bop.getInput().get(0).getInput().get(1)) //prevent mv
-				&& bop.getInput().get(0).getInput().get(0).getDataType()==DataType.MATRIX)
-			{
-			    Hop lleft = bop.getInput().get(0).getInput().get(0); 
-			    Hop lright = bop.getInput().get(0).getInput().get(1); 
-                
-			    //a) sum ((X - W * (U %*% t(V))) ^ 2)
-			    int wuvIndex = -1;
-			    if( lright instanceof BinaryOp && lright.getInput().get(1) instanceof AggBinaryOp ){
-			    	wuvIndex = 1;
-			    }
-			    //b) sum ((W * (U %*% t(V)) - X) ^ 2)
-			    else if( lleft instanceof BinaryOp && lleft.getInput().get(1) instanceof AggBinaryOp ){
-			    	wuvIndex = 0;
-			    }
-			    
-			    if( wuvIndex >= 0 ) //rewrite match
-			    {
-			    	Hop X = bop.getInput().get(0).getInput().get((wuvIndex==0)?1:0);
-			    	Hop tmp = bop.getInput().get(0).getInput().get(wuvIndex); //(W * (U %*% t(V)))
-    				
-    				if( ((BinaryOp)tmp).getOp()==OpOp2.MULT
-    					&& tmp.getInput().get(0).getDataType() == DataType.MATRIX	
-    					&& HopRewriteUtils.isEqualSize(tmp.getInput().get(0), tmp.getInput().get(1)) //prevent mv
-    					&& HopRewriteUtils.isSingleBlock(tmp.getInput().get(1).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
-    				{
-    					Hop W = tmp.getInput().get(0); 
-    					Hop U = tmp.getInput().get(1).getInput().get(0);
-    					Hop V = tmp.getInput().get(1).getInput().get(1);
-    					
-    					if( !HopRewriteUtils.isTransposeOperation(V) ) { 
-    						V = HopRewriteUtils.createTranspose(V);
-    					}
-    					else {
-    						V = V.getInput().get(0);
-    					}
-    					
-    					hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
-    							  OpOp4.WSLOSS, X, U, V, W, false);
-    					HopRewriteUtils.setOutputParametersForScalar(hnew);
-    
-    					appliedPattern = true;
-    					LOG.debug("Applied simplifyWeightedSquaredLoss2"+wuvIndex+" (line "+hi.getBeginLine()+")");	
-    				}
-			    }
-			}
-			
-			//Pattern 3) sum ((X - (U %*% t(V))) ^ 2) (no weighting)
-			//alternative pattern: sum (((U %*% t(V)) - X) ^ 2)
-			if( !appliedPattern
-				&& bop.getOp()==OpOp2.POW && bop.getInput().get(1) instanceof LiteralOp
-				&& HopRewriteUtils.getDoubleValue((LiteralOp)bop.getInput().get(1))==2
-				&& bop.getInput().get(0) instanceof BinaryOp	
-				&& bop.getInput().get(0).getDataType()==DataType.MATRIX	
-				&& ((BinaryOp)bop.getInput().get(0)).getOp()==OpOp2.MINUS
-				&& HopRewriteUtils.isEqualSize(bop.getInput().get(0).getInput().get(0), bop.getInput().get(0).getInput().get(1)) //prevent mv
-				&& bop.getInput().get(0).getInput().get(0).getDataType()==DataType.MATRIX)
-			{
-				Hop lleft = bop.getInput().get(0).getInput().get(0);
-				Hop lright = bop.getInput().get(0).getInput().get(1);
-                
-				//a) sum ((X - (U %*% t(V))) ^ 2)
-				int uvIndex = -1;
-				if( lright instanceof AggBinaryOp //ba gurantees matrices
-					&& HopRewriteUtils.isSingleBlock(lright.getInput().get(0),true) )  //BLOCKSIZE CONSTRAINT
-				{
-					uvIndex = 1;
-				}
-				//b) sum (((U %*% t(V)) - X) ^ 2)
-				else if( lleft instanceof AggBinaryOp //ba gurantees matrices
-						&& HopRewriteUtils.isSingleBlock(lleft.getInput().get(0),true) )  //BLOCKSIZE CONSTRAINT
-				{
-					uvIndex = 0;
-				}
-			    
-				if( uvIndex >= 0 ) //rewrite match
-				{
-					Hop X = bop.getInput().get(0).getInput().get((uvIndex==0)?1:0);
-					Hop tmp = bop.getInput().get(0).getInput().get(uvIndex); //(U %*% t(V))
-					Hop W = new LiteralOp(1); //no weighting 
-					Hop U = tmp.getInput().get(0);
-					Hop V = tmp.getInput().get(1);
-	
-					if( !HopRewriteUtils.isTransposeOperation(V) ) { 
-						V = HopRewriteUtils.createTranspose(V);
-					}
-					else {
-						V = V.getInput().get(0);
-					}
-					
-					hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, 
-							  OpOp4.WSLOSS, X, U, V, W, false);
-					HopRewriteUtils.setOutputParametersForScalar(hnew);
-
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedSquaredLoss3"+uvIndex+" (line "+hi.getBeginLine()+")");	
-				}
-			}			
-		}
-		
-		//relink new hop into original position
-		if( hnew != null ) {
-			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
-			HopRewriteUtils.addChildReference(parent, hnew, pos);
-			hi = hnew;
-		}
-		
-		return hi;
-	}
-	
-	/**
-	 * 
-	 * @param parent
-	 * @param hi
-	 * @param pos
-	 * @return
-	 * @throws HopsException
-	 */
-	private Hop simplifyWeightedSigmoidMMChains(Hop parent, Hop hi, int pos) 
-		throws HopsException
-	{
-		Hop hnew = null;
 		
-		if(    hi instanceof BinaryOp //all patterns subrooted by W *
-			&& ((BinaryOp) hi).getOp()==OpOp2.MULT
-			&& hi.getDim2() > 1       //not applied for vector-vector mult
-			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
-			&& hi.getInput().get(0).getDataType()==DataType.MATRIX 
-			&& hi.getInput().get(1) instanceof UnaryOp ) //sigmoid/log
-		{
-			UnaryOp uop = (UnaryOp) hi.getInput().get(1);
-			boolean appliedPattern = false;
-			
-			//Pattern 1) W * sigmoid(Y%*%t(X)) (basic)
-			if(    uop.getOp() == OpOp1.SIGMOID 
-				&& uop.getInput().get(0) instanceof AggBinaryOp
-				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(0),true) )
-			{
-				Hop W = hi.getInput().get(0); 
-				Hop Y = uop.getInput().get(0).getInput().get(0);
-				Hop tX = uop.getInput().get(0).getInput().get(1);
-				
-				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
-					tX = HopRewriteUtils.createTranspose(tX);
-				}
-				else 
-					tX = tX.getInput().get(0);
-				
-				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-						  OpOp4.WSIGMOID, W, Y, tX, false, false);
-				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-				appliedPattern = true;
-				LOG.debug("Applied simplifyWeightedSigmoid1 (line "+hi.getBeginLine()+")");	
-			}
-			
-			//Pattern 2) W * sigmoid(-(Y%*%t(X))) (minus)
-			if(    !appliedPattern 
-				&& uop.getOp() == OpOp1.SIGMOID 
-				&& uop.getInput().get(0) instanceof BinaryOp
-				&& ((BinaryOp)uop.getInput().get(0)).getOp()==OpOp2.MINUS
-				&& uop.getInput().get(0).getInput().get(0) instanceof LiteralOp
-				&& HopRewriteUtils.getDoubleValueSafe(
-				   (LiteralOp)uop.getInput().get(0).getInput().get(0))==0
-				&& uop.getInput().get(0).getInput().get(1) instanceof AggBinaryOp
-				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(1).getInput().get(0),true))
-			{
-				Hop W = hi.getInput().get(0); 
-				Hop Y = uop.getInput().get(0).getInput().get(1).getInput().get(0);
-				Hop tX = uop.getInput().get(0).getInput().get(1).getInput().get(1);
-				
-				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
-					tX = HopRewriteUtils.createTranspose(tX);
-				}
-				else 
-					tX = tX.getInput().get(0);
-				
-				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-						  OpOp4.WSIGMOID, W, Y, tX, false, true);
-				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-				appliedPattern = true;
-				LOG.debug("Applied simplifyWeightedSigmoid2 (line "+hi.getBeginLine()+")");	
-			}
-			
-			//Pattern 3) W * log(sigmoid(Y%*%t(X))) (log)			
-			if(    !appliedPattern 
-				&& uop.getOp() == OpOp1.LOG
-				&& uop.getInput().get(0) instanceof UnaryOp
-				&& ((UnaryOp)uop.getInput().get(0)).getOp() == OpOp1.SIGMOID 
-				&& uop.getInput().get(0).getInput().get(0) instanceof AggBinaryOp
-				&& HopRewriteUtils.isSingleBlock(uop.getInput().get(0).getInput().get(0).getInput().get(0),true) )
-			{
-				Hop W = hi.getInput().get(0); 
-				Hop Y = uop.getInput().get(0).getInput().get(0).getInput().get(0);
-				Hop tX = uop.getInput().get(0).getInput().get(0).getInput().get(1);
-				
-				if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
-					tX = HopRewriteUtils.createTranspose(tX);
-				}
-				else 
-					tX = tX.getInput().get(0);
-				
-				hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-						  OpOp4.WSIGMOID, W, Y, tX, true, false);
-				HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-				appliedPattern = true;
-				LOG.debug("Applied simplifyWeightedSigmoid3 (line "+hi.getBeginLine()+")");	
-			}			
-			
-			//Pattern 4) W * log(sigmoid(-(Y%*%t(X)))) (log_minus)
-			if(    !appliedPattern 
-				&& uop.getOp() == OpOp1.LOG
-				&& uop.getInput().get(0) instanceof UnaryOp
-				&& ((UnaryOp)uop.getInput().get(0)).getOp() == OpOp1.SIGMOID 
-				&& uop.getInput().get(0).getInput().get(0) instanceof BinaryOp )
-			{
-				BinaryOp bop = (BinaryOp) uop.getInput().get(0).getInput().get(0);
-				
-				if(    bop.getOp() == OpOp2.MINUS 
-					&& bop.getInput().get(0) instanceof LiteralOp
-					&& HopRewriteUtils.getDoubleValueSafe((LiteralOp)bop.getInput().get(0))==0
-					&& bop.getInput().get(1) instanceof AggBinaryOp
-					&& HopRewriteUtils.isSingleBlock(bop.getInput().get(1).getInput().get(0),true))
-				{
-					Hop W = hi.getInput().get(0); 
-					Hop Y = bop.getInput().get(1).getInput().get(0);
-					Hop tX = bop.getInput().get(1).getInput().get(1);
-					
-					if( !HopRewriteUtils.isTransposeOperation(tX) ) { 
-						tX = HopRewriteUtils.createTranspose(tX);
-					}
-					else 
-						tX = tX.getInput().get(0);
-					
-					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-							  OpOp4.WSIGMOID, W, Y, tX, true, true);
-					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-	
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedSigmoid4 (line "+hi.getBeginLine()+")");	
-				}
-			}
-		}
-		
-		//relink new hop into original position
-		if( hnew != null ) {
-			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
-			HopRewriteUtils.addChildReference(parent, hnew, pos);
-			hi = hnew;
-		}
-		
-		return hi;
-	}
-
-	/**
-	 * 
-	 * @param parent
-	 * @param hi
-	 * @param pos
-	 * @return
-	 * @throws HopsException
-	 */
-	private Hop simplifyWeightedDivMM(Hop parent, Hop hi, int pos) 
-		throws HopsException
-	{
-		Hop hnew = null;
-		boolean appliedPattern = false;
-		
-		//left/right patterns rooted by 'ab - b(div)' or 'ab - b(mult)'
-		//note: we do not rewrite t(X)%*%(w*(X%*%v)) where w and v are vectors (see mmchain ops) 
-		if( hi instanceof AggBinaryOp && ((AggBinaryOp)hi).isMatrixMultiply()  
-			&& (hi.getInput().get(0) instanceof BinaryOp
-			&& HopRewriteUtils.isValidOp(((BinaryOp)hi.getInput().get(0)).getOp(), LOOKUP_VALID_WDIVMM_BINARY)
-			|| hi.getInput().get(1) instanceof BinaryOp 
-			&& hi.getDim2() > 1 //not applied for vector-vector mult
-			&& HopRewriteUtils.isValidOp(((BinaryOp)hi.getInput().get(1)).getOp(), LOOKUP_VALID_WDIVMM_BINARY)) ) 
-		{
-			Hop left = hi.getInput().get(0);
-			Hop right = hi.getInput().get(1);
-			
-			//Pattern 1) t(U) %*% (W/(U%*%t(V)))
-			//alternative pattern: t(U) %*% (W*(U%*%t(V)))
-			if( right instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)right).getOp(),LOOKUP_VALID_WDIVMM_BINARY)	
-				&& HopRewriteUtils.isEqualSize(right.getInput().get(0), right.getInput().get(1)) //prevent mv
-				&& right.getInput().get(1) instanceof AggBinaryOp
-				&& HopRewriteUtils.isSingleBlock(right.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
-			{
-				Hop W = right.getInput().get(0); 
-				Hop U = right.getInput().get(1).getInput().get(0);
-				Hop V = right.getInput().get(1).getInput().get(1);
-				
-				if( HopRewriteUtils.isTransposeOfItself(left, U) ) 
-				{
-					if( !HopRewriteUtils.isTransposeOperation(V) )
-						V = HopRewriteUtils.createTranspose(V);
-					else 
-						V = V.getInput().get(0);
-					
-					boolean mult = ((BinaryOp)right).getOp() == OpOp2.MULT;
-					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-							  OpOp4.WDIVMM, W, U, V, 1, mult, false);
-					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-					
-					//add output transpose for efficient target indexing (redundant t() removed by other rewrites)
-					hnew = HopRewriteUtils.createTranspose(hnew);
-					
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedDivMM1 (line "+hi.getBeginLine()+")");					
-				}
-			}	
-			
-			//Pattern 2) (W/(U%*%t(V))) %*% V
-			//alternative pattern: (W*(U%*%t(V))) %*% V
-			if( !appliedPattern
-				&& left instanceof BinaryOp && HopRewriteUtils.isValidOp(((BinaryOp)left).getOp(), LOOKUP_VALID_WDIVMM_BINARY)	
-				&& HopRewriteUtils.isEqualSize(left.getInput().get(0), left.getInput().get(1)) //prevent mv
-				&& left.getInput().get(1) instanceof AggBinaryOp
-				&& HopRewriteUtils.isSingleBlock(left.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
-			{
-				Hop W = left.getInput().get(0); 
-				Hop U = left.getInput().get(1).getInput().get(0);
-				Hop V = left.getInput().get(1).getInput().get(1);
-				
-				if( HopRewriteUtils.isTransposeOfItself(right, V) ) 
-				{
-					if( !HopRewriteUtils.isTransposeOperation(V) )
-						V = right;
-					else 
-						V = V.getInput().get(0);
-					
-					boolean mult = ((BinaryOp)left).getOp() == OpOp2.MULT;
-					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-							  OpOp4.WDIVMM, W, U, V, 2, mult, false);
-					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedDivMM2 (line "+hi.getBeginLine()+")");	
-				}
-			}
-			
-			//Pattern 3) t(U) %*% ((X!=0)*(U%*%t(V)-X))
-			if( right instanceof BinaryOp && ((BinaryOp)right).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT
-				&& right.getInput().get(1) instanceof BinaryOp && ((BinaryOp)right.getInput().get(1)).getOp()==OpOp2.MINUS	
-				&& right.getInput().get(1).getInput().get(0) instanceof AggBinaryOp
-                && right.getInput().get(1).getInput().get(1).getDataType() == DataType.MATRIX
-				&& HopRewriteUtils.isSingleBlock(right.getInput().get(1).getInput().get(0).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
-			{
-				Hop W = right.getInput().get(0); 
-				Hop U = right.getInput().get(1).getInput().get(0).getInput().get(0);
-				Hop V = right.getInput().get(1).getInput().get(0).getInput().get(1);
-				Hop X = right.getInput().get(1).getInput().get(1);
-				
-				if(    HopRewriteUtils.isNonZeroIndicator(W, X)        //W-X constraint
-				    && HopRewriteUtils.isTransposeOfItself(left, U) )  //t(U)-U constraint
-				{
-					if( !HopRewriteUtils.isTransposeOperation(V) )
-						V = HopRewriteUtils.createTranspose(V);
-					else 
-						V = V.getInput().get(0);
-					
-					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-							  OpOp4.WDIVMM, X, U, V, 1, true, true);
-					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-					
-					//add output transpose for efficient target indexing (redundant t() removed by other rewrites)
-					hnew = HopRewriteUtils.createTranspose(hnew);
-					
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedDivMM3 (line "+hi.getBeginLine()+")");					
-				}
-			}	
-			
-			//Pattern 4) ((X!=0)*(U%*%t(V)-X)) %*% V
-			if( !appliedPattern
-				&& left instanceof BinaryOp && ((BinaryOp)left).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT	
-				&& left.getInput().get(1) instanceof BinaryOp && ((BinaryOp)left.getInput().get(1)).getOp()==OpOp2.MINUS	
-				&& left.getInput().get(1).getInput().get(0) instanceof AggBinaryOp
-                && left.getInput().get(1).getInput().get(1).getDataType() == DataType.MATRIX
-				&& HopRewriteUtils.isSingleBlock(left.getInput().get(1).getInput().get(0).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
-			{
-				Hop W = left.getInput().get(0); 
-				Hop U = left.getInput().get(1).getInput().get(0).getInput().get(0);
-				Hop V = left.getInput().get(1).getInput().get(0).getInput().get(1);
-				Hop X = left.getInput().get(1).getInput().get(1);
-				
-				if(    HopRewriteUtils.isNonZeroIndicator(W, X)        //W-X constraint
-					&& HopRewriteUtils.isTransposeOfItself(right, V) )  //V-t(V) constraint
-				{
-					if( !HopRewriteUtils.isTransposeOperation(V) )
-						V = right;
-					else 
-						V = V.getInput().get(0);
-					
-					hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-							  OpOp4.WDIVMM, X, U, V, 2, true, true);
-					HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-					appliedPattern = true;
-					LOG.debug("Applied simplifyWeightedDivMM4 (line "+hi.getBeginLine()+")");	
-				}
-			}
-		}
-		
-		//Pattern 5) (W*(U%*%t(V)))
-		if( !appliedPattern
-			&& hi instanceof BinaryOp && ((BinaryOp)hi).getOp()==LOOKUP_VALID_WDIVMM_BINARY[0] //MULT	
-			&& HopRewriteUtils.isEqualSize(hi.getInput().get(0), hi.getInput().get(1)) //prevent mv
-			&& hi.getDim2() > 1 //not applied for vector-vector mult
-			&& hi.getInput().get(0).getDataType() == DataType.MATRIX 
-			&& hi.getInput().get(0).getDim2() > hi.getInput().get(0).getColsInBlock()
-			&& hi.getInput().get(1) instanceof AggBinaryOp
-			&& (((AggBinaryOp) hi.getInput().get(1)).checkMapMultChain() == ChainType.NONE || hi.getInput().get(1).getInput().get(1).getDim2() > 1) //no mmchain
-			&& HopRewriteUtils.isSingleBlock(hi.getInput().get(1).getInput().get(0),true) ) //BLOCKSIZE CONSTRAINT
-		{
-			Hop W = hi.getInput().get(0); 
-			Hop U = hi.getInput().get(1).getInput().get(0);
-			Hop V = hi.getInput().get(1).getInput().get(1);
-			
-			if( !HopRewriteUtils.isTransposeOperation(V) )
-				V = HopRewriteUtils.createTranspose(V);
-			else 
-				V = V.getInput().get(0);
-				
-			hnew = new QuaternaryOp(hi.getName(), DataType.MATRIX, ValueType.DOUBLE, 
-					  OpOp4.WDIVMM, W, U, V, 0, true, false);
-			HopRewriteUtils.setOutputBlocksizes(hnew, W.getRowsInBlock(), W.getColsInBlock());
-
-			appliedPattern = true;
-			LOG.debug("Applied simplifyWeightedDivMM5 (line "+hi.getBeginLine()+")");	
-		}
-		
-		//relink new hop into original position
-		if( hnew != null ) {
-			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
-			HopRewriteUtils.addChildReference(parent, hnew, pos);
-			hi = hnew;
-		}
-		
-		return hi;
-	}
-
-	/**
-	 * 
-	 * @param parent
-	 * @param hi
-	 * @param pos
-	 * @return
-	 * @throws HopsException
-	 */
-	private Hop simplifyWeightedCrossEntropy(Hop parent, Hop hi, int pos) 
-		throws HopsException
-	{
-		Hop hnew = null;
-		
-		//Pattern 1) sum( X * log(U %*% t(V)))
-		if( hi instanceof AggUnaryOp && ((AggUnaryOp)hi).getDirection()==Direction.RowCol
-			&& ((AggUnaryOp)hi).getOp() == AggOp.SUM     //pattern rooted by sum()
-			&& hi.getInput().get(0) instanceof BinaryOp  //pattern subrooted by binary op
-			&& hi.getInput().get(0).getDim2() > 1   )    //not applied for vector-vector mult
-		{
-			BinaryOp bop = (BinaryOp) hi.getInput().get(0);
-			Hop left = bop.getInput().get(0);
-			Hop right = bop.getInput().get(1);
-			
-			if( bop.getOp()==OpOp2.MULT && left.getDataType()==DataType.MATRIX		
-				&& HopRewriteUtils.isEqualSize(left, right)  //prevent mb
-				&& right instanceof UnaryOp	&& ((UnaryOp)right).getOp()==OpOp1.LOG
-				&& right.getInput().get(0) instanceof AggBinaryOp  //ba gurantees matrices
-				&& HopRewriteUtils.isSingleBlock(right.getInput().get(0).getInput().get(0),true)) //BLOCKSIZE CONSTRAINT
-			{
-				Hop X = left; 
-				Hop U = right.getInput().get(0).getInput().get(0);
-				Hop V = right.getInput().get(0).getInput().get(1);
-				
-				if( !HopRewriteUtils.isTransposeOperation(V) )
-					V = HopRewriteUtils.createTranspose(V);
-				else 
-					V = V.getInput().get(0);
-					
-				hnew = new QuaternaryOp(hi.getName(), DataType.SCALAR, ValueType.DOUBLE, OpOp4.WCEMM, X, U, V);
-				HopRewriteUtils.setOutputBlocksizes(hnew, X.getRowsInBlock(), X.getColsInBlock());
-					
-				LOG.debug("Applied simplifyWeightedCEMM (line "+hi.getBeginLine()+")");					
-			}
-		}
-		
-		//relink new hop into original position
-		if( hnew != null ) {
-			HopRewriteUtils.removeChildReferenceByPos(parent, hi, pos);
-			HopRewriteUtils.addChildReference(parent, hnew, pos);
-			hi = hnew;
-=======
->>>>>>> 04aa86c Fix rewrite 'fuse sum_sq' (after wsloss rewrite), for kmeans_predict 
-		}
-		
-		return hi;
-	}
-	
 	/**
 	 * 
 	 * @param parent