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 2020/04/30 20:23:18 UTC

[systemml] branch master updated: [SYSTEMDS-338] Extended lineage tracing and partial reuse

This is an automated email from the ASF dual-hosted git repository.

mboehm7 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/systemml.git


The following commit(s) were added to refs/heads/master by this push:
     new dc01c4d  [SYSTEMDS-338] Extended lineage tracing and partial reuse
dc01c4d is described below

commit dc01c4db8b46bc413ceabf4998e0b8d44969db73
Author: arnabp <ar...@tugraz.at>
AuthorDate: Thu Apr 30 22:01:51 2020 +0200

    [SYSTEMDS-338] Extended lineage tracing and partial reuse
    
    This patch contains
    - two new partial rewrites which are specializations of existing
    rewrites,
    - bug fixes and optimizations in partial rewrites,
    - lineage tracing for removeEmpty,
    - a new test class to test algorithms and builtins with reuse,
    - extension of lineage tracing of list objects.
    Note that lineage doesn't work with most of the list handling methods
    today. Due to that the current generalized grid search builtin is far
    from working with lineage framework.
    
    Closes #897.
---
 docs/Tasks.txt                                     |   1 +
 .../instructions/cp/DataGenCPInstruction.java      |   4 +
 .../instructions/cp/ListIndexingCPInstruction.java |   7 +
 .../cp/ParameterizedBuiltinCPInstruction.java      |   8 +
 .../apache/sysds/runtime/lineage/LineageCache.java |   4 +-
 .../sysds/runtime/lineage/LineageCacheConfig.java  |   9 +-
 .../apache/sysds/runtime/lineage/LineageMap.java   |   3 +-
 .../sysds/runtime/lineage/LineageRewriteReuse.java | 244 +++++++++++++++++----
 .../functions/lineage/FunctionFullReuseTest.java   |   5 -
 ...tionFullReuseTest.java => LineageReuseAlg.java} |  60 ++---
 .../test/functions/lineage/LineageRewriteTest.java |  10 +-
 ...FunctionFullReuse5.dml => LineageReuseAlg1.dml} |   2 -
 .../scripts/functions/lineage/LineageReuseAlg2.dml |  60 +++++
 .../{FunctionFullReuse5.dml => RewriteTest13.dml}  |  24 +-
 14 files changed, 333 insertions(+), 108 deletions(-)

diff --git a/docs/Tasks.txt b/docs/Tasks.txt
index 97fa914..35d07e6 100644
--- a/docs/Tasks.txt
+++ b/docs/Tasks.txt
@@ -264,6 +264,7 @@ SYSTEMDS-330 Lineage Tracing, Reuse and Integration
  * 335 Weighted eviction policy (function of size & computetime)
  * 336 Better use of cache status to handle multithreading
  * 337 Adjust disk I/O speed by recording actual time taken           OK
+ * 338 Extended lineage tracing (rmEmpty, lists), partial rewrites    OK
  
 SYSTEMDS-340 Compiler Assisted Lineage Caching and Reuse
  * 341 Finalize unmarking of loop dependent operations
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/DataGenCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/DataGenCPInstruction.java
index c29c539..7a37608 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/DataGenCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/DataGenCPInstruction.java
@@ -164,6 +164,10 @@ public class DataGenCPInstruction extends UnaryCPInstruction {
 	public long getSeed() {
 		return seed;
 	}
+	
+	public boolean isOnesCol() {
+		return minValue == maxValue && minValue == 1 && sparsity == 1 && getCols() == 1;
+	}
 
 	public static DataGenCPInstruction parseInstruction(String str)
 	{
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/ListIndexingCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/ListIndexingCPInstruction.java
index f15ad65..523eceb 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/ListIndexingCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/ListIndexingCPInstruction.java
@@ -25,6 +25,8 @@ import org.apache.sysds.common.Types.ValueType;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.controlprogram.caching.CacheableData;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
+import org.apache.sysds.runtime.lineage.LineageItem;
+import org.apache.sysds.runtime.lineage.LineageItemUtils;
 
 public final class ListIndexingCPInstruction extends IndexingCPInstruction {
 
@@ -93,4 +95,9 @@ public final class ListIndexingCPInstruction extends IndexingCPInstruction {
 		else
 			throw new DMLRuntimeException("Invalid opcode (" + opcode +") encountered in ListIndexingCPInstruction.");
 	}
+	@Override
+	public LineageItem[] getLineageItems(ExecutionContext ec) {
+		return new LineageItem[]{new LineageItem(output.getName(), getOpcode(),
+			LineageItemUtils.getLineage(ec, input1,input2,input3,rowLower,rowUpper))};
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
index 81a4367..99ce686 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
@@ -410,6 +410,14 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
 			return new LineageItem[]{new LineageItem(output.getName(),
 				getOpcode(), LineageItemUtils.getLineage(ec, target, groups, weights, fn, ngroups))};
 		}
+		else if (opcode.equalsIgnoreCase("rmempty")) {
+			CPOperand target = new CPOperand(params.get("target"), ValueType.FP64, DataType.MATRIX);
+			CPOperand margin = new CPOperand(params.get("margin"), ValueType.STRING, DataType.SCALAR, true);
+			String sl = params.containsKey("select") ? params.get("select") : String.valueOf(-1);
+			CPOperand select = new CPOperand(sl, ValueType.FP64, DataType.MATRIX); 
+			return new LineageItem[]{new LineageItem(output.getName(),
+				getOpcode(), LineageItemUtils.getLineage(ec, target, margin, select))};
+		}
 		//TODO: generic interface to support all the ops
 		else
 			return new LineageItem[]{new LineageItem(output.getName(),
diff --git a/src/main/java/org/apache/sysds/runtime/lineage/LineageCache.java b/src/main/java/org/apache/sysds/runtime/lineage/LineageCache.java
index 5e945d6..a42a376 100644
--- a/src/main/java/org/apache/sysds/runtime/lineage/LineageCache.java
+++ b/src/main/java/org/apache/sysds/runtime/lineage/LineageCache.java
@@ -237,8 +237,10 @@ public class LineageCache
 			synchronized( _cache ) {
 				if (data instanceof MatrixObject)
 					_cache.get(item).setValue(((MatrixObject)data).acquireReadAndRelease(), computetime);
-				else
+				else if (data instanceof ScalarObject)
 					_cache.get(item).setValue((ScalarObject)data, computetime);
+				else
+					throw new DMLRuntimeException("Lineage Cache: unsupported data: "+data.getDataType());
 				long size = _cache.get(item).getSize();
 				
 				if (!isBelowThreshold(size))
diff --git a/src/main/java/org/apache/sysds/runtime/lineage/LineageCacheConfig.java b/src/main/java/org/apache/sysds/runtime/lineage/LineageCacheConfig.java
index efe35f7..3268840 100644
--- a/src/main/java/org/apache/sysds/runtime/lineage/LineageCacheConfig.java
+++ b/src/main/java/org/apache/sysds/runtime/lineage/LineageCacheConfig.java
@@ -24,6 +24,7 @@ import org.apache.sysds.api.DMLScript;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.instructions.Instruction;
 import org.apache.sysds.runtime.instructions.cp.ComputationCPInstruction;
+import org.apache.sysds.runtime.instructions.cp.ListIndexingCPInstruction;
 
 import java.util.ArrayList;
 
@@ -101,9 +102,11 @@ public class LineageCacheConfig {
 	}
 	
 	public static boolean isReusable (Instruction inst, ExecutionContext ec) {
-		return inst instanceof ComputationCPInstruction
-			&& (ArrayUtils.contains(REUSE_OPCODES, inst.getOpcode())
-				|| (inst.getOpcode().equals("append") && isVectorAppend(inst, ec)));
+		boolean insttype = inst instanceof ComputationCPInstruction 
+			&& !(inst instanceof ListIndexingCPInstruction);
+		boolean rightop = (ArrayUtils.contains(REUSE_OPCODES, inst.getOpcode())
+			|| (inst.getOpcode().equals("append") && isVectorAppend(inst, ec)));
+		return insttype && rightop;
 	}
 	
 	private static boolean isVectorAppend(Instruction inst, ExecutionContext ec) {
diff --git a/src/main/java/org/apache/sysds/runtime/lineage/LineageMap.java b/src/main/java/org/apache/sysds/runtime/lineage/LineageMap.java
index 0c4947e..b766cb0 100644
--- a/src/main/java/org/apache/sysds/runtime/lineage/LineageMap.java
+++ b/src/main/java/org/apache/sysds/runtime/lineage/LineageMap.java
@@ -25,6 +25,7 @@ import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.instructions.Instruction;
 import org.apache.sysds.runtime.instructions.cp.CPOperand;
+import org.apache.sysds.runtime.instructions.cp.EvalNaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.FunctionCallCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.VariableCPInstruction;
 import org.apache.sysds.runtime.instructions.spark.WriteSPInstruction;
@@ -51,7 +52,7 @@ public class LineageMap {
 	}
 	
 	public void trace(Instruction inst, ExecutionContext ec) {
-		if( inst instanceof FunctionCallCPInstruction )
+		if( inst instanceof FunctionCallCPInstruction || inst instanceof EvalNaryCPInstruction)
 			return; // no need for lineage tracing
 		if (!(inst instanceof LineageTraceable))
 			throw new DMLRuntimeException("Unknown Instruction (" + inst.getOpcode() + ") traced.");
diff --git a/src/main/java/org/apache/sysds/runtime/lineage/LineageRewriteReuse.java b/src/main/java/org/apache/sysds/runtime/lineage/LineageRewriteReuse.java
index 4be8aef..dc1d2da 100644
--- a/src/main/java/org/apache/sysds/runtime/lineage/LineageRewriteReuse.java
+++ b/src/main/java/org/apache/sysds/runtime/lineage/LineageRewriteReuse.java
@@ -26,12 +26,17 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.common.Types.AggOp;
+import org.apache.sysds.common.Types.Direction;
 import org.apache.sysds.common.Types.OpOp2;
 import org.apache.sysds.common.Types.OpOpN;
 import org.apache.sysds.common.Types.ParamBuiltinOp;
 import org.apache.sysds.common.Types.ValueType;
 import org.apache.sysds.hops.AggBinaryOp;
+import org.apache.sysds.hops.AggUnaryOp;
 import org.apache.sysds.hops.BinaryOp;
 import org.apache.sysds.hops.DataOp;
 import org.apache.sysds.hops.Hop;
@@ -50,9 +55,12 @@ import org.apache.sysds.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContextFactory;
 import org.apache.sysds.runtime.instructions.Instruction;
+import org.apache.sysds.runtime.instructions.InstructionParser;
 import org.apache.sysds.runtime.instructions.cp.ComputationCPInstruction;
+import org.apache.sysds.runtime.instructions.cp.DataGenCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.ParameterizedBuiltinCPInstruction;
 import org.apache.sysds.runtime.lineage.LineageCacheConfig.ReuseCacheType;
+import org.apache.sysds.runtime.lineage.LineageItem.LineageItemType;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.meta.MetaData;
 import org.apache.sysds.utils.Explain;
@@ -65,6 +73,15 @@ public class LineageRewriteReuse
 	private static ExecutionContext _lrEC = null;
 	private static final Log LOG = LogFactory.getLog(LineageRewriteReuse.class.getName());
 	
+	private static boolean LDEBUG = false; //internal debugging
+	
+	static {
+		if( LDEBUG ) {
+			Logger.getLogger("org.apache.sysds.runtime.lineage.LineageRewriteReuse")
+				.setLevel(Level.DEBUG);
+		}
+	}
+	
 	public static boolean executeRewrites (Instruction curr, ExecutionContext ec)
 	{
 		ExecutionContext lrwec = getExecutionContext();
@@ -74,14 +91,18 @@ public class LineageRewriteReuse
 		DMLScript.EXPLAIN = ExplainType.NONE;
 		
 		//check applicability and apply rewrite
+		//tsmm(cbind(X, ones)) -> rbind(t(colSums(cbind(X, ones))[, 1:ncol-1]), colSums(cbind(X, ones)))
+		ArrayList<Instruction> newInst = rewriteTsmmCbindOnes(curr, ec, lrwec);
 		//tsmm(cbind(X, deltaX)) -> rbind(cbind(tsmm(X), t(X) %*% deltaX), cbind(t(deltaX) %*%X, tsmm(deltaX)))
-		ArrayList<Instruction> newInst = rewriteTsmmCbind(curr, ec, lrwec);
+		newInst = (newInst == null) ? rewriteTsmmCbind(curr, ec, lrwec) : newInst;
 		//tsmm(cbind(cbind(X, deltaX), ones)) -> TODO
 		newInst = (newInst == null) ? rewriteTsmm2Cbind(curr, ec, lrwec) : newInst;
 		//tsmm(rbind(X, deltaX)) -> tsmm(X) + tsmm(deltaX)
 		newInst = (newInst == null) ? rewriteTsmmRbind(curr, ec, lrwec) : newInst;
 		//rbind(X,deltaX) %*% Y -> rbind(X %*% Y, deltaX %*% Y)
 		newInst = (newInst == null) ? rewriteMatMulRbindLeft(curr, ec, lrwec) : newInst;
+		//X %*% cbind(Y,ones)) -> cbind(X %*% Y, rowSums(X))
+		newInst = (newInst == null) ? rewriteMatMulCbindRightOnes(curr, ec, lrwec) : newInst;
 		//X %*% cbind(Y,deltaY)) -> cbind(X %*% Y, X %*% deltaY)
 		newInst = (newInst == null) ? rewriteMatMulCbindRight(curr, ec, lrwec) : newInst;
 		//rbind(X, deltaX) * rbind(Y, deltaY) -> rbind(X * Y, deltaX * deltaY)
@@ -133,7 +154,7 @@ public class LineageRewriteReuse
 		lrwec.setVariable("oldMatrix", mo);
 		DataOp newMatrix = HopRewriteUtils.createTransientRead("oldMatrix", mo);
 		IndexingOp oldMatrix = HopRewriteUtils.createIndexingOp(newMatrix, new LiteralOp(1), 
-				new LiteralOp(mo.getNumRows()), new LiteralOp(1), new LiteralOp(mo.getNumColumns()-1));
+			new LiteralOp(mo.getNumRows()), new LiteralOp(1), new LiteralOp(mo.getNumColumns()-1));
 		Hop lastCol;
 		// Use deltaX from cache, or create rightIndex
 		if (inCache.containsKey("deltaX")) {
@@ -143,14 +164,14 @@ public class LineageRewriteReuse
 		}
 		else
 			lastCol = HopRewriteUtils.createIndexingOp(newMatrix, new LiteralOp(1), new LiteralOp(mo.getNumRows()), 
-					new LiteralOp(mo.getNumColumns()), new LiteralOp(mo.getNumColumns()));
+				new LiteralOp(mo.getNumColumns()), new LiteralOp(mo.getNumColumns()));
 		// cell topRight = t(oldMatrix) %*% lastCol
 		ReorgOp tOldM = HopRewriteUtils.createTranspose(oldMatrix);
 		AggBinaryOp topRight = HopRewriteUtils.createMatrixMultiply(tOldM, lastCol);
-		// cell bottomLeft = t(lastCol) %*% oldMatrix
-		ReorgOp tLastCol = HopRewriteUtils.createTranspose(lastCol);
-		AggBinaryOp bottomLeft = HopRewriteUtils.createMatrixMultiply(tLastCol, oldMatrix);
+		// cell bottomLeft = t(lastCol) %*% oldMatrix = t(topRight)
+		ReorgOp bottomLeft = HopRewriteUtils.createTranspose(topRight);
 		// bottomRight = t(lastCol) %*% lastCol
+		ReorgOp tLastCol = HopRewriteUtils.createTranspose(lastCol);
 		AggBinaryOp bottomRight = HopRewriteUtils.createMatrixMultiply(tLastCol, lastCol);
 		// rowOne = cbind(lastRes, topRight)
 		BinaryOp rowOne = HopRewriteUtils.createBinary(lastRes, topRight, OpOp2.CBIND);
@@ -161,7 +182,50 @@ public class LineageRewriteReuse
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteTsmmCbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteTsmmCbind APPLIED");
+		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
+
+		if (DMLScript.STATISTICS) {
+			LineageCacheStatistics.incrementPRewriteTime(System.nanoTime() - t0);
+			LineageCacheStatistics.incrementPRewrites();
+		}
+		return inst;
+	}
+
+	private static ArrayList<Instruction> rewriteTsmmCbindOnes (Instruction curr, ExecutionContext ec, ExecutionContext lrwec) 
+	{
+		// This is a specialization of rewriteTsmmCbind. This qualifies if 
+		// the appended matrix is a column matrix of 1s (deltaX = 1s). 
+		// Check the applicability of this rewrite.
+		Map<String, MatrixBlock> inCache = new HashMap<>();
+		if(!isTsmmCbindOnes(curr, ec, inCache))
+			return null;
+		
+		long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		// Create a transient read op over the cached tsmm result
+		MatrixBlock cachedEntry = inCache.get("lastMatrix");
+		lrwec.setVariable("cachedEntry", convMBtoMO(cachedEntry));
+		DataOp lastRes = HopRewriteUtils.createTransientRead("cachedEntry", cachedEntry);
+		// Create a transient read op over current input
+		MatrixObject mo = ec.getMatrixObject(((ComputationCPInstruction)curr).input1);
+		lrwec.setVariable("newMatrix", mo);
+		DataOp newMatrix = HopRewriteUtils.createTransientRead("newMatrix", mo);
+		// rowTwo = colSums(newMatrix)
+		AggUnaryOp rowTwo = HopRewriteUtils.createAggUnaryOp(newMatrix, AggOp.SUM, Direction.Col);
+		// topRight = t(rowTwo[, 1:ncols-1])
+		IndexingOp tmp = HopRewriteUtils.createIndexingOp(rowTwo, new LiteralOp(1), new LiteralOp(1), 
+			new LiteralOp(1), new LiteralOp(mo.getNumColumns()-1));
+		ReorgOp topRight = HopRewriteUtils.createTranspose(tmp);
+		// rowOne = cbind(lastRes, topRight)
+		BinaryOp rowOne = HopRewriteUtils.createBinary(lastRes, topRight, OpOp2.CBIND);
+		// rbind(rowOne, rowTwo)
+		BinaryOp lrwHop= HopRewriteUtils.createBinary(rowOne, rowTwo, OpOp2.RBIND);
+		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
+
+		// generate runtime instructions
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteTsmmCbindOnes APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -204,7 +268,8 @@ public class LineageRewriteReuse
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteTsmmRbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteTsmmRbind APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -241,7 +306,7 @@ public class LineageRewriteReuse
 		}
 		else
 			lastCol = HopRewriteUtils.createIndexingOp(newMatrix, new LiteralOp(1), new LiteralOp(mo.getNumRows()), 
-					new LiteralOp(mo.getNumColumns()-1), new LiteralOp(mo.getNumColumns()-1));
+				new LiteralOp(mo.getNumColumns()-1), new LiteralOp(mo.getNumColumns()-1));
 		// apply t(lastCol) on i/p matrix to get the result vectors.
 		ReorgOp tlastCol = HopRewriteUtils.createTranspose(lastCol);
 		AggBinaryOp newCol = HopRewriteUtils.createMatrixMultiply(tlastCol, newMatrix);
@@ -249,24 +314,25 @@ public class LineageRewriteReuse
 
 		// push the result row & column inside the cashed block as 2nd last row and col respectively.
 		IndexingOp topLeft = HopRewriteUtils.createIndexingOp(lastRes, new LiteralOp(1), new LiteralOp(newmo.getNumRows()-1), 
-				new LiteralOp(1), new LiteralOp(newmo.getNumColumns()-1));
+			new LiteralOp(1), new LiteralOp(newmo.getNumColumns()-1));
 		IndexingOp topRight = HopRewriteUtils.createIndexingOp(lastRes, new LiteralOp(1), new LiteralOp(newmo.getNumRows()-1), 
-				new LiteralOp(newmo.getNumColumns()), new LiteralOp(newmo.getNumColumns()));
+			new LiteralOp(newmo.getNumColumns()), new LiteralOp(newmo.getNumColumns()));
 		IndexingOp bottomLeft = HopRewriteUtils.createIndexingOp(lastRes, new LiteralOp(newmo.getNumRows()), 
-				new LiteralOp(newmo.getNumRows()), new LiteralOp(1), new LiteralOp(newmo.getNumColumns()-1));
+			new LiteralOp(newmo.getNumRows()), new LiteralOp(1), new LiteralOp(newmo.getNumColumns()-1));
 		IndexingOp bottomRight = HopRewriteUtils.createIndexingOp(lastRes, new LiteralOp(newmo.getNumRows()), 
-				new LiteralOp(newmo.getNumRows()), new LiteralOp(newmo.getNumColumns()), new LiteralOp(newmo.getNumColumns()));
+			new LiteralOp(newmo.getNumRows()), new LiteralOp(newmo.getNumColumns()), new LiteralOp(newmo.getNumColumns()));
 		IndexingOp topCol = HopRewriteUtils.createIndexingOp(tnewCol, new LiteralOp(1), new LiteralOp(mo.getNumColumns()-2), 
-				new LiteralOp(1), new LiteralOp(1));
+			new LiteralOp(1), new LiteralOp(1));
 		IndexingOp bottomCol = HopRewriteUtils.createIndexingOp(tnewCol, new LiteralOp(mo.getNumColumns()), 
-				new LiteralOp(mo.getNumColumns()), new LiteralOp(1), new LiteralOp(1));
+			new LiteralOp(mo.getNumColumns()), new LiteralOp(1), new LiteralOp(1));
 		NaryOp rowOne = HopRewriteUtils.createNary(OpOpN.CBIND, topLeft, topCol, topRight);
 		NaryOp rowTwo = HopRewriteUtils.createNary(OpOpN.CBIND, bottomLeft, bottomCol, bottomRight);
 		NaryOp lrwHop = HopRewriteUtils.createNary(OpOpN.RBIND, rowOne, newCol, rowTwo);
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteTsmm2Cbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteTsmm2Cbind APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -303,14 +369,15 @@ public class LineageRewriteReuse
 			lastRow = HopRewriteUtils.createTransientRead("deltaX", cachedRI);
 		}
 		lastRow = HopRewriteUtils.createIndexingOp(leftMatrix, new LiteralOp(moL.getNumRows()), 
-				new LiteralOp(moL.getNumRows()), new LiteralOp(1), new LiteralOp(moL.getNumColumns()));
+			new LiteralOp(moL.getNumRows()), new LiteralOp(1), new LiteralOp(moL.getNumColumns()));
 		// ba+*(X+lastRow, Y) = rbind(ba+*(X, Y), ba+*(lastRow, Y))
 		AggBinaryOp rowTwo = HopRewriteUtils.createMatrixMultiply(lastRow, rightMatrix);
 		BinaryOp lrwHop= HopRewriteUtils.createBinary(lastRes, rowTwo, OpOp2.RBIND);
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteMetMulRbindLeft APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteMetMulRbindLeft APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -347,14 +414,49 @@ public class LineageRewriteReuse
 			lastCol = HopRewriteUtils.createTransientRead("deltaY", cachedRI);
 		}
 		lastCol = HopRewriteUtils.createIndexingOp(rightMatrix, new LiteralOp(1), new LiteralOp(moR.getNumRows()), 
-				new LiteralOp(moR.getNumColumns()), new LiteralOp(moR.getNumColumns()));
-		// ba+*(X, Y+lastCol) = cbind(ba+*(X, Y), ba+*(X, lastCol))
-		AggBinaryOp rowTwo = HopRewriteUtils.createMatrixMultiply(leftMatrix, lastCol);
-		BinaryOp lrwHop= HopRewriteUtils.createBinary(lastRes, rowTwo, OpOp2.CBIND);
+			new LiteralOp(moR.getNumColumns()), new LiteralOp(moR.getNumColumns()));
+		// ba+*(X, cbind(Y, lastCol)) = cbind(ba+*(X, Y), ba+*(X, lastCol))
+		AggBinaryOp colTwo = HopRewriteUtils.createMatrixMultiply(leftMatrix, lastCol);
+		BinaryOp lrwHop= HopRewriteUtils.createBinary(lastRes, colTwo, OpOp2.CBIND);
+		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
+
+		// generate runtime instructions
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteMatMulCbindRight APPLIED");
+		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
+
+		if (DMLScript.STATISTICS) {
+			LineageCacheStatistics.incrementPRewriteTime(System.nanoTime() - t0);
+			LineageCacheStatistics.incrementPRewrites();
+		}
+		return inst;
+	}
+
+	private static ArrayList<Instruction> rewriteMatMulCbindRightOnes (Instruction curr, ExecutionContext ec, ExecutionContext lrwec) 
+	{
+		// This is a specialization of rewriteMatMulCbindRight. This qualifies
+		// if the right matrix is appended with a matrix of 1s (deltaY == 1s).
+		// Check the applicability of this rewrite.
+		Map<String, MatrixBlock> inCache = new HashMap<>();
+		if (!isMatMulCbindRightOnes(curr, ec, inCache))
+			return null;
+
+		long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0;
+		// Create a transient read op over the last ba+* result
+		MatrixBlock cachedEntry = inCache.get("lastMatrix");
+		lrwec.setVariable("cachedEntry", convMBtoMO(cachedEntry));
+		DataOp lastRes = HopRewriteUtils.createTransientRead("cachedEntry", cachedEntry);
+		MatrixObject moL = ec.getMatrixObject(((ComputationCPInstruction)curr).input1);
+		lrwec.setVariable("leftMatrix", moL);
+		DataOp leftMatrix = HopRewriteUtils.createTransientRead("leftMatrix", moL);
+		// ba+*(X, cbind(Y, ones)) = cbind(ba+*(X, Y), rowSums(X))
+		AggUnaryOp colTwo = HopRewriteUtils.createAggUnaryOp(leftMatrix, AggOp.SUM, Direction.Row);
+		BinaryOp lrwHop= HopRewriteUtils.createBinary(lastRes, colTwo, OpOp2.CBIND);
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteMatMulCbindRight APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteMatMulCbindRightOnes APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -389,7 +491,7 @@ public class LineageRewriteReuse
 			lrwec.setVariable("leftMatrix", moL);
 			DataOp leftMatrix = HopRewriteUtils.createTransientRead("leftMatrix", moL);
 			lastRowL = HopRewriteUtils.createIndexingOp(leftMatrix, new LiteralOp(moL.getNumRows()), 
-					new LiteralOp(moL.getNumRows()), new LiteralOp(1), new LiteralOp(moL.getNumColumns()));
+				new LiteralOp(moL.getNumRows()), new LiteralOp(1), new LiteralOp(moL.getNumColumns()));
 		}
 		if (inCache.containsKey("deltaY")) {
 			MatrixBlock cachedRI = inCache.get("deltaY");
@@ -409,7 +511,8 @@ public class LineageRewriteReuse
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteElementMulRbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteElementMulRbind APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -464,7 +567,8 @@ public class LineageRewriteReuse
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteElementMulCbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteElementMulCbind APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -496,7 +600,7 @@ public class LineageRewriteReuse
 		DataOp groups = HopRewriteUtils.createTransientRead("groups", moG);
 		String fn = params.get(Statement.GAGG_FN);
 		int ngroups = (params.get(Statement.GAGG_NUM_GROUPS) != null) ? 
-				(int)Double.parseDouble(params.get(Statement.GAGG_NUM_GROUPS)) : -1;
+			(int)Double.parseDouble(params.get(Statement.GAGG_NUM_GROUPS)) : -1;
 		Hop lastCol;
 		// Use deltaX from cache, or create rightIndex
 		if (inCache.containsKey("deltaX")) {
@@ -506,7 +610,7 @@ public class LineageRewriteReuse
 		}
 		else
 			lastCol = HopRewriteUtils.createIndexingOp(newMatrix, new LiteralOp(1), new LiteralOp(mo.getNumRows()), 
-					new LiteralOp(mo.getNumColumns()), new LiteralOp(mo.getNumColumns()));
+				new LiteralOp(mo.getNumColumns()), new LiteralOp(mo.getNumColumns()));
 		// aggregate(target=X+lastCol,...) = cbind(aggregate(target=X,...), aggregate(target=lastCol,...))
 		LinkedHashMap<String, Hop> args = new LinkedHashMap<>();
 		args.put("target", lastCol);
@@ -519,7 +623,8 @@ public class LineageRewriteReuse
 		DataOp lrwWrite = HopRewriteUtils.createTransientWrite(LR_VAR, lrwHop);
 
 		// generate runtime instructions
-		LOG.debug("LINEAGE REWRITE rewriteElementMulCbind APPLIED");
+		if (LOG.isDebugEnabled())
+			LOG.debug("LINEAGE REWRITE rewriteElementMulCbind APPLIED");
 		ArrayList<Instruction> inst = genInst(lrwWrite, lrwec);
 
 		if (DMLScript.STATISTICS) {
@@ -539,8 +644,8 @@ public class LineageRewriteReuse
 
 		// If the input to tsmm came from cbind, look for both the inputs in cache.
 		LineageItem[] items = ((ComputationCPInstruction) curr).getLineageItems(ec);
-		LineageItem item = items[0];
-		for (LineageItem source : item.getInputs())
+		if (curr.getOpcode().equalsIgnoreCase("tsmm")) {
+			LineageItem source = items[0].getInputs()[0];
 			if (source.getOpcode().equalsIgnoreCase("cbind")) {
 				//for (LineageItem input : source.getInputs()) {
 				// create tsmm lineage on top of the input of last append
@@ -552,10 +657,40 @@ public class LineageRewriteReuse
 				if (LineageCache.probe(source.getInputs()[1])) 
 					inCache.put("deltaX", LineageCache.getMatrix(source.getInputs()[1]));
 			}
+		}
 		// return true only if the last tsmm is found
 		return inCache.containsKey("lastMatrix") ? true : false;
 	}
 
+	private static boolean isTsmmCbindOnes(Instruction curr, ExecutionContext ec, Map<String, MatrixBlock> inCache)
+	{
+		if (!LineageCacheConfig.isReusable(curr, ec)) {
+			return false;
+		}
+
+		// If the input to tsmm came from cbind, look for both the inputs in cache.
+		LineageItem[] items = ((ComputationCPInstruction) curr).getLineageItems(ec);
+		if (curr.getOpcode().equalsIgnoreCase("tsmm")) {
+			LineageItem source = items[0].getInputs()[0];
+			if (source.getOpcode().equalsIgnoreCase("cbind")) {
+				// check if the appended column is a matrix of 1s
+				LineageItem input2 = source.getInputs()[1];
+				if (input2.getType() != LineageItemType.Creation)
+					return false;
+				Instruction ins = InstructionParser.parseSingleInstruction(input2.getData());
+				if (!((DataGenCPInstruction)ins).isOnesCol())
+					return false;
+				// create tsmm lineage on top of the input of last append
+				LineageItem input1 = source.getInputs()[0];
+				LineageItem tmp = new LineageItem("toProbe", curr.getOpcode(), new LineageItem[] {input1});
+				if (LineageCache.probe(tmp)) 
+					inCache.put("lastMatrix", LineageCache.getMatrix(tmp));
+			}
+		}
+		// return true only if the last tsmm result is found
+		return inCache.containsKey("lastMatrix") ? true : false;
+	}
+
 	private static boolean isTsmmRbind(Instruction curr, ExecutionContext ec, Map<String, MatrixBlock> inCache)
 	{
 		if (!LineageCacheConfig.isReusable(curr, ec))
@@ -563,8 +698,8 @@ public class LineageRewriteReuse
 
 		// If the input to tsmm came from rbind, look for both the inputs in cache.
 		LineageItem[] items = ((ComputationCPInstruction) curr).getLineageItems(ec);
-		LineageItem item = items[0];
-		for (LineageItem source : item.getInputs())
+		if (curr.getOpcode().equalsIgnoreCase("tsmm")) {
+			LineageItem source = items[0].getInputs()[0];
 			if (source.getOpcode().equalsIgnoreCase("rbind")) {
 				// create tsmm lineage on top of the input of last append
 				LineageItem input1 = source.getInputs()[0];
@@ -575,6 +710,7 @@ public class LineageRewriteReuse
 				if (LineageCache.probe(source.getInputs()[1])) 
 					inCache.put("deltaX", LineageCache.getMatrix(source.getInputs()[1]));
 			}
+		}
 		// return true only if the last tsmm is found
 		return inCache.containsKey("lastMatrix") ? true : false;
 	}
@@ -587,9 +723,9 @@ public class LineageRewriteReuse
 		//TODO: support nary cbind
 		// If the input to tsmm came from cbind, look for both the inputs in cache.
 		LineageItem[] items = ((ComputationCPInstruction) curr).getLineageItems(ec);
-		LineageItem item = items[0];
 		// look for two consecutive cbinds
-		for (LineageItem source : item.getInputs())
+		if (curr.getOpcode().equalsIgnoreCase("tsmm")) {
+			LineageItem source = items[0].getInputs()[0];
 			if (source.getOpcode().equalsIgnoreCase("cbind")) {
 				LineageItem input = source.getInputs()[0];
 				if (input.getOpcode().equalsIgnoreCase("cbind")) {
@@ -603,6 +739,7 @@ public class LineageRewriteReuse
 						inCache.put("deltaX", LineageCache.getMatrix(input.getInputs()[1]));
 				}
 			}
+		}
 		// return true only if the last tsmm is found
 		return inCache.containsKey("lastMatrix") ? true : false;
 	}
@@ -656,6 +793,34 @@ public class LineageRewriteReuse
 		return inCache.containsKey("lastMatrix") ? true : false;
 	}
 
+	private static boolean isMatMulCbindRightOnes(Instruction curr, ExecutionContext ec, Map<String, MatrixBlock> inCache)
+	{
+		if (!LineageCacheConfig.isReusable(curr, ec))
+			return false;
+
+		// If the right input to ba+* came from cbind of a matrix and ones.
+		LineageItem[] items = ((ComputationCPInstruction) curr).getLineageItems(ec);
+		if (curr.getOpcode().equalsIgnoreCase("ba+*")) {
+			LineageItem left = items[0].getInputs()[0];
+			LineageItem right = items[0].getInputs()[1];
+			if (right.getOpcode().equalsIgnoreCase("cbind")) {
+				LineageItem rightSource1 = right.getInputs()[0]; //left input of cbind is X
+				LineageItem rightSource2 = right.getInputs()[1]; 
+				// check if the right input to cbind is a matrix of 1s.
+				if (rightSource2.getType() != LineageItemType.Creation)
+					return false;
+				Instruction ins = InstructionParser.parseSingleInstruction(rightSource2.getData());
+				if (!((DataGenCPInstruction)ins).isOnesCol())
+					return false;
+				// create ba+* lineage on top of the input of last append
+				LineageItem tmp = new LineageItem("toProbe", curr.getOpcode(), new LineageItem[] {left, rightSource1});
+				if (LineageCache.probe(tmp))
+					inCache.put("lastMatrix", LineageCache.getMatrix(tmp));
+			}
+		}
+		return inCache.containsKey("lastMatrix") ? true : false;
+	}
+
 	private static boolean isElementMulRbind(Instruction curr, ExecutionContext ec, Map<String, MatrixBlock> inCache)
 	{
 		if (!LineageCacheConfig.isReusable(curr, ec))
@@ -747,8 +912,11 @@ public class LineageRewriteReuse
 		ArrayList<Instruction> newInst = Recompiler.recompileHopsDag(hops, ec.getVariables(), null, true, true, 0);
 		if (DMLScript.STATISTICS) 
 			LineageCacheStatistics.incrementPRwExecTime(System.nanoTime()-t0);
-		LOG.debug("EXPLAIN LINEAGE REWRITE \nGENERIC (line "+hops.getBeginLine()+"):\n" + Explain.explain(hops,1));
-		LOG.debug("EXPLAIN LINEAGE REWRITE \nGENERIC (line "+hops.getBeginLine()+"):\n" + Explain.explain(newInst,1));
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("COMPENSATION PLAN: ");
+			LOG.debug("EXPLAIN LINEAGE REWRITE (HOP) \n" + Explain.explain(hops,1));
+			LOG.debug("EXPLAIN LINEAGE REWRITE (INSTRUCTION) \n" + Explain.explain(newInst,1));
+		}
 		return newInst;
 	}
 	
@@ -793,4 +961,4 @@ public class LineageRewriteReuse
 			_lrPB = new BasicProgramBlock(new Program());
 		return _lrPB;
 	}
-}
\ No newline at end of file
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java b/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java
index 8fc7f78..b0ff854 100644
--- a/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java
@@ -74,11 +74,6 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 	}
 
 	@Test
-	public void testStepLM() {
-		testLineageTrace(TEST_NAME+"5");
-	}
-	
-	@Test
 	public void testParforIssue1() {
 		testLineageTrace(TEST_NAME+"6");
 	}
diff --git a/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java b/src/test/java/org/apache/sysds/test/functions/lineage/LineageReuseAlg.java
similarity index 74%
copy from src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java
copy to src/test/java/org/apache/sysds/test/functions/lineage/LineageReuseAlg.java
index 8fc7f78..631588f 100644
--- a/src/test/java/org/apache/sysds/test/functions/lineage/FunctionFullReuseTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/lineage/LineageReuseAlg.java
@@ -19,16 +19,13 @@
 
 package org.apache.sysds.test.functions.lineage;
 
-import org.junit.Assert;
 import org.junit.Test;
-
 import org.apache.sysds.common.Types.ExecMode;
 import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.hops.recompile.Recompiler;
 import org.apache.sysds.lops.LopProperties.ExecType;
 import org.apache.sysds.runtime.lineage.Lineage;
 import org.apache.sysds.runtime.lineage.LineageCacheConfig.ReuseCacheType;
-import org.apache.sysds.runtime.lineage.LineageCacheStatistics;
 import org.apache.sysds.runtime.matrix.data.MatrixValue;
 import org.apache.sysds.test.AutomatedTestBase;
 import org.apache.sysds.test.TestConfiguration;
@@ -38,13 +35,12 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 
-public class FunctionFullReuseTest extends AutomatedTestBase
-{
-	protected static final String TEST_DIR = "functions/lineage/";
-	protected static final String TEST_NAME = "FunctionFullReuse";
-	protected static final int TEST_VARIANTS = 7;
+public class LineageReuseAlg extends AutomatedTestBase {
 	
-	protected String TEST_CLASS_DIR = TEST_DIR + FunctionFullReuseTest.class.getSimpleName() + "/";
+	protected static final String TEST_DIR = "functions/lineage/";
+	protected static final String TEST_NAME = "LineageReuseAlg";
+	protected static final int TEST_VARIANTS = 2;
+	protected String TEST_CLASS_DIR = TEST_DIR + LineageReuseAlg.class.getSimpleName() + "/";
 	
 	@Override
 	public void setUp() {
@@ -54,41 +50,16 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 	}
 	
 	@Test
-	public void testCacheHit() {
-		testLineageTrace(TEST_NAME+"1");
-	}
-	
-	@Test
-	public void testCacheMiss() {
-		testLineageTrace(TEST_NAME+"2");
-	}
-
-	@Test
-	public void testMultipleReturns() {
-		testLineageTrace(TEST_NAME+"3");
-	}
-
-	@Test
-	public void testNestedFunc() {
-		testLineageTrace(TEST_NAME+"4");
-	}
-
-	@Test
 	public void testStepLM() {
-		testLineageTrace(TEST_NAME+"5");
-	}
-	
-	@Test
-	public void testParforIssue1() {
-		testLineageTrace(TEST_NAME+"6");
+		testLineageTrace(TEST_NAME+"1", ReuseCacheType.REUSE_HYBRID.name().toLowerCase());
 	}
 	
 	@Test
-	public void testParforIssue2() {
-		testLineageTrace(TEST_NAME+"7");
+	public void testGridSearchLM() {
+		testLineageTrace(TEST_NAME+"2", ReuseCacheType.REUSE_HYBRID.name().toLowerCase());
 	}
-	
-	public void testLineageTrace(String testname) {
+
+	public void testLineageTrace(String testname, String reuseType) {
 		boolean old_simplification = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;
 		boolean old_sum_product = OptimizerUtils.ALLOW_SUM_PRODUCT_REWRITES;
 		ExecMode platformOld = setExecMode(ExecType.CP);
@@ -106,7 +77,6 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 			List<String> proArgs = new ArrayList<>();
 			proArgs.add("-stats");
 			proArgs.add("-lineage");
-			proArgs.add("-explain");
 			proArgs.add("-args");
 			proArgs.add(output("X"));
 			programArgs = proArgs.toArray(new String[proArgs.size()]);
@@ -118,8 +88,9 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 			// With lineage-based reuse enabled
 			proArgs.clear();
 			proArgs.add("-stats");
+			//proArgs.add("-explain");
 			proArgs.add("-lineage");
-			proArgs.add(ReuseCacheType.REUSE_MULTILEVEL.name().toLowerCase());
+			proArgs.add(reuseType);
 			proArgs.add("-args");
 			proArgs.add(output("X"));
 			programArgs = proArgs.toArray(new String[proArgs.size()]);
@@ -129,12 +100,7 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 			runTest(true, EXCEPTION_NOT_EXPECTED, null, -1);
 			HashMap<MatrixValue.CellIndex, Double> X_reused = readDMLMatrixFromHDFS("X");
 			Lineage.setLinReuseNone();
-			
 			TestUtils.compareMatrices(X_orig, X_reused, 1e-6, "Origin", "Reused");
-			if( testname.endsWith("6") ) { // parfor fn reuse
-				Assert.assertEquals(9L, LineageCacheStatistics.getMultiLevelFnHits() 
-					+ LineageCacheStatistics.getMultiLevelSBHits());
-			}
 		}
 		finally {
 			OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = old_simplification;
@@ -143,4 +109,4 @@ public class FunctionFullReuseTest extends AutomatedTestBase
 			Recompiler.reinitRecompiler();
 		}
 	}
-}
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/sysds/test/functions/lineage/LineageRewriteTest.java b/src/test/java/org/apache/sysds/test/functions/lineage/LineageRewriteTest.java
index 6850f5c..9942099 100644
--- a/src/test/java/org/apache/sysds/test/functions/lineage/LineageRewriteTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/lineage/LineageRewriteTest.java
@@ -41,6 +41,7 @@ public class LineageRewriteTest extends AutomatedTestBase {
 	protected static final String TEST_NAME6 = "RewriteTest10";
 	protected static final String TEST_NAME7 = "RewriteTest11";
 	protected static final String TEST_NAME8 = "RewriteTest12";
+	protected static final String TEST_NAME9 = "RewriteTest13";
 	
 	protected String TEST_CLASS_DIR = TEST_DIR + LineageRewriteTest.class.getSimpleName() + "/";
 	
@@ -58,6 +59,7 @@ public class LineageRewriteTest extends AutomatedTestBase {
 		addTestConfiguration(TEST_NAME6, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME6));
 		addTestConfiguration(TEST_NAME7, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME7));
 		addTestConfiguration(TEST_NAME8, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME8));
+		addTestConfiguration(TEST_NAME9, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME9));
 	}
 	
 	@Test
@@ -100,6 +102,12 @@ public class LineageRewriteTest extends AutomatedTestBase {
 		testRewrite(TEST_NAME8, false, 2);
 	}
 
+	@Test
+	public void testTsmmCbindOnes() {
+		// This also tests testMatmulCbindRightOnes.
+		testRewrite(TEST_NAME9, false, 0);
+	}
+
 	private void testRewrite(String testname, boolean elementwise, int classes) {
 		try {
 			getAndLoadTestConfiguration(testname);
@@ -130,8 +138,6 @@ public class LineageRewriteTest extends AutomatedTestBase {
 			HashMap<MatrixValue.CellIndex, Double> R_orig = readDMLMatrixFromHDFS("Res");
 
 			proArgs.clear();
-			proArgs.add("-explain");
-			proArgs.add("recompile_hops");
 			proArgs.add("-stats");
 			proArgs.add("-lineage");
 			proArgs.add("reuse_hybrid");
diff --git a/src/test/scripts/functions/lineage/FunctionFullReuse5.dml b/src/test/scripts/functions/lineage/LineageReuseAlg1.dml
similarity index 95%
copy from src/test/scripts/functions/lineage/FunctionFullReuse5.dml
copy to src/test/scripts/functions/lineage/LineageReuseAlg1.dml
index 94ae80e..f87ac40 100644
--- a/src/test/scripts/functions/lineage/FunctionFullReuse5.dml
+++ b/src/test/scripts/functions/lineage/LineageReuseAlg1.dml
@@ -19,8 +19,6 @@
 #
 #-------------------------------------------------------------
 
-# Increase rows and cols for better performance gains
-
 X = rand(rows=100, cols=10, sparsity=1.0, seed=1);
 y = X %*% rand(rows=10, cols=1, sparsity=1.0, seed=1);
 R = matrix(0, 101, 2);
diff --git a/src/test/scripts/functions/lineage/LineageReuseAlg2.dml b/src/test/scripts/functions/lineage/LineageReuseAlg2.dml
new file mode 100644
index 0000000..8639f20
--- /dev/null
+++ b/src/test/scripts/functions/lineage/LineageReuseAlg2.dml
@@ -0,0 +1,60 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+l2norm = function(Matrix[Double] X, Matrix[Double] y, Matrix[Double] B) return (Matrix[Double] loss) {
+  loss = as.matrix(sum((y - X%*%B)^2));
+}
+
+randColSet = function(Matrix[Double] X, Integer seed, Double sample) return (Matrix[Double] Xi) {
+  temp = rand(rows=ncol(X), cols=1, min = 0, max = 1, sparsity=1, seed=seed) <= sample
+  sel = diag(temp)
+  sel = removeEmpty(target = sel, margin = "cols")
+  Xi = X %*% sel
+}
+
+X = rand(rows=100, cols=100, sparsity=1.0, seed=1);
+y = rand(rows=100, cols=1, sparsity=1.0, seed=1);
+
+Rbeta = matrix(0, rows=525, cols=ncol(X)); #nrows = 5*5*3*7 = 525
+k = 1;
+for (i in 1:5) 
+{
+  #randomly select 15% columns in every iteration
+  Xi = randColSet(X, i, 0.15);
+
+  #TODO: Use generalized gridsearch builtin after lineage integration with list.
+  for (h1 in -4:0) {       #reg - values:10^-4 to 10^0
+    for (h2 in 0:2) {      #icpt - range: 0, 1, 2
+      for (h3 in -12:-6) { #tol -values: 10^-12 to 10^-6
+        reg = 10^h1;
+        icpt = h2;
+        tol = 10^h3;
+        beta = lm(X=Xi, y=y, icpt=icpt, reg=reg, tol=tol, maxi=0, verbose=FALSE);
+        Rbeta[k, 1:nrow(beta)] = t(beta);
+        k = k + 1;
+      }
+    }
+  }
+}
+
+while(FALSE) {}
+write(Rbeta, $1, format="text");
+
diff --git a/src/test/scripts/functions/lineage/FunctionFullReuse5.dml b/src/test/scripts/functions/lineage/RewriteTest13.dml
similarity index 73%
rename from src/test/scripts/functions/lineage/FunctionFullReuse5.dml
rename to src/test/scripts/functions/lineage/RewriteTest13.dml
index 94ae80e..c55a173 100644
--- a/src/test/scripts/functions/lineage/FunctionFullReuse5.dml
+++ b/src/test/scripts/functions/lineage/RewriteTest13.dml
@@ -19,15 +19,21 @@
 #
 #-------------------------------------------------------------
 
-# Increase rows and cols for better performance gains
+X = read($1);
+y = rand(rows=nrow(X), cols=1, sparsity=1.0, seed=2);
 
-X = rand(rows=100, cols=10, sparsity=1.0, seed=1);
-y = X %*% rand(rows=10, cols=1, sparsity=1.0, seed=1);
-R = matrix(0, 101, 2);
+R = matrix(0, 1, 1);
 
-[C, S] = steplm(X=X, y=y, icpt=2);
-S = cbind(S, matrix(1, 1, 1));
-R[1:nrow(C) ,1] = C;
-R[1:ncol(S) ,2] = t(S);
+Res1 = t(X) %*% X;
+Res2 = t(X) %*% y;
 
-write(R, $1, format="text");
+while(FALSE) {}
+
+X = cbind(X, matrix(1, rows=nrow(X), cols=1));
+Res11 = t(X) %*% X;
+Res22 = t(X) %*% y;
+while(FALSE) {}
+R[1,1] = sum(Res1) + sum(Res2) + sum(Res11) + sum(Res22);
+
+
+write(R, $3, format="text");