You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by ba...@apache.org on 2021/09/20 11:27:36 UTC

[systemds] branch master updated (f80c238 -> 61f40bd)

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

baunsgaard pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/systemds.git.


    from f80c238  [MINOR] Fix robustness sparsity-related component tests
     new 67bb803  [SYSTEMDS-2610] CLA updates
     new e607544  [SYSTEMDS-3123] Rewrite c bind 0 Matrix Multiplication
     new 61f40bd  [SYSTEMDS-2990] Workload tree move decompression to input hop

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/sysds/hops/OptimizerUtils.java |   3 +
 src/main/java/org/apache/sysds/hops/UnaryOp.java   |  12 +-
 .../sysds/hops/estim/EstimatorDensityMap.java      |   4 +-
 .../ipa/IPAPassCompressionWorkloadAnalysis.java    |  21 +-
 .../RewriteAlgebraicSimplificationDynamic.java     |  58 +++-
 .../hops/rewrite/RewriteCompressedReblock.java     |   6 +-
 .../runtime/compress/CompressedMatrixBlock.java    | 136 ++++++--
 .../compress/CompressedMatrixBlockFactory.java     | 108 ++++++-
 .../runtime/compress/CompressionSettings.java      |  14 +-
 .../compress/CompressionSettingsBuilder.java       |   8 +-
 .../runtime/compress/CompressionStatistics.java    |   4 +-
 .../runtime/compress/DMLCompressionException.java  |  12 +-
 .../runtime/compress/cocode/AColumnCoCoder.java    |  22 +-
 .../runtime/compress/cocode/CoCodeBinPacking.java  |   4 +-
 .../runtime/compress/cocode/CoCodeGreedy.java      |  10 +-
 .../runtime/compress/cocode/CoCodePriorityQue.java |   4 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |   4 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  20 +-
 .../runtime/compress/colgroup/ColGroupValue.java   |   6 -
 .../compress/colgroup/mapping/MapToFactory.java    |  17 +-
 .../compress/cost/ComputationCostEstimator.java    |  48 ++-
 .../compress/cost/CostEstimatorBuilder.java        |  39 ++-
 .../compress/cost/CostEstimatorFactory.java        |  10 +-
 .../compress/cost/InstructionTypeCounter.java      |   2 +
 .../compress/estim/CompressedSizeEstimator.java    |  43 ++-
 .../estim/CompressedSizeEstimatorExact.java        |   8 +-
 .../estim/CompressedSizeEstimatorFactory.java      |  30 +-
 .../estim/CompressedSizeEstimatorSample.java       |  14 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |   4 +-
 .../runtime/compress/estim/EstimationFactors.java  |   2 +-
 .../sysds/runtime/compress/lib/CLALibAppend.java   |  31 +-
 .../runtime/compress/lib/CLALibBinaryCellOp.java   | 136 +++++---
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |  60 ++--
 .../runtime/compress/lib/CLALibLeftMultBy.java     | 102 +++---
 .../runtime/compress/lib/CLALibRelationalOp.java   | 267 ----------------
 .../runtime/compress/lib/CLALibRightMultBy.java    |  82 ++---
 .../sysds/runtime/compress/lib/CLALibScalar.java   |   7 -
 .../readers/ReaderCompressedSelection.java         |   2 -
 .../runtime/compress/workload/AWTreeNode.java      |   6 +-
 .../apache/sysds/runtime/compress/workload/Op.java |  33 +-
 .../runtime/compress/workload/OpDecompressing.java |  39 ---
 .../runtime/compress/workload/OpMetadata.java      |  15 +-
 .../sysds/runtime/compress/workload/OpNormal.java  |   5 -
 .../compress/workload/OpOverlappingDecompress.java |  38 ---
 .../sysds/runtime/compress/workload/OpSided.java   |  15 +-
 .../sysds/runtime/compress/workload/WTreeRoot.java |  31 +-
 .../compress/workload/WorkloadAnalyzer.java        | 351 +++++++++++++--------
 .../context/SparkExecutionContext.java             |  14 +-
 .../instructions/cp/SpoofCPInstruction.java        |   3 +-
 .../spark/BinUaggChainSPInstruction.java           |   6 +
 .../runtime/matrix/data/LibMatrixBincell.java      |  10 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |  96 +++---
 .../apache/sysds/runtime/util/DataConverter.java   |   2 +-
 .../compress/AbstractCompressedUnaryTests.java     |   2 -
 .../component/compress/CompressedMatrixTest.java   | 194 ++++++++++++
 .../component/compress/CompressedTestBase.java     |  36 +--
 .../component/compress/workload/WorkloadTest.java  |  49 +--
 .../test/component/frame/DataCorruptionTest.java   |   8 +-
 .../test/functions/codegen/RowAggTmplTest.java     |   6 +-
 .../functions/compress/CompressInstruction.java    |   1 -
 .../compress/CompressInstructionRewrite.java       |  19 +-
 .../functions/compress/CompressRewriteSpark.java   |  43 ++-
 .../compress/configuration/CompressBase.java       |  16 +-
 .../compress/configuration/CompressCost.java       |  70 ----
 .../compress/configuration/CompressForce.java      |  36 +++
 .../compress/configuration/CompressLossy.java      |  52 ---
 .../compress/configuration/CompressLossyCost.java  |  52 ---
 .../compress/workload/WorkloadAlgorithmTest.java   |  40 ++-
 .../compress/workload/WorkloadAnalysisTest.java    |   9 +-
 .../rewrite/RewriteMMCBindZeroVector.java          | 155 +++++++++
 .../component/compress/{1-1.csv => 1-1_y.csv}      |   0
 .../compress/{1-1.csv.mtd => 1-1_y.csv.mtd}        |   2 +-
 src/test/resources/component/compress/README.md    |   2 +-
 .../workload/functions/{l2svm.dml => l2svm_Y.dml}  |   4 +-
 .../functions/compress/compress_mmr_sum.dml        |   2 +-
 ...press_mmr_sum.dml => compress_mmr_sum_plus.dml} |   2 +-
 ...ess_mmr_sum.dml => compress_mmr_sum_plus_2.dml} |   2 +-
 ...dAnalysisLmCG.dml => WorkloadAnalysisL2SVM.dml} |   6 +-
 .../compress/workload/WorkloadAnalysisMLogReg.dml  |  13 +-
 .../RewritMMCBindZeroVectorOp.dml}                 |   9 +-
 80 files changed, 1594 insertions(+), 1260 deletions(-)
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/workload/OpDecompressing.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/workload/OpOverlappingDecompress.java
 delete mode 100644 src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressCost.java
 delete mode 100644 src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossy.java
 delete mode 100644 src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossyCost.java
 create mode 100644 src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
 copy src/test/resources/component/compress/{1-1.csv => 1-1_y.csv} (100%)
 copy src/test/resources/component/compress/{1-1.csv.mtd => 1-1_y.csv.mtd} (88%)
 copy src/test/scripts/component/compress/workload/functions/{l2svm.dml => l2svm_Y.dml} (93%)
 copy src/test/scripts/functions/compress/{compress_mmr_sum.dml => compress_mmr_sum_plus.dml} (98%)
 copy src/test/scripts/functions/compress/{compress_mmr_sum.dml => compress_mmr_sum_plus_2.dml} (98%)
 copy src/test/scripts/functions/compress/workload/{WorkloadAnalysisLmCG.dml => WorkloadAnalysisL2SVM.dml} (90%)
 copy src/test/scripts/functions/{codegen/SumAdditionChain.dml => rewrite/RewritMMCBindZeroVectorOp.dml} (92%)

[systemds] 01/03: [SYSTEMDS-2610] CLA updates

Posted by ba...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 67bb8034611a766ca93c0314ee735a3252e79290
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Mon Aug 30 16:28:31 2021 +0200

    [SYSTEMDS-2610] CLA updates
    
    - Compressed matrix factory improvements
    - Add decompression if the data is serialized and larger in compressed format
    - Decompress on write to HDFS
    - Abort compression after cocode if the compression sizes are bad
    - Make c bind decompressing in workload tree (worst case)
    - Add a minimum compression ratio argument to the CompressionSettings
    - Reduce the sampling size in c bind compression and set high minimum compression ratio
    - Fix order of operations in compressed append
    - Add compressed output size to unary hops
    - More utilization of the cached decompressed matrix if it fits in
    memory by looking for soft reference of uncompressed in certain cases
---
 src/main/java/org/apache/sysds/hops/UnaryOp.java   |  12 +-
 .../ipa/IPAPassCompressionWorkloadAnalysis.java    |   1 +
 .../hops/rewrite/RewriteCompressedReblock.java     |   2 +-
 .../runtime/compress/CompressedMatrixBlock.java    |  29 ++-
 .../compress/CompressedMatrixBlockFactory.java     |  98 +++++++-
 .../runtime/compress/CompressionSettings.java      |   8 +-
 .../compress/CompressionSettingsBuilder.java       |   8 +-
 .../runtime/compress/cocode/AColumnCoCoder.java    |  22 +-
 .../runtime/compress/cocode/CoCodeBinPacking.java  |   4 +-
 .../runtime/compress/cocode/CoCodeGreedy.java      |   2 +-
 .../runtime/compress/cocode/CoCodePriorityQue.java |   4 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  19 +-
 .../compress/colgroup/mapping/MapToFactory.java    |  12 +-
 .../compress/cost/CostEstimatorBuilder.java        |   5 +
 .../compress/cost/CostEstimatorFactory.java        |   2 +-
 .../compress/estim/CompressedSizeEstimator.java    |  43 +++-
 .../estim/CompressedSizeEstimatorExact.java        |   5 +-
 .../estim/CompressedSizeEstimatorFactory.java      |  30 ++-
 .../estim/CompressedSizeEstimatorSample.java       |  11 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |   2 +-
 .../sysds/runtime/compress/lib/CLALibAppend.java   |  32 +--
 .../runtime/compress/lib/CLALibBinaryCellOp.java   |  42 ++--
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |  51 ++--
 .../runtime/compress/lib/CLALibRelationalOp.java   | 267 ---------------------
 .../sysds/runtime/compress/lib/CLALibScalar.java   |   7 -
 .../compress/workload/WorkloadAnalyzer.java        |   5 +-
 .../context/SparkExecutionContext.java             |   8 +-
 .../spark/BinUaggChainSPInstruction.java           |   6 +
 .../component/compress/CompressedMatrixTest.java   |  33 +++
 .../component/compress/CompressedTestBase.java     |  27 ---
 .../component/compress/workload/WorkloadTest.java  |  18 +-
 .../compress/workload/WorkloadAlgorithmTest.java   |   2 +-
 32 files changed, 379 insertions(+), 438 deletions(-)

diff --git a/src/main/java/org/apache/sysds/hops/UnaryOp.java b/src/main/java/org/apache/sysds/hops/UnaryOp.java
index b7df277..38199b2 100644
--- a/src/main/java/org/apache/sysds/hops/UnaryOp.java
+++ b/src/main/java/org/apache/sysds/hops/UnaryOp.java
@@ -545,7 +545,7 @@ public class UnaryOp extends MultiThreadedHop
 			setDim2(1);
 		}
 		else if(_op == OpOp1.TYPEOF || _op == OpOp1.DETECTSCHEMA || _op == OpOp1.COLNAMES) {
-			//TODO theses three builtins should rather be moved to unary aggregates
+			//TODO these three builtins should rather be moved to unary aggregates
 			setDim1(1);
 			setDim2(input.getDim2());
 		}
@@ -564,6 +564,16 @@ public class UnaryOp extends MultiThreadedHop
 			{
 				setNnz( input.getNnz() );
 			}
+
+			// if the input is compressed then set the output to be compressed as well.
+			if(input._compressedOutput && ! (_op==OpOp1.DECOMPRESS)){
+				setCompressedOutput(true);
+				// Setting the compressed output to be 2 x larger.
+				// Just in case we change the compressed structure slightly.
+				// this value is overwritten with correct size once the hop is executed
+				// TODO handle overlapping state, since some operations would not lead to compressed output.
+				setCompressedSize(input.compressedSize() * 2);
+			}
 		}
 	}
 	
diff --git a/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java b/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
index 324b272..b23397f 100644
--- a/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
+++ b/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
@@ -58,6 +58,7 @@ public class IPAPassCompressionWorkloadAnalysis extends IPAPass {
 			WTreeRoot tree = e.getValue();
 			CostEstimatorBuilder b = new CostEstimatorBuilder(tree);
 			// filter out compression plans that is known bad
+			
 			if(b.shouldTryToCompress()){
 				tree.getRoot().setRequiresCompression(tree);
 				for(Hop h : tree.getDecompressList())
diff --git a/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java b/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
index 6b51a51..2f0f1f6c 100644
--- a/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
+++ b/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
@@ -128,7 +128,7 @@ public class RewriteCompressedReblock extends StatementBlockRewriteRule {
 
 	public static boolean satisfiesSizeConstraintsForCompression(Hop hop) {
 		if(hop.getDim2() >= 1) {
-			return (hop.getDim1() >= 1000 && hop.getDim2() < 100) || hop.getDim1() / hop.getDim2() >= 75;
+			return (hop.getDim1() >= 1000 && hop.getDim2() < 100) || hop.getDim1() / hop.getDim2() >= 75 || (hop.getSparsity() < 0.0001 && hop.getDim1() > 1000);
 		}
 		return false;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
index d374d3c..5dcc406 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -302,7 +302,10 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		return ret;
 	}
 
-	private MatrixBlock getCachedDecompressed() {
+	/**
+	 * Get the cached decompressed matrix (if it exists otherwise null)
+	 */
+	public MatrixBlock getCachedDecompressed() {
 		if(decompressedVersion != null) {
 			final MatrixBlock mb = decompressedVersion.get();
 			if(mb != null) {
@@ -453,6 +456,16 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public void write(DataOutput out) throws IOException {
+		if(getExactSizeOnDisk() > MatrixBlock.estimateSizeOnDisk(rlen, clen, nonZeros)) {
+			// decompress and make a uncompressed column group. this is then used for the serialization, since it is
+			// smaller.
+			// throw new NotImplementedException("Decompressing serialization is not implemented");
+
+			MatrixBlock uncompressed = getUncompressed("Decompressing serialization for smaller serialization");
+			ColGroupUncompressed cg = new ColGroupUncompressed(uncompressed);
+			allocateColGroup(cg);
+			nonZeros = cg.getNumberNonZeros();
+		}
 		// serialize compressed matrix block
 		out.writeInt(rlen);
 		out.writeInt(clen);
@@ -492,11 +505,15 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock binaryOperations(BinaryOperator op, MatrixValue thatValue, MatrixValue result) {
-		return CLALibBinaryCellOp.binaryOperations(op, this, thatValue, result);
+		MatrixBlock that = thatValue == null ? null : (MatrixBlock) thatValue;
+		MatrixBlock ret = result == null ? null : (MatrixBlock) result;
+		return CLALibBinaryCellOp.binaryOperations(op, this, that, ret);
 	}
 
 	public MatrixBlock binaryOperationsLeft(BinaryOperator op, MatrixValue thatValue, MatrixValue result) {
-		return CLALibBinaryCellOp.binaryOperationsLeft(op, this, thatValue, result);
+		MatrixBlock that = thatValue == null ? null : (MatrixBlock) thatValue;
+		MatrixBlock ret = result == null ? null : (MatrixBlock) result;
+		return CLALibBinaryCellOp.binaryOperationsLeft(op, this, that, ret);
 	}
 
 	@Override
@@ -686,8 +703,8 @@ public class CompressedMatrixBlock extends MatrixBlock {
 				.aggregateUnaryOperations(op, result, blen, indexesIn, inCP);
 
 		}
-
-		return CLALibCompAgg.aggregateUnary(this, result, op, blen, indexesIn, inCP);
+		MatrixBlock ret = (result == null) ? null : (MatrixBlock) result;
+		return CLALibCompAgg.aggregateUnary(this, ret, op, blen, indexesIn, inCP);
 	}
 
 	@Override
@@ -1080,7 +1097,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		boolean m2C = m2 instanceof CompressedMatrixBlock;
 		boolean m3C = m3 instanceof CompressedMatrixBlock;
 		printDecompressWarning("aggregateTernaryOperations " + op.aggOp.getClass().getSimpleName() + " "
-			+ op.indexFn.getClass().getSimpleName() + "  " + op.aggOp.increOp.fn.getClass().getSimpleName() + " "
+			+ op.indexFn.getClass().getSimpleName() + " " + op.aggOp.increOp.fn.getClass().getSimpleName() + " "
 			+ op.binaryFn.getClass().getSimpleName() + " m1,m2,m3 " + m1C + " " + m2C + " " + m3C);
 		MatrixBlock left = getUncompressed();
 		MatrixBlock right1 = getUncompressed(m2);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
index 20beaf5..5f45d56 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -32,18 +32,21 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupEmpty;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupFactory;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
+import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.compress.cost.CostEstimatorFactory;
 import org.apache.sysds.runtime.compress.cost.ICostEstimate;
-import org.apache.sysds.runtime.compress.cost.MemoryCostEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorFactory;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.utils.DblArrayIntListHashMap;
 import org.apache.sysds.runtime.compress.utils.DoubleCountHashMap;
 import org.apache.sysds.runtime.compress.workload.WTreeRoot;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
+import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.DMLCompressionStatistics;
@@ -65,7 +68,7 @@ public class CompressedMatrixBlockFactory {
 	private final CompressionSettings compSettings;
 	/** The main cost estimator used for the compression */
 	private final ICostEstimate costEstimator;
-	
+
 	/** Time stamp of last phase */
 	private double lastPhase;
 	/** Pointer to the original matrix Block that is about to be compressed. */
@@ -153,6 +156,20 @@ public class CompressedMatrixBlockFactory {
 	}
 
 	/**
+	 * Generate a CompressedMatrixBlock Object that contains a single uncompressed matrix block column group.
+	 * 
+	 * @param mb The matrix block to be contained in the uncompressed matrix block column,
+	 * @return a CompressedMatrixBlock
+	 */
+	public static CompressedMatrixBlock genUncompressedCompressedMatrixBlock(MatrixBlock mb) {
+		CompressedMatrixBlock ret = new CompressedMatrixBlock(mb.getNumRows(), mb.getNumColumns());
+		AColGroup cg = new ColGroupUncompressed(mb);
+		ret.allocateColGroup(cg);
+		ret.setNonZeros(mb.getNonZeros());
+		return ret;
+	}
+
+	/**
 	 * Method for constructing a compressed matrix out of an constant input.
 	 * 
 	 * Since the input is a constant value it is trivially compressable, therefore we skip the entire compression
@@ -191,9 +208,13 @@ public class CompressedMatrixBlockFactory {
 
 		res = new CompressedMatrixBlock(mb); // copy metadata and allocate soft reference
 
-		classifyPhase();
-		if(coCodeColGroups == null)
-			return abortCompression();
+		looksLikeOneHot();
+
+		if(coCodeColGroups == null) {
+			classifyPhase();
+			if(coCodeColGroups == null)
+				return abortCompression();
+		}
 
 		transposePhase();
 		compressPhase();
@@ -217,7 +238,13 @@ public class CompressedMatrixBlockFactory {
 		_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
 		logPhase();
 
-		if(!(costEstimator instanceof MemoryCostEstimator) || _stats.estimatedSizeCols < _stats.originalSize)
+		final boolean isValidForComputeBasedCompression = isComputeBasedCompression() &&
+			(compSettings.minimumCompressionRatio != 1.0) ? _stats.estimatedSizeCols *
+				compSettings.minimumCompressionRatio < _stats.originalSize : true;
+		final boolean isValidForMemoryBasedCompression = _stats.estimatedSizeCols *
+			compSettings.minimumCompressionRatio < _stats.originalSize;
+
+		if(isValidForComputeBasedCompression || isValidForMemoryBasedCompression)
 			coCodePhase(sizeEstimator, sizeInfos, costEstimator);
 		else {
 			LOG.info("Estimated Size of singleColGroups: " + _stats.estimatedSizeCols);
@@ -225,13 +252,72 @@ public class CompressedMatrixBlockFactory {
 		}
 	}
 
+	private boolean isComputeBasedCompression() {
+		return costEstimator instanceof ComputationCostEstimator;
+	}
+
 	private void coCodePhase(CompressedSizeEstimator sizeEstimator, CompressedSizeInfo sizeInfos,
 		ICostEstimate costEstimator) {
 		coCodeColGroups = CoCoderFactory.findCoCodesByPartitioning(sizeEstimator, sizeInfos, k, costEstimator,
 			compSettings);
 
 		_stats.estimatedSizeCoCoded = coCodeColGroups.memoryEstimate();
+
 		logPhase();
+
+		// if cocode is estimated larger than uncompressed abort compression.
+		if(isComputeBasedCompression() &&
+			_stats.estimatedSizeCoCoded * compSettings.minimumCompressionRatio > _stats.originalSize) {
+
+			coCodeColGroups = null;
+			LOG.info("Aborting compression because the cocoded size : " + _stats.estimatedSizeCoCoded);
+			LOG.info("Vs original size                              : " + _stats.originalSize);
+		}
+
+	}
+
+	private void looksLikeOneHot() {
+		final int numColumns = mb.getNumColumns();
+		final int numRows = mb.getNumRows();
+		final long nnz = mb.getNonZeros();
+		final int colGroupSize = 100;
+		if(nnz == numRows) {
+			boolean onlyOneValues = true;
+			LOG.debug("Looks like one hot encoded.");
+			if(mb.isInSparseFormat()) {
+				final SparseBlock sb = mb.getSparseBlock();
+				for(double v : sb.get(0).values()) {
+					onlyOneValues = v == 1.0;
+					if(!onlyOneValues) {
+						break;
+					}
+				}
+			}
+			else {
+				final double[] vals = mb.getDenseBlock().values(0);
+				for(int i = 0; i < Math.min(vals.length, 1000); i++) {
+					double v = vals[i];
+					onlyOneValues = v == 1.0 || v == 0.0;
+					if(!onlyOneValues) {
+						break;
+					}
+				}
+			}
+			if(onlyOneValues) {
+				List<CompressedSizeInfoColGroup> ng = new ArrayList<>(numColumns / colGroupSize + 1);
+				for(int i = 0; i < numColumns; i += colGroupSize) {
+					int[] columnIds = new int[Math.min(colGroupSize, numColumns - i)];
+					for(int j = 0; j < columnIds.length; j++)
+						columnIds[j] = i + j;
+					ng.add(new CompressedSizeInfoColGroup(columnIds, Math.min(numColumns, colGroupSize), numRows));
+				}
+				coCodeColGroups = new CompressedSizeInfo(ng);
+
+				LOG.debug("Concluded that it probably is one hot encoded skipping analysis");
+				// skipping two phases
+				phase += 2;
+			}
+		}
 	}
 
 	private void transposePhase() {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
index 6dfcdae..ea04e8e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -103,10 +103,15 @@ public class CompressionSettings {
 	 */
 	public boolean transposed = false;
 
+	/**
+	 * The minimum compression ratio to achieve.
+	 */
+	public final double minimumCompressionRatio;
+
 	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput,
 		 int seed, boolean lossy, EnumSet<CompressionType> validCompressions,
 		boolean sortValuesByLength, PartitionerType columnPartitioner, int maxColGroupCoCode, double coCodePercentage,
-		int minimumSampleSize, EstimationType estimationType, CostType costComputationType) {
+		int minimumSampleSize, EstimationType estimationType, CostType costComputationType, double minimumCompressionRatio) {
 		this.samplingRatio = samplingRatio;
 		this.allowSharedDictionary = allowSharedDictionary;
 		this.transposeInput = transposeInput;
@@ -120,6 +125,7 @@ public class CompressionSettings {
 		this.minimumSampleSize = minimumSampleSize;
 		this.estimationType = estimationType;
 		this.costComputationType = costComputationType;
+		this.minimumCompressionRatio = minimumCompressionRatio;
 		if(LOG.isDebugEnabled())
 			LOG.debug(this);
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index a56285b..2864118 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -45,6 +45,7 @@ public class CompressionSettingsBuilder {
 	private EstimationType estimationType = EstimationType.HassAndStokes;
 	private PartitionerType columnPartitioner;
 	private CostType costType;
+	private double minimumCompressionRatio = 1.0;
 
 	public CompressionSettingsBuilder() {
 
@@ -267,6 +268,11 @@ public class CompressionSettingsBuilder {
 		return this;
 	}
 
+	public CompressionSettingsBuilder setMinimumCompressionRatio(double ratio){
+		this.minimumCompressionRatio = ratio;
+		return this;
+	}
+
 	/**
 	 * Create the CompressionSettings object to use in the compression.
 	 * 
@@ -275,6 +281,6 @@ public class CompressionSettingsBuilder {
 	public CompressionSettings create() {
 		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, seed, lossy,
 			validCompressions, sortValuesByLength, columnPartitioner, maxColGroupCoCode, coCodePercentage,
-			minimumSampleSize, estimationType, costType);
+			minimumSampleSize, estimationType, costType,minimumCompressionRatio);
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
index 41d986c..3bdcf5d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
@@ -26,7 +26,6 @@ import org.apache.sysds.runtime.compress.cost.ICostEstimate;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
-import org.apache.sysds.runtime.compress.utils.Util;
 
 public abstract class AColumnCoCoder {
 
@@ -52,26 +51,17 @@ public abstract class AColumnCoCoder {
 	 */
 	protected abstract CompressedSizeInfo coCodeColumns(CompressedSizeInfo colInfos, int k);
 
-	protected CompressedSizeInfoColGroup join(CompressedSizeInfoColGroup lhs, CompressedSizeInfoColGroup rhs,
-		boolean analyze) {
-		return analyze ? joinWithAnalysis(lhs, rhs) : joinWithoutAnalysis(lhs, rhs);
+	protected CompressedSizeInfoColGroup join(int[] joined, CompressedSizeInfoColGroup lhs,
+		CompressedSizeInfoColGroup rhs, boolean analyze) {
+		return analyze ? _sest.estimateJoinCompressedSize(joined, lhs, rhs) : joinWithoutAnalysis(joined, lhs, rhs);
 	}
 
-	protected CompressedSizeInfoColGroup joinWithAnalysis(CompressedSizeInfoColGroup lhs,
+	protected CompressedSizeInfoColGroup joinWithoutAnalysis(int[] joined, CompressedSizeInfoColGroup lhs,
 		CompressedSizeInfoColGroup rhs) {
-		return _sest.estimateJoinCompressedSize(lhs, rhs);
-	}
-
-	protected CompressedSizeInfoColGroup joinWithoutAnalysis(CompressedSizeInfoColGroup lhs,
-		CompressedSizeInfoColGroup rhs) {
-		int[] joined = Util.join(lhs.getColumns(), rhs.getColumns());
 		final int lhsV = lhs.getNumVals();
 		final int rhsV = rhs.getNumVals();
-		final int numVals = lhsV * rhsV;
-		if(numVals < 0 || numVals > _sest.getNumRows())
-			return null;
-		else
-			return new CompressedSizeInfoColGroup(joined, numVals, _sest.getNumRows());
+		final int joinedMaxDistinct = (int) Math.min((long) lhsV * (long) rhsV, (long) _sest.getNumRows());
+		return new CompressedSizeInfoColGroup(joined, joinedMaxDistinct, _sest.getNumRows());
 	}
 
 	protected CompressedSizeInfoColGroup analyze(CompressedSizeInfoColGroup g) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
index 1887bdc..1731ef2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
@@ -141,7 +141,7 @@ public class CoCodeBinPacking extends AColumnCoCoder {
 			for(int j = 0; j < bins.size(); j++) {
 				double newBinWeight = binWeights[j] - c.getCardinalityRatio();
 				if(newBinWeight >= 0 && bins.get(j).getColumns().length < MAX_COL_PER_GROUP - 1) {
-					bins.set(j, joinWithoutAnalysis(bins.get(j), c));
+					bins.set(j, joinWithoutAnalysis(Util.join(bins.get(j).getColumns(), c.getColumns()),bins.get(j), c));
 					binWeights[j] = newBinWeight;
 					assigned = true;
 					break;
@@ -291,7 +291,7 @@ public class CoCodeBinPacking extends AColumnCoCoder {
 					g = CompressedSizeInfoColGroup.addConstGroup(c, left, _cs.validCompressions);
 				else {
 					st3++;
-					g = _sest.estimateJoinCompressedSize(left, right);
+					g = _sest.estimateJoinCompressedSize(c, left, right);
 				}
 
 				if(leftConst || rightConst)
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
index 51bfa46..3091ba0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
@@ -163,7 +163,7 @@ public class CoCodeGreedy extends AColumnCoCoder {
 					g = CompressedSizeInfoColGroup.addConstGroup(c, left, _cs.validCompressions);
 				else {
 					st3++;
-					g = _sest.estimateJoinCompressedSize(left, right);
+					g = _sest.estimateJoinCompressedSize(c, left, right);
 				}
 
 				if(leftConst || rightConst)
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodePriorityQue.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodePriorityQue.java
index 2e0e7fb..27b678c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodePriorityQue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodePriorityQue.java
@@ -68,7 +68,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 			while(que.peek() != null) {
 
 				CompressedSizeInfoColGroup r = que.poll();
-				final CompressedSizeInfoColGroup g = joinWithAnalysis(l, r);
+				final CompressedSizeInfoColGroup g = _sest.estimateJoinCompressedSize(l, r);
 				if(g != null) {
 					final double costOfJoin = _cest.getCostOfCollectionOfGroups(que, g);
 					if(costOfJoin < costBeforeJoin) {
@@ -93,7 +93,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 			while(que.peek() != null) {
 				CompressedSizeInfoColGroup r = que.peek();
 				if(_cest.shouldTryJoin(l, r)) {
-					CompressedSizeInfoColGroup g = joinWithAnalysis(l, r);
+					CompressedSizeInfoColGroup g = _sest.estimateJoinCompressedSize(l, r);
 					if(g != null) {
 						double costOfJoin = _cest.getCostOfColumnGroup(g);
 						double costIndividual = _cest.getCostOfColumnGroup(l) + _cest.getCostOfColumnGroup(r);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
index ab94646..b1b0540 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
@@ -132,6 +132,23 @@ public class ColGroupUncompressed extends AColGroup {
 		_data = data;
 	}
 
+	/**
+	 * Constructor for allocating a single uncompressed column group.
+	 * 
+	 * @param data matrix block
+	 */
+	public ColGroupUncompressed(MatrixBlock data) {
+		super(generateColumnList(data.getNumColumns()));
+		_data = data;
+	}
+
+	private static int[] generateColumnList(int nCol){
+		int[] cols = new int[nCol];
+		for(int i = 0; i< nCol; i++)
+			cols[i] = i;
+		return cols;
+	}
+
 	@Override
 	public CompressionType getCompType() {
 		return CompressionType.UNCOMPRESSED;
@@ -632,7 +649,7 @@ public class ColGroupUncompressed extends AColGroup {
 			double[] dv = colSum.getDenseBlockValues();
 			for(int i = 0; i < _colIndexes.length; i++)
 				c[_colIndexes[i]] += dv[i];
-			
+
 		}
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
index 5a72ff7..6c0b9fa 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
@@ -108,16 +108,13 @@ public final class MapToFactory {
 		final int nVL = left.getUnique();
 		final int nVR = right.getUnique();
 		final int size = left.size();
-		final int maxUnique = nVL * nVR;
+		final long maxUnique = nVL * nVR;
+		if(maxUnique > (long)Integer.MAX_VALUE)
+			throw new DMLCompressionException("Joining impossible using linearized join, since each side has a large number of unique values");
 		if(size != right.size())
 			throw new DMLCompressionException("Invalid input maps to join, must contain same number of rows");
 
-		try {
-			return computeJoin(left, right, size, nVL, maxUnique);
-		}
-		catch(Exception e) {
-			throw new DMLCompressionException("Joining failed max unique expected:" + maxUnique, e);
-		}
+		return computeJoin(left, right, size, nVL, (int)maxUnique);
 	}
 
 	private static AMapToData computeJoin(AMapToData left, AMapToData right, int size, int nVL, int maxUnique) {
@@ -141,7 +138,6 @@ public final class MapToFactory {
 		}
 
 		tmp.setUnique(newUID-1);
-		// LOG.error(Arrays.toString(map));
 		return tmp;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
index d7f305c..3d58bff 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
@@ -125,6 +125,11 @@ public final class CostEstimatorBuilder implements Serializable {
 		numberOps += counter.scans + counter.leftMultiplications * 2 + counter.rightMultiplications +
 			counter.compressedMultiplications * 4 + counter.dictionaryOps;
 		numberOps -= counter.decompressions + counter.overlappingDecompressions;
+
+		if(counter.decompressions > 1 &&
+			counter.leftMultiplications + counter.rightMultiplications + counter.compressedMultiplications < 1)
+			// This condition is added for l2svm and mLogReg y dataset, that is compressing while it should not.
+			return false;
 		return numberOps > 4;
 
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
index e8bff06..83b794e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
@@ -45,7 +45,7 @@ public final class CostEstimatorFactory {
 					return b.create(nRows, nCols);
 				}
 				else
-					return new DistinctCostEstimator(nRows, cs);
+					return new MemoryCostEstimator();
 			case MEMORY:
 			default:
 				return new MemoryCostEstimator();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
index 3dd092c..ef7d9dd 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
@@ -82,7 +82,7 @@ public abstract class CompressedSizeEstimator {
 		return _numCols;
 	}
 
-	public MatrixBlock getData(){
+	public MatrixBlock getData() {
 		return _data;
 	}
 
@@ -163,7 +163,7 @@ public abstract class CompressedSizeEstimator {
 	 * @param colIndexes The columns to group together inside a ColGroup
 	 * @return The CompressedSizeInformation associated with the selected ColGroups.
 	 */
-	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes){
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
 		return estimateCompressedColGroupSize(colIndexes, 8, getNumRows());
 	}
 
@@ -173,7 +173,7 @@ public abstract class CompressedSizeEstimator {
 	 * the number estimated in sub groups of the given colIndexes.
 	 * 
 	 * @param colIndexes         The columns to extract compression information from
-	 * @param estimate 			 An estimate of number of unique elements in these columns
+	 * @param estimate           An estimate of number of unique elements in these columns
 	 * @param nrUniqueUpperBound The upper bound of unique elements allowed in the estimate, can be calculated from the
 	 *                           number of unique elements estimated in sub columns multiplied together. This is
 	 *                           flexible in the sense that if the sample is small then this unique can be manually
@@ -181,13 +181,16 @@ public abstract class CompressedSizeEstimator {
 	 * 
 	 * @return The CompressedSizeInfoColGroup fro the given column indexes.
 	 */
-	public abstract CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate, int nrUniqueUpperBound);
+	public abstract CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate,
+		int nrUniqueUpperBound);
 
 	/**
 	 * Join two analyzed column groups together. without materializing the dictionaries of either side.
 	 * 
-	 * If either side was constructed without analysis then fall back to default materialization of double arrays.
+	 * if the number of distinct elements in both sides multiplied is larger than Integer, return null.
 	 * 
+	 * If either side was constructed without analysis then fall back to default materialization of double arrays.
+	 * O
 	 * @param g1 First group
 	 * @param g2 Second group
 	 * @return A joined compressed size estimation for the group.
@@ -195,19 +198,39 @@ public abstract class CompressedSizeEstimator {
 	public CompressedSizeInfoColGroup estimateJoinCompressedSize(CompressedSizeInfoColGroup g1,
 		CompressedSizeInfoColGroup g2) {
 		final int[] joined = Util.join(g1.getColumns(), g2.getColumns());
+		return estimateJoinCompressedSize(joined, g1, g2);
+	}
+
+	/**
+	 * Join two analyzed column groups together. without materializing the dictionaries of either side.
+	 * 
+	 * if the number of distinct elements in both sides multiplied is larger than Integer, return null.
+	 * 
+	 * If either side was constructed without analysis then fall back to default materialization of double arrays.
+	 * 
+	 * @param joined The joined column indexes.
+	 * @param g1     First group
+	 * @param g2     Second group
+	 * @return A joined compressed size estimation for the group.
+	 */
+	public CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
+		CompressedSizeInfoColGroup g2) {
 		final int g1V = g1.getNumVals();
 		final int g2V = g2.getNumVals();
-		if(g1V * g2V < 0 || g1V * g2V > getNumRows())
+		if((long) g1V * g2V > (long) Integer.MAX_VALUE)
 			return null;
-		else if((g1.getMap() == null && g2V != 0) || (g2.getMap() == null && g2V != 0))
-			return estimateCompressedColGroupSize(joined, Math.max(g1V + 1, g2V+ 1), Math.min((g1V + 1) * (g2V + 1), getNumRows()));
+
+		final int joinedMaxDistinct = (int) Math.min((long) g1V * (long) g2V, getNumRows());
+		if((g1.getMap() == null && g2V != 0) || (g2.getMap() == null && g2V != 0))
+			return estimateCompressedColGroupSize(joined, Math.max(g1V + 1, g2V + 1),
+				Math.min((g1V + 1) * (g2V + 1), getNumRows()));
 		else
-			return estimateJoinCompressedSize(joined, g1, g2);
+			return estimateJoinCompressedSize(joined, g1, g2, joinedMaxDistinct);
 
 	}
 
 	protected abstract CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joinedcols,
-		CompressedSizeInfoColGroup g1, CompressedSizeInfoColGroup g2);
+		CompressedSizeInfoColGroup g1, CompressedSizeInfoColGroup g2, int joinedMaxDistinct);
 
 	/**
 	 * Method used to extract the CompressedSizeEstimationFactors from an constructed UncompressedBitmap. Note this
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
index 824034a..1531781 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -45,8 +45,9 @@ public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
-		CompressedSizeInfoColGroup g2) {
+	protected CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
+		CompressedSizeInfoColGroup g2, int joinedMaxDistinct) {
+
 		AMapToData map = MapToFactory.join(g1.getMap(), g2.getMap());
 		EstimationFactors em = EstimationFactors.computeSizeEstimation(joined, map,
 			_cs.validCompressions.contains(CompressionType.RLE), _numRows, false);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
index 8b42258..6e9208a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
@@ -31,22 +31,31 @@ public class CompressedSizeEstimatorFactory {
 
 	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, CompressionSettings cs, int k) {
 
-		final int nRows = cs.transposed ? data.getNumColumns() :  data.getNumRows();
+		final int nRows = cs.transposed ? data.getNumColumns() : data.getNumRows();
 		final int nCols = cs.transposed ? data.getNumRows() : data.getNumColumns();
 		final int nnzRows = (int) Math.ceil(data.getNonZeros() / nCols);
 
 		final double sampleRatio = cs.samplingRatio;
 		final int sampleSize = Math.min(getSampleSize(sampleRatio, nRows, cs.minimumSampleSize), maxSampleSize);
-		if(shouldUseExactEstimator(cs, nRows, sampleSize, nnzRows)) {
-			if(sampleSize > nnzRows && nRows > 10000 && nCols > 10 && !cs.transposed) {
-				data = LibMatrixReorg.transpose(data,
-					new MatrixBlock(data.getNumColumns(), data.getNumRows(), data.isInSparseFormat()), k);
-				cs.transposed = true;
-			}
-			return new CompressedSizeEstimatorExact(data, cs);
+
+		if(nCols > 1000) {
+			return tryToMakeSampleEstimator(data, cs, sampleRatio, sampleSize / 10, nRows, nnzRows, k);
 		}
 		else {
-			return tryToMakeSampleEstimator(data, cs, sampleRatio, sampleSize, nRows, nnzRows, k);
+			if(shouldUseExactEstimator(cs, nRows, sampleSize, nnzRows)) {
+				if(sampleSize > nnzRows && nRows > 10000 && nCols > 10 && !cs.transposed) {
+					LOG.info("Transposing for exact estimator");
+					data = LibMatrixReorg.transpose(data,
+						new MatrixBlock(data.getNumColumns(), data.getNumRows(), data.isInSparseFormat()), k);
+					cs.transposed = true;
+				}
+				LOG.info("Using Exact estimator");
+				return new CompressedSizeEstimatorExact(data, cs);
+			}
+			else {
+				LOG.info("Trying sample size: " + sampleSize);
+				return tryToMakeSampleEstimator(data, cs, sampleRatio, sampleSize, nRows, nnzRows, k);
+			}
 		}
 
 	}
@@ -54,8 +63,9 @@ public class CompressedSizeEstimatorFactory {
 	private static CompressedSizeEstimator tryToMakeSampleEstimator(MatrixBlock data, CompressionSettings cs,
 		double sampleRatio, int sampleSize, int nRows, int nnzRows, int k) {
 		CompressedSizeEstimatorSample estS = new CompressedSizeEstimatorSample(data, cs, sampleSize, k);
+		int double_number = 1;
 		while(estS.getSample() == null) {
-			LOG.warn("Doubling sample size");
+			LOG.error("Warining doubling sample size " + double_number++);
 			sampleSize = sampleSize * 2;
 			if(shouldUseExactEstimator(cs, nRows, sampleSize, nnzRows))
 				return new CompressedSizeEstimatorExact(data, cs);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
index 24ee358..218e12b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -107,18 +107,17 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
-		CompressedSizeInfoColGroup g2) {
-		final int g1V = g1.getMap().getUnique();
-		final int g2V = g2.getMap().getUnique();
-		final int nrUniqueUpperBound = g1V * g2V;
+	protected CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
+		CompressedSizeInfoColGroup g2, int joinedMaxDistinct) {
+		if((long)g1.getNumVals() * g2.getNumVals() >(long)Integer.MAX_VALUE )
+			return null;
 
 		final AMapToData map = MapToFactory.join(g1.getMap(), g2.getMap());
 		EstimationFactors sampleFacts = EstimationFactors.computeSizeEstimation(joined, map,
 			_cs.validCompressions.contains(CompressionType.RLE), map.size(), false);
 
 		// result facts
-		EstimationFactors em = estimateCompressionFactors(sampleFacts, map, joined, nrUniqueUpperBound);
+		EstimationFactors em = estimateCompressionFactors(sampleFacts, map, joined, joinedMaxDistinct);
 		return new CompressedSizeInfoColGroup(em, _cs.validCompressions, map);
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
index e2b7491..1026aa2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -66,7 +66,7 @@ public class CompressedSizeInfoColGroup {
 		_facts = new EstimationFactors(columns, numVals, numRows);
 		_cardinalityRatio = (double) numVals / numRows;
 		_sizes = null;
-		_bestCompressionType = null;
+		_bestCompressionType = CompressionType.DDC;
 		_minSize = ColGroupSizes.estimateInMemorySizeDDC(columns.length, numVals, numRows, 1.0, false);
 		_map = null;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
index fc086e6..ab60d9f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
@@ -22,12 +22,10 @@ package org.apache.sysds.runtime.compress.lib;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
-import org.apache.sysds.runtime.compress.CompressionStatistics;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupEmpty;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -41,24 +39,18 @@ public class CLALibAppend {
 		if(left.isEmpty() && right instanceof CompressedMatrixBlock)
 			return appendLeftEmpty(left, (CompressedMatrixBlock) right);
 		else if(right.isEmpty() && left instanceof CompressedMatrixBlock)
-			return appendRightEmpty((CompressedMatrixBlock)left, right);
+			return appendRightEmpty((CompressedMatrixBlock) left, right);
 
 		final int m = left.getNumRows();
 		final int n = left.getNumColumns() + right.getNumColumns();
 
-		// try to compress both sides (if not already compressed).
 		if(!(left instanceof CompressedMatrixBlock) && m > 1000) {
-			LOG.warn("Compressing left for append operation");
-			Pair<MatrixBlock, CompressionStatistics> x = CompressedMatrixBlockFactory.compress(left);
-			if(x.getRight().getRatio() > 3.0)
-				left = x.getLeft();
-
+			LOG.info("Appending uncompressed column group left");
+			left = CompressedMatrixBlockFactory.genUncompressedCompressedMatrixBlock(left);
 		}
 		if(!(right instanceof CompressedMatrixBlock) && m > 1000) {
-			LOG.warn("Compressing right for append operation");
-			Pair<MatrixBlock, CompressionStatistics> x = CompressedMatrixBlockFactory.compress(right);
-			if(x.getRight().getRatio() > 3.0)
-				right = x.getLeft();
+			LOG.warn("Appending uncompressed column group right");
+			left = CompressedMatrixBlockFactory.genUncompressedCompressedMatrixBlock(right);
 		}
 
 		// if compression failed then use default append method.
@@ -72,14 +64,22 @@ public class CLALibAppend {
 		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
 
 		ret = appendColGroups(ret, leftC.getColGroups(), rightC.getColGroups(), leftC.getNumColumns());
-		return ret;
+
+		double compressedSize = ret.getInMemorySize();
+		double uncompressedSize = MatrixBlock.estimateSizeInMemory(m,n, ret.getSparsity());
+
+		
+		if(compressedSize * 10 < uncompressedSize)
+			return ret;
+		else
+			return ret.getUncompressed("Decompressing c bind matrix");
 	}
 
 	private static MatrixBlock appendRightEmpty(CompressedMatrixBlock left, MatrixBlock right) {
 
 		final int m = left.getNumRows();
 		final int n = left.getNumColumns() + right.getNumColumns();
-		CompressedMatrixBlock ret = new CompressedMatrixBlock(m,n);
+		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
 
 		List<AColGroup> newGroup = new ArrayList<>(1);
 		newGroup.add(ColGroupEmpty.generate(right.getNumColumns(), right.getNumRows()));
@@ -91,7 +91,7 @@ public class CLALibAppend {
 	private static MatrixBlock appendLeftEmpty(MatrixBlock left, CompressedMatrixBlock right) {
 		final int m = left.getNumRows();
 		final int n = left.getNumColumns() + right.getNumColumns();
-		CompressedMatrixBlock ret = new CompressedMatrixBlock(m,n);
+		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
 
 		List<AColGroup> newGroup = new ArrayList<>(1);
 		newGroup.add(ColGroupEmpty.generate(left.getNumColumns(), left.getNumRows()));
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
index 79cf1c3..99d9c92 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
@@ -19,7 +19,6 @@
 
 package org.apache.sysds.runtime.compress.lib;
 
-import java.lang.ref.SoftReference;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -59,33 +58,41 @@ public class CLALibBinaryCellOp {
 
 	private static final Log LOG = LogFactory.getLog(CLALibBinaryCellOp.class.getName());
 
-	public static MatrixBlock binaryOperations(BinaryOperator op, CompressedMatrixBlock m1, MatrixValue thatValue,
-		MatrixValue result) {
+	public static MatrixBlock binaryOperations(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock thatValue,
+		MatrixBlock result) {
 		MatrixBlock that = CompressedMatrixBlock.getUncompressed(thatValue, "Decompressing right side in BinaryOps");
 		if(m1.getNumRows() <= 0)
 			LOG.error(m1);
-		if(thatValue.getNumRows() <= 0)
-			LOG.error(thatValue);
+		if(that.getNumRows() <= 0)
+			LOG.error(that);
 		LibMatrixBincell.isValidDimensionsBinary(m1, that);
-		thatValue = that;
 		BinaryAccessType atype = LibMatrixBincell.getBinaryAccessType(m1, that);
-		return selectProcessingBasedOnAccessType(op, m1, thatValue, result, atype, false);
+		return selectProcessingBasedOnAccessType(op, m1, that, result, atype, false);
 	}
 
-	public static MatrixBlock binaryOperationsLeft(BinaryOperator op, CompressedMatrixBlock m1, MatrixValue thatValue,
-		MatrixValue result) {
+	public static MatrixBlock binaryOperationsLeft(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock thatValue,
+		MatrixBlock result) {
 		MatrixBlock that = CompressedMatrixBlock.getUncompressed(thatValue, "Decompressing left side in BinaryOps");
 		LibMatrixBincell.isValidDimensionsBinary(that, m1);
 		thatValue = that;
 		BinaryAccessType atype = LibMatrixBincell.getBinaryAccessType(that, m1);
-		return selectProcessingBasedOnAccessType(op, m1, thatValue, result, atype, true);
+		return selectProcessingBasedOnAccessType(op, m1, that, result, atype, true);
 	}
 
 	private static MatrixBlock selectProcessingBasedOnAccessType(BinaryOperator op, CompressedMatrixBlock m1,
-		MatrixValue thatValue, MatrixValue result, BinaryAccessType atype, boolean left) {
-		MatrixBlock that = (MatrixBlock) thatValue;
-		if(atype == BinaryAccessType.MATRIX_COL_VECTOR)
-			return binaryMVCol(m1, that, op, left);
+		MatrixBlock that, MatrixBlock result, BinaryAccessType atype, boolean left) {
+		if(atype == BinaryAccessType.MATRIX_COL_VECTOR) {
+			MatrixBlock d_compressed = m1.getCachedDecompressed();
+			if(d_compressed != null) {
+				if(left)
+					return that.binaryOperations(op, d_compressed, result);
+				else
+					return d_compressed.binaryOperations(op, that, result);
+			}
+			else
+				return binaryMVCol(m1, that, op, left);
+
+		}
 		else if(atype == BinaryAccessType.MATRIX_MATRIX) {
 			if(that.isEmpty()) {
 				ScalarOperator sop = left ? new LeftScalarOperator(op.fn, 0, -1) : new RightScalarOperator(op.fn, 0,
@@ -93,8 +100,7 @@ public class CLALibBinaryCellOp {
 				return CLALibScalar.scalarOperations(sop, m1, result);
 			}
 			else {
-				SoftReference<MatrixBlock> msf = m1.getSoftReferenceToDecompressed();
-				MatrixBlock d_compressed = msf != null ? msf.get() : null;
+				MatrixBlock d_compressed = m1.getCachedDecompressed();
 				if(d_compressed != null) {
 					// copy the decompressed matrix if there is a decompressed matrix already.
 					MatrixBlock tmp = d_compressed;
@@ -117,7 +123,7 @@ public class CLALibBinaryCellOp {
 			return bincellOp(m1, that, setupCompressedReturnMatrixBlock(m1, result), op, left);
 		else {
 			LOG.warn("Decompressing since Binary Ops" + op.fn + " is not supported compressed");
-			return CompressedMatrixBlock.getUncompressed(m1).binaryOperations(op, thatValue, result);
+			return CompressedMatrixBlock.getUncompressed(m1).binaryOperations(op, that, result);
 		}
 	}
 
@@ -295,7 +301,7 @@ public class CLALibBinaryCellOp {
 		final int blkz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int k = op.getNumThreads();
 		long nnz = 0;
-		;
+
 		if(k <= 1) {
 			for(int i = 0; i * blkz < m1.getNumRows(); i++) {
 				if(left)
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
index 892ec95..5c9f7b2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
@@ -34,6 +34,7 @@ import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
+import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.functionobjects.IndexFunction;
@@ -50,7 +51,6 @@ import org.apache.sysds.runtime.matrix.data.LibMatrixAgg;
 import org.apache.sysds.runtime.matrix.data.LibMatrixBincell;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
-import org.apache.sysds.runtime.matrix.data.MatrixValue;
 import org.apache.sysds.runtime.matrix.data.MatrixValue.CellIndex;
 import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
@@ -71,43 +71,64 @@ public class CLALibCompAgg {
 		}
 	};
 
-	public static MatrixBlock aggregateUnary(CompressedMatrixBlock inputMatrix, MatrixValue result,
+	public static MatrixBlock aggregateUnary(CompressedMatrixBlock inputMatrix, MatrixBlock result,
 		AggregateUnaryOperator op, int blen, MatrixIndexes indexesIn, boolean inCP) {
 
 		// prepare output dimensions
 		CellIndex tempCellIndex = new CellIndex(-1, -1);
 		op.indexFn.computeDimension(inputMatrix.getNumRows(), inputMatrix.getNumColumns(), tempCellIndex);
 
+		if(requireDecompression(inputMatrix, op)) {
+			// Decide if we should use the cached decompressed Version, or we should decompress.
+			final double denseSize = MatrixBlock.estimateSizeDenseInMemory(inputMatrix.getNumRows(),
+				inputMatrix.getNumColumns());
+			final double currentSize = inputMatrix.getInMemorySize();
+			final double localMaxMemory = InfrastructureAnalyzer.getLocalMaxMemory();
+
+			if(denseSize < 5 * currentSize && inputMatrix.getColGroups().size() > 5 &&
+				denseSize <= localMaxMemory / 2) {
+				LOG.info("Decompressing for unaryAggregate because of overlapping state");
+				inputMatrix.decompress(op.getNumThreads());
+			}
+			MatrixBlock decomp = inputMatrix.getCachedDecompressed();
+			if(decomp != null)
+				return decomp.aggregateUnaryOperations(op, result, blen, indexesIn, inCP);
+		}
+		
 		// initialize and allocate the result
 		if(result == null)
 			result = new MatrixBlock(tempCellIndex.row, tempCellIndex.column, false);
 		else
 			result.reset(tempCellIndex.row, tempCellIndex.column, false);
-		MatrixBlock ret = (MatrixBlock) result;
 
-		ret.allocateDenseBlock();
+		result.allocateDenseBlock();
 
 		AggregateUnaryOperator opm = replaceKahnOperations(op);
 
 		if(inputMatrix.getColGroups() != null) {
-			fillStart(ret, opm);
+			fillStart(result, opm);
 
-			if(inputMatrix.isOverlapping() &&
-				(opm.aggOp.increOp.fn instanceof KahanPlusSq || (opm.aggOp.increOp.fn instanceof Builtin &&
-					(((Builtin) opm.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MIN ||
-						((Builtin) opm.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MAX))))
-				aggregateUnaryOverlapping(inputMatrix, ret, opm, indexesIn, inCP);
+			if(requireDecompression(inputMatrix, opm))
+				aggregateUnaryOverlapping(inputMatrix, result, opm, indexesIn, inCP);
 			else
-				aggregateUnaryNormalCompressedMatrixBlock(inputMatrix, ret, opm, blen, indexesIn, inCP);
+				aggregateUnaryNormalCompressedMatrixBlock(inputMatrix, result, opm, blen, indexesIn, inCP);
 		}
-		ret.recomputeNonZeros();
+		
+		result.recomputeNonZeros();
 		if(op.aggOp.existsCorrection() && !inCP) {
-			ret = addCorrection(ret, op);
+			result = addCorrection(result, op);
 			if(op.aggOp.increOp.fn instanceof Mean)
-				ret = addCellCount(ret, op, inputMatrix.getNumRows(), inputMatrix.getNumColumns());
+				result = addCellCount(result, op, inputMatrix.getNumRows(), inputMatrix.getNumColumns());
 		}
-		return ret;
+		return result;
+
+	}
 
+	private static boolean requireDecompression(CompressedMatrixBlock inputMatrix, AggregateUnaryOperator op) {
+		return inputMatrix.isOverlapping() &&
+			(op.aggOp.increOp.fn instanceof KahanPlusSq || (op.aggOp.increOp.fn instanceof Builtin &&
+				(((Builtin) op.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MIN ||
+					((Builtin) op.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MAX)));
 	}
 
 	private static MatrixBlock addCorrection(MatrixBlock ret, AggregateUnaryOperator op) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
deleted file mode 100644
index d4fbb7e..0000000
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-// package org.apache.sysds.runtime.compress.lib;
-
-// import java.util.ArrayList;
-// import java.util.Arrays;
-// import java.util.List;
-// import java.util.concurrent.Callable;
-// import java.util.concurrent.ExecutionException;
-// import java.util.concurrent.ExecutorService;
-// import java.util.concurrent.Future;
-
-// import org.apache.sysds.hops.OptimizerUtils;
-// import org.apache.sysds.runtime.DMLRuntimeException;
-// import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
-// import org.apache.sysds.runtime.compress.CompressionSettings;
-// import org.apache.sysds.runtime.compress.colgroup.AColGroup;
-// import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
-// import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
-// import org.apache.sysds.runtime.functionobjects.Equals;
-// import org.apache.sysds.runtime.functionobjects.GreaterThan;
-// import org.apache.sysds.runtime.functionobjects.GreaterThanEquals;
-// import org.apache.sysds.runtime.functionobjects.LessThan;
-// import org.apache.sysds.runtime.functionobjects.LessThanEquals;
-// import org.apache.sysds.runtime.functionobjects.NotEquals;
-// import org.apache.sysds.runtime.matrix.data.MatrixBlock;
-// import org.apache.sysds.runtime.matrix.operators.LeftScalarOperator;
-// import org.apache.sysds.runtime.matrix.operators.RightScalarOperator;
-// import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
-// import org.apache.sysds.runtime.util.CommonThreadPool;
-
-// /**
-//  * This class is used for relational operators that return binary values depending on individual cells values in the
-//  * compression. This indicate that the resulting vectors/matrices are amenable to compression since they only contain
-//  * two distinct values, true or false.
-//  * 
-//  */
-// public class CLALibRelationalOp {
-// 	// private static final Log LOG = LogFactory.getLog(LibRelationalOp.class.getName());
-
-// 	/** Thread pool matrix Block for materializing decompressed groups. */
-// 	private static ThreadLocal<MatrixBlock> memPool = new ThreadLocal<MatrixBlock>() {
-// 		@Override
-// 		protected MatrixBlock initialValue() {
-// 			return null;
-// 		}
-// 	};
-
-// 	protected static boolean isValidForRelationalOperation(ScalarOperator sop, CompressedMatrixBlock m1) {
-// 		return m1.isOverlapping() &&
-// 			(sop.fn instanceof LessThan || sop.fn instanceof LessThanEquals || sop.fn instanceof GreaterThan ||
-// 				sop.fn instanceof GreaterThanEquals || sop.fn instanceof Equals || sop.fn instanceof NotEquals);
-// 	}
-
-// 	// public static MatrixBlock overlappingRelativeRelationalOperation(ScalarOperator sop, CompressedMatrixBlock m1) {
-
-// 	// 	List<AColGroup> colGroups = m1.getColGroups();
-// 	// 	boolean less = ((sop.fn instanceof LessThan || sop.fn instanceof LessThanEquals) &&
-// 	// 		sop instanceof LeftScalarOperator) ||
-// 	// 		(sop instanceof RightScalarOperator &&
-// 	// 			(sop.fn instanceof GreaterThan || sop.fn instanceof GreaterThanEquals));
-// 	// 	double v = sop.getConstant();
-// 	// 	double min = m1.min();
-// 	// 	double max = m1.max();
-
-// 	// 	// Shortcut:
-// 	// 	// If we know worst case min and worst case max and the values to compare to in all cases is
-// 	// 	// less then or greater than worst then we can return a full matrix with either 1 or 0.
-
-// 	// 	if(v < min || v > max) {
-// 	// 		if(sop.fn instanceof Equals) {
-// 	// 			return makeConstZero(m1.getNumRows(), m1.getNumColumns());
-// 	// 		}
-// 	// 		else if(sop.fn instanceof NotEquals) {
-// 	// 			return makeConstOne(m1.getNumRows(), m1.getNumColumns());
-// 	// 		}
-// 	// 		else if(less) {
-// 	// 			if(v < min || ((sop.fn instanceof LessThanEquals || sop.fn instanceof GreaterThan) && v <= min))
-// 	// 				return makeConstOne(m1.getNumRows(), m1.getNumColumns());
-// 	// 			else
-// 	// 				return makeConstZero(m1.getNumRows(), m1.getNumColumns());
-// 	// 		}
-// 	// 		else {
-// 	// 			if(v > max || ((sop.fn instanceof LessThanEquals || sop.fn instanceof GreaterThan) && v >= max))
-// 	// 				return makeConstOne(m1.getNumRows(), m1.getNumColumns());
-// 	// 			else
-// 	// 				return makeConstZero(m1.getNumRows(), m1.getNumColumns());
-// 	// 		}
-// 	// 	}
-// 	// 	else {
-// 	// 		return processNonConstant(sop, minMax, min, max, m1.getNumRows(), m1.getNumColumns(), less);
-// 	// 	}
-
-// 	// }
-
-// 	private static MatrixBlock makeConstOne(int rows, int cols) {
-// 	// 	List<AColGroup> newColGroups = new ArrayList<>();
-// 	// 	int[] colIndexes = new int[cols];
-// 	// 	for(int i = 0; i < colIndexes.length; i++) {
-// 	// 		colIndexes[i] = i;
-// 	// 	}
-// 	// 	double[] values = new double[cols];
-// 	// 	Arrays.fill(values, 1);
-
-// 	// 	newColGroups.add(new ColGroupConst(colIndexes, rows, new Dictionary(values)));
-// 	// 	CompressedMatrixBlock ret = new CompressedMatrixBlock(rows, cols);
-// 	// 	ret.allocateColGroupList(newColGroups);
-// 	// 	ret.setNonZeros(cols * rows);
-// 	// 	ret.setOverlapping(false);
-// 	// 	return ret;
-// 	// }
-
-// 	// private static MatrixBlock makeConstZero(int rows, int cols) {
-// 	// 	MatrixBlock sb = new MatrixBlock(rows, cols, true, 0);
-// 	// 	return sb;
-// 	// }
-
-// 	// private static MatrixBlock processNonConstant(ScalarOperator sop, MinMaxGroup[] minMax, double minS, double maxS,
-// 	// 	final int rows, final int cols, boolean less) {
-
-// 	// 	// BitSet res = new BitSet(ret.getNumColumns() * ret.getNumRows());
-// 	// 	MatrixBlock res = new MatrixBlock(rows, cols, true, 0).allocateBlock();
-// 	// 	int k = OptimizerUtils.getConstrainedNumThreads(-1);
-// 	// 	int outRows = rows;
-// 	// 	long nnz = 0;
-// 	// 	if(k == 1) {
-// 	// 		final int b = CompressionSettings.BITMAP_BLOCK_SZ / cols;
-// 	// 		final int blkz = (outRows < b) ? outRows : b;
-
-// 	// 		MatrixBlock tmp = new MatrixBlock(blkz, cols, false, -1).allocateBlock();
-// 	// 		for(int i = 0; i * blkz < outRows; i++) {
-// 	// 			for(MinMaxGroup mmg : minMax) 
-// 	// 				mmg.g.decompressToBlockUnSafe(tmp, i * blkz, Math.min((i + 1) * blkz, rows), 0);
-				
-// 	// 			for(int row = 0; row < blkz && row < rows - i * blkz; row++) {
-// 	// 				int off = (row + i * blkz);
-// 	// 				for(int col = 0; col < cols; col++) {
-// 	// 					res.quickSetValue(off, col, sop.executeScalar(tmp.quickGetValue(row, col)));
-// 	// 					if(res.quickGetValue(off, col) != 0) {
-// 	// 						nnz++;
-// 	// 					}
-// 	// 				}
-// 	// 			}
-// 	// 		}
-// 	// 		tmp.reset();
-// 	// 		res.setNonZeros(nnz);
-// 	// 	}
-// 	// 	else {
-// 	// 		final int blkz = CompressionSettings.BITMAP_BLOCK_SZ / 2;
-// 	// 		ExecutorService pool = CommonThreadPool.get(k);
-// 	// 		ArrayList<RelationalTask> tasks = new ArrayList<>();
-
-// 	// 		try {
-// 	// 			for(int i = 0; i * blkz < outRows; i++) {
-// 	// 				RelationalTask rt = new RelationalTask(minMax, i, blkz, res, rows, cols, sop);
-// 	// 				tasks.add(rt);
-// 	// 			}
-// 	// 			List<Future<Object>> futures = pool.invokeAll(tasks);
-// 	// 			pool.shutdown();
-// 	// 			for(Future<Object> f : futures)
-// 	// 				f.get();
-// 	// 		}
-// 	// 		catch(InterruptedException | ExecutionException e) {
-// 	// 			e.printStackTrace();
-// 	// 			throw new DMLRuntimeException(e);
-// 	// 		}
-
-// 	// 	}
-// 	// 	memPool.remove();
-
-// 	// 	return res;
-// 	// }
-
-// 	// protected static class MinMaxGroup implements Comparable<MinMaxGroup> {
-// 	// 	double min;
-// 	// 	double max;
-// 	// 	AColGroup g;
-// 	// 	double[] values;
-
-// 	// 	public MinMaxGroup(double min, double max, AColGroup g) {
-// 	// 		this.min = min;
-// 	// 		this.max = max;
-// 	// 		this.g = g;
-
-// 	// 		this.values = g.getValues();
-// 	// 	}
-
-// 	// 	@Override
-// 	// 	public int compareTo(MinMaxGroup o) {
-// 	// 		double t = max - min;
-// 	// 		double ot = o.max - o.min;
-// 	// 		return Double.compare(t, ot);
-// 	// 	}
-
-// 	// 	@Override
-// 	// 	public String toString() {
-// 	// 		StringBuilder sb = new StringBuilder();
-// 	// 		sb.append("MMG: ");
-// 	// 		sb.append("[" + min + "," + max + "]");
-// 	// 		sb.append(" " + g.getClass().getSimpleName());
-// 	// 		return sb.toString();
-// 	// 	}
-// 	// }
-
-// 	// private static class RelationalTask implements Callable<Object> {
-// 	// 	private final MinMaxGroup[] _minMax;
-// 	// 	private final int _i;
-// 	// 	private final int _blkz;
-// 	// 	private final MatrixBlock _res;
-// 	// 	private final int _rows;
-// 	// 	private final int _cols;
-// 	// 	private final ScalarOperator _sop;
-
-// 	// 	protected RelationalTask(MinMaxGroup[] minMax, int i, int blkz, MatrixBlock res, int rows, int cols,
-// 	// 		ScalarOperator sop) {
-// 	// 		_minMax = minMax;
-// 	// 		_i = i;
-// 	// 		_blkz = blkz;
-// 	// 		_res = res;
-// 	// 		_rows = rows;
-// 	// 		_cols = cols;
-// 	// 		_sop = sop;
-// 	// 	}
-
-// 	// 	@Override
-// 	// 	public Object call() {
-// 	// 		MatrixBlock tmp = memPool.get();
-// 	// 		if(tmp == null) {
-// 	// 			memPool.set(new MatrixBlock(_blkz, _cols, false, -1).allocateBlock());
-// 	// 			tmp = memPool.get();
-// 	// 		}
-// 	// 		else {
-// 	// 			tmp = memPool.get();
-// 	// 			tmp.reset(_blkz, _cols, false, -1);
-// 	// 		}
-
-// 	// 		for(MinMaxGroup mmg : _minMax) {
-// 	// 			if(mmg.g.getNumberNonZeros() != 0)
-// 	// 				mmg.g.decompressToBlockUnSafe(tmp, _i * _blkz, Math.min((_i + 1) * _blkz, mmg.g.getNumRows()), 0);
-// 	// 		}
-
-// 	// 		for(int row = 0, off = _i * _blkz; row < _blkz && row < _rows - _i * _blkz; row++, off++) {
-// 	// 			for(int col = 0; col < _cols; col++) {
-// 	// 				_res.appendValue(off, col, _sop.executeScalar(tmp.quickGetValue(row, col)));
-// 	// 			}
-// 	// 		}
-// 	// 		return null;
-// 	// 	}
-// 	// }
-// }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
index aea5488..1822685 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
@@ -56,13 +56,6 @@ public class CLALibScalar {
 	private static final int MINIMUM_PARALLEL_SIZE = 8096;
 
 	public static MatrixBlock scalarOperations(ScalarOperator sop, CompressedMatrixBlock m1, MatrixValue result) {
-		// Special case handling of overlapping relational operations
-		// if(CLALibRelationalOp.isValidForRelationalOperation(sop, m1)) {
-		// 	MatrixBlock ret =  CLALibRelationalOp.overlappingRelativeRelationalOperation(sop, m1);
-		// 	ret.recomputeNonZeros();
-		// 	return ret;
-		// }
-
 		if(isInvalidForCompressedOutput(m1, sop)) {
 			LOG.warn("scalar overlapping not supported for op: " + sop.fn);
 			MatrixBlock m1d = m1.decompress(sop.getNumThreads());
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
index b37acc1..31b3714 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
@@ -404,7 +404,10 @@ public class WorkloadAnalyzer {
 				ArrayList<Hop> in = hop.getInput();
 				if(isOverlapping(in.get(0)) || isOverlapping(in.get(1)))
 					overlapping.add(hop.getHopID());
-				return new OpNormal(hop, true);
+				// CBind is in worst case decompressing, but can be compressing the other side if it is trivially compressable.
+				// to make the optimizer correct we need to mark this operation as decompressing, since it is the worst possible outcome.
+				// Currently we dont optimize for operations that are located past a cbind.
+				return new OpDecompressing(hop);
 			}
 			else if(HopRewriteUtils.isBinary(hop, OpOp2.RBIND)) {
 				ArrayList<Hop> in = hop.getInput();
diff --git a/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
index 880f31f..faef2c7 100644
--- a/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
@@ -66,6 +66,7 @@ import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.data.TensorBlock;
 import org.apache.sysds.runtime.data.TensorIndexes;
 import org.apache.sysds.runtime.instructions.cp.Data;
+import org.apache.sysds.runtime.instructions.spark.DeCompressionSPInstruction;
 import org.apache.sysds.runtime.instructions.spark.data.BroadcastObject;
 import org.apache.sysds.runtime.instructions.spark.data.LineageObject;
 import org.apache.sysds.runtime.instructions.spark.data.PartitionedBlock;
@@ -1280,16 +1281,21 @@ public class SparkExecutionContext extends ExecutionContext
 		return out;
 	}
 
-	@SuppressWarnings("unchecked")
+	// @SuppressWarnings("unchecked")
 	public static long writeMatrixRDDtoHDFS( RDDObject rdd, String path, FileFormat fmt )
 	{
 		JavaPairRDD<MatrixIndexes,MatrixBlock> lrdd = (JavaPairRDD<MatrixIndexes, MatrixBlock>) rdd.getRDD();
 		InputOutputInfo oinfo = InputOutputInfo.get(DataType.MATRIX, fmt);
 		
+		// if compression is enabled decompress all blocks before writing to disk TEMPORARY MODIFICATION UNTILL MATRIXBLOCK IS MERGED WITH COMPRESSEDMATRIXBLOCK
+		if(ConfigurationManager.isCompressionEnabled())
+			lrdd = lrdd.mapValues(new DeCompressionSPInstruction.DeCompressionFunction());
+
 		//piggyback nnz maintenance on write
 		LongAccumulator aNnz = getSparkContextStatic().sc().longAccumulator("nnz");
 		lrdd = lrdd.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
 
+
 		//save file is an action which also triggers nnz maintenance
 		lrdd.saveAsHadoopFile(path,
 			oinfo.keyClass,
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/BinUaggChainSPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/BinUaggChainSPInstruction.java
index 4ea0284..39b2408 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/BinUaggChainSPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/BinUaggChainSPInstruction.java
@@ -30,6 +30,12 @@ import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 
+/**
+ * Instruction that performs
+ * 
+ * res = X / rowsum(x)
+ * 
+ */
 public class BinUaggChainSPInstruction extends UnarySPInstruction {
 	// operators
 	private BinaryOperator _bOp = null;
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
index 6115654..a998a74 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
@@ -29,6 +29,7 @@ import java.io.DataOutputStream;
 import java.util.Collection;
 
 import org.apache.commons.math3.random.Well1024a;
+import org.apache.sysds.common.Types.CorrectionLocationType;
 import org.apache.sysds.conf.ConfigurationManager;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
@@ -36,10 +37,15 @@ import org.apache.sysds.runtime.compress.CompressionSettingsBuilder;
 import org.apache.sysds.runtime.compress.CompressionStatistics;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
+import org.apache.sysds.runtime.functionobjects.KahanPlus;
+import org.apache.sysds.runtime.functionobjects.Multiply;
+import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.matrix.data.LibMatrixCountDistinct;
 import org.apache.sysds.runtime.matrix.data.LibMatrixDatagen;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.data.RandomMatrixGenerator;
+import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysds.runtime.matrix.operators.AggregateTernaryOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.CountDistinctOperator;
 import org.apache.sysds.runtime.matrix.operators.CountDistinctOperator.CountDistinctTypes;
@@ -570,6 +576,33 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		cmb.copy(cmb);
 	}
 
+	@Test
+	public void testAggregateTernaryOperation() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
+				return;
+			CorrectionLocationType corr = CorrectionLocationType.LASTCOLUMN;
+			AggregateOperator agg = new AggregateOperator(0, KahanPlus.getKahanPlusFnObject(), corr);
+			AggregateTernaryOperator op = new AggregateTernaryOperator(Multiply.getMultiplyFnObject(), agg,
+				ReduceAll.getReduceAllFnObject());
+
+			int nrow = mb.getNumRows();
+			int ncol = mb.getNumColumns();
+
+			MatrixBlock m2 = new MatrixBlock(nrow, ncol, 13.0);
+			MatrixBlock m3 = new MatrixBlock(nrow, ncol, 14.0);
+
+			MatrixBlock ret1 = cmb.aggregateTernaryOperations(cmb, m2, m3, null, op, true);
+			MatrixBlock ret2 = mb.aggregateTernaryOperations(mb, m2, m3, null, op, true);
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
 	private static long getJolSize(CompressedMatrixBlock cmb, CompressionStatistics cStat) {
 		Layouter l = new HotSpotLayouter(new X86_64_DataModel());
 		long jolEstimate = 0;
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
index eb5ceca..c291235 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
@@ -31,7 +31,6 @@ import java.util.List;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysds.common.Types.CorrectionLocationType;
 import org.apache.sysds.lops.MMTSJ.MMTSJType;
 import org.apache.sysds.lops.MapMultChain.ChainType;
 import org.apache.sysds.runtime.DMLRuntimeException;
@@ -55,22 +54,18 @@ import org.apache.sysds.runtime.functionobjects.Divide;
 import org.apache.sysds.runtime.functionobjects.Equals;
 import org.apache.sysds.runtime.functionobjects.GreaterThan;
 import org.apache.sysds.runtime.functionobjects.GreaterThanEquals;
-import org.apache.sysds.runtime.functionobjects.KahanPlus;
 import org.apache.sysds.runtime.functionobjects.LessThan;
 import org.apache.sysds.runtime.functionobjects.LessThanEquals;
 import org.apache.sysds.runtime.functionobjects.Minus;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.functionobjects.Power2;
-import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.functionobjects.SwapIndex;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.functionobjects.Xor;
 import org.apache.sysds.runtime.instructions.InstructionUtils;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateBinaryOperator;
-import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
-import org.apache.sysds.runtime.matrix.operators.AggregateTernaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.LeftScalarOperator;
 import org.apache.sysds.runtime.matrix.operators.ReorgOperator;
@@ -1253,28 +1248,6 @@ public abstract class CompressedTestBase extends TestBase {
 	}
 
 	@Test
-	public void aggregateTernaryOperations() {
-		if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
-			return;
-
-		MatrixBlock m1 = new MatrixBlock();
-		MatrixBlock m2 = new MatrixBlock();
-		MatrixBlock m3 = null;
-		CorrectionLocationType corr = CorrectionLocationType.LASTCOLUMN;
-		AggregateOperator agg = new AggregateOperator(0, KahanPlus.getKahanPlusFnObject(), corr);
-		AggregateTernaryOperator op = new AggregateTernaryOperator(Multiply.getMultiplyFnObject(), agg,
-			ReduceAll.getReduceAllFnObject(), _k);
-
-		boolean inCP = true;
-
-		MatrixBlock ret1 = mb.aggregateTernaryOperations(m1, m2, m3, null, op, inCP);
-		MatrixBlock ret2 = cmb.aggregateTernaryOperations(m1, m2, m3, null, op, inCP);
-
-		compareResultMatrices(ret1, ret2, 1);
-
-	}
-
-	@Test
 	public void unaryOperations() {
 		if(!(cmb instanceof CompressedMatrixBlock) || cmb.getNumColumns() < 2)
 			return;
diff --git a/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java b/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
index aacb716..9422e58 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
@@ -119,7 +119,7 @@ public class WorkloadTest {
 		args.put("$2", "FALSE");
 		args.put("$3", "0");
 
-		tests.add(new Object[] {0, 0, 0, 1, 1, 1, 6, 0, true, false, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 1, 0, 1, 1, 1, 6, 0, true, false, "functions/lmDS.dml", args});
 		tests.add(new Object[] {0, 0, 0, 1, 0, 1, 0, 0, true, true, "functions/lmDS.dml", args});
 		tests.add(new Object[] {0, 0, 0, 1, 10, 10, 1, 0, true, true, "functions/lmCG.dml", args});
 
@@ -134,15 +134,15 @@ public class WorkloadTest {
 		args.put("$1", testFile);
 		args.put("$2", "TRUE");
 		args.put("$3", "1");
-		tests.add(new Object[] {0, 0, 1, 1, 1, 1, 1, 0, true, true, "functions/lmDS.dml", args});
-		tests.add(new Object[] {0, 0, 1, 1, 11, 10, 2, 0, true, true, "functions/lmCG.dml", args});
+		tests.add(new Object[] {0, 1, 0, 0, 0, 0, 1, 0, false, true, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 1, 1, 1, 11, 10, 2, 0, true, true, "functions/lmCG.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "TRUE");
 		args.put("$3", "2");
-		tests.add(new Object[] {0, 0, 1, 1, 1, 1, 3, 0, true, true, "functions/lmDS.dml", args});
-		tests.add(new Object[] {0, 0, 1, 1, 11, 10, 4, 0, true, true, "functions/lmCG.dml", args});
+		tests.add(new Object[] {0, 1, 0, 0, 0, 0, 1, 0, false, true, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 1, 1, 1, 11, 10, 2, 0, true, true, "functions/lmCG.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
@@ -176,7 +176,7 @@ public class WorkloadTest {
 			CostEstimatorBuilder ceb = new CostEstimatorBuilder(wtr);
 			InstructionTypeCounter itc = ceb.getCounter();
 
-			verify(wtr, itc, ceb);
+			verify(wtr, itc, ceb, scriptName, args);
 		}
 		catch(Exception e) {
 			e.printStackTrace();
@@ -184,9 +184,9 @@ public class WorkloadTest {
 		}
 	}
 
-	private void verify(WTreeRoot wtr, InstructionTypeCounter itc, CostEstimatorBuilder ceb) {
+	private void verify(WTreeRoot wtr, InstructionTypeCounter itc, CostEstimatorBuilder ceb, String name, Map<String, String> args) {
 
-		String errorString = wtr + "\n" + itc + " \n ";
+		String errorString = wtr + "\n" + itc + " \n " + name + "  -- " + args +  "\n";
 		Assert.assertEquals(errorString + "scans:", scans, itc.getScans());
 		Assert.assertEquals(errorString + "decompressions", decompressions, itc.getDecompressions());
 		Assert.assertEquals(errorString + "overlappingDecompressions", overlappingDecompressions,
@@ -197,7 +197,7 @@ public class WorkloadTest {
 			itc.getCompressedMultiplications());
 		Assert.assertEquals(errorString + "dictionaryOps", dictionaryOps, itc.getDictionaryOps());
 		Assert.assertEquals(errorString + "lookup", indexing, itc.getIndexing());
-		Assert.assertEquals(shouldCompress, ceb.shouldTryToCompress());
+		Assert.assertEquals(errorString + "Should Compresss", shouldCompress, ceb.shouldTryToCompress());
 	}
 
 	private static WTreeRoot getWorkloadTree(DMLProgram prog) {
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
index 9ffeb02..5de8880 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
@@ -73,7 +73,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 
 	@Test
 	public void testMLogRegCP() {
-		runWorkloadAnalysisTest(TEST_NAME1, ExecMode.HYBRID, 2, false);
+		runWorkloadAnalysisTest(TEST_NAME1, ExecMode.HYBRID, 1, false);
 	}
 
 	@Test

[systemds] 03/03: [SYSTEMDS-2990] Workload tree move decompression to input hop

Posted by ba...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 61f40bd6af0291c2340974d3a2e1074181a51822
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Wed Sep 8 17:04:04 2021 +0200

    [SYSTEMDS-2990] Workload tree move decompression to input hop
    
    This commit move the decompression instruction to the input hop
    of a decompressing instruction execution on the workload trees.
    this in practice means that if a variable is used in a forloop
    and needs decompression it is taken into account that the decompression
    only happens once, and outside the for loop.
    
    - Update to allow return of constant column groups in various cases
    - Remove System.out.println from estim estimatorDensityMap
    - Add Compression in loop right mult with decompression
    - Fix Binary matrix matrix operation Compressed
    - Add isDensifying boolean to compression cost, to allow compression
    to compare to dense allocation.
    - add missing minus on empty operator
    
    Closes #1385
---
 .../java/org/apache/sysds/hops/OptimizerUtils.java |   3 +
 .../sysds/hops/estim/EstimatorDensityMap.java      |   4 +-
 .../ipa/IPAPassCompressionWorkloadAnalysis.java    |  22 +-
 .../RewriteAlgebraicSimplificationDynamic.java     |  26 +-
 .../hops/rewrite/RewriteCompressedReblock.java     |   6 +-
 .../runtime/compress/CompressedMatrixBlock.java    | 119 +++++--
 .../compress/CompressedMatrixBlockFactory.java     |  18 +-
 .../runtime/compress/CompressionSettings.java      |   8 +-
 .../compress/CompressionSettingsBuilder.java       |   4 +-
 .../runtime/compress/CompressionStatistics.java    |   4 +-
 .../runtime/compress/DMLCompressionException.java  |  12 +-
 .../runtime/compress/cocode/CoCodeGreedy.java      |   8 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |   4 +-
 .../compress/colgroup/ColGroupUncompressed.java    |   7 +-
 .../runtime/compress/colgroup/ColGroupValue.java   |   6 -
 .../compress/colgroup/mapping/MapToFactory.java    |  11 +-
 .../compress/cost/ComputationCostEstimator.java    |  48 ++-
 .../compress/cost/CostEstimatorBuilder.java        |  34 +-
 .../compress/cost/CostEstimatorFactory.java        |  10 +-
 .../compress/cost/InstructionTypeCounter.java      |   2 +
 .../compress/estim/CompressedSizeEstimator.java    |   4 +-
 .../estim/CompressedSizeEstimatorExact.java        |   3 +-
 .../estim/CompressedSizeEstimatorFactory.java      |   2 +-
 .../estim/CompressedSizeEstimatorSample.java       |   5 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |   2 +-
 .../runtime/compress/estim/EstimationFactors.java  |   2 +-
 .../sysds/runtime/compress/lib/CLALibAppend.java   |   7 +-
 .../runtime/compress/lib/CLALibBinaryCellOp.java   | 116 ++++---
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |  17 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     | 102 +++---
 .../runtime/compress/lib/CLALibRightMultBy.java    |  82 ++---
 .../readers/ReaderCompressedSelection.java         |   2 -
 .../runtime/compress/workload/AWTreeNode.java      |   6 +-
 .../apache/sysds/runtime/compress/workload/Op.java |  33 +-
 .../runtime/compress/workload/OpDecompressing.java |  39 ---
 .../runtime/compress/workload/OpMetadata.java      |  15 +-
 .../sysds/runtime/compress/workload/OpNormal.java  |   5 -
 .../compress/workload/OpOverlappingDecompress.java |  38 ---
 .../sysds/runtime/compress/workload/OpSided.java   |  15 +-
 .../sysds/runtime/compress/workload/WTreeRoot.java |  31 +-
 .../compress/workload/WorkloadAnalyzer.java        | 358 +++++++++++++--------
 .../context/SparkExecutionContext.java             |  12 +-
 .../instructions/cp/SpoofCPInstruction.java        |   3 +-
 .../runtime/matrix/data/LibMatrixBincell.java      |  10 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |  96 +++---
 .../apache/sysds/runtime/util/DataConverter.java   |   2 +-
 .../compress/AbstractCompressedUnaryTests.java     |   2 -
 .../component/compress/CompressedMatrixTest.java   | 163 +++++++++-
 .../component/compress/CompressedTestBase.java     |   9 +
 .../component/compress/workload/WorkloadTest.java  |  43 +--
 .../test/component/frame/DataCorruptionTest.java   |   8 +-
 .../test/functions/codegen/RowAggTmplTest.java     |   6 +-
 .../functions/compress/CompressInstruction.java    |   1 -
 .../compress/CompressInstructionRewrite.java       |  19 +-
 .../functions/compress/CompressRewriteSpark.java   |  43 ++-
 .../compress/configuration/CompressBase.java       |  16 +-
 .../compress/configuration/CompressCost.java       |  70 ----
 .../compress/configuration/CompressForce.java      |  36 +++
 .../compress/configuration/CompressLossy.java      |  52 ---
 .../compress/configuration/CompressLossyCost.java  |  52 ---
 .../compress/workload/WorkloadAlgorithmTest.java   |  12 +-
 .../compress/workload/WorkloadAnalysisTest.java    |   9 +-
 .../rewrite/RewriteMMCBindZeroVector.java          |  36 ++-
 src/test/resources/component/compress/1-1_y.csv    |   1 +
 .../resources/component/compress/1-1_y.csv.mtd     |   8 +
 src/test/resources/component/compress/README.md    |   2 +-
 .../compress/workload/functions/l2svm_Y.dml}       |  10 +-
 .../functions/compress/compress_mmr_sum.dml        |   2 +-
 ...press_mmr_sum.dml => compress_mmr_sum_plus.dml} |   2 +-
 ...ess_mmr_sum.dml => compress_mmr_sum_plus_2.dml} |   2 +-
 .../WorkloadAnalysisL2SVM.dml}                     |  29 +-
 71 files changed, 1122 insertions(+), 874 deletions(-)

diff --git a/src/main/java/org/apache/sysds/hops/OptimizerUtils.java b/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
index d63b2cb..be916d9 100644
--- a/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
+++ b/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
@@ -1139,6 +1139,9 @@ public class OptimizerUtils
 		// default is worst-case estimate for robustness
 		double ret = 1.0;
 		
+		if(op == null) // If Unknown op, assume the worst
+			return ret;
+
 		if( worstcase )
 		{
 			//NOTE: for matrix-scalar operations this estimate is too conservative, because 
diff --git a/src/main/java/org/apache/sysds/hops/estim/EstimatorDensityMap.java b/src/main/java/org/apache/sysds/hops/estim/EstimatorDensityMap.java
index 9b7169e..65a8463 100644
--- a/src/main/java/org/apache/sysds/hops/estim/EstimatorDensityMap.java
+++ b/src/main/java/org/apache/sysds/hops/estim/EstimatorDensityMap.java
@@ -234,7 +234,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 			_map = init(in);
 			_scaled = false;
 			if( !isPow2(_b) )
-				System.out.println("WARN: Invalid block size: "+_b);
+				LOG.warn("Invalid block size: "+_b);
 		}
 		
 		public DensityMap(MatrixBlock map, int rlenOrig, int clenOrig, int b, boolean scaled) {
@@ -244,7 +244,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 			_map = map;
 			_scaled = scaled;
 			if( !isPow2(_b) )
-				System.out.println("WARN: Invalid block size: "+_b);
+				LOG.warn("Invalid block size: "+_b);
 		}
 		
 		public MatrixBlock getMap() {
diff --git a/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java b/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
index b23397f..11c3ad6 100644
--- a/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
+++ b/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
@@ -24,7 +24,6 @@ import java.util.Map.Entry;
 
 import org.apache.sysds.conf.ConfigurationManager;
 import org.apache.sysds.conf.DMLConfig;
-import org.apache.sysds.hops.Hop;
 import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.lops.Compression.CompressConfig;
 import org.apache.sysds.parser.DMLProgram;
@@ -54,18 +53,19 @@ public class IPAPassCompressionWorkloadAnalysis extends IPAPass {
 		Map<Long, WTreeRoot> map = WorkloadAnalyzer.getAllCandidateWorkloads(prog);
 
 		// Add compression instruction to all remaining locations
-		for(Entry<Long, WTreeRoot> e : map.entrySet()){
-			WTreeRoot tree = e.getValue();
-			CostEstimatorBuilder b = new CostEstimatorBuilder(tree);
-			// filter out compression plans that is known bad
-			
-			if(b.shouldTryToCompress()){
+		for(Entry<Long, WTreeRoot> e : map.entrySet()) {
+			final WTreeRoot tree = e.getValue();
+			final CostEstimatorBuilder b = new CostEstimatorBuilder(tree);
+			final boolean shouldCompress = b.shouldTryToCompress();
+			if(LOG.isTraceEnabled())
+				LOG.trace("IPAPass Should Compress:\n" + tree + "\n" + b + "\n Should Compress: " + shouldCompress);
+
+			// Filter out compression plans that is known to be bad
+			if(shouldCompress) 
 				tree.getRoot().setRequiresCompression(tree);
-				for(Hop h : tree.getDecompressList())
-					h.setRequiresDeCompression();
-			}
+
 		}
-		
+
 		return map != null;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 0b91e5d..d0b1ac5 100644
--- a/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -2801,22 +2801,26 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 
 		// cbind((X %*% Y), matrix(0, nrow(X), 1)) ->
 		// X %*% (cbind(Y, matrix(0, nrow(Y), 1)))
-		// if nRows of x is larger than nCols of y
+		// if nRows of x is larger than nRows of y
 		// rewrite used in MLogReg first level loop.
-		
+
 		if(HopRewriteUtils.isBinary(hi, OpOp2.CBIND) && HopRewriteUtils.isMatrixMultiply(hi.getInput(0)) &&
-			HopRewriteUtils.isDataGenOpWithConstantValue(hi.getInput(1), 0) && hi.getDim1() > hi.getDim2() * 2) {
-			final Hop oldGen = hi.getInput(1);
+			HopRewriteUtils.isDataGenOpWithConstantValue(hi.getInput(1), 0) && hi.getInput(0).getInput(0).dimsKnown() &&
+			hi.getInput(0).getInput(1).dimsKnown()) {
 			final Hop y = hi.getInput(0).getInput(1);
 			final Hop x = hi.getInput(0).getInput(0);
-			final Hop newGen = HopRewriteUtils.createDataGenOp(y, oldGen, 0);
-			final Hop newCBind = HopRewriteUtils.createBinary(y, newGen, OpOp2.CBIND);
-			final Hop newMM = HopRewriteUtils.createMatrixMultiply(x, newCBind);
-
-			HopRewriteUtils.replaceChildReference(parent, hi, newMM, pos);
-			LOG.debug("Applied MMCBind Zero algebraic simplification (line " +hi.getBeginLine()+")." );
-			return newMM;
+			final long m = x.getDim1(); // number of rows in output or X
+			final long n = y.getDim1(); // number of rows in Y or common dimension
+			if(m > n * 2) {
+				final Hop oldGen = hi.getInput(1);
+				final Hop newGen = HopRewriteUtils.createDataGenOp(y, oldGen, 0);
+				final Hop newCBind = HopRewriteUtils.createBinary(y, newGen, OpOp2.CBIND);
+				final Hop newMM = HopRewriteUtils.createMatrixMultiply(x, newCBind);
 
+				HopRewriteUtils.replaceChildReference(parent, hi, newMM, pos);
+				LOG.debug("Applied MMCBind Zero algebraic simplification (line " + hi.getBeginLine() + ").");
+				return newMM;
+			}
 		}
 		return hi;
 	}
diff --git a/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java b/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
index 2f0f1f6c..96e1469 100644
--- a/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
+++ b/src/main/java/org/apache/sysds/hops/rewrite/RewriteCompressedReblock.java
@@ -128,7 +128,11 @@ public class RewriteCompressedReblock extends StatementBlockRewriteRule {
 
 	public static boolean satisfiesSizeConstraintsForCompression(Hop hop) {
 		if(hop.getDim2() >= 1) {
-			return (hop.getDim1() >= 1000 && hop.getDim2() < 100) || hop.getDim1() / hop.getDim2() >= 75 || (hop.getSparsity() < 0.0001 && hop.getDim1() > 1000);
+			return 
+				// If number of rows is above 1000 and either very sparse or number of columns is less than 100.
+				(hop.getDim1() >= 1000 && (hop.getDim2() < 100) || hop.getSparsity() < 0.0001)
+				// If relative ratio between number of rows and columns is better than 75, aka 75 rows per one column.
+				|| hop.getDim1() / hop.getDim2() >= 75;
 		}
 		return false;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
index 5dcc406..bff9a21 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -27,6 +27,7 @@ import java.io.ObjectOutput;
 import java.lang.ref.SoftReference;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.Callable;
@@ -71,8 +72,11 @@ import org.apache.sysds.runtime.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
 import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
 import org.apache.sysds.runtime.functionobjects.Mean;
+import org.apache.sysds.runtime.functionobjects.MinusMultiply;
 import org.apache.sysds.runtime.functionobjects.Multiply;
+import org.apache.sysds.runtime.functionobjects.PlusMultiply;
 import org.apache.sysds.runtime.functionobjects.SwapIndex;
+import org.apache.sysds.runtime.functionobjects.TernaryValueFunction.ValueFunctionWithConstant;
 import org.apache.sysds.runtime.instructions.InstructionUtils;
 import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysds.runtime.instructions.cp.ScalarObject;
@@ -82,6 +86,7 @@ import org.apache.sysds.runtime.matrix.data.IJV;
 import org.apache.sysds.runtime.matrix.data.LibMatrixBincell;
 import org.apache.sysds.runtime.matrix.data.LibMatrixDatagen;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
+import org.apache.sysds.runtime.matrix.data.LibMatrixTercell;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysds.runtime.matrix.data.MatrixValue;
@@ -245,6 +250,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 		ret.allocateDenseBlock();
 
+		if(isOverlapping()){
+			Comparator<AColGroup> comp = Comparator.comparing(x -> effect(x));
+			_colGroups.sort(comp);
+		}
+
 		if(k == 1)
 			decompress(ret);
 		else
@@ -265,6 +275,10 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		return ret;
 	}
 
+	private double effect(AColGroup x){
+		return - Math.max(x.getMax(), Math.abs(x.getMin()));
+	}
+
 	private MatrixBlock decompress(MatrixBlock ret) {
 
 		ret.setNonZeros(nonZeros == -1 && !this.isOverlapping() ? recomputeNonZeros() : nonZeros);
@@ -303,7 +317,12 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	/**
-	 * Get the cached decompressed matrix (if it exists otherwise null)
+	 * Get the cached decompressed matrix (if it exists otherwise null).
+	 * 
+	 * This in practice means that if some other instruction have materialized the decompressed version it can be
+	 * accessed though this method with a guarantee that it did not go through the entire decompression phase.
+	 * 
+	 * @return The cached decompressed matrix, if it does not exist return null
 	 */
 	public MatrixBlock getCachedDecompressed() {
 		if(decompressedVersion != null) {
@@ -457,14 +476,15 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	@Override
 	public void write(DataOutput out) throws IOException {
 		if(getExactSizeOnDisk() > MatrixBlock.estimateSizeOnDisk(rlen, clen, nonZeros)) {
-			// decompress and make a uncompressed column group. this is then used for the serialization, since it is
-			// smaller.
-			// throw new NotImplementedException("Decompressing serialization is not implemented");
-
-			MatrixBlock uncompressed = getUncompressed("Decompressing serialization for smaller serialization");
+			// If the size of this matrixBlock is smaller in uncompressed format, then
+			// decompress and save inside an uncompressed column group.
+			MatrixBlock uncompressed = getUncompressed("for smaller serialization");
 			ColGroupUncompressed cg = new ColGroupUncompressed(uncompressed);
 			allocateColGroup(cg);
 			nonZeros = cg.getNumberNonZeros();
+			// clear the soft reference to the decompressed version, since the one column group is perfectly,
+			// representing the decompressed version.
+			decompressedVersion = null;
 		}
 		// serialize compressed matrix block
 		out.writeInt(rlen);
@@ -651,11 +671,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 		if(transposeOutput) {
 			ReorgOperator r_op = new ReorgOperator(SwapIndex.getSwapIndexFnObject(), op.getNumThreads());
-			return ret.reorgOperations(r_op, new MatrixBlock(), 0, 0, 0);
+			ret = ret.reorgOperations(r_op, new MatrixBlock(), 0, 0, 0);
 		}
-		else
-			return ret;
-
+		
+		if(ret.getNumRows() == 0 || ret.getNumColumns() == 0)
+			throw new DMLCompressionException("Error in outputted MM no dimensions");
+		
+		return ret;
 	}
 
 	private MatrixBlock doubleCompressedAggregateBinaryOperations(CompressedMatrixBlock m1, CompressedMatrixBlock m2,
@@ -667,7 +689,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return aggregateBinaryOperations(m1, getUncompressed(m2), ret, op, transposeLeft, transposeRight);
 		}
 		else if(transposeLeft && !transposeRight) {
-			// Select witch compressed matrix to decompress.
 			if(m1.getNumColumns() > m2.getNumColumns()) {
 				ret = CLALibLeftMultBy.leftMultByMatrixTransposed(m1, m2, ret, op.getNumThreads());
 				ReorgOperator r_op = new ReorgOperator(SwapIndex.getSwapIndexFnObject(), op.getNumThreads());
@@ -1099,10 +1120,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		printDecompressWarning("aggregateTernaryOperations " + op.aggOp.getClass().getSimpleName() + " "
 			+ op.indexFn.getClass().getSimpleName() + " " + op.aggOp.increOp.fn.getClass().getSimpleName() + " "
 			+ op.binaryFn.getClass().getSimpleName() + " m1,m2,m3 " + m1C + " " + m2C + " " + m3C);
-		MatrixBlock left = getUncompressed();
+		MatrixBlock left = getUncompressed(m1);
 		MatrixBlock right1 = getUncompressed(m2);
 		MatrixBlock right2 = getUncompressed(m3);
-		return left.aggregateTernaryOperations(left, right1, right2, ret, op, inCP);
+		ret = left.aggregateTernaryOperations(left, right1, right2, ret, op, inCP);
+		if(ret.getNumRows() == 0 || ret.getNumColumns() == 0)
+			throw new DMLCompressionException("Invalid output");
+		return ret;
 	}
 
 	@Override
@@ -1184,10 +1208,54 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock ternaryOperations(TernaryOperator op, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret) {
-		MatrixBlock left = getUncompressed("ternaryOperations  " + op.fn);
-		MatrixBlock right1 = getUncompressed(m2);
-		MatrixBlock right2 = getUncompressed(m3);
-		return left.ternaryOperations(op, right1, right2, ret);
+
+		// prepare inputs
+		final int r1 = getNumRows();
+		final int r2 = m2.getNumRows();
+		final int r3 = m3.getNumRows();
+		final int c1 = getNumColumns();
+		final int c2 = m2.getNumColumns();
+		final int c3 = m3.getNumColumns();
+		final boolean s1 = (r1 == 1 && c1 == 1);
+		final boolean s2 = (r2 == 1 && c2 == 1);
+		final boolean s3 = (r3 == 1 && c3 == 1);
+		final double d1 = s1 ? quickGetValue(0, 0) : Double.NaN;
+		final double d2 = s2 ? m2.quickGetValue(0, 0) : Double.NaN;
+		final double d3 = s3 ? m3.quickGetValue(0, 0) : Double.NaN;
+		final int m = Math.max(Math.max(r1, r2), r3);
+		final int n = Math.max(Math.max(c1, c2), c3);
+
+		ternaryOperationCheck(s1, s2, s3, m, r1, r2, r3, n, c1, c2, c3);
+
+		final boolean PM_Or_MM = (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply);
+		if(PM_Or_MM && ((s2 && d2 == 0) || (s3 && d3 == 0))) {
+			ret = new CompressedMatrixBlock();
+			ret.copy(this);
+			return ret;
+		}
+
+		if(m2 instanceof CompressedMatrixBlock)
+			m2 = ((CompressedMatrixBlock) m2)
+				.getUncompressed("Ternay Operator arg2 " + op.fn.getClass().getSimpleName());
+		if(m3 instanceof CompressedMatrixBlock)
+			m3 = ((CompressedMatrixBlock) m3)
+				.getUncompressed("Ternay Operator arg3 " + op.fn.getClass().getSimpleName());
+
+		if(s2 != s3 && (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply)) {
+			// SPECIAL CASE for sparse-dense combinations of common +* and -*
+			BinaryOperator bop = ((ValueFunctionWithConstant) op.fn).setOp2Constant(s2 ? d2 : d3);
+			ret = CLALibBinaryCellOp.binaryOperations(bop, this, s2 ? m3 : m2, ret);
+		}
+		else {
+			final boolean sparseOutput = evalSparseFormatInMemory(m, n,
+				(s1 ? m * n * (d1 != 0 ? 1 : 0) : getNonZeros()) +
+					Math.min(s2 ? m * n : m2.getNonZeros(), s3 ? m * n : m3.getNonZeros()));
+			ret.reset(m, n, sparseOutput);
+			final MatrixBlock thisUncompressed = getUncompressed("Ternary Operation not supported");
+			LibMatrixTercell.tercellOp(thisUncompressed, m2, m3, ret, op);
+			ret.examSparsity();
+		}
+		return ret;
 	}
 
 	@Override
@@ -1240,19 +1308,22 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	public MatrixBlock getUncompressed(String operation) {
+		MatrixBlock d_compressed = getCachedDecompressed();
+		if(d_compressed != null)
+			return d_compressed;
+		if(isEmpty())
+			return new MatrixBlock(getNumRows(), getNumColumns(), true);
 		printDecompressWarning(operation);
 		return getUncompressed();
 	}
 
 	private static void printDecompressWarning(String operation) {
-		LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
+		LOG.warn("Decompressing because: " + operation);
 	}
 
 	private static void printDecompressWarning(String operation, MatrixBlock m2) {
 		if(isCompressed(m2))
-			LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
-		else
-			LOG.warn("Operation '" + operation + "' not supported yet - decompressing'");
+			printDecompressWarning(operation);
 	}
 
 	@Override
@@ -1300,13 +1371,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	private void copyCompressedMatrix(CompressedMatrixBlock that) {
-		this.rlen = that.rlen;
-		this.clen = that.clen;
+		this.rlen = that.getNumRows();
+		this.clen = that.getNumColumns();
 		this.sparseBlock = null;
 		this.denseBlock = null;
 		this.nonZeros = that.getNonZeros();
 
-		this._colGroups = new ArrayList<>();
+		this._colGroups = new ArrayList<>(that.getColGroups().size());
 		for(AColGroup cg : that._colGroups)
 			_colGroups.add(cg.copy());
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
index 5f45d56..98567e0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -185,6 +185,9 @@ public class CompressedMatrixBlockFactory {
 		AColGroup cg = ColGroupFactory.genColGroupConst(numRows, numCols, value);
 		block.allocateColGroup(cg);
 		block.recomputeNonZeros();
+		if(block.getNumRows() == 0 || block.getNumColumns() == 0) {
+			throw new DMLCompressionException("Invalid size of allocated constant compressed matrix block");
+		}
 		return block;
 	}
 
@@ -238,14 +241,17 @@ public class CompressedMatrixBlockFactory {
 		_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
 		logPhase();
 
+		final double sizeToCompare = (costEstimator instanceof ComputationCostEstimator &&
+			((ComputationCostEstimator) costEstimator).isDense()) ? _stats.denseSize : _stats.originalSize;
+
 		final boolean isValidForComputeBasedCompression = isComputeBasedCompression() &&
 			(compSettings.minimumCompressionRatio != 1.0) ? _stats.estimatedSizeCols *
-				compSettings.minimumCompressionRatio < _stats.originalSize : true;
+				compSettings.minimumCompressionRatio < sizeToCompare : true;
 		final boolean isValidForMemoryBasedCompression = _stats.estimatedSizeCols *
-			compSettings.minimumCompressionRatio < _stats.originalSize;
+			compSettings.minimumCompressionRatio < sizeToCompare;
 
 		if(isValidForComputeBasedCompression || isValidForMemoryBasedCompression)
-			coCodePhase(sizeEstimator, sizeInfos, costEstimator);
+			coCodePhase(sizeEstimator, sizeInfos, costEstimator, sizeToCompare);
 		else {
 			LOG.info("Estimated Size of singleColGroups: " + _stats.estimatedSizeCols);
 			LOG.info("Original size                    : " + _stats.originalSize);
@@ -257,7 +263,7 @@ public class CompressedMatrixBlockFactory {
 	}
 
 	private void coCodePhase(CompressedSizeEstimator sizeEstimator, CompressedSizeInfo sizeInfos,
-		ICostEstimate costEstimator) {
+		ICostEstimate costEstimator, double sizeToCompare) {
 		coCodeColGroups = CoCoderFactory.findCoCodesByPartitioning(sizeEstimator, sizeInfos, k, costEstimator,
 			compSettings);
 
@@ -267,7 +273,7 @@ public class CompressedMatrixBlockFactory {
 
 		// if cocode is estimated larger than uncompressed abort compression.
 		if(isComputeBasedCompression() &&
-			_stats.estimatedSizeCoCoded * compSettings.minimumCompressionRatio > _stats.originalSize) {
+			_stats.estimatedSizeCoCoded * compSettings.minimumCompressionRatio > sizeToCompare) {
 
 			coCodeColGroups = null;
 			LOG.info("Aborting compression because the cocoded size : " + _stats.estimatedSizeCoCoded);
@@ -281,7 +287,7 @@ public class CompressedMatrixBlockFactory {
 		final int numRows = mb.getNumRows();
 		final long nnz = mb.getNonZeros();
 		final int colGroupSize = 100;
-		if(nnz == numRows) {
+		if(nnz == numRows && numColumns != 1) {
 			boolean onlyOneValues = true;
 			LOG.debug("Looks like one hot encoded.");
 			if(mb.isInSparseFormat()) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
index ea04e8e..b347031 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -108,10 +108,10 @@ public class CompressionSettings {
 	 */
 	public final double minimumCompressionRatio;
 
-	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput,
-		 int seed, boolean lossy, EnumSet<CompressionType> validCompressions,
-		boolean sortValuesByLength, PartitionerType columnPartitioner, int maxColGroupCoCode, double coCodePercentage,
-		int minimumSampleSize, EstimationType estimationType, CostType costComputationType, double minimumCompressionRatio) {
+	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput, int seed,
+		boolean lossy, EnumSet<CompressionType> validCompressions, boolean sortValuesByLength,
+		PartitionerType columnPartitioner, int maxColGroupCoCode, double coCodePercentage, int minimumSampleSize,
+		EstimationType estimationType, CostType costComputationType, double minimumCompressionRatio) {
 		this.samplingRatio = samplingRatio;
 		this.allowSharedDictionary = allowSharedDictionary;
 		this.transposeInput = transposeInput;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index 2864118..156fdfc 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -268,7 +268,7 @@ public class CompressionSettingsBuilder {
 		return this;
 	}
 
-	public CompressionSettingsBuilder setMinimumCompressionRatio(double ratio){
+	public CompressionSettingsBuilder setMinimumCompressionRatio(double ratio) {
 		this.minimumCompressionRatio = ratio;
 		return this;
 	}
@@ -281,6 +281,6 @@ public class CompressionSettingsBuilder {
 	public CompressionSettings create() {
 		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, seed, lossy,
 			validCompressions, sortValuesByLength, columnPartitioner, maxColGroupCoCode, coCodePercentage,
-			minimumSampleSize, estimationType, costType,minimumCompressionRatio);
+			minimumSampleSize, estimationType, costType, minimumCompressionRatio);
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
index 0ad1900..63b579c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
@@ -97,7 +97,7 @@ public class CompressionStatistics {
 		return size == 0.0 ? Double.POSITIVE_INFINITY : (double) originalSize / size;
 	}
 
-	public double getDenseRatio(){
+	public double getDenseRatio() {
 		return size == 0.0 ? Double.POSITIVE_INFINITY : (double) denseSize / size;
 	}
 
@@ -109,7 +109,7 @@ public class CompressionStatistics {
 		sb.append("\nOriginal Size         : " + originalSize);
 		sb.append("\nCompressed Size       : " + size);
 		sb.append("\nCompressionRatio      : " + getRatio());
-		if(colGroupCounts != null){
+		if(colGroupCounts != null) {
 			sb.append("\nCompressionTypes      : " + getGroupsTypesString());
 			sb.append("\nCompressionGroupSizes : " + getGroupsSizesString());
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/DMLCompressionException.java b/src/main/java/org/apache/sysds/runtime/compress/DMLCompressionException.java
index 4761e5e..bc95246 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/DMLCompressionException.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/DMLCompressionException.java
@@ -21,22 +21,22 @@ package org.apache.sysds.runtime.compress;
 
 import org.apache.sysds.runtime.DMLRuntimeException;
 
-public class DMLCompressionException extends DMLRuntimeException{
+public class DMLCompressionException extends DMLRuntimeException {
 	private static final long serialVersionUID = 1L;
 
-	public DMLCompressionException(){
+	public DMLCompressionException() {
 		super("Invalid execution on Compressed MatrixBlock");
 	}
 
 	public DMLCompressionException(String string) {
 		super(string);
 	}
-	
+
 	public DMLCompressionException(Exception e) {
 		super(e);
 	}
 
-	public DMLCompressionException(String string, Exception ex){
-		super(string,ex);
+	public DMLCompressionException(String string, Exception ex) {
+		super(string, ex);
 	}
-}
\ No newline at end of file
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
index 3091ba0..cf366e7 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeGreedy.java
@@ -108,8 +108,8 @@ public class CoCodeGreedy extends AColumnCoCoder {
 			else
 				break;
 		}
-
-		LOG.debug(mem.stats());
+		if(LOG.isDebugEnabled())
+			LOG.debug("Memorizer stats:" + mem.stats());
 		mem.resetStats();
 
 		List<CompressedSizeInfoColGroup> ret = new ArrayList<>(workset.size());
@@ -132,10 +132,6 @@ public class CoCodeGreedy extends AColumnCoCoder {
 			mem.put(new ColIndexes(g.getColumns()), g);
 		}
 
-		// public CompressedSizeInfoColGroup get(CompressedSizeInfoColGroup g) {
-		// 	return mem.get(new ColIndexes(g.getColumns()));
-		// }
-
 		public CompressedSizeInfoColGroup get(ColIndexes c) {
 			return mem.get(c);
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java
index 9416618..d8b85c1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java
@@ -458,7 +458,9 @@ public final class ColGroupFactory {
 	}
 
 	public static AColGroup genColGroupConst(int numRows, int numCols, double value) {
-
+		if(numRows <= 0 || numCols <= 0)
+			throw new DMLCompressionException(
+				"Invalid construction of constant column group with rows/cols: " + numRows + "/" + numCols);
 		int[] colIndices = new int[numCols];
 		for(int i = 0; i < numCols; i++)
 			colIndices[i] = i;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
index b1b0540..35ba409 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
@@ -142,9 +142,9 @@ public class ColGroupUncompressed extends AColGroup {
 		_data = data;
 	}
 
-	private static int[] generateColumnList(int nCol){
+	private static int[] generateColumnList(int nCol) {
 		int[] cols = new int[nCol];
-		for(int i = 0; i< nCol; i++)
+		for(int i = 0; i < nCol; i++)
 			cols[i] = i;
 		return cols;
 	}
@@ -456,7 +456,6 @@ public class ColGroupUncompressed extends AColGroup {
 	@Override
 	public AColGroup copy() {
 		MatrixBlock newData = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), _data.isInSparseFormat());
-		// _data.copy(newData);
 		newData.copy(_data);
 		return new ColGroupUncompressed(_colIndexes, newData);
 	}
@@ -640,7 +639,6 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public void computeColSums(double[] c) {
-		// TODO Auto-generated method stub
 		MatrixBlock colSum = _data.colSum();
 		if(colSum.isInSparseFormat()) {
 			throw new NotImplementedException();
@@ -649,7 +647,6 @@ public class ColGroupUncompressed extends AColGroup {
 			double[] dv = colSum.getDenseBlockValues();
 			for(int i = 0; i < _colIndexes.length; i++)
 				c[_colIndexes[i]] += dv[i];
-
 		}
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index 6844095..c931880 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -896,8 +896,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		}
 	}
 
-	private static boolean logMM = true;
-
 	/**
 	 * Matrix Multiply the two matrices, note that the left side is transposed,
 	 * 
@@ -978,10 +976,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 		}
 		catch(Exception e) {
-			if(logMM) {
-				LOG.error("\nLeft (transposed):\n" + left + "\nRight:\n" + right);
-				logMM = false;
-			}
 			throw new DMLCompressionException("MM of pre aggregated colGroups failed", e);
 		}
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
index 6c0b9fa..63fecc5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToFactory.java
@@ -109,12 +109,13 @@ public final class MapToFactory {
 		final int nVR = right.getUnique();
 		final int size = left.size();
 		final long maxUnique = nVL * nVR;
-		if(maxUnique > (long)Integer.MAX_VALUE)
-			throw new DMLCompressionException("Joining impossible using linearized join, since each side has a large number of unique values");
+		if(maxUnique > (long) Integer.MAX_VALUE)
+			throw new DMLCompressionException(
+				"Joining impossible using linearized join, since each side has a large number of unique values");
 		if(size != right.size())
 			throw new DMLCompressionException("Invalid input maps to join, must contain same number of rows");
 
-		return computeJoin(left, right, size, nVL, (int)maxUnique);
+		return computeJoin(left, right, size, nVL, (int) maxUnique);
 	}
 
 	private static AMapToData computeJoin(AMapToData left, AMapToData right, int size, int nVL, int maxUnique) {
@@ -133,11 +134,11 @@ public final class MapToFactory {
 				tmp.set(i, newUID - 1);
 				map[nv] = newUID++;
 			}
-			else 
+			else
 				tmp.set(i, mapV - 1);
 		}
 
-		tmp.setUnique(newUID-1);
+		tmp.setUnique(newUID - 1);
 		return tmp;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/ComputationCostEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/cost/ComputationCostEstimator.java
index 8db6729..1653f51 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/ComputationCostEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/ComputationCostEstimator.java
@@ -31,7 +31,7 @@ public class ComputationCostEstimator implements ICostEstimate {
 	private final boolean _isCompareAll;
 
 	private final int _nRows;
-	// private final int _nColsInMatrix;
+	private final int _nCols;
 
 	// Iteration through each row of decompressed.
 	private final int _scans;
@@ -44,6 +44,8 @@ public class ComputationCostEstimator implements ICostEstimate {
 	// private final int _rowBasedOps;
 	private final int _dictionaryOps;
 
+	private final boolean _isDensifying;
+
 	/**
 	 * A Cost based estimator based on the WTree that is parsed in IPA.
 	 * 
@@ -51,7 +53,7 @@ public class ComputationCostEstimator implements ICostEstimate {
 	 */
 	protected ComputationCostEstimator(int nRows, int nCols, boolean compareAll, InstructionTypeCounter counts) {
 		_nRows = nRows;
-		// _nColsInMatrix = nCols;
+		_nCols = nCols;
 		_isCompareAll = compareAll;
 		_scans = counts.scans;
 		_decompressions = counts.decompressions;
@@ -60,11 +62,31 @@ public class ComputationCostEstimator implements ICostEstimate {
 		_compressedMultiplication = counts.compressedMultiplications;
 		_rightMultiplications = counts.rightMultiplications;
 		_dictionaryOps = counts.dictionaryOps;
+		_isDensifying = counts.isDensifying;
 		// _rowBasedOps = counts.rowBasedOps;
 		if(LOG.isDebugEnabled())
 			LOG.debug(this);
 	}
 
+	public ComputationCostEstimator(int nRows, int nCols, int scans, int decompressions, int overlappingDecompressions,
+		int leftMultiplictions, int compressedMultiplication, int rightMultiplications, int dictioanaryOps, boolean isDensifying) {
+		_nRows = nRows;
+		_nCols = nCols;
+		_isCompareAll = false;
+		_scans = scans;
+		_decompressions = decompressions;
+		_overlappingDecompressions = overlappingDecompressions;
+		_leftMultiplications = leftMultiplictions;
+		_compressedMultiplication = compressedMultiplication;
+		_rightMultiplications = rightMultiplications;
+		_dictionaryOps = dictioanaryOps;
+		_isDensifying = isDensifying;
+	}
+
+	public static ComputationCostEstimator genDefaultCostCase(int nRows, int nCols){
+		return new ComputationCostEstimator(nRows,nCols, 1, 1, 0, 1, 1, 1, 10, true);
+	}
+
 	@Override
 	public double getUncompressedCost(int nRows, int nCols, int sparsity) {
 		throw new NotImplementedException();
@@ -84,6 +106,7 @@ public class ComputationCostEstimator implements ICostEstimate {
 		// 16 is assuming that the right side is 16 rows.
 		double rmc = rightMultCost(g) * 16;
 		cost += _rightMultiplications * rmc;
+
 		// cost += _compressedMultiplication * (lmc + rmc);
 		cost += _compressedMultiplication * _compressedMultCost(g);
 		cost += _dictionaryOps * dictionaryOpsCost(g);
@@ -110,28 +133,29 @@ public class ComputationCostEstimator implements ICostEstimate {
 	}
 
 	private double _compressedMultCost(CompressedSizeInfoColGroup g) {
-		final int nCols = g.getColumns().length;
+		final int nColsInGroup = g.getColumns().length;
 		final double mcf = g.getMostCommonFraction();
 		final double preAggregateCost = mcf > 0.6 ? _nRows * (1 - 0.7 * mcf) : _nRows;
 
 		final double numberTuples = (float) g.getNumVals();
 		final double tupleSparsity = g.getTupleSparsity();
-		final double postScalingCost = (nCols > 1 && tupleSparsity > 0.4) ? numberTuples * nCols * tupleSparsity *
-			1.4 : numberTuples * nCols;
+		final double postScalingCost = (nColsInGroup > 1 && tupleSparsity > 0.4) ? numberTuples * nColsInGroup * tupleSparsity *
+			1.4 : numberTuples * nColsInGroup;
 		if(numberTuples < 64000)
 			return preAggregateCost + postScalingCost;
 		else
 			return preAggregateCost * (numberTuples / 64000) + postScalingCost * (numberTuples / 64000);
 	}
 
-	private static double rightMultCost(CompressedSizeInfoColGroup g) {
-		final int nCols = g.getColumns().length;
+	private double rightMultCost(CompressedSizeInfoColGroup g) {
+		final int nColsInGroup = g.getColumns().length;
 		final int numberTuples = g.getNumVals() * 10;
 		final double tupleSparsity = g.getTupleSparsity();
-		final double postScalingCost = (nCols > 1 && tupleSparsity > 0.4) ? numberTuples * nCols * tupleSparsity *
-			1.4 : numberTuples * nCols;
+		final double postScalingCost = (nColsInGroup > 1 && tupleSparsity > 0.4) ? numberTuples * nColsInGroup * tupleSparsity *
+			1.4 : numberTuples * nColsInGroup;
+		final double postAllocationCost = _nCols * numberTuples;
 
-		return postScalingCost;
+		return postScalingCost + postAllocationCost;
 	}
 
 	private double decompressionCost(CompressedSizeInfoColGroup g) {
@@ -227,6 +251,10 @@ public class ComputationCostEstimator implements ICostEstimate {
 		return true;
 	}
 
+	public boolean isDense(){
+		return _isDensifying;
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
index 3d58bff..cfd22dd 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorBuilder.java
@@ -26,10 +26,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.hops.AggUnaryOp;
 import org.apache.sysds.hops.Hop;
 import org.apache.sysds.hops.IndexingOp;
-import org.apache.sysds.hops.ParameterizedBuiltinOp;
 import org.apache.sysds.runtime.compress.workload.Op;
 import org.apache.sysds.runtime.compress.workload.OpMetadata;
-import org.apache.sysds.runtime.compress.workload.OpOverlappingDecompress;
 import org.apache.sysds.runtime.compress.workload.OpSided;
 import org.apache.sysds.runtime.compress.workload.WTreeNode;
 import org.apache.sysds.runtime.compress.workload.WTreeRoot;
@@ -44,6 +42,8 @@ public final class CostEstimatorBuilder implements Serializable {
 
 	public CostEstimatorBuilder(WTreeRoot root) {
 		counter = new InstructionTypeCounter();
+		if(root.isDecompressing())
+			counter.decompressions++;
 		for(Op o : root.getOps())
 			addOp(1, o, counter);
 		for(WTreeNode n : root.getChildNodes())
@@ -59,9 +59,7 @@ public final class CostEstimatorBuilder implements Serializable {
 	}
 
 	private static void addNode(int count, WTreeNode n, InstructionTypeCounter counter) {
-
 		int mult = n.getReps();
-
 		for(Op o : n.getOps())
 			addOp(count * mult, o, counter);
 		for(WTreeNode nc : n.getChildNodes())
@@ -69,27 +67,29 @@ public final class CostEstimatorBuilder implements Serializable {
 	}
 
 	private static void addOp(int count, Op o, InstructionTypeCounter counter) {
+		if(o.isDecompressing()) {
+			if(o.isOverlapping())
+				counter.overlappingDecompressions += count;
+			else
+				counter.decompressions += count;
+		}
+		if(o.isDensifying()){
+			counter.isDensifying = true;
+		}
+
 		if(o instanceof OpSided) {
 			OpSided os = (OpSided) o;
 			if(os.isLeftMM())
 				counter.leftMultiplications += count;
-			else if(os.isRightMM()) {
+			else if(os.isRightMM())
 				counter.rightMultiplications += count;
-				if(os.isDecompressing())
-					counter.overlappingDecompressions += count;
-			}
 			else
 				counter.compressedMultiplications += count;
 		}
 		else if(o instanceof OpMetadata) {
 			// ignore it
 		}
-		else if(o instanceof OpOverlappingDecompress) {
-			counter.overlappingDecompressions += count;
-		}
 		else {
-			if(o.isDecompressing())
-				counter.decompressions += count;
 			Hop h = o.getHop();
 			if(h instanceof AggUnaryOp) {
 				AggUnaryOp agop = (AggUnaryOp) o.getHop();
@@ -108,12 +108,6 @@ public final class CostEstimatorBuilder implements Serializable {
 					counter.indexing++;
 				else if(idxO.isAllRows())
 					counter.dictionaryOps += count; // Technically not correct but better than decompression
-				else
-					counter.decompressions += count;
-			}
-			else if(h instanceof ParameterizedBuiltinOp) {
-				// ParameterizedBuiltinOp pop = (ParameterizedBuiltinOp) h;
-				counter.decompressions += count;
 			}
 			else
 				counter.dictionaryOps += count;
@@ -122,7 +116,7 @@ public final class CostEstimatorBuilder implements Serializable {
 
 	public boolean shouldTryToCompress() {
 		int numberOps = 0;
-		numberOps += counter.scans + counter.leftMultiplications * 2 + counter.rightMultiplications +
+		numberOps += counter.scans + counter.leftMultiplications * 2 + counter.rightMultiplications * 2 +
 			counter.compressedMultiplications * 4 + counter.dictionaryOps;
 		numberOps -= counter.decompressions + counter.overlappingDecompressions;
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
index 83b794e..2e64781 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/CostEstimatorFactory.java
@@ -38,14 +38,8 @@ public final class CostEstimatorFactory {
 				return new DistinctCostEstimator(nRows, cs);
 			case W_TREE:
 			case AUTO:
-				if(root != null) {
-					CostEstimatorBuilder b = new CostEstimatorBuilder(root);
-					if(LOG.isDebugEnabled())
-						LOG.debug(b);
-					return b.create(nRows, nCols);
-				}
-				else
-					return new MemoryCostEstimator();
+				return root != null ? new CostEstimatorBuilder(root).create(nRows, nCols) : ComputationCostEstimator
+					.genDefaultCostCase(nRows, nCols);
 			case MEMORY:
 			default:
 				return new MemoryCostEstimator();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/InstructionTypeCounter.java b/src/main/java/org/apache/sysds/runtime/compress/cost/InstructionTypeCounter.java
index 8c2b594..215983b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/InstructionTypeCounter.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/InstructionTypeCounter.java
@@ -34,6 +34,8 @@ public final class InstructionTypeCounter implements Serializable {
 	protected int dictionaryOps = 0; // base cost is one pass of dictionary
 	protected int indexing = 0;
 
+	protected boolean isDensifying = false;
+
 	protected InstructionTypeCounter() {
 
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
index ef7d9dd..912321c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
@@ -189,8 +189,8 @@ public abstract class CompressedSizeEstimator {
 	 * 
 	 * if the number of distinct elements in both sides multiplied is larger than Integer, return null.
 	 * 
-	 * If either side was constructed without analysis then fall back to default materialization of double arrays.
-	 * O
+	 * If either side was constructed without analysis then fall back to default materialization of double arrays. O
+	 * 
 	 * @param g1 First group
 	 * @param g2 Second group
 	 * @return A joined compressed size estimation for the group.
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
index 1531781..2e80532 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -37,7 +37,8 @@ public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate, int nrUniqueUpperBound) {
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate,
+		int nrUniqueUpperBound) {
 		// exact estimator can ignore upper bound.
 		ABitmap entireBitMap = BitmapEncoder.extractBitmap(colIndexes, _data, _transposed, estimate);
 		EstimationFactors em = estimateCompressedColGroupSize(entireBitMap, colIndexes);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
index 6e9208a..a4b1c99 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
@@ -65,7 +65,7 @@ public class CompressedSizeEstimatorFactory {
 		CompressedSizeEstimatorSample estS = new CompressedSizeEstimatorSample(data, cs, sampleSize, k);
 		int double_number = 1;
 		while(estS.getSample() == null) {
-			LOG.error("Warining doubling sample size " + double_number++);
+			LOG.warn("Doubling sample size " + double_number++);
 			sampleSize = sampleSize * 2;
 			if(shouldUseExactEstimator(cs, nRows, sampleSize, nnzRows))
 				return new CompressedSizeEstimatorExact(data, cs);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
index 218e12b..8b9df2c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -94,7 +94,8 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate, int nrUniqueUpperBound) {
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int estimate,
+		int nrUniqueUpperBound) {
 
 		// extract statistics from sample
 		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, _sample, _transposed, estimate);
@@ -109,7 +110,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	@Override
 	protected CompressedSizeInfoColGroup estimateJoinCompressedSize(int[] joined, CompressedSizeInfoColGroup g1,
 		CompressedSizeInfoColGroup g2, int joinedMaxDistinct) {
-		if((long)g1.getNumVals() * g2.getNumVals() >(long)Integer.MAX_VALUE )
+		if((long) g1.getNumVals() * g2.getNumVals() > (long) Integer.MAX_VALUE)
 			return null;
 
 		final AMapToData map = MapToFactory.join(g1.getMap(), g2.getMap());
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
index 1026aa2..5e29272 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -183,7 +183,7 @@ public class CompressedSizeInfoColGroup {
 		return _facts.cols;
 	}
 
-	public int getNumRows(){
+	public int getNumRows() {
 		return _facts.numRows;
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java b/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
index 28d0821..d6a4005 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
@@ -169,7 +169,7 @@ public class EstimationFactors {
 			if(zerosLargestOffset)
 				largestOffs = zerosOffs;
 
-			double overAllSparsity = (double) overallNonZeroCount / ((long)numRows * (long)cols.length);
+			double overAllSparsity = (double) overallNonZeroCount / ((long) numRows * (long) cols.length);
 			double tupleSparsity = (double) tupleNonZeroCount / (numVals * cols.length);
 
 			return new EstimationFactors(cols, numVals, numOffs, largestOffs, frequencies, numRuns, numSingle, numRows,
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
index ab60d9f..daef9ef 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibAppend.java
@@ -49,7 +49,7 @@ public class CLALibAppend {
 			left = CompressedMatrixBlockFactory.genUncompressedCompressedMatrixBlock(left);
 		}
 		if(!(right instanceof CompressedMatrixBlock) && m > 1000) {
-			LOG.warn("Appending uncompressed column group right");
+			LOG.info("Appending uncompressed column group right");
 			left = CompressedMatrixBlockFactory.genUncompressedCompressedMatrixBlock(right);
 		}
 
@@ -66,13 +66,12 @@ public class CLALibAppend {
 		ret = appendColGroups(ret, leftC.getColGroups(), rightC.getColGroups(), leftC.getNumColumns());
 
 		double compressedSize = ret.getInMemorySize();
-		double uncompressedSize = MatrixBlock.estimateSizeInMemory(m,n, ret.getSparsity());
+		double uncompressedSize = MatrixBlock.estimateSizeInMemory(m, n, ret.getSparsity());
 
-		
 		if(compressedSize * 10 < uncompressedSize)
 			return ret;
 		else
-			return ret.getUncompressed("Decompressing c bind matrix");
+			return ret.getUncompressed("Decompressing c bind matrix because it had to small compression ratio");
 	}
 
 	private static MatrixBlock appendRightEmpty(CompressedMatrixBlock left, MatrixBlock right) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
index 99d9c92..e330ab8 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
@@ -31,7 +31,9 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
@@ -39,6 +41,7 @@ import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictiona
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Divide;
 import org.apache.sysds.runtime.functionobjects.Minus;
+import org.apache.sysds.runtime.functionobjects.Minus1Multiply;
 import org.apache.sysds.runtime.functionobjects.MinusMultiply;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.Plus;
@@ -58,72 +61,107 @@ public class CLALibBinaryCellOp {
 
 	private static final Log LOG = LogFactory.getLog(CLALibBinaryCellOp.class.getName());
 
-	public static MatrixBlock binaryOperations(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock thatValue,
+	public static MatrixBlock binaryOperations(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock that,
 		MatrixBlock result) {
-		MatrixBlock that = CompressedMatrixBlock.getUncompressed(thatValue, "Decompressing right side in BinaryOps");
-		if(m1.getNumRows() <= 0)
-			LOG.error(m1);
-		if(that.getNumRows() <= 0)
-			LOG.error(that);
-		LibMatrixBincell.isValidDimensionsBinary(m1, that);
-		BinaryAccessType atype = LibMatrixBincell.getBinaryAccessType(m1, that);
-		return selectProcessingBasedOnAccessType(op, m1, that, result, atype, false);
+		try {
+			if(that.getNumRows() == 1 && that.getNumColumns() == 1) {
+				ScalarOperator sop = new RightScalarOperator(op.fn, that.getValue(0, 0), op.getNumThreads());
+				return CLALibScalar.scalarOperations(sop, m1, result);
+			}
+			if(that.isEmpty())
+				return binaryOperationsEmpty(op, m1, that, result);
+			that = CompressedMatrixBlock.getUncompressed(that, "Decompressing right side in BinaryOps");
+			LibMatrixBincell.isValidDimensionsBinary(m1, that);
+			BinaryAccessType atype = LibMatrixBincell.getBinaryAccessType(m1, that);
+			return selectProcessingBasedOnAccessType(op, m1, that, result, atype, false);
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException("Failed to perform compressed binary operation: " + op, e);
+		}
 	}
 
-	public static MatrixBlock binaryOperationsLeft(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock thatValue,
+	public static MatrixBlock binaryOperationsLeft(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock that,
 		MatrixBlock result) {
-		MatrixBlock that = CompressedMatrixBlock.getUncompressed(thatValue, "Decompressing left side in BinaryOps");
+		if(that.getNumRows() == 1 && that.getNumColumns() == 1) {
+			ScalarOperator sop = new LeftScalarOperator(op.fn, that.getValue(0, 0), op.getNumThreads());
+			return CLALibScalar.scalarOperations(sop, m1, result);
+		}
+		if(that.isEmpty())
+			throw new NotImplementedException("Not handling left empty yet");
+
+		that = CompressedMatrixBlock.getUncompressed(that, "Decompressing left side in BinaryOps");
 		LibMatrixBincell.isValidDimensionsBinary(that, m1);
-		thatValue = that;
 		BinaryAccessType atype = LibMatrixBincell.getBinaryAccessType(that, m1);
 		return selectProcessingBasedOnAccessType(op, m1, that, result, atype, true);
 	}
 
+	private static MatrixBlock binaryOperationsEmpty(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock that,
+		MatrixBlock result) {
+		final ValueFunction fn = op.fn;
+		if((m1.getNumRows() == that.getNumRows() && m1.getNumColumns() == that.getNumColumns()) ||
+			m1.getNumColumns() == that.getNumColumns()) {
+
+			if(fn instanceof Multiply)
+				result = CompressedMatrixBlockFactory.createConstant(m1.getNumRows(), m1.getNumColumns(), 0);
+			else if(fn instanceof Minus1Multiply)
+				result = CompressedMatrixBlockFactory.createConstant(m1.getNumRows(), m1.getNumColumns(), 1);
+			else if(fn instanceof Minus || fn instanceof Plus || fn instanceof MinusMultiply || fn instanceof PlusMultiply){
+				CompressedMatrixBlock ret = new CompressedMatrixBlock();
+				ret.copy(m1);
+				return ret;
+			}
+			else
+				throw new NotImplementedException("Function Type: " + fn);
+
+			return result;
+		}
+		else {
+			final long lr = m1.getNumRows();
+			final long rr = that.getNumRows();
+			final long lc = m1.getNumColumns();
+			final long rc = that.getNumColumns();
+			throw new NotImplementedException(
+				"Not Implemented sizes: left(" + lr + ", " + lc + ") right(" + rr + ", " + rc + ")");
+		}
+	}
+
 	private static MatrixBlock selectProcessingBasedOnAccessType(BinaryOperator op, CompressedMatrixBlock m1,
 		MatrixBlock that, MatrixBlock result, BinaryAccessType atype, boolean left) {
+		final int outRows = m1.getNumRows();
+		final int outCols = m1.getNumColumns();
+		// TODO optimize to allow for sparse outputs.
+		final int outCells = outRows * outCols;
 		if(atype == BinaryAccessType.MATRIX_COL_VECTOR) {
+			result.reset(outRows, Math.max(outCols, that.getNumColumns()), outCells);
 			MatrixBlock d_compressed = m1.getCachedDecompressed();
 			if(d_compressed != null) {
 				if(left)
-					return that.binaryOperations(op, d_compressed, result);
+					LibMatrixBincell.bincellOp(that, d_compressed, result, op);
 				else
-					return d_compressed.binaryOperations(op, that, result);
+					LibMatrixBincell.bincellOp(d_compressed, that, result, op);
+				return result;
 			}
 			else
 				return binaryMVCol(m1, that, op, left);
 
 		}
 		else if(atype == BinaryAccessType.MATRIX_MATRIX) {
-			if(that.isEmpty()) {
-				ScalarOperator sop = left ? new LeftScalarOperator(op.fn, 0, -1) : new RightScalarOperator(op.fn, 0,
-					-1);
-				return CLALibScalar.scalarOperations(sop, m1, result);
-			}
-			else {
-				MatrixBlock d_compressed = m1.getCachedDecompressed();
-				if(d_compressed != null) {
-					// copy the decompressed matrix if there is a decompressed matrix already.
-					MatrixBlock tmp = d_compressed;
-					d_compressed = new MatrixBlock(m1.getNumRows(), m1.getNumColumns(), false);
-					d_compressed.copy(tmp);
-				}
-				else {
-					d_compressed = m1.decompress(op.getNumThreads());
-					m1.clearSoftReferenceToDecompressed();
-				}
+			result.reset(outRows, outCols, outCells);
 
-				if(left)
-					return LibMatrixBincell.bincellOpInPlaceLeft(d_compressed, that, op);
-				else
-					return LibMatrixBincell.bincellOpInPlaceRight(d_compressed, that, op);
-
-			}
+			MatrixBlock d_compressed = m1.getCachedDecompressed();
+			if(d_compressed == null)
+				d_compressed = m1.getUncompressed("MatrixMatrix " + op);
+			
+			if(left)
+				LibMatrixBincell.bincellOp(that, d_compressed, result, op);
+			else
+				LibMatrixBincell.bincellOp(d_compressed, that, result, op);
+			return result;
 		}
 		else if(isSupportedBinaryCellOp(op.fn))
 			return bincellOp(m1, that, setupCompressedReturnMatrixBlock(m1, result), op, left);
 		else {
-			LOG.warn("Decompressing since Binary Ops" + op.fn + " is not supported compressed");
-			return CompressedMatrixBlock.getUncompressed(m1).binaryOperations(op, that, result);
+			return CompressedMatrixBlock.getUncompressed(m1, "BinaryOp: " + op.fn).binaryOperations(op, that, result);
 		}
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
index 5c9f7b2..62a1c7a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
@@ -27,8 +27,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.CompressionSettings;
@@ -58,10 +56,7 @@ import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class CLALibCompAgg {
-
-	private static final Log LOG = LogFactory.getLog(CLALibCompAgg.class.getName());
-
-	// private static final long MIN_PAR_AGG_THRESHOLD = 8 * 1024 * 1024;
+	// private static final Log LOG = LogFactory.getLog(CLALibCompAgg.class.getName());
 	private static final long MIN_PAR_AGG_THRESHOLD = 8 * 1024;
 
 	private static ThreadLocal<MatrixBlock> memPool = new ThreadLocal<MatrixBlock>() {
@@ -87,14 +82,13 @@ public class CLALibCompAgg {
 
 			if(denseSize < 5 * currentSize && inputMatrix.getColGroups().size() > 5 &&
 				denseSize <= localMaxMemory / 2) {
-				LOG.info("Decompressing for unaryAggregate because of overlapping state");
-				inputMatrix.decompress(op.getNumThreads());
+				inputMatrix.getUncompressed("Decompressing for unaryAggregate because of overlapping state");
 			}
 			MatrixBlock decomp = inputMatrix.getCachedDecompressed();
 			if(decomp != null)
 				return decomp.aggregateUnaryOperations(op, result, blen, indexesIn, inCP);
 		}
-		
+
 		// initialize and allocate the result
 		if(result == null)
 			result = new MatrixBlock(tempCellIndex.row, tempCellIndex.column, false);
@@ -113,7 +107,7 @@ public class CLALibCompAgg {
 			else
 				aggregateUnaryNormalCompressedMatrixBlock(inputMatrix, result, opm, blen, indexesIn, inCP);
 		}
-		
+
 		result.recomputeNonZeros();
 		if(op.aggOp.existsCorrection() && !inCP) {
 			result = addCorrection(result, op);
@@ -256,8 +250,7 @@ public class CLALibCompAgg {
 					f.get();
 		}
 		catch(InterruptedException | ExecutionException e) {
-			LOG.error("Aggregate In parallel failed.");
-			throw new DMLRuntimeException(e);
+			throw new DMLRuntimeException("Aggregate In parallel failed.", e);
 		}
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index 5fc9016..8b78e58 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -58,11 +58,11 @@ public class CLALibLeftMultBy {
 		return ret;
 	}
 
-	public static MatrixBlock leftMultByMatrixTransposed(CompressedMatrixBlock m1, CompressedMatrixBlock m2,
+	public static MatrixBlock leftMultByMatrixTransposed(CompressedMatrixBlock right, CompressedMatrixBlock left,
 		MatrixBlock ret, int k) {
-
-		ret = prepareReturnMatrix(m1, m2, ret, true);
-		leftMultByCompressedTransposedMatrix(m1.getColGroups(), m2, ret, k, m1.getNumColumns(), m1.isOverlapping());
+		LOG.warn("Compressed Compressed matrix multiplication");
+		ret = prepareReturnMatrix(right, left, ret, true);
+		leftMultByCompressedTransposedMatrix(right, left, ret, k);
 
 		ret.recomputeNonZeros();
 		return ret;
@@ -79,6 +79,15 @@ public class CLALibLeftMultBy {
 		return ret;
 	}
 
+	/**
+	 * Prepare the output matrix.
+	 * 
+	 * @param m1          The right hand side matrix
+	 * @param m2          The left hand side matrix
+	 * @param ret         The output matrix to reallocate
+	 * @param doTranspose Boolean specifying if the m2 (left side) matrix should be considered transposed
+	 * @return the result matrix allocated.
+	 */
 	private static MatrixBlock prepareReturnMatrix(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret,
 		boolean doTranspose) {
 		final int numRowsOutput = doTranspose ? m2.getNumColumns() : m2.getNumRows();
@@ -87,6 +96,8 @@ public class CLALibLeftMultBy {
 			ret = new MatrixBlock(numRowsOutput, numColumnsOutput, false, numRowsOutput * numColumnsOutput);
 		else if(!(ret.getNumColumns() == numColumnsOutput && ret.getNumRows() == numRowsOutput && ret.isAllocated()))
 			ret.reset(numRowsOutput, numColumnsOutput, false, numRowsOutput * numColumnsOutput);
+
+		ret.allocateDenseBlock();
 		return ret;
 	}
 
@@ -94,11 +105,9 @@ public class CLALibLeftMultBy {
 		final boolean overlapping = cmb.isOverlapping();
 		final List<AColGroup> groups = cmb.getColGroups();
 
-		result.allocateDenseBlock();
-
 		if(overlapping) {
 			LOG.warn("Inefficient TSMM with overlapping matrix could be implemented multi-threaded but is not yet.");
-			leftMultByCompressedTransposedMatrix(groups, groups, result);
+			multAllColGroups(groups, groups, result);
 		}
 		else {
 			final boolean containsSDC = containsSDC(groups);
@@ -153,26 +162,29 @@ public class CLALibLeftMultBy {
 
 	}
 
-	private static MatrixBlock leftMultByCompressedTransposedMatrix(List<AColGroup> colGroups,
-		CompressedMatrixBlock that, MatrixBlock ret, int k, int numColumns, boolean overlapping) {
+	private static MatrixBlock leftMultByCompressedTransposedMatrix(CompressedMatrixBlock right,
+		CompressedMatrixBlock left, MatrixBlock ret, int k) {
 
-		ret.allocateDenseBlock();
-		List<AColGroup> thatCGs = that.getColGroups();
+		final List<AColGroup> thisCGs = right.getColGroups();
+		final List<AColGroup> thatCGs = left.getColGroups();
+
+		final boolean thisOverlapping = right.isOverlapping();
+		final boolean thatOverlapping = left.isOverlapping();
+		final boolean anyOverlap = thisOverlapping || thatOverlapping;
 
-		if(k <= 1 || overlapping || that.isOverlapping()) {
-			if(overlapping || that.isOverlapping())
+		if(k <= 1 || anyOverlap) {
+			if(anyOverlap)
 				LOG.warn("Inefficient Compressed multiplication with overlapping matrix"
 					+ " could be implemented multi-threaded but is not yet.");
-			leftMultByCompressedTransposedMatrix(colGroups, thatCGs, ret);
+			multAllColGroups(thisCGs, thatCGs, ret);
 		}
 		else {
-
 			try {
 				ExecutorService pool = CommonThreadPool.get(k);
 				ArrayList<Callable<Object>> tasks = new ArrayList<>();
-				for(int i = 0; i < thatCGs.size(); i++) {
-					tasks.add(new LeftMultByCompressedTransposedMatrixTask(colGroups, thatCGs.get(i), ret));
-				}
+				for(int i = 0; i < thatCGs.size(); i++)
+					for(int j = 0; j < thisCGs.size(); j++)
+						tasks.add(new LeftMultByCompressedTransposedMatrixTask(thisCGs.get(j), thatCGs.get(i), ret));
 
 				for(Future<Object> tret : pool.invokeAll(tasks))
 					tret.get();
@@ -182,29 +194,29 @@ public class CLALibLeftMultBy {
 				throw new DMLRuntimeException(e);
 			}
 		}
+
 		ret.recomputeNonZeros();
 		return ret;
 	}
 
 	private static class LeftMultByCompressedTransposedMatrixTask implements Callable<Object> {
-		private final List<AColGroup> _groups;
+		private final AColGroup _right;
 		private final AColGroup _left;
 		private final MatrixBlock _ret;
-		private final int _start;
-		private final int _end;
 
-		protected LeftMultByCompressedTransposedMatrixTask(List<AColGroup> groups, AColGroup left, MatrixBlock ret) {
-			_groups = groups;
+		protected LeftMultByCompressedTransposedMatrixTask(AColGroup right, AColGroup left, MatrixBlock ret) {
+			_right = right;
 			_left = left;
 			_ret = ret;
-			_start = 0;
-			_end = groups.size();
 		}
 
 		@Override
 		public Object call() {
 			try {
-				leftMultByCompressedTransposedMatrix(_left, _groups, _ret, _start, _end);
+				if(_right != _left)
+					_right.leftMultByAColGroup(_left, _ret);
+				else
+					_right.tsmm(_ret);
 			}
 			catch(Exception e) {
 				e.printStackTrace();
@@ -214,21 +226,16 @@ public class CLALibLeftMultBy {
 		}
 	}
 
-	private static void leftMultByCompressedTransposedMatrix(List<AColGroup> thisCG, List<AColGroup> thatCG,
-		MatrixBlock ret) {
-		for(AColGroup lhs : thatCG)
-			leftMultByCompressedTransposedMatrix(lhs, thisCG, ret, 0, thisCG.size());
-	}
-
-	private static void leftMultByCompressedTransposedMatrix(AColGroup lhs, List<AColGroup> thisCG, MatrixBlock ret,
-		int colGroupStart, int colGroupEnd) {
-
-		for(; colGroupStart < colGroupEnd; colGroupStart++) {
-			AColGroup rhs = thisCG.get(colGroupStart);
-			if(rhs != lhs)
-				rhs.leftMultByAColGroup(lhs, ret);
-			else
-				rhs.tsmm(ret);
+	private static void multAllColGroups(List<AColGroup> right, List<AColGroup> left, MatrixBlock ret) {
+		for(int i = 0; i < left.size(); i++) {
+			AColGroup leftCG = left.get(i);
+			for(int j = 0; j < right.size(); j++) {
+				AColGroup rightCG = right.get(j);
+				if(rightCG != leftCG)
+					rightCG.leftMultByAColGroup(leftCG, ret);
+				else
+					rightCG.tsmm(ret);
+			}
 		}
 	}
 
@@ -244,14 +251,12 @@ public class CLALibLeftMultBy {
 		try {
 			ExecutorService pool = CommonThreadPool.get(k);
 			ArrayList<Callable<Object>> tasks = new ArrayList<>();
-			// if(groups.size()< 10){
 
-			// }
 			final int numColGroups = groups.size();
 			for(int i = 0; i < numColGroups; i++) {
 				tasks.add(new tsmmSelfColGroupTask(groups.get(i), ret));
-				for(int j = i +1; j < numColGroups; j++)
-					tasks.add(new tsmmColGroupTask(groups, filteredGroups, ret, i, j, j+1));
+				for(int j = i + 1; j < numColGroups; j++)
+					tasks.add(new tsmmColGroupTask(groups, filteredGroups, ret, i, j, j + 1));
 			}
 
 			for(Future<Object> tret : pool.invokeAll(tasks))
@@ -273,7 +278,7 @@ public class CLALibLeftMultBy {
 		final AColGroup full_lhs = groups.get(i);
 		final AColGroup lhs = filteredGroups.get(i);
 		boolean isSDC = full_lhs instanceof ColGroupSDC || full_lhs instanceof ColGroupSDCSingle;
-		for(int id = start ; id < end; id++) {
+		for(int id = start; id < end; id++) {
 			final AColGroup full_rhs = groups.get(id);
 			final AColGroup rhs = filteredGroups.get(id);
 			if(isSDC && (full_rhs instanceof ColGroupSDC || full_rhs instanceof ColGroupSDCSingle))
@@ -299,7 +304,6 @@ public class CLALibLeftMultBy {
 		final double[] constV = containsSDC ? new double[numColumnsOut] : null;
 		final List<AColGroup> filteredGroups = filterSDCGroups(colGroups, constV);
 
-		ret.allocateDenseBlock();
 		final double[] rowSums = containsSDC ? new double[that.getNumRows()] : null;
 
 		if(k == 1) {
@@ -329,7 +333,6 @@ public class CLALibLeftMultBy {
 				}
 				else {
 					final int numberSplits = Math.max((k / (ret.getNumRows() / rowBlockSize)), 1);
-					// LOG.error("RowBLockSize:" +rowBlockSize + " Splits " + numberSplits);
 					if(numberSplits == 1) {
 						for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
 							tasks.add(new LeftMatrixColGroupMultTaskNew(filteredGroups, that, ret, blo,
@@ -465,7 +468,8 @@ public class CLALibLeftMultBy {
 		private final int _start;
 		private final int _end;
 
-		protected tsmmColGroupTask(List<AColGroup> groups, List<AColGroup> filteredGroups, MatrixBlock ret, int i, int start, int end) {
+		protected tsmmColGroupTask(List<AColGroup> groups, List<AColGroup> filteredGroups, MatrixBlock ret, int i,
+			int start, int end) {
 			_groups = groups;
 			_filteredGroups = filteredGroups;
 			_ret = ret;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
index edc68b5..fb4be0d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
@@ -28,8 +28,6 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
@@ -38,68 +36,52 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class CLALibRightMultBy {
-	private static final Log LOG = LogFactory.getLog(CLALibRightMultBy.class.getName());
+	// private static final Log LOG = LogFactory.getLog(CLALibRightMultBy.class.getName());
 
 	public static MatrixBlock rightMultByMatrix(CompressedMatrixBlock m1, MatrixBlock m2, MatrixBlock ret, int k,
 		boolean allowOverlap) {
-		ret = rightMultByMatrix(m1.getColGroups(), m2, ret, k, allowOverlap);
-		ret.recomputeNonZeros();
-		return ret;
-	}
-
-	private static MatrixBlock rightMultByMatrix(List<AColGroup> colGroups, MatrixBlock that, MatrixBlock ret, int k,
-		boolean allowOverlap) {
-
-		if(that instanceof CompressedMatrixBlock)
-			LOG.warn("Decompression Right matrix");
-
-		that = that instanceof CompressedMatrixBlock ? ((CompressedMatrixBlock) that).decompress(k) : that;
-
-		MatrixBlock m = rightMultByMatrixOverlapping(colGroups, that, ret, k);
-
-		if(m instanceof CompressedMatrixBlock)
-			if(allowOverlappingOutput(colGroups, allowOverlap))
-				return m;
+		if(m2.isEmpty()) {
+			if(ret == null)
+				ret = new MatrixBlock(m1.getNumRows(), m2.getNumColumns(), 0);
 			else
-				return ((CompressedMatrixBlock) m).decompress(k);
-		else
-			return m;
-	}
+				ret.reset(m1.getNumRows(), m2.getNumColumns(), 0);
+		}
+		else {
+			if(m2 instanceof CompressedMatrixBlock) {
+				CompressedMatrixBlock m2C = (CompressedMatrixBlock) m2;
+				m2 = m2C.getUncompressed("Uncompressed right side of right MM");
+			}
 
-	private static boolean allowOverlappingOutput(List<AColGroup> colGroups, boolean allowOverlap) {
+			ret = rightMultByMatrixOverlapping(m1, m2, ret, k);
 
-		if(!allowOverlap) {
-			LOG.debug("Not Overlapping because it is not allowed");
-			return false;
+			if(ret instanceof CompressedMatrixBlock) {
+				if(!allowOverlap) {
+					ret = ((CompressedMatrixBlock) ret).getUncompressed("Overlapping not allowed");
+				}
+				else {
+					final double compressedSize = ret.getInMemorySize();
+					final double uncompressedSize = MatrixBlock.estimateSizeDenseInMemory(ret.getNumRows(),
+						ret.getNumColumns());
+					if(compressedSize * 2 > uncompressedSize)
+						ret = ((CompressedMatrixBlock) ret).getUncompressed(
+							"Overlapping rep to big: " + compressedSize + " vs Uncompressed " + uncompressedSize);
+				}
+			}
 		}
-		else
-			return true;
-		// int distinctCount = 0;
-		// for(AColGroup g : colGroups) {
-		// if(g instanceof ColGroupCompressed)
-		// distinctCount += ((ColGroupCompressed) g).getNumValues();
-		// else {
-		// LOG.debug("Not Overlapping because there is an un-compressed column group");
-		// return false;
-		// }
-		// }
-		// final int threshold = colGroups.get(0).getNumRows() / 2;
-		// boolean allow = distinctCount <= threshold;
-		// if(LOG.isDebugEnabled() && !allow)
-		// LOG.debug("Not Allowing Overlap because of number of distinct items in compression: " + distinctCount
-		// + " is greater than threshold: " + threshold);
-		// return allow;
 
+		ret.recomputeNonZeros();
+
+		return ret;
 	}
 
-	private static MatrixBlock rightMultByMatrixOverlapping(List<AColGroup> colGroups, MatrixBlock that,
-		MatrixBlock ret, int k) {
-		int rl = colGroups.get(0).getNumRows();
+	private static MatrixBlock rightMultByMatrixOverlapping(CompressedMatrixBlock m1, MatrixBlock that, MatrixBlock ret,
+		int k) {
+		int rl = m1.getNumRows();
 		int cl = that.getNumColumns();
 		// Create an overlapping compressed Matrix Block.
 		ret = new CompressedMatrixBlock(rl, cl);
 		CompressedMatrixBlock retC = (CompressedMatrixBlock) ret;
-		ret = rightMultByMatrixCompressed(colGroups, that, retC, k);
+		ret = rightMultByMatrixCompressed(m1.getColGroups(), that, retC, k);
 		return ret;
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
index 47941ab..5e347af 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
@@ -63,9 +63,7 @@ public class ReaderCompressedSelection extends ReaderColumnSelection {
 			reusableArr[i] = bl.get(offset, _colIndexes[i]);
 			bl.set(offset, _colIndexes[i], 0);
 		}
-		// LOG.error(reusableReturn);
 		return reusableReturn;
-
 	}
 
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/AWTreeNode.java b/src/main/java/org/apache/sysds/runtime/compress/workload/AWTreeNode.java
index ce965c8..4941def 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/AWTreeNode.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/AWTreeNode.java
@@ -46,8 +46,8 @@ public abstract class AWTreeNode {
 	}
 
 	private final WTNodeType _type;
-	private final List<WTreeNode> _children = new ArrayList<>();
-	private final List<Op> _ops = new ArrayList<>();
+	protected final List<WTreeNode> _children = new ArrayList<>();
+	protected final List<Op> _ops = new ArrayList<>();
 
 	public AWTreeNode(WTNodeType type) {
 		_type = type;
@@ -102,7 +102,7 @@ public abstract class AWTreeNode {
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append("\n---------------------------Workload Tree:---------------------------------------\n");
-		sb.append(this.explain(1));
+		sb.append(this.explain(0));
 		sb.append("--------------------------------------------------------------------------------\n");
 		return sb.toString();
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/Op.java b/src/main/java/org/apache/sysds/runtime/compress/workload/Op.java
index 32b4b34..1825a02 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/Op.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/Op.java
@@ -24,6 +24,9 @@ import org.apache.sysds.hops.Hop;
 public abstract class Op {
 
 	protected final Hop _op;
+	protected boolean isDecompressing = false;
+	protected boolean isOverlapping = false;
+	private boolean isDensifying = false;
 
 	public Op(Hop op) {
 		_op = op;
@@ -35,11 +38,35 @@ public abstract class Op {
 
 	@Override
 	public String toString() {
-		return _op.toString();
+		return _op.getHopID() + " " + _op.toString() + " CompressedOutput: " + isCompressedOutput()
+			+ " IsDecompressing: " + isDecompressing();
 	}
 
-	public abstract boolean isCompressedOutput();
+	public boolean isCompressedOutput(){
+		return true;
+	}
+
+	public final boolean isDecompressing() {
+		return isDecompressing;
+	}
 
-	public abstract boolean isDecompressing();
+	public final void setDecompressing() {
+		isDecompressing = true;
+	}
+
+	public boolean isOverlapping() {
+		return isOverlapping;
+	}
 
+	public void setOverlapping() {
+		isOverlapping = true;
+	}
+
+	public boolean isDensifying(){
+		return isDensifying;
+	}
+
+	public void setDensifying(){
+		isDensifying = true;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/OpDecompressing.java b/src/main/java/org/apache/sysds/runtime/compress/workload/OpDecompressing.java
deleted file mode 100644
index 8141b3c..0000000
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/OpDecompressing.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.runtime.compress.workload;
-
-import org.apache.sysds.hops.Hop;
-
-public class OpDecompressing extends Op {
-
-	public OpDecompressing(Hop op) {
-		super(op);
-	}
-
-	@Override
-	public boolean isCompressedOutput() {
-		return false;
-	}
-
-	@Override
-	public boolean isDecompressing(){
-		return true;
-	}
-}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/OpMetadata.java b/src/main/java/org/apache/sysds/runtime/compress/workload/OpMetadata.java
index 1792eb6..386e9c4 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/OpMetadata.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/OpMetadata.java
@@ -23,17 +23,14 @@ import org.apache.sysds.hops.Hop;
 
 public class OpMetadata extends Op {
 
-	public OpMetadata(Hop op) {
-		super(op);
-	}
+	final Hop parent;
 
-	@Override
-	public boolean isCompressedOutput() {
-		return true;
+	public OpMetadata(Hop op, Hop parent) {
+		super(op);
+		this.parent = parent;
 	}
 
-	@Override
-	public boolean isDecompressing() {
-		return false;
+	public Hop getParent(){
+		return parent;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/OpNormal.java b/src/main/java/org/apache/sysds/runtime/compress/workload/OpNormal.java
index 0c1abdc..e13848f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/OpNormal.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/OpNormal.java
@@ -34,9 +34,4 @@ public class OpNormal extends Op {
 	public boolean isCompressedOutput() {
 		return outC;
 	}
-
-	@Override
-	public boolean isDecompressing() {
-		return false;
-	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/OpOverlappingDecompress.java b/src/main/java/org/apache/sysds/runtime/compress/workload/OpOverlappingDecompress.java
deleted file mode 100644
index 76307c4..0000000
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/OpOverlappingDecompress.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.runtime.compress.workload;
-
-import org.apache.sysds.hops.Hop;
-
-public class OpOverlappingDecompress extends Op {
-	public OpOverlappingDecompress(Hop op) {
-		super(op);
-	}
-
-	@Override
-	public boolean isCompressedOutput() {
-		return false;
-	}
-
-	@Override
-	public boolean isDecompressing() {
-		return true;
-	}
-}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/OpSided.java b/src/main/java/org/apache/sysds/runtime/compress/workload/OpSided.java
index 4b8549e..d13912e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/OpSided.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/OpSided.java
@@ -30,15 +30,12 @@ public class OpSided extends Op {
 	private final boolean _tLeft;
 	private final boolean _tRight;
 
-	private boolean _overlappingDecompression = false;
-
 	public OpSided(Hop op, boolean cLeft, boolean cRight, boolean tLeft, boolean tRight) {
 		super(op);
 		_cLeft = cLeft;
 		_cRight = cRight;
 		_tLeft = tLeft;
 		_tRight = tRight;
-
 	}
 
 	public boolean getLeft() {
@@ -60,7 +57,6 @@ public class OpSided extends Op {
 	@Override
 	public String toString() {
 		return super.toString() + " L:" + _cLeft + " R:" + _cRight + " tL:" + _tLeft + " tR:" + _tRight + " ";
-
 	}
 
 	public boolean isLeftMM() {
@@ -77,15 +73,8 @@ public class OpSided extends Op {
 
 	@Override
 	public boolean isCompressedOutput() {
-		return isRightMM() && !_overlappingDecompression;
+		// if the output is transposed after a right matrix multiplication the compression is decompressed
+		return _cLeft && !_cRight && !_tLeft;
 	}
 
-	protected void setOverlappingDecompression(boolean v) {
-		_overlappingDecompression = v;
-	}
-
-	@Override
-	public boolean isDecompressing() {
-		return _overlappingDecompression;
-	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/WTreeRoot.java b/src/main/java/org/apache/sysds/runtime/compress/workload/WTreeRoot.java
index b08ec64..c3b3818 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/WTreeRoot.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/WTreeRoot.java
@@ -19,8 +19,6 @@
 
 package org.apache.sysds.runtime.compress.workload;
 
-import java.util.List;
-
 import org.apache.sysds.hops.Hop;
 
 /**
@@ -32,12 +30,11 @@ public class WTreeRoot extends AWTreeNode {
 
 	private final Hop _root;
 
-	private final List<Hop> _decompressList;
+	private boolean isDecompressing = false;
 
-	public WTreeRoot(Hop root, List<Hop> decompressList) {
+	public WTreeRoot(Hop root) {
 		super(WTNodeType.ROOT);
 		_root = root;
-		_decompressList = decompressList;
 	}
 
 	/**
@@ -49,7 +46,27 @@ public class WTreeRoot extends AWTreeNode {
 		return _root;
 	}
 
-	public List<Hop> getDecompressList() {
-		return _decompressList;
+	public boolean isDecompressing() {
+		return isDecompressing;
+	}
+
+	public void setDecompressing() {
+		isDecompressing = true;
 	}
+
+	@Override
+	protected String explain(int level) {
+		StringBuilder sb = new StringBuilder();
+	
+		// append node summary
+		sb.append("ROOT : " + _root.toString());
+		sb.append("\n");
+
+		// append child nodes
+		if(!_children.isEmpty())
+			for(AWTreeNode n : _children)
+				sb.append(n.explain(level + 1));
+		return sb.toString();
+	}
+
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
index c865507..e086974 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
@@ -30,20 +30,25 @@ import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.common.Types.AggOp;
 import org.apache.sysds.common.Types.OpOp1;
 import org.apache.sysds.common.Types.OpOp2;
+import org.apache.sysds.common.Types.OpOp3;
 import org.apache.sysds.common.Types.OpOpData;
 import org.apache.sysds.common.Types.ReOrgOp;
 import org.apache.sysds.hops.AggBinaryOp;
+import org.apache.sysds.hops.AggUnaryOp;
+import org.apache.sysds.hops.BinaryOp;
 import org.apache.sysds.hops.FunctionOp;
 import org.apache.sysds.hops.Hop;
 import org.apache.sysds.hops.IndexingOp;
 import org.apache.sysds.hops.LiteralOp;
+import org.apache.sysds.hops.ParameterizedBuiltinOp;
 import org.apache.sysds.hops.ReorgOp;
+import org.apache.sysds.hops.UnaryOp;
 import org.apache.sysds.hops.rewrite.HopRewriteUtils;
 import org.apache.sysds.hops.rewrite.RewriteCompressedReblock;
 import org.apache.sysds.parser.DMLProgram;
-import org.apache.sysds.parser.DataIdentifier;
 import org.apache.sysds.parser.ForStatement;
 import org.apache.sysds.parser.ForStatementBlock;
 import org.apache.sysds.parser.FunctionStatement;
@@ -54,8 +59,9 @@ import org.apache.sysds.parser.ParForStatementBlock;
 import org.apache.sysds.parser.StatementBlock;
 import org.apache.sysds.parser.WhileStatement;
 import org.apache.sysds.parser.WhileStatementBlock;
-import org.apache.sysds.runtime.compress.cost.CostEstimatorBuilder;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.workload.AWTreeNode.WTNodeType;
+import org.apache.sysds.utils.Explain;
 
 public class WorkloadAnalyzer {
 	private static final Log LOG = LogFactory.getLog(WorkloadAnalyzer.class.getName());
@@ -64,30 +70,29 @@ public class WorkloadAnalyzer {
 	// avoid wtree construction for assumptionly already compressed intermediates
 	// (due to conditional control flow this might miss compression opportunities)
 	public static boolean PRUNE_COMPRESSED_INTERMEDIATES = true;
-	
+
 	private final Set<Hop> visited;
 	private final Set<Long> compressed;
 	private final Set<Long> transposed;
-	private final Set<String> transientCompressed;
+	private final Map<String, Long> transientCompressed;
 	private final Set<Long> overlapping;
-	private final Set<String> transientOverlapping;
 	private final DMLProgram prog;
-	private final List<Hop> decompressHops;
+	private final Map<Long, Op> treeLookup;
 
 	public static Map<Long, WTreeRoot> getAllCandidateWorkloads(DMLProgram prog) {
 		// extract all compression candidates from program (in program order)
 		List<Hop> candidates = getCandidates(prog);
-		
+
 		// for each candidate, create pruned workload tree
 		List<WorkloadAnalyzer> allWAs = new LinkedList<>();
 		Map<Long, WTreeRoot> map = new HashMap<>();
 		for(Hop cand : candidates) {
-			//prune already covered candidate (intermediate already compressed)
-			if( PRUNE_COMPRESSED_INTERMEDIATES )
-				if( allWAs.stream().anyMatch(w -> w.containsCompressed(cand)) )
-					continue; //intermediate already compressed
-			
-			//construct workload tree for candidate
+			// prune already covered candidate (intermediate already compressed)
+			if(PRUNE_COMPRESSED_INTERMEDIATES)
+				if(allWAs.stream().anyMatch(w -> w.containsCompressed(cand)))
+					continue; // intermediate already compressed
+
+			// construct workload tree for candidate
 			WorkloadAnalyzer wa = new WorkloadAnalyzer(prog);
 			WTreeRoot tree = wa.createWorkloadTree(cand);
 			map.put(cand.getHopID(), tree);
@@ -102,10 +107,9 @@ public class WorkloadAnalyzer {
 		this.visited = new HashSet<>();
 		this.compressed = new HashSet<>();
 		this.transposed = new HashSet<>();
-		this.transientCompressed = new HashSet<>();
+		this.transientCompressed = new HashMap<>();
 		this.overlapping = new HashSet<>();
-		this.transientOverlapping = new HashSet<>();
-		this.decompressHops = new ArrayList<>();
+		this.treeLookup = new HashMap<>();
 	}
 
 	protected WorkloadAnalyzer(DMLProgram prog, Set<Long> overlapping) {
@@ -113,28 +117,25 @@ public class WorkloadAnalyzer {
 		this.visited = new HashSet<>();
 		this.compressed = new HashSet<>();
 		this.transposed = new HashSet<>();
-		this.transientCompressed = new HashSet<>();
+		this.transientCompressed = new HashMap<>();
 		this.overlapping = overlapping;
-		this.transientOverlapping = new HashSet<>();
-		this.decompressHops = new ArrayList<>();
+		this.treeLookup = new HashMap<>();
 	}
 
-	protected WorkloadAnalyzer(DMLProgram prog, Set<Long> compressed, Set<String> transientCompressed,
-		Set<Long> transposed, Set<Long> overlapping, Set<String> transientOverlapping) {
+	protected WorkloadAnalyzer(DMLProgram prog, Set<Long> compressed, HashMap<String, Long> transientCompressed,
+		Set<Long> transposed, Set<Long> overlapping, Map<Long, Op> treeLookup) {
 		this.prog = prog;
 		this.visited = new HashSet<>();
 		this.compressed = compressed;
 		this.transposed = transposed;
 		this.transientCompressed = transientCompressed;
 		this.overlapping = overlapping;
-		this.transientOverlapping = transientOverlapping;
-		this.decompressHops = new ArrayList<>();
+		this.treeLookup = treeLookup;
 	}
 
 	protected WTreeRoot createWorkloadTree(Hop candidate) {
-		WTreeRoot main = new WTreeRoot(candidate, decompressHops);
+		WTreeRoot main = new WTreeRoot(candidate);
 		compressed.add(candidate.getHopID());
-		transientCompressed.add(candidate.getName());
 		for(StatementBlock sb : prog.getStatementBlocks())
 			createWorkloadTree(main, sb, prog, new HashSet<>());
 		pruneWorkloadTree(main);
@@ -144,7 +145,7 @@ public class WorkloadAnalyzer {
 	protected boolean containsCompressed(Hop hop) {
 		return compressed.contains(hop.getHopID());
 	}
-	
+
 	private static List<Hop> getCandidates(DMLProgram prog) {
 		List<Hop> candidates = new ArrayList<>();
 		for(StatementBlock sb : prog.getStatementBlocks()) {
@@ -208,9 +209,8 @@ public class WorkloadAnalyzer {
 		if(hop.isVisited())
 			return;
 		// evaluate and add candidates (type and size)
-		if( (  RewriteCompressedReblock.satisfiesAggressiveCompressionCondition(hop)
-				& ALLOW_INTERMEDIATE_CANDIDATES)
-			|| RewriteCompressedReblock.satisfiesCompressionCondition(hop))
+		if((RewriteCompressedReblock.satisfiesAggressiveCompressionCondition(hop) & ALLOW_INTERMEDIATE_CANDIDATES) ||
+			RewriteCompressedReblock.satisfiesCompressionCondition(hop))
 			cands.add(hop);
 
 		// recursively process children (inputs)
@@ -287,23 +287,27 @@ public class WorkloadAnalyzer {
 							FunctionStatementBlock fsb = prog.getFunctionStatementBlock(fop.getFunctionKey());
 							if(fsb == null)
 								continue;
-							FunctionStatement fstmt = (FunctionStatement) fsb.getStatement(0);
-							Set<String> fCompressed = new HashSet<>();
+
+							HashMap<String, Long> fCompressed = new HashMap<>();
 							// handle propagation of compressed intermediates into functions
-							List<DataIdentifier> fArgs = fstmt.getInputParams();
-							for(int i = 0; i < fArgs.size(); i++)
-								if(compressed.contains(fop.getInput(i).getHopID()) ||
-									transientCompressed.contains(fop.getInput(i).getName()))
-									fCompressed.add(fArgs.get(i).getName());
+
+							String[] ins = fop.getInputVariableNames();
+							for(int i = 0; i < ins.length; i++) {
+								final String name = ins[i];
+								final Long outsideID = fop.getInput(i).getHopID();
+								if(compressed.contains(outsideID))
+									fCompressed.put(name, outsideID);
+							}
+
 							WorkloadAnalyzer fa = new WorkloadAnalyzer(prog, compressed, fCompressed, transposed,
-								overlapping, transientOverlapping);
+								overlapping, treeLookup);
 							fa.createWorkloadTree(n, fsb, prog, fStack);
-							List<DataIdentifier> fOut = fstmt.getOutputParams();
 							String[] outs = fop.getOutputVariableNames();
-							for(int i = 0; i < outs.length; i++)
-								if(fCompressed.contains(fOut.get(i).getName())) {
-									transientCompressed.add(outs[i]);
-								}
+							for(int i = 0; i < outs.length; i++) {
+								Long id = fCompressed.get(outs[i]);
+								if(id != null)
+									transientCompressed.put(outs[i], id);
+							}
 							fStack.remove(fop.getFunctionKey());
 						}
 					}
@@ -325,10 +329,9 @@ public class WorkloadAnalyzer {
 
 		// map statement block propagation to hop propagation
 		if(HopRewriteUtils.isData(hop, OpOpData.PERSISTENTREAD, OpOpData.TRANSIENTREAD) &&
-			transientCompressed.contains(hop.getName())) {
+			transientCompressed.containsKey(hop.getName())) {
 			compressed.add(hop.getHopID());
-			if(transientOverlapping.contains(hop.getName()))
-				overlapping.add(hop.getHopID());
+			treeLookup.put(hop.getHopID(), treeLookup.get(transientCompressed.get(hop.getName())));
 		}
 
 		if(LOG.isTraceEnabled()) {
@@ -338,37 +341,52 @@ public class WorkloadAnalyzer {
 
 		// collect operations on compressed intermediates or inputs
 		// if any input is compressed we collect this hop as a compressed operation
-		if(hop.getInput().stream().anyMatch(h -> compressed.contains(h.getHopID()))) {
-
-			if(isCompressedOp(hop)) {
-				Op o = createOp(hop);
-				parent.addOp(o);
-				if(o.isCompressedOutput())
-					compressed.add(hop.getHopID());
-			}
-			else if(HopRewriteUtils.isData(hop, OpOpData.TRANSIENTWRITE)) {
-				Hop in = hop.getInput().get(0);
-				if(compressed.contains(hop.getHopID()) || compressed.contains(in.getHopID()) ||
-					transientCompressed.contains(in.getName())) {
-					transientCompressed.add(hop.getName());
-				}
-				if(overlapping.contains(hop.getHopID()) || overlapping.contains(in.getHopID()) ||
-					transientOverlapping.contains(in.getName())) {
-					transientOverlapping.add(hop.getName());
-				}
-			}
-		}
+		if(hop.getInput().stream().anyMatch(h -> compressed.contains(h.getHopID())))
+			createOp(hop, parent);
 
 		visited.add(hop);
 	}
 
-	private Op createOp(Hop hop) {
+	private void createOp(Hop hop, AWTreeNode parent) {
 		if(hop.getDataType().isMatrix()) {
-			if(hop instanceof ReorgOp && ((ReorgOp) hop).getOp() == ReOrgOp.TRANS) {
+			Op o = null;
+			if(HopRewriteUtils.isData(hop, OpOpData.PERSISTENTREAD, OpOpData.TRANSIENTREAD)) {
+				return;
+			}
+			else if(HopRewriteUtils.isData(hop, OpOpData.TRANSIENTWRITE, OpOpData.PERSISTENTWRITE)) {
+				transientCompressed.put(hop.getName(), hop.getInput(0).getHopID());
+				compressed.add(hop.getHopID());
+				o = new OpMetadata(hop, hop.getInput(0));
+				if(isOverlapping(hop.getInput(0)))
+					o.setOverlapping();
+			}
+			else if(hop instanceof ReorgOp && ((ReorgOp) hop).getOp() == ReOrgOp.TRANS) {
 				transposed.add(hop.getHopID());
 				compressed.add(hop.getHopID());
-				transientCompressed.add(hop.getName());
-				return new OpMetadata(hop);
+				// hack add to transient compressed since the decompression is marking the parents.
+				transientCompressed.put(hop.getName(), hop.getHopID());
+				// transientCompressed.add(hop.getName());
+				o = new OpMetadata(hop, hop.getInput(0));
+				if(isOverlapping(hop.getInput(0)))
+					o.setOverlapping();
+			}
+			else if(hop instanceof AggUnaryOp) {
+				if((isOverlapping(hop.getInput().get(0)) &&
+					!HopRewriteUtils.isAggUnaryOp(hop, AggOp.SUM, AggOp.MEAN)) ||
+					HopRewriteUtils.isAggUnaryOp(hop, AggOp.TRACE)) {
+					setDecompressionOnAllInputs(hop, parent);
+					return;
+				}
+				else {
+					o = new OpNormal(hop, false);
+				}
+			}
+			else if(hop instanceof UnaryOp && !HopRewriteUtils.isUnary(hop, OpOp1.MULT2, OpOp1.MINUS1_MULT,
+				OpOp1.MINUS_RIGHT, OpOp1.CAST_AS_MATRIX)) {
+				if(isOverlapping(hop.getInput(0))) {
+					treeLookup.get(hop.getInput(0).getHopID()).setDecompressing();
+					return;
+				}
 			}
 			else if(hop instanceof AggBinaryOp) {
 				AggBinaryOp agbhop = (AggBinaryOp) hop;
@@ -376,90 +394,168 @@ public class WorkloadAnalyzer {
 				boolean transposedLeft = transposed.contains(in.get(0).getHopID());
 				boolean transposedRight = transposed.contains(in.get(1).getHopID());
 				boolean left = compressed.contains(in.get(0).getHopID()) ||
-					transientCompressed.contains(in.get(0).getName());
+					transientCompressed.containsKey(in.get(0).getName());
 				boolean right = compressed.contains(in.get(1).getHopID()) ||
-					transientCompressed.contains(in.get(1).getName());
+					transientCompressed.containsKey(in.get(1).getName());
 				OpSided ret = new OpSided(hop, left, right, transposedLeft, transposedRight);
-				if(ret.isRightMM()) {
-					// HashSet<Long> overlapping2 = new HashSet<>();
-					// overlapping2.add(hop.getHopID());
-					// WorkloadAnalyzer overlappingAnalysis = new WorkloadAnalyzer(prog, overlapping2);
-					// WTreeRoot r = overlappingAnalysis.createWorkloadTree(hop);
 
-					// CostEstimatorBuilder b = new CostEstimatorBuilder(r);
-					// if(LOG.isTraceEnabled())
-					// 	LOG.trace("Workload for overlapping: " + r + "\n" + b);
-
-					// if(b.shouldUseOverlap())
+				if(ret.isRightMM()) {
 					overlapping.add(hop.getHopID());
-					// else {
-					// 	decompressHops.add(hop);
-					// 	ret.setOverlappingDecompression(true);
-					// }
+					ret.setOverlapping();
+					if(!ret.isCompressedOutput())
+						ret.setDecompressing();
 				}
 
-				return ret;
-			}
-			else if(HopRewriteUtils.isBinary(hop, OpOp2.CBIND)) {
-				ArrayList<Hop> in = hop.getInput();
-				if(isOverlapping(in.get(0)) || isOverlapping(in.get(1)))
-					overlapping.add(hop.getHopID());
-				// CBind is in worst case decompressing, but can be compressing the other side if it is trivially compressable.
-				// to make the optimizer correct we need to mark this operation as decompressing, since it is the worst possible outcome.
-				// Currently we dont optimize for operations that are located past a cbind.
-				return new OpDecompressing(hop);
-			}
-			else if(HopRewriteUtils.isBinary(hop, OpOp2.RBIND)) {
-				ArrayList<Hop> in = hop.getInput();
-				if(isOverlapping(in.get(0)) || isOverlapping(in.get(1)))
-					return new OpOverlappingDecompress(hop);
-				else
-					return new OpDecompressing(hop);
+				o = ret;
+
 			}
-			else if(HopRewriteUtils.isBinaryMatrixScalarOperation(hop) ||
-				HopRewriteUtils.isBinaryMatrixRowVectorOperation(hop)) {
-				ArrayList<Hop> in = hop.getInput();
-				if(isOverlapping(in.get(0)) || isOverlapping(in.get(1)))
-					overlapping.add(hop.getHopID());
+			else if(hop instanceof BinaryOp) {
+				if(HopRewriteUtils.isBinary(hop, OpOp2.CBIND)) {
+					ArrayList<Hop> in = hop.getInput();
+					o = new OpNormal(hop, true);
+					if(isOverlapping(in.get(0)) || isOverlapping(in.get(1))) {
+						overlapping.add(hop.getHopID());
+						o.setOverlapping();
+					}
+					// assume that CBind have to decompress, but only such that it also have the compressed version
+					// available. Therefore add a new OpNormal, set to decompressing.
+					o.setDecompressing();
+				}
+				else if(HopRewriteUtils.isBinary(hop, OpOp2.RBIND)) {
+					setDecompressionOnAllInputs(hop, parent);
+					return;
+				}
+				// shortcut instead of comparing to MatrixScalar or RowVector.
+				else if(hop.getInput(1).getDim1() == 1 || hop.getInput(1).isScalar() || hop.getInput(0).isScalar()) {
+
+					ArrayList<Hop> in = hop.getInput();
+					final boolean ol0 = isOverlapping(in.get(0));
+					final boolean ol1 = isOverlapping(in.get(1));
+					final boolean ol = ol0 || ol1;
+					if(ol && HopRewriteUtils.isBinary(hop, OpOp2.PLUS, OpOp2.MULT, OpOp2.DIV, OpOp2.MINUS)) {
+						overlapping.add(hop.getHopID());
+						o = new OpNormal(hop, true);
+						o.setOverlapping();
+					}
+					else if(ol) {
+						treeLookup.get(in.get(0).getHopID()).setDecompressing();
+						return;
+					}
+					else {
+						o = new OpNormal(hop, true);
+					}
+					if(!HopRewriteUtils.isBinarySparseSafe(hop))
+						o.setDensifying();
+
+				}
+				else if(HopRewriteUtils.isBinaryMatrixMatrixOperation(hop) ||
+					HopRewriteUtils.isBinaryMatrixColVectorOperation(hop) ||
+					HopRewriteUtils.isBinaryMatrixMatrixOperationWithSharedInput(hop)) {
+					setDecompressionOnAllInputs(hop, parent);
+					return;
+				}
+				else {
+					String ex = "Setting decompressed because input Binary Op is unknown, please add the case to WorkloadAnalyzer:\n"
+						+ Explain.explain(hop);
+					LOG.warn(ex);
+					setDecompressionOnAllInputs(hop, parent);
+				}
 
-				return new OpNormal(hop, true);
 			}
 			else if(hop instanceof IndexingOp) {
 				IndexingOp idx = (IndexingOp) hop;
 				final boolean isOverlapping = isOverlapping(hop.getInput(0));
 				final boolean fullColumn = HopRewriteUtils.isFullColumnIndexing(idx);
-				if(fullColumn && isOverlapping)
-					overlapping.add(hop.getHopID());
 
-				if(fullColumn)
-					return new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
-				else
-					return new OpDecompressing(hop);
+				if(fullColumn) {
+					o = new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
+					if(isOverlapping) {
+						overlapping.add(hop.getHopID());
+						o.setOverlapping();
+					}
+				}
+				else {
+					// This decompression is a little different, since it does not decompress the entire matrix
+					// but only a sub part. therefore create a new op node and set it to decompressing.
+					o = new OpNormal(hop, false);
+					o.setDecompressing();
+				}
 			}
-			else if(HopRewriteUtils.isBinaryMatrixMatrixOperation(hop) ||
-				HopRewriteUtils.isBinaryMatrixColVectorOperation(hop)) {
-				ArrayList<Hop> in = hop.getInput();
-				if(isOverlapping(in.get(0)) || isOverlapping(in.get(1)))
-					return new OpOverlappingDecompress(hop);
-
-				return new OpDecompressing(hop);
+			else if(HopRewriteUtils.isTernary(hop, OpOp3.MINUS_MULT, OpOp3.PLUS_MULT, OpOp3.QUANTILE, OpOp3.CTABLE)) {
+				setDecompressionOnAllInputs(hop, parent);
+				return;
 			}
+			else if(HopRewriteUtils.isTernary(hop, OpOp3.IFELSE)) {
+				final Hop o1 = hop.getInput(1);
+				final Hop o2 = hop.getInput(2);
+				if(isCompressed(o1) && isCompressed(o2)) {
+					o = new OpMetadata(hop, o1);
+					if(isOverlapping(o1) || isOverlapping(o2))
+						o.setOverlapping();
+				}
+				else if(isCompressed(o1)) {
+					o = new OpMetadata(hop, o1);
+					if(isOverlapping(o1))
+						o.setOverlapping();
+				}
+				else if(isCompressed(o2)) {
+					o = new OpMetadata(hop, o2);
+					if(isOverlapping(o2))
+						o.setOverlapping();
+				}
+				else {
+					setDecompressionOnAllInputs(hop, parent);
+				}
+			}
+			else if(hop instanceof ParameterizedBuiltinOp) {
+				setDecompressionOnAllInputs(hop, parent);
+				return;
+			}
+			else
+				throw new DMLCompressionException("Unknown Hop: " + Explain.explain(hop));
+
+			o = o != null ? o : new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
+			treeLookup.put(hop.getHopID(), o);
+			parent.addOp(o);
 
-			// if the output size also qualifies for compression, we propagate this status
-			// return new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
-			return new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
+			if(o.isCompressedOutput())
+				compressed.add(hop.getHopID());
+		}
+		else {
+			parent.addOp(new OpNormal(hop, false));
 		}
-		else
-			return new OpNormal(hop, false);
 	}
 
-	private boolean isOverlapping(Hop hop) {
-		return overlapping.contains(hop.getHopID()) || transientOverlapping.contains(hop.getName());
+	private boolean isCompressed(Hop hop) {
+		return compressed.contains(hop.getHopID());
+	}
+
+	private void setDecompressionOnAllInputs(Hop hop, AWTreeNode parent) {
+		if(parent instanceof WTreeRoot)
+			((WTreeRoot) parent).setDecompressing();
+		for(Hop h : hop.getInput()) {
+			Op ol = treeLookup.get(h.getHopID());
+			if(ol != null) {
+				while(ol instanceof OpMetadata) {
+					// go up through operations and mark the first known as decompressing.
+					// The first known usually is the root of the work tree.
+					Op oln = treeLookup.get(((OpMetadata) ol).getParent().getHopID());
+					if(oln == null)
+						break;
+					else
+						ol = oln;
+				}
+				ol.setDecompressing();
+			}
+		}
 	}
 
-	private static boolean isCompressedOp(Hop hop) {
-		return !(HopRewriteUtils.isData(hop, OpOpData.PERSISTENTREAD, // all, but data ops
-			OpOpData.TRANSIENTREAD, OpOpData.TRANSIENTWRITE));
+	private boolean isOverlapping(Hop hop) {
+		Op o = treeLookup.get(hop.getHopID());
+		if(o != null)
+			return o.isOverlapping();
+		else
+			return false;
 	}
 
 	private static boolean isNoOp(Hop hop) {
diff --git a/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
index faef2c7..ca3f69c 100644
--- a/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/org/apache/sysds/runtime/controlprogram/context/SparkExecutionContext.java
@@ -1065,6 +1065,7 @@ public class SparkExecutionContext extends ExecutionContext
 
 			//copy blocks one-at-a-time into output matrix block
 			long aNnz = 0;
+			boolean firstCompressed = true;
 			for( Tuple2<MatrixIndexes,MatrixBlock> keyval : list )
 			{
 				//unpack index-block pair
@@ -1079,7 +1080,12 @@ public class SparkExecutionContext extends ExecutionContext
 				
 				//handle compressed blocks (decompress for robustness)
 				if( block instanceof CompressedMatrixBlock ){
-					block = ((CompressedMatrixBlock)block).decompress(InfrastructureAnalyzer.getLocalParallelism());
+					if(firstCompressed){
+						// with warning.
+						block =((CompressedMatrixBlock)block).getUncompressed("Spark RDD block to MatrixBlock Decompressing");
+						firstCompressed = false;
+					}else
+						block = ((CompressedMatrixBlock)block).decompress(InfrastructureAnalyzer.getLocalParallelism());
 				}
 
 				//append block
@@ -1281,13 +1287,13 @@ public class SparkExecutionContext extends ExecutionContext
 		return out;
 	}
 
-	// @SuppressWarnings("unchecked")
+	@SuppressWarnings("unchecked")
 	public static long writeMatrixRDDtoHDFS( RDDObject rdd, String path, FileFormat fmt )
 	{
 		JavaPairRDD<MatrixIndexes,MatrixBlock> lrdd = (JavaPairRDD<MatrixIndexes, MatrixBlock>) rdd.getRDD();
 		InputOutputInfo oinfo = InputOutputInfo.get(DataType.MATRIX, fmt);
 		
-		// if compression is enabled decompress all blocks before writing to disk TEMPORARY MODIFICATION UNTILL MATRIXBLOCK IS MERGED WITH COMPRESSEDMATRIXBLOCK
+		// if compression is enabled decompress all blocks before writing to disk to ensure type of MatrixBlock.
 		if(ConfigurationManager.isCompressionEnabled())
 			lrdd = lrdd.mapValues(new DeCompressionSPInstruction.DeCompressionFunction());
 
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/SpoofCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/SpoofCPInstruction.java
index 97047a4..15d35b7 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/SpoofCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/SpoofCPInstruction.java
@@ -86,8 +86,7 @@ public class SpoofCPInstruction extends ComputationCPInstruction {
 				MatrixBlock mb = ec.getMatrixInput(input.getName());
 				//FIXME fused codegen operators already support compressed main inputs 
 				if(mb instanceof CompressedMatrixBlock){
-					LOG.warn("Spoof instruction decompressed matrix");
-					mb = ((CompressedMatrixBlock) mb).decompress(_numThreads);
+					mb = ((CompressedMatrixBlock) mb).getUncompressed("Spoof instruction");
 				}
 				inputs.add(mb);
 			}
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixBincell.java b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixBincell.java
index 6d53ff7..f949c6c 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixBincell.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixBincell.java
@@ -359,10 +359,10 @@ public class LibMatrixBincell {
 
 	public static void isValidDimensionsBinary(MatrixBlock m1, MatrixBlock m2)
 	{
-		int rlen1 = m1.rlen;
-		int clen1 = m1.clen;
-		int rlen2 = m2.rlen;
-		int clen2 = m2.clen;
+		final int rlen1 = m1.rlen;
+		final int clen1 = m1.clen;
+		final int rlen2 = m2.rlen;
+		final int clen2 = m2.clen;
 		
 		//currently we support three major binary cellwise operations:
 		//1) MM (where both dimensions need to match)
@@ -376,7 +376,7 @@ public class LibMatrixBincell {
 
 		if( !isValid ) {
 			throw new RuntimeException("Block sizes are not matched for binary " +
-					"cell operations: " + m1.rlen + "x" + m1.clen + " vs " + m2.rlen + "x" + m2.clen);
+					"cell operations: " + rlen1 + "x" + clen1 + " vs " + rlen2 + "x" + clen2);
 		}
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
index 6c7cb0a..07afc4c 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
@@ -522,7 +522,6 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		return isEmptyBlock(true);
 	}
 	
-	
 	public boolean isEmptyBlock(boolean safe)
 	{
 		boolean ret = ( sparse && sparseBlock==null ) || ( !sparse && denseBlock==null );
@@ -2955,37 +2954,31 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	}
 	
 	public MatrixBlock ternaryOperations(TernaryOperator op, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret) {
-		if(m2 instanceof CompressedMatrixBlock)
-			m2 = ((CompressedMatrixBlock) m2).getUncompressed("Ternay Operator arg2 " + op.fn.getClass().getSimpleName());
-		if(m3 instanceof CompressedMatrixBlock)
-			m3 = ((CompressedMatrixBlock) m3).getUncompressed("Ternay Operator arg3 " + op.fn.getClass().getSimpleName());
-
+		
 		//prepare inputs
-		final boolean s1 = (rlen==1 && clen==1);
-		final boolean s2 = (m2.rlen==1 && m2.clen==1);
-		final boolean s3 = (m3.rlen==1 && m3.clen==1);
+		final int r1 = getNumRows();
+		final int r2 = m2.getNumRows();
+		final int r3 = m3.getNumRows();
+		final int c1 = getNumColumns();
+		final int c2 = m2.getNumColumns();
+		final int c3 = m3.getNumColumns();
+		final boolean s1 = (r1 == 1 && c1 == 1);
+		final boolean s2 = (r2 == 1 && c2 == 1);
+		final boolean s3 = (r3 == 1 && c3 == 1);
 		final double d1 = s1 ? quickGetValue(0, 0) : Double.NaN;
 		final double d2 = s2 ? m2.quickGetValue(0, 0) : Double.NaN;
 		final double d3 = s3 ? m3.quickGetValue(0, 0) : Double.NaN;
-		final int m = Math.max(Math.max(rlen, m2.rlen), m3.rlen);
-		final int n = Math.max(Math.max(clen, m2.clen), m3.clen);
+		final int m = Math.max(Math.max(r1, r2), r3);
+		final int n = Math.max(Math.max(c1, c2), c3);
 		final long nnz = nonZeros;
 		
-		//error handling 
-		if( (!s1 && (rlen != m || clen != n))
-			|| (!s2 && (m2.rlen != m || m2.clen != n))
-			|| (!s3 && (m3.rlen != m || m3.clen != n)) ) {
-			throw new DMLRuntimeException("Block sizes are not matched for ternary cell operations: "
-				+ rlen + "x" + clen + " vs " + m2.rlen + "x" + m2.clen + " vs " + m3.rlen + "x" + m3.clen);
-		}
+		ternaryOperationCheck(s1, s2, s3, m, r1, r2, r3, n, c1, c2, c3);
 		
 		//prepare result
-		boolean sparseOutput = (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply)?
-			evalSparseFormatInMemory(m, n, (s1?m*n*(d1!=0?1:0):getNonZeros())
-				+ Math.min(s2?m*n:m2.getNonZeros(), s3?m*n:m3.getNonZeros())) : false;
-		ret.reset(m, n, sparseOutput);
-		
 		if( op.fn instanceof IfElse && (s1 || nnz==0 || nnz==(long)m*n) ) {
+			
+			ret.reset(m, n, false);
+			
 			//SPECIAL CASE for shallow-copy if-else
 			boolean expr = s1 ? (d1 != 0) : (nnz==(long)m*n);
 			MatrixBlock tmp = expr ? m2 : m3;
@@ -3003,24 +2996,55 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 				}
 			}
 		}
-		else if (s2 != s3 && (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply) ) {
-			//SPECIAL CASE for sparse-dense combinations of common +* and -*
-			BinaryOperator bop = ((ValueFunctionWithConstant)op.fn).setOp2Constant(s2 ? d2 : d3);
-			if( op.getNumThreads() > 1 )
-				LibMatrixBincell.bincellOp(this, s2 ? m3 : m2, ret, bop, op.getNumThreads());
-			else
-				LibMatrixBincell.bincellOp(this, s2 ? m3 : m2, ret, bop);
-		}
-		else {
-			//DEFAULT CASE
-			LibMatrixTercell.tercellOp(this, m2, m3, ret, op);
+		else{
+			final boolean PM_Or_MM = (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply);
 			
-			//ensure correct output representation
-			ret.examSparsity();
+			if(PM_Or_MM && ((s2 && d2 == 0) || (s3 && d3 == 0))) {
+				ret.copy(this);
+				return ret;
+			}
+			
+			final boolean sparseOutput = evalSparseFormatInMemory(m, n, (s1 ? m * n * (d1 != 0 ? 1 : 0) : getNonZeros()) +
+				Math.min(s2 ? m * n : m2.getNonZeros(), s3 ? m * n : m3.getNonZeros()));
+
+			if(m2 instanceof CompressedMatrixBlock)
+				m2 = ((CompressedMatrixBlock) m2)
+					.getUncompressed("Ternay Operator arg2 " + op.fn.getClass().getSimpleName());
+			if(m3 instanceof CompressedMatrixBlock)
+				m3 = ((CompressedMatrixBlock) m3)
+					.getUncompressed("Ternay Operator arg3 " + op.fn.getClass().getSimpleName());
+
+			ret.reset(m, n, sparseOutput);
+
+			if (s2 != s3 && (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply) ) {
+				//SPECIAL CASE for sparse-dense combinations of common +* and -*
+				BinaryOperator bop = ((ValueFunctionWithConstant)op.fn).setOp2Constant(s2 ? d2 : d3);
+				if( op.getNumThreads() > 1 )
+					LibMatrixBincell.bincellOp(this, s2 ? m3 : m2, ret, bop, op.getNumThreads());
+				else
+					LibMatrixBincell.bincellOp(this, s2 ? m3 : m2, ret, bop);
+			}
+			else {
+				//DEFAULT CASE
+				LibMatrixTercell.tercellOp(this, m2, m3, ret, op);
+				
+				//ensure correct output representation
+				ret.examSparsity();
+			}
 		}
 		
 		return ret;
 	}
+
+	protected static void ternaryOperationCheck(boolean s1, boolean s2, boolean s3, int m, int r1, int r2, int r3, int n, int c1, int c2, int c3){
+		//error handling 
+		if( (!s1 && (r1 != m || c1 != n))
+		|| (!s2 && (r2 != m || c2 != n))
+		|| (!s3 && (r3 != m || c3 != n)) ) {
+			throw new DMLRuntimeException("Block sizes are not matched for ternary cell operations: "
+			+ r1 + "x" + c1 + " vs " + r2 + "x" + c2 + " vs " + r3 + "x" + c3);
+		}
+	}
 	
 	@Override
 	public void incrementalAggregate(AggregateOperator aggOp, MatrixValue correction, MatrixValue newWithCorrection, boolean deep) {
diff --git a/src/main/java/org/apache/sysds/runtime/util/DataConverter.java b/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
index 6850a97..1339a68 100644
--- a/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
+++ b/src/main/java/org/apache/sysds/runtime/util/DataConverter.java
@@ -260,7 +260,7 @@ public class DataConverter {
 		int cols = mb.getNumColumns();
 		double[][] ret = new double[rows][cols]; //0-initialized
 		if(mb instanceof CompressedMatrixBlock){
-			mb = ((CompressedMatrixBlock)mb).decompress();
+			mb = ((CompressedMatrixBlock)mb).getUncompressed("convert to Double Matrix");
 		}
 		if( mb.getNonZeros() > 0 ) {
 			if( mb.isInSparseFormat() ) {
diff --git a/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java b/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
index 61968d8..ec969e3 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
@@ -249,8 +249,6 @@ public abstract class AbstractCompressedUnaryTests extends CompressedTestBase {
 			MatrixBlock ret1 = mb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, inCP);
 			// matrix-vector compressed
 			MatrixBlock ret2 = cmb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, inCP);
-			// LOG.error(ret1 + "\nvs\n" + ret2);
-			// LOG.error(cmb);
 			// compare result with input
 
 			assertTrue("dim 1 is not equal in compressed res  should be : " + ret1.getNumRows() + "  but is: "
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
index a998a74..cc47036 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
@@ -38,7 +38,12 @@ import org.apache.sysds.runtime.compress.CompressionStatistics;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
+import org.apache.sysds.runtime.functionobjects.Minus;
+import org.apache.sysds.runtime.functionobjects.Minus1Multiply;
+import org.apache.sysds.runtime.functionobjects.MinusMultiply;
 import org.apache.sysds.runtime.functionobjects.Multiply;
+import org.apache.sysds.runtime.functionobjects.Plus;
+import org.apache.sysds.runtime.functionobjects.PlusMultiply;
 import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.matrix.data.LibMatrixCountDistinct;
 import org.apache.sysds.runtime.matrix.data.LibMatrixDatagen;
@@ -47,8 +52,12 @@ import org.apache.sysds.runtime.matrix.data.RandomMatrixGenerator;
 import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateTernaryOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.CountDistinctOperator;
 import org.apache.sysds.runtime.matrix.operators.CountDistinctOperator.CountDistinctTypes;
+import org.apache.sysds.runtime.matrix.operators.RightScalarOperator;
+import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysds.runtime.matrix.operators.TernaryOperator;
 import org.apache.sysds.runtime.util.DataConverter;
 import org.apache.sysds.test.TestUtils;
 import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
@@ -579,7 +588,7 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 	@Test
 	public void testAggregateTernaryOperation() {
 		try {
-			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
+			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 1000)
 				return;
 			CorrectionLocationType corr = CorrectionLocationType.LASTCOLUMN;
 			AggregateOperator agg = new AggregateOperator(0, KahanPlus.getKahanPlusFnObject(), corr);
@@ -603,6 +612,158 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 	}
 
+	@Test
+	public void testAggregateTernaryOperationZero() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
+				return;
+			CorrectionLocationType corr = CorrectionLocationType.LASTCOLUMN;
+			AggregateOperator agg = new AggregateOperator(0, KahanPlus.getKahanPlusFnObject(), corr);
+			AggregateTernaryOperator op = new AggregateTernaryOperator(Multiply.getMultiplyFnObject(), agg,
+				ReduceAll.getReduceAllFnObject());
+
+			int nrow = mb.getNumRows();
+			int ncol = mb.getNumColumns();
+
+			MatrixBlock m2 = new MatrixBlock(nrow, ncol, 0);
+			MatrixBlock m3 = new MatrixBlock(nrow, ncol, 14.0);
+
+			MatrixBlock ret1 = cmb.aggregateTernaryOperations(cmb, m2, m3, null, op, true);
+			MatrixBlock ret2 = mb.aggregateTernaryOperations(mb, m2, m3, null, op, true);
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	@Test
+	public void testTernaryOperation() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
+				return;
+			TernaryOperator op = new TernaryOperator(PlusMultiply.getFnObject(), _k);
+
+			int nrow = mb.getNumRows();
+			int ncol = mb.getNumColumns();
+
+			MatrixBlock m2 = new MatrixBlock(1, 1, 0);
+			MatrixBlock m3 = new MatrixBlock(nrow, ncol, 14.0);
+			MatrixBlock ret1 = cmb.ternaryOperations(op, m2, m3, new MatrixBlock());
+			MatrixBlock ret2 = mb.ternaryOperations(op, m2, m3, new MatrixBlock());
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	@Test
+	public void testBinaryEmptyScalarOp() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+			BinaryOperator op = new BinaryOperator(Multiply.getMultiplyFnObject());
+
+			MatrixBlock m2 = new MatrixBlock(1, 1, 0);
+			MatrixBlock ret1 = cmb.binaryOperations(op, m2, new MatrixBlock());
+			ScalarOperator sop = new RightScalarOperator(op.fn, m2.getValue(0, 0), op.getNumThreads());
+			MatrixBlock ret2 = mb.scalarOperations(sop, new MatrixBlock());
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	@Test
+	public void testBinaryEmptyMatrixMultiplicationOp() {
+		BinaryOperator op = new BinaryOperator(Multiply.getMultiplyFnObject());
+		testBinaryEmptyMatrixOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyMatrixMinusOp() {
+		BinaryOperator op = new BinaryOperator(Minus.getMinusFnObject());
+		testBinaryEmptyMatrixOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyMatrixPlusOp() {
+		BinaryOperator op = new BinaryOperator(Plus.getPlusFnObject());
+		testBinaryEmptyMatrixOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyMatrixMinusMultiplyOp() {
+		BinaryOperator op = MinusMultiply.getFnObject().setOp2Constant(42);
+		testBinaryEmptyMatrixOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyMatrixMinus1MultiplyOp() {
+		BinaryOperator op = new BinaryOperator(Minus1Multiply.getMinus1MultiplyFnObject());
+		testBinaryEmptyMatrixOp(op);
+	}
+
+	public void testBinaryEmptyMatrixOp(BinaryOperator op) {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+
+			MatrixBlock m2 = new MatrixBlock(cmb.getNumRows(), cmb.getNumColumns(), 0);
+			MatrixBlock ret1 = cmb.binaryOperations(op, m2, new MatrixBlock());
+			MatrixBlock ret2 = mb.binaryOperations(op, m2, new MatrixBlock());
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	@Test
+	public void testBinaryEmptyRowVectorMultiplicationOp() {
+		BinaryOperator op = new BinaryOperator(Multiply.getMultiplyFnObject());
+		testBinaryEmptyRowVectorOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyRowVectorMinusOp() {
+		BinaryOperator op = new BinaryOperator(Minus.getMinusFnObject());
+		testBinaryEmptyRowVectorOp(op);
+	}
+
+	@Test
+	public void testBinaryEmptyRowVectorPlusOp() {
+		BinaryOperator op = new BinaryOperator(Plus.getPlusFnObject());
+		testBinaryEmptyRowVectorOp(op);
+	}
+
+	public void testBinaryEmptyRowVectorOp(BinaryOperator op) {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+
+			MatrixBlock m2 = new MatrixBlock(1, cmb.getNumColumns(), 0);
+			MatrixBlock ret1 = cmb.binaryOperations(op, m2, new MatrixBlock());
+			MatrixBlock ret2 = mb.binaryOperations(op, m2, new MatrixBlock());
+
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
+		}
+	}
+
 	private static long getJolSize(CompressedMatrixBlock cmb, CompressionStatistics cStat) {
 		Layouter l = new HotSpotLayouter(new X86_64_DataModel());
 		long jolEstimate = 0;
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
index c291235..ef8441e 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
@@ -1282,14 +1282,23 @@ public abstract class CompressedTestBase extends TestBase {
 	}
 
 	protected void compareResultMatrices(MatrixBlock expected, MatrixBlock result, double toleranceMultiplier) {
+		compareDimensions(expected, result);
 		if(expected instanceof CompressedMatrixBlock)
 			expected = ((CompressedMatrixBlock) expected).decompress();
 		if(result instanceof CompressedMatrixBlock)
 			result = ((CompressedMatrixBlock) result).decompress();
+		
+		compareDimensions(expected, result);
+			
 
 		// compare result with input
 		double[][] d1 = DataConverter.convertToDoubleMatrix(expected);
 		double[][] d2 = DataConverter.convertToDoubleMatrix(result);
 		compareResultMatrices(d1, d2, toleranceMultiplier);
 	}
+
+	protected static void compareDimensions(MatrixBlock expected, MatrixBlock result){
+		assertEquals(expected.getNumRows(), result.getNumRows());
+		assertEquals(expected.getNumColumns(), result.getNumColumns());
+	}
 }
diff --git a/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java b/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
index 9422e58..f1682a7 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/workload/WorkloadTest.java
@@ -47,6 +47,7 @@ public class WorkloadTest {
 
 	private static final String basePath = "src/test/scripts/component/compress/workload/";
 	private static final String testFile = "src/test/resources/component/compress/1-1.csv";
+	private static final String yFile = "src/test/resources/component/compress/1-1_y.csv";
 
 	@Parameterized.Parameter(0)
 	public int scans;
@@ -83,17 +84,13 @@ public class WorkloadTest {
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 1, 0, false, false, "sum.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 1, 0, false, false, "mean.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 1, 1, false, false, "plus.dml", args});
-		tests.add(new Object[] {0, 1, 0, 0, 0, 0, 1, 0, false, false, "sliceCols.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 2, 0, false, false, "sliceCols.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 2, 0, false, false, "sliceIndex.dml", args});
 		tests.add(new Object[] {0, 0, 0, 1, 0, 0, 0, 0, false, false, "leftMult.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 1, 0, 1, 0, false, false, "rightMult.dml", args});
 		tests.add(new Object[] {0, 0, 0, 1, 0, 0, 0, 0, false, false, "TLeftMult.dml", args});
 
-		// https://issues.apache.org/jira/browse/SYSTEMDS-3025 Transposed layout.
-		// (the t right mult here would be much faster if a transposed layout is allowed.)
-		// Also the decompression is not detected.
-		// nr 8:
-		tests.add(new Object[] {0, 0, 0, 0, 1, 0, 1, 0, false, false, "TRightMult.dml", args});
+		tests.add(new Object[] {0, 0, 1, 0, 1, 0, 0, 0, false, false, "TRightMult.dml", args});
 
 		// Loops:
 		tests.add(new Object[] {0, 0, 0, 11, 0, 0, 0, 0, true, false, "loop/leftMult.dml", args});
@@ -104,22 +101,24 @@ public class WorkloadTest {
 
 		// Builtins:
 		// nr 11:
-		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 7, 0, true, false, "functions/scale.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 6, 0, true, false, "functions/scale.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 5, 0, true, true, "functions/scale.dml", args});
-		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 8, 0, true, false, "functions/scale_continued.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 6, 0, true, false, "functions/scale_continued.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 6, 0, true, true, "functions/scale_continued.dml", args});
 		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 2, 0, false, true, "functions/scale_onlySide.dml", args});
-		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 8, 0, true, false, "functions/scale_onlySide.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 0, 0, 6, 0, true, false, "functions/scale_onlySide.dml", args});
 
-		tests.add(new Object[] {0, 0, 0, 0, 1, 1, 9, 0, true, false, "functions/pca.dml", args});
-		tests.add(new Object[] {0, 0, 0, 0, 1, 1, 6, 0, true, true, "functions/pca.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 1, 1, 8, 0, true, false, "functions/pca.dml", args});
+		tests.add(new Object[] {0, 0, 0, 0, 1, 1, 5, 0, true, true, "functions/pca.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "FALSE");
 		args.put("$3", "0");
 
-		tests.add(new Object[] {0, 1, 0, 1, 1, 1, 6, 0, true, false, "functions/lmDS.dml", args});
+		// no recompile
+		tests.add(new Object[] {0, 1, 1, 1, 1, 1, 6, 0, true, false, "functions/lmDS.dml", args});
+		// with recompile
 		tests.add(new Object[] {0, 0, 0, 1, 0, 1, 0, 0, true, true, "functions/lmDS.dml", args});
 		tests.add(new Object[] {0, 0, 0, 1, 10, 10, 1, 0, true, true, "functions/lmCG.dml", args});
 
@@ -127,30 +126,34 @@ public class WorkloadTest {
 		args.put("$1", testFile);
 		args.put("$2", "TRUE");
 		args.put("$3", "0");
-		tests.add(new Object[] {0, 0, 1, 1, 1, 1, 0, 0, true, true, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 1, 1, 1, 1, 1, 0, 0, true, true, "functions/lmDS.dml", args});
 		tests.add(new Object[] {0, 0, 1, 1, 11, 10, 1, 0, true, true, "functions/lmCG.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "TRUE");
 		args.put("$3", "1");
-		tests.add(new Object[] {0, 1, 0, 0, 0, 0, 1, 0, false, true, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 2, 1, 1, 1, 1, 1, 0, true, true, "functions/lmDS.dml", args});
 		tests.add(new Object[] {0, 1, 1, 1, 11, 10, 2, 0, true, true, "functions/lmCG.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "TRUE");
 		args.put("$3", "2");
-		tests.add(new Object[] {0, 1, 0, 0, 0, 0, 1, 0, false, true, "functions/lmDS.dml", args});
-		tests.add(new Object[] {0, 1, 1, 1, 11, 10, 2, 0, true, true, "functions/lmCG.dml", args});
+		tests.add(new Object[] {0, 2, 1, 1, 1, 1, 3, 0, true, true, "functions/lmDS.dml", args});
+		tests.add(new Object[] {0, 1, 1, 1, 11, 10, 4, 0, true, true, "functions/lmCG.dml", args});
 
 		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "FALSE");
-		// Currently l2svm detects that decompression is needed after right mult
 		tests.add(new Object[] {0, 0, 10, 11, 10, 0, 1, 0, true, true, "functions/l2svm.dml", args});
 
 		args = new HashMap<>();
+		args.put("$1", yFile);
+		args.put("$2", "FALSE");
+		tests.add(new Object[] {0, 1, 0, 1, 0, 0, 10, 0, true, true, "functions/l2svm_Y.dml", args});
+
+		args = new HashMap<>();
 		args.put("$1", testFile);
 		args.put("$2", "100");
 		args.put("$3", "16");
@@ -184,9 +187,10 @@ public class WorkloadTest {
 		}
 	}
 
-	private void verify(WTreeRoot wtr, InstructionTypeCounter itc, CostEstimatorBuilder ceb, String name, Map<String, String> args) {
+	private void verify(WTreeRoot wtr, InstructionTypeCounter itc, CostEstimatorBuilder ceb, String name,
+		Map<String, String> args) {
 
-		String errorString = wtr + "\n" + itc + " \n " + name + "  -- " + args +  "\n";
+		String errorString = wtr + "\n" + itc + " \n " + name + "  -- " + args + "\n";
 		Assert.assertEquals(errorString + "scans:", scans, itc.getScans());
 		Assert.assertEquals(errorString + "decompressions", decompressions, itc.getDecompressions());
 		Assert.assertEquals(errorString + "overlappingDecompressions", overlappingDecompressions,
@@ -214,7 +218,6 @@ public class WorkloadTest {
 			String filePath = basePath + name;
 			String dmlScript = DMLScript.readDMLScript(isFile, filePath);
 			return ParserFactory.createParser().parse(DMLOptions.defaultOptions.filePath, dmlScript, args);
-
 		}
 		catch(Exception e) {
 			throw new DMLRuntimeException("Error in parsing", e);
diff --git a/src/test/java/org/apache/sysds/test/component/frame/DataCorruptionTest.java b/src/test/java/org/apache/sysds/test/component/frame/DataCorruptionTest.java
index 316e062..51fa497 100644
--- a/src/test/java/org/apache/sysds/test/component/frame/DataCorruptionTest.java
+++ b/src/test/java/org/apache/sysds/test/component/frame/DataCorruptionTest.java
@@ -90,7 +90,7 @@ public class DataCorruptionTest
 				assertTrue("Row "+i+" was marked with typos, but it has not been changed.", checkTypo);
 			}
 		}
-		System.out.println("Test typos: number of changed rows: " + numch);
+		// Test typos: number of changed rows: 
 		assertEquals("The number of changed rows is not approx. 20%", 0.2, numch/Xp.getNumRows(), 0.05);
 	}
 	
@@ -114,7 +114,7 @@ public class DataCorruptionTest
 				assertTrue("Row "+i+" was marked with missing values, but it has not been changed.", dropped>0);
 			}
 		}
-		System.out.println("Test missing: number of changed rows: " + numch);
+		// Test missing: number of changed rows:
 		assertEquals("The number of changed rows is not approx. 20%", 0.2, numch/Xp.getNumRows(), 0.05);
 	}
 	
@@ -147,7 +147,7 @@ public class DataCorruptionTest
 				assertTrue("Row "+i+" was marked with outliers, but it has not been changed.", checkOut);
 			}
 		}
-		System.out.println("Test outliers: number of changed rows: " + numch);
+		// Test outliers: number of changed rows:
 		assertEquals("The number of changed rows is not approx. 20%", 0.2, numch/Xp.getNumRows(), 0.05);
 	}
 	
@@ -192,7 +192,7 @@ public class DataCorruptionTest
 				assertTrue("Row "+i+" was marked with outliers, but it has not been changed.",checkSwap);
 			}
 		}
-		System.out.println("Test swap: number of changed rows: " + numch);
+		// Test swap: number of changed rows:
 		assertEquals("The number of changed rows is not approx. 20%", 0.2, numch/changed.getNumRows(), 0.05);
 		
 	}
diff --git a/src/test/java/org/apache/sysds/test/functions/codegen/RowAggTmplTest.java b/src/test/java/org/apache/sysds/test/functions/codegen/RowAggTmplTest.java
index 540b07b..5ae3196 100644
--- a/src/test/java/org/apache/sysds/test/functions/codegen/RowAggTmplTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/codegen/RowAggTmplTest.java
@@ -25,14 +25,15 @@ import java.util.HashMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.common.Types.ExecMode;
-import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.common.Types.ExecType;
+import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.lops.RightIndex;
 import org.apache.sysds.runtime.matrix.data.MatrixValue.CellIndex;
 import org.apache.sysds.test.AutomatedTestBase;
 import org.apache.sysds.test.TestConfiguration;
 import org.apache.sysds.test.TestUtils;
 import org.junit.Assert;
+import org.junit.Ignore;
 import org.junit.Test;
 
 public class RowAggTmplTest extends AutomatedTestBase
@@ -357,6 +358,9 @@ public class RowAggTmplTest extends AutomatedTestBase
 	}
 	
 	@Test
+	@Ignore
+	// Since adding the rewrite (simplyfyMMCBindZeroVector) CodeGen is unable to
+	// combine the instructions.
 	public void testCodegenRowAggRewrite18CP() {
 		testCodegenIntegration( TEST_NAME18, true, ExecType.CP );
 	}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/CompressInstruction.java b/src/test/java/org/apache/sysds/test/functions/compress/CompressInstruction.java
index 929c202..9498e77 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/CompressInstruction.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/CompressInstruction.java
@@ -95,7 +95,6 @@ public class CompressInstruction extends AutomatedTestBase {
 			programArgs = new String[] {"-stats", "100", "-nvargs", "cols=" + cols, "rows=" + rows,
 				"sparsity=" + sparsity, "min=" + min, "max= " + max};
 			runTest(null);
-			// LOG.error(runTest(null));
 
 		}
 		catch(Exception e) {
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/CompressInstructionRewrite.java b/src/test/java/org/apache/sysds/test/functions/compress/CompressInstructionRewrite.java
index c005115..b1697e7 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/CompressInstructionRewrite.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/CompressInstructionRewrite.java
@@ -21,11 +21,8 @@ package org.apache.sysds.test.functions.compress;
 
 import static org.junit.Assert.assertTrue;
 
-import java.io.ByteArrayOutputStream;
 import java.io.File;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.common.Types;
 import org.apache.sysds.common.Types.ExecType;
 import org.apache.sysds.test.AutomatedTestBase;
@@ -37,7 +34,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class CompressInstructionRewrite extends AutomatedTestBase {
-	private static final Log LOG = LogFactory.getLog(CompressInstructionRewrite.class.getName());
+	// private static final Log LOG = LogFactory.getLog(CompressInstructionRewrite.class.getName());
 
 	private String TEST_CONF = "SystemDS-config-compress-cost.xml";
 	private File TEST_CONF_FILE = new File(SCRIPT_DIR + getTestDir(), TEST_CONF);
@@ -91,12 +88,12 @@ public class CompressInstructionRewrite extends AutomatedTestBase {
 
 	@Test
 	public void testCompressInstruction_07() {
-		compressTest(6, 6000, 0.2, ExecType.CP, 0, 5, 0, 1, "07");
+		compressTest(10, 6000, 0.2, ExecType.CP, 0, 3, 0, 1, "07");
 	}
 
 	@Test
 	public void testCompressInstruction_08() {
-		compressTest(6, 6000, 0.2, ExecType.CP, 0, 5, 0, 1, "08");
+		compressTest(10, 6000, 0.2, ExecType.CP, 0, 3, 0, 1, "08");
 	}
 
 	@Test
@@ -109,7 +106,6 @@ public class CompressInstructionRewrite extends AutomatedTestBase {
 		compressTest(1, 1000, 1.0, ExecType.CP, 5, 5, 0, 0, "10");
 	}
 
-
 	public void compressTest(int cols, int rows, double sparsity, ExecType instType, int min, int max,
 		int decompressionCountExpected, int compressionCountsExpected, String name) {
 
@@ -122,16 +118,13 @@ public class CompressInstructionRewrite extends AutomatedTestBase {
 			programArgs = new String[] {"-explain", "-stats", "100", "-nvargs", "cols=" + cols, "rows=" + rows,
 				"sparsity=" + sparsity, "min=" + min, "max= " + max};
 
-			ByteArrayOutputStream stdout = runTest(null);
-
-			if(LOG.isDebugEnabled())
-				LOG.debug(stdout);
+			String stdout = runTest(null).toString();
 
 			int decompressCount = DMLCompressionStatistics.getDecompressionCount();
 			long compressionCount = Statistics.getCPHeavyHitterCount("compress");
 
-			Assert.assertEquals(compressionCountsExpected, compressionCount);
-			Assert.assertEquals(decompressionCountExpected, decompressCount);
+			Assert.assertEquals(stdout, compressionCountsExpected, compressionCount);
+			Assert.assertEquals(stdout, decompressionCountExpected, decompressCount);
 			if(decompressionCountExpected > 0)
 				Assert.assertTrue(heavyHittersContainsString("decompress", decompressionCountExpected));
 		}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/CompressRewriteSpark.java b/src/test/java/org/apache/sysds/test/functions/compress/CompressRewriteSpark.java
index fe3cda5..dc5d205 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/CompressRewriteSpark.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/CompressRewriteSpark.java
@@ -23,8 +23,6 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.common.Types;
 import org.apache.sysds.common.Types.ExecMode;
 import org.apache.sysds.test.AutomatedTestBase;
@@ -33,8 +31,10 @@ import org.apache.sysds.test.TestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+
+@net.jcip.annotations.NotThreadSafe
 public class CompressRewriteSpark extends AutomatedTestBase {
-	private static final Log LOG = LogFactory.getLog(CompressRewriteSpark.class.getName());
+	// private static final Log LOG = LogFactory.getLog(CompressRewriteSpark.class.getName());
 
 	private static final String dataPath = "src/test/scripts/functions/compress/densifying/";
 	private final static String TEST_DIR = "functions/compress/";
@@ -66,27 +66,24 @@ public class CompressRewriteSpark extends AutomatedTestBase {
 		compressTest(ExecMode.HYBRID, "02", "large.ijv");
 	}
 
-
-	@Test 
-	public void testCompressionInstruction_colmean(){
-		compressTest(ExecMode.HYBRID,"submean", "large.ijv");
+	@Test
+	public void testCompressionInstruction_colmean() {
+		compressTest(ExecMode.HYBRID, "submean", "large.ijv");
 	}
 
-
-	@Test 
-	public void testCompressionInstruction_scale(){
-		compressTest(ExecMode.HYBRID,"scale", "large.ijv");
+	@Test
+	public void testCompressionInstruction_scale() {
+		compressTest(ExecMode.HYBRID, "scale", "large.ijv");
 	}
 
-
-	@Test 
-	public void testCompressionInstruction_seq_large(){
-		compressTest(ExecMode.HYBRID,"seq", "large.ijv");
+	@Test
+	public void testCompressionInstruction_seq_large() {
+		compressTest(ExecMode.HYBRID, "seq", "large.ijv");
 	}
 
-	@Test 
-	public void testCompressionInstruction_pca_large(){
-		compressTest(ExecMode.HYBRID,"pca", "large.ijv");
+	@Test
+	public void testCompressionInstruction_pca_large() {
+		compressTest(ExecMode.HYBRID, "pca", "large.ijv");
 	}
 
 	public void compressTest(ExecMode instType, String name, String data) {
@@ -98,12 +95,14 @@ public class CompressRewriteSpark extends AutomatedTestBase {
 
 			fullDMLScriptName = SCRIPT_DIR + "/" + getTestDir() + "compress_" + name + ".dml";
 
-			programArgs = new String[] {"-stats", "100", "-explain", "-args", dataPath + data};
+			programArgs = new String[] {"-stats", "100","-explain", "-args", dataPath + data};
 
-			LOG.debug(runTest(null));
+			String out = runTest(null).toString();
 
-			Assert.assertTrue(!heavyHittersContainsString("sp_compress"));
-			Assert.assertTrue(!heavyHittersContainsString("sp_+"));
+			Assert.assertTrue(out + "\nShould not containing spark compression instruction",
+				!heavyHittersContainsString("sp_compress"));
+			Assert.assertTrue(out + "\nShould not contain spark instruction on compressed input",
+				!heavyHittersContainsString("sp_+"));
 
 		}
 		catch(Exception e) {
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressBase.java b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressBase.java
index c466e17..d38bb32 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressBase.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressBase.java
@@ -21,8 +21,6 @@ package org.apache.sysds.test.functions.compress.configuration;
 
 import static org.junit.Assert.assertTrue;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.common.Types;
 import org.apache.sysds.common.Types.ExecType;
 import org.apache.sysds.runtime.meta.MatrixCharacteristics;
@@ -34,7 +32,7 @@ import org.apache.sysds.utils.Statistics;
 import org.junit.Assert;
 
 public abstract class CompressBase extends AutomatedTestBase {
-	private static final Log LOG = LogFactory.getLog(CompressBase.class.getName());
+	// private static final Log LOG = LogFactory.getLog(CompressBase.class.getName());
 
 	protected abstract String getTestClassDir();
 
@@ -49,7 +47,7 @@ public abstract class CompressBase extends AutomatedTestBase {
 	}
 
 	public void runTest(int rows, int cols, int decompressCount, int compressCount, ExecType ex, String name) {
-		compressTest(rows, cols, 1.0, ex, 1, 10, 1.4, decompressCount, compressCount, name);
+		compressTest(rows, cols, 1.0, ex, 1, 5, 1.4, decompressCount, compressCount, name);
 	}
 
 	public void compressTest(int rows, int cols, double sparsity, ExecType instType, int min, int max, double delta,
@@ -64,19 +62,17 @@ public abstract class CompressBase extends AutomatedTestBase {
 			writeInputMatrixWithMTD("A", A, false, new MatrixCharacteristics(rows, cols, 1000, rows * cols));
 
 			fullDMLScriptName = SCRIPT_DIR + "/functions/compress/compress_" + name + ".dml";
+			programArgs = new String[] {"-stats", "100", "-nvargs", "A=" + input("A")};
 
-			// programArgs = new String[] {"-stats", "100" , "-explain", "-nvargs", "A=" + input("A")};
-			programArgs = new String[] {"-stats", "100",  "-nvargs", "A=" + input("A")};
-
-			LOG.debug(runTest(null));
+			String out = runTest(null).toString();
 
 			int decompressCount = DMLCompressionStatistics.getDecompressionCount();
 			long compressionCount = (instType == ExecType.SPARK) ? Statistics
 				.getCPHeavyHitterCount("sp_compress") : Statistics.getCPHeavyHitterCount("compress");
 			DMLCompressionStatistics.reset();
 
-			Assert.assertEquals("Expected compression count   : ", compressionCount, compressionCountsExpected);
-			Assert.assertEquals("Expected Decompression count : ", decompressionCountExpected, decompressCount);
+			Assert.assertEquals(out + "\ncompression count   wrong : ", compressionCount, compressionCountsExpected);
+			Assert.assertEquals(out + "\nDecompression count wrong : ", decompressionCountExpected, decompressCount);
 
 		}
 		catch(Exception e) {
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressCost.java b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressCost.java
deleted file mode 100644
index a64e8de..0000000
--- a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressCost.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.test.functions.compress.configuration;
-
-import java.io.File;
-
-import org.apache.sysds.common.Types.ExecType;
-import org.junit.Test;
-
-public class CompressCost extends CompressBase {
-
-	public String TEST_NAME = "compress";
-	public String TEST_DIR = "functions/compress/cost/";
-	public String TEST_CLASS_DIR = TEST_DIR + CompressCost.class.getSimpleName() + "/";
-	private String TEST_CONF = "SystemDS-config-compress-cost.xml";
-	private File TEST_CONF_FILE = new File(SCRIPT_DIR + TEST_DIR, TEST_CONF);
-
-	protected String getTestClassDir() {
-		return TEST_CLASS_DIR;
-	}
-
-	protected String getTestName() {
-		return TEST_NAME;
-	}
-
-	protected String getTestDir() {
-		return TEST_DIR;
-	}
-
-	@Test
-	public void testTranspose() {
-		runTest(100, 20, 0, 0, ExecType.CP, "transpose");
-	}
-
-	@Test
-	public void testSum() {
-		runTest(100, 20, 0, 0, ExecType.CP, "sum");
-	}
-
-	@Test
-	public void testRowAggregate() {
-		runTest(100, 20, 0, 0, ExecType.CP, "row_min");
-	}
-
-	/**
-	 * Override default configuration with custom test configuration to ensure scratch space and local temporary
-	 * directory locations are also updated.
-	 */
-	@Override
-	protected File getConfigTemplateFile() {
-		return TEST_CONF_FILE;
-	}
-}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressForce.java b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressForce.java
index 84be096..dc5b17f 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressForce.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressForce.java
@@ -151,11 +151,47 @@ public class CompressForce extends CompressBase {
 	}
 
 	@Test
+	public void testMatrixMultRightSum_Larger_CP() {
+		runTest(1500, 11, 0, 1, ExecType.CP, "mmr_sum");
+	}
+
+	@Test
 	public void testMatrixMultRightSum_Larger_SP() {
 		runTest(1500, 11, 0, 1, ExecType.SPARK, "mmr_sum");
 	}
 
 	@Test
+	public void testMatrixMultRightSumPlus_Larger_CP() {
+		runTest(1500, 11, 0, 1, ExecType.CP, "mmr_sum_plus");
+	}
+
+	@Test
+	public void testMatrixMultRightSumPlus_Larger_SP() {
+		runTest(1500, 11, 0, 1, ExecType.SPARK, "mmr_sum_plus");
+	}
+
+	@Test
+	public void testMatrixMultRightSumPlusOnOverlap_Larger_CP() {
+		runTest(1500, 11, 0, 1, ExecType.CP, "mmr_sum_plus_2");
+	}
+
+	@Test
+	public void testMatrixMultRightSumPlusOnOverlap_Larger_SP() {
+		// be aware that with multiple blocks it is likely that the small blocks
+		// initially compress, but is to large for overlapping state will decompress.
+		// In this test it does not decompress
+		runTest(1010, 11, 0, 1, ExecType.SPARK, "mmr_sum_plus_2");
+	}
+
+	@Test
+	public void testMatrixMultRightSumPlusOnOverlapDecompress_Larger_SP() {
+		// be aware that with multiple blocks it is likely that the small blocks
+		// initially compress, but is to large for overlapping state therefor will decompress.
+		// In this test it decompress the second small block but keeps the first in overlapping state.
+		runTest(1110, 30, 1, 1, ExecType.SPARK, "mmr_sum_plus_2");
+	}
+
+	@Test
 	public void testMatrixMultLeftSum_CP() {
 		runTest(1500, 1, 0, 1, ExecType.CP, "mml_sum");
 	}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossy.java b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossy.java
deleted file mode 100644
index 68da2e5..0000000
--- a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossy.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.test.functions.compress.configuration;
-
-import java.io.File;
-
-public class CompressLossy extends CompressForce {
-
-	public String TEST_NAME = "compress";
-	public String TEST_DIR = "functions/compress/force/";
-	public String TEST_CLASS_DIR = TEST_DIR + CompressLossy.class.getSimpleName() + "/";
-	private String TEST_CONF = "SystemDS-config-compress-lossy.xml";
-	private File TEST_CONF_FILE = new File(SCRIPT_DIR + TEST_DIR, TEST_CONF);
-
-	protected String getTestClassDir() {
-		return TEST_CLASS_DIR;
-	}
-
-	protected String getTestName() {
-		return TEST_NAME;
-	}
-
-	protected String getTestDir() {
-		return TEST_DIR;
-	}
-
-	/**
-	 * Override default configuration with custom test configuration to ensure scratch space and local temporary
-	 * directory locations are also updated.
-	 */
-	@Override
-	protected File getConfigTemplateFile() {
-		return TEST_CONF_FILE;
-	}
-}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossyCost.java b/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossyCost.java
deleted file mode 100644
index e99e791..0000000
--- a/src/test/java/org/apache/sysds/test/functions/compress/configuration/CompressLossyCost.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.test.functions.compress.configuration;
-
-import java.io.File;
-
-public class CompressLossyCost extends CompressCost {
-
-	public String TEST_NAME = "compress";
-	public String TEST_DIR = "functions/compress/cost";
-	public String TEST_CLASS_DIR = TEST_DIR + CompressLossyCost.class.getSimpleName() + "/";
-	private String TEST_CONF = "SystemDS-config-compress-cost-lossy.xml";
-	private File TEST_CONF_FILE = new File(SCRIPT_DIR + TEST_DIR, TEST_CONF);
-
-	protected String getTestClassDir() {
-		return TEST_CLASS_DIR;
-	}
-
-	protected String getTestName() {
-		return TEST_NAME;
-	}
-
-	protected String getTestDir() {
-		return TEST_DIR;
-	}
-
-	/**
-	 * Override default configuration with custom test configuration to ensure scratch space and local temporary
-	 * directory locations are also updated.
-	 */
-	@Override
-	protected File getConfigTemplateFile() {
-		return TEST_CONF_FILE;
-	}
-}
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
index af05bdc..c35b209 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
@@ -44,6 +44,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 	private final static String TEST_NAME4 = "WorkloadAnalysisSliceLine";
 	private final static String TEST_NAME5 = "WorkloadAnalysisSliceFinder";
 	private final static String TEST_NAME6 = "WorkloadAnalysisLmCG";
+	private final static String TEST_NAME7 = "WorkloadAnalysisL2SVM";
 	private final static String TEST_DIR = "functions/compress/workload/";
 	private final static String TEST_CLASS_DIR = TEST_DIR + WorkloadAnalysisTest.class.getSimpleName() + "/";
 
@@ -69,6 +70,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 		addTestConfiguration(TEST_NAME4, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME4, new String[] {"B"}));
 		addTestConfiguration(TEST_NAME5, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME5, new String[] {"B"}));
 		addTestConfiguration(TEST_NAME6, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME6, new String[] {"B"}));
+		addTestConfiguration(TEST_NAME7, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME7, new String[] {"B"}));
 	}
 
 	@Test
@@ -126,6 +128,11 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 		runWorkloadAnalysisTest(TEST_NAME6, ExecMode.SINGLE_NODE, 2, false);
 	}
 
+	@Test
+	public void testL2SVMCP() {
+		runWorkloadAnalysisTest(TEST_NAME7, ExecMode.SINGLE_NODE, 2, false);
+	}
+
 	// private void runWorkloadAnalysisTest(String testname, ExecMode mode, int compressionCount) {
 	private void runWorkloadAnalysisTest(String testname, ExecMode mode, int compressionCount, boolean intermediates) {
 		ExecMode oldPlatform = setExecMode(mode);
@@ -137,8 +144,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 
 			String HOME = SCRIPT_DIR + TEST_DIR;
 			fullDMLScriptName = HOME + testname + ".dml";
-			programArgs = new String[] {"-stats", "20", "-args", input("X"), input("y"),
-				output("B")};
+			programArgs = new String[] {"-stats", "20", "-args", input("X"), input("y"), output("B")};
 
 			writeInputMatrixWithMTD("X", X, false);
 			writeInputMatrixWithMTD("y", y, false);
@@ -153,7 +159,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 			Assert.assertEquals("Assert that the compression counts expeted matches actual: " + compressionCount
 				+ " vs " + actualCompressionCount, compressionCount, actualCompressionCount);
 			if(compressionCount > 0)
-				Assert.assertTrue(mode == ExecMode.SINGLE_NODE || mode == ExecMode.HYBRID ?  heavyHittersContainsString(
+				Assert.assertTrue(mode == ExecMode.SINGLE_NODE || mode == ExecMode.HYBRID ? heavyHittersContainsString(
 					"compress") : heavyHittersContainsString("sp_compress"));
 			if(!testname.equals(TEST_NAME4))
 				Assert.assertFalse(heavyHittersContainsString("m_scale"));
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAnalysisTest.java b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAnalysisTest.java
index a9c2853..f1efd34 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAnalysisTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAnalysisTest.java
@@ -45,15 +45,14 @@ public class WorkloadAnalysisTest extends AutomatedTestBase {
 		addTestConfiguration(TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2, new String[] {"B"}));
 	}
 
-
-	@Test 
-	public void testLeftMultiplicationLoop(){
+	@Test
+	public void testLeftMultiplicationLoop() {
 		runWorkloadAnalysisTest(TEST_NAME1, ExecMode.HYBRID, 1);
 	}
 
 	@Test
-	public void testRightMultiplicationLoop(){
-		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.HYBRID,1);
+	public void testRightMultiplicationLoop() {
+		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.HYBRID, 1);
 	}
 
 	private void runWorkloadAnalysisTest(String testname, ExecMode mode, int compressionCount) {
diff --git a/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java b/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
index 1cc1cca..6a8bfd7 100644
--- a/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
+++ b/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
@@ -58,25 +58,35 @@ public class RewriteMMCBindZeroVector extends AutomatedTestBase {
 
 	@Test
 	public void testNoRewritesCP() {
-		testRewrite(TEST_NAME1, false, ExecType.CP, 100, 3, 10);
+		testRewrite(TEST_NAME1, false, ExecType.CP, 100, 3, 10, false);
 	}
 
 	@Test
 	public void testNoRewritesSP() {
-		testRewrite(TEST_NAME1, false, ExecType.SPARK, 100, 3, 10);
+		testRewrite(TEST_NAME1, false, ExecType.SPARK, 100, 3, 10, false);
 	}
 
 	@Test
 	public void testRewritesCP() {
-		testRewrite(TEST_NAME1, true, ExecType.CP, 100, 3, 10);
+		testRewrite(TEST_NAME1, true, ExecType.CP, 100, 3, 10, true);
 	}
 
 	@Test
 	public void testRewritesSP() {
-		testRewrite(TEST_NAME1, true, ExecType.SPARK, 100, 3, 10);
+		testRewrite(TEST_NAME1, true, ExecType.SPARK, 100, 3, 10, true);
 	}
 
-	private void testRewrite(String testname, boolean rewrites, ExecType et, int leftRows, int rightCols, int shared) {
+	@Test
+	public void testRewritesCP_ButToSmall() {
+		testRewrite(TEST_NAME1, true, ExecType.CP, 100, 10, 55, false);
+	}
+
+	@Test
+	public void testRewritesSP_ButToSmall() {
+		testRewrite(TEST_NAME1, true, ExecType.SPARK, 100, 10, 55, false);
+	}
+
+	private void testRewrite(String testname, boolean rewrites, ExecType et, int leftRows, int rightCols, int shared, boolean rewriteShouldBeExecuted) {
 		ExecMode platformOld = rtplatform;
 		switch(et) {
 			case SPARK:
@@ -100,7 +110,7 @@ public class RewriteMMCBindZeroVector extends AutomatedTestBase {
 
 			String HOME = SCRIPT_DIR + TEST_DIR;
 			fullDMLScriptName = HOME + testname + ".dml";
-			programArgs = new String[] {"-explain", "-stats", "-args", input("X"), input("Y"),
+			programArgs = new String[] {"-explain", "hops","-stats", "-args", input("X"), input("Y"),
 				output("R")};
 			fullRScriptName = HOME + testname + ".R";
 			rCmd = getRCmd(inputDir(), expectedDir());
@@ -114,20 +124,20 @@ public class RewriteMMCBindZeroVector extends AutomatedTestBase {
 			String out = runTest(null).toString();
 
 			for(String line : out.split("\n")) {
-				if(rewrites) {
-					if(line.contains("append"))
+				if(rewrites && rewriteShouldBeExecuted) {
+					if(line.contains("b(cbind)"))
 						break;
-					else if(line.contains("ba+*"))
+					else if(line.contains("ba(+*)"))
 						fail(
-							"invalid execution matrix multiplication is done before append, therefore the rewrite did not tricker.\n\n"
+							"invalid execution matrix multiplication is done before b(cbind), therefore the rewrite did not tricker.\n\n"
 								+ out);
 				}
 				else {
-					if(line.contains("ba+*"))
+					if(line.contains("ba(+*)"))
 						break;
-					else if(line.contains("append"))
+					else if(line.contains("b(cbind)"))
 						fail(
-							"invalid execution append was done before multiplication, therefore the rewrite did tricker when not allowed.\n\n"
+							"invalid execution b(cbind) was done before multiplication, therefore the rewrite did tricker when not allowed.\n\n"
 								+ out);
 				}
 
diff --git a/src/test/resources/component/compress/1-1_y.csv b/src/test/resources/component/compress/1-1_y.csv
new file mode 100644
index 0000000..56a6051
--- /dev/null
+++ b/src/test/resources/component/compress/1-1_y.csv
@@ -0,0 +1 @@
+1
\ No newline at end of file
diff --git a/src/test/resources/component/compress/1-1_y.csv.mtd b/src/test/resources/component/compress/1-1_y.csv.mtd
new file mode 100644
index 0000000..2db0d5e
--- /dev/null
+++ b/src/test/resources/component/compress/1-1_y.csv.mtd
@@ -0,0 +1,8 @@
+{  
+    "data_type": "matrix",
+    "value_type": "double",  
+    "rows": 1000000,
+    "cols": 1,
+    "nnz": 1,
+    "format": "csv"
+}
diff --git a/src/test/resources/component/compress/README.md b/src/test/resources/component/compress/README.md
index 183e4d6..a6669df 100644
--- a/src/test/resources/component/compress/README.md
+++ b/src/test/resources/component/compress/README.md
@@ -19,5 +19,5 @@ limitations under the License.
 
 # Test files
 
-It is intensional that the mtd file says that the file is 1 million rows, and the actual file is 1 row.
+It is intensional that the mtd files says that the file is 1 million rows, and the actual file is 1 row.
 Since the tests using this file is intended for simulating the workload, not actually executing the script.
diff --git a/src/test/scripts/functions/compress/compress_mmr_sum.dml b/src/test/scripts/component/compress/workload/functions/l2svm_Y.dml
similarity index 87%
copy from src/test/scripts/functions/compress/compress_mmr_sum.dml
copy to src/test/scripts/component/compress/workload/functions/l2svm_Y.dml
index a3bedfb..c270663 100644
--- a/src/test/scripts/functions/compress/compress_mmr_sum.dml
+++ b/src/test/scripts/component/compress/workload/functions/l2svm_Y.dml
@@ -19,8 +19,8 @@
 #
 #-------------------------------------------------------------
 
-x = read($A)
-v = rand(rows=ncol(x), cols=10, min=0.0, max=1.0, seed= 13);
-r = x %*% v  + 1
-s = sum(r)
-print(s)
+b = read($1)
+A = rand(rows= nrow(b), cols = 10, min = -1, max = 1)
+b = round(b)
+m = l2svm(X=A ,Y=b, verbose=$2)
+print(mean(m))
diff --git a/src/test/scripts/functions/compress/compress_mmr_sum.dml b/src/test/scripts/functions/compress/compress_mmr_sum.dml
index a3bedfb..b46ef4b 100644
--- a/src/test/scripts/functions/compress/compress_mmr_sum.dml
+++ b/src/test/scripts/functions/compress/compress_mmr_sum.dml
@@ -21,6 +21,6 @@
 
 x = read($A)
 v = rand(rows=ncol(x), cols=10, min=0.0, max=1.0, seed= 13);
-r = x %*% v  + 1
+r = x %*% v
 s = sum(r)
 print(s)
diff --git a/src/test/scripts/functions/compress/compress_mmr_sum.dml b/src/test/scripts/functions/compress/compress_mmr_sum_plus.dml
similarity index 98%
copy from src/test/scripts/functions/compress/compress_mmr_sum.dml
copy to src/test/scripts/functions/compress/compress_mmr_sum_plus.dml
index a3bedfb..506ffbe 100644
--- a/src/test/scripts/functions/compress/compress_mmr_sum.dml
+++ b/src/test/scripts/functions/compress/compress_mmr_sum_plus.dml
@@ -21,6 +21,6 @@
 
 x = read($A)
 v = rand(rows=ncol(x), cols=10, min=0.0, max=1.0, seed= 13);
-r = x %*% v  + 1
+r = (x + 1) %*% v
 s = sum(r)
 print(s)
diff --git a/src/test/scripts/functions/compress/compress_mmr_sum.dml b/src/test/scripts/functions/compress/compress_mmr_sum_plus_2.dml
similarity index 98%
copy from src/test/scripts/functions/compress/compress_mmr_sum.dml
copy to src/test/scripts/functions/compress/compress_mmr_sum_plus_2.dml
index a3bedfb..162309a 100644
--- a/src/test/scripts/functions/compress/compress_mmr_sum.dml
+++ b/src/test/scripts/functions/compress/compress_mmr_sum_plus_2.dml
@@ -21,6 +21,6 @@
 
 x = read($A)
 v = rand(rows=ncol(x), cols=10, min=0.0, max=1.0, seed= 13);
-r = x %*% v  + 1
+r = (x %*% v) + 1
 s = sum(r)
 print(s)
diff --git a/src/test/scripts/functions/compress/compress_mmr_sum.dml b/src/test/scripts/functions/compress/workload/WorkloadAnalysisL2SVM.dml
similarity index 70%
copy from src/test/scripts/functions/compress/compress_mmr_sum.dml
copy to src/test/scripts/functions/compress/workload/WorkloadAnalysisL2SVM.dml
index a3bedfb..1ea0528 100644
--- a/src/test/scripts/functions/compress/compress_mmr_sum.dml
+++ b/src/test/scripts/functions/compress/workload/WorkloadAnalysisL2SVM.dml
@@ -7,9 +7,9 @@
 # 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
@@ -19,8 +19,23 @@
 #
 #-------------------------------------------------------------
 
-x = read($A)
-v = rand(rows=ncol(x), cols=10, min=0.0, max=1.0, seed= 13);
-r = x %*% v  + 1
-s = sum(r)
-print(s)
+X = read($1);
+y = read($2);
+
+# Limit to one classification
+y = y == min(y)
+
+print("")
+print("LMCG")
+
+X = scale(X=X, scale=TRUE, center=TRUE);
+B = l2svm(X=X, Y=y, verbose=TRUE);
+[y_pred, n] = l2svmPredict(X=X, W=B, verbose=TRUE);
+
+classifications = (y_pred > 0.1)
+
+acc = sum(classifications == y)  / nrow(y)
+
+if(acc < 0.80)
+    stop("ERROR: to low accuracy achieved")
+print(acc)

[systemds] 02/03: [SYSTEMDS-3123] Rewrite c bind 0 Matrix Multiplication

Posted by ba...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e607544ea6ab1f2ec1c9f2c8370c38c86c346170
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Tue Sep 7 19:14:56 2021 +0200

    [SYSTEMDS-3123] Rewrite c bind 0 Matrix Multiplication
    
    ```
     cbind((X %*% Y), matrix(0, nrow(X), 1))
    
     ->
    
     X %*% (cbind(Y, matrix(0, nrow(Y), 1)))
    ```
    
    This commit contains a rewrite that change the sequences if number
    of rows in X is 2x larger than Y:
    
    This rewrite effects MLogReg in line 215 to not force allocation of the
    large X twice.
---
 .../RewriteAlgebraicSimplificationDynamic.java     |  54 +++++---
 .../compress/workload/WorkloadAnalyzer.java        |  30 ++---
 .../compress/workload/WorkloadAlgorithmTest.java   |  34 ++---
 .../rewrite/RewriteMMCBindZeroVector.java          | 145 +++++++++++++++++++++
 .../compress/workload/WorkloadAnalysisMLogReg.dml  |  13 +-
 .../RewritMMCBindZeroVectorOp.dml}                 |  26 +---
 6 files changed, 229 insertions(+), 73 deletions(-)

diff --git a/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java b/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
index 63a05a4..0b91e5d 100644
--- a/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
+++ b/src/main/java/org/apache/sysds/hops/rewrite/RewriteAlgebraicSimplificationDynamic.java
@@ -26,6 +26,20 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 
+import org.apache.sysds.common.Types.AggOp;
+import org.apache.sysds.common.Types.DataType;
+import org.apache.sysds.common.Types.Direction;
+import org.apache.sysds.common.Types.OpOp1;
+import org.apache.sysds.common.Types.OpOp2;
+import org.apache.sysds.common.Types.OpOp3;
+import org.apache.sysds.common.Types.OpOp4;
+import org.apache.sysds.common.Types.OpOpDG;
+import org.apache.sysds.common.Types.OpOpN;
+import org.apache.sysds.common.Types.ParamBuiltinOp;
+import org.apache.sysds.common.Types.ReOrgOp;
+import org.apache.sysds.common.Types.ValueType;
+import org.apache.sysds.conf.ConfigurationManager;
+import org.apache.sysds.conf.DMLConfig;
 import org.apache.sysds.hops.AggBinaryOp;
 import org.apache.sysds.hops.AggUnaryOp;
 import org.apache.sysds.hops.BinaryOp;
@@ -41,22 +55,8 @@ import org.apache.sysds.hops.QuaternaryOp;
 import org.apache.sysds.hops.ReorgOp;
 import org.apache.sysds.hops.TernaryOp;
 import org.apache.sysds.hops.UnaryOp;
-import org.apache.sysds.common.Types.AggOp;
-import org.apache.sysds.common.Types.Direction;
-import org.apache.sysds.common.Types.OpOp1;
-import org.apache.sysds.common.Types.OpOp2;
-import org.apache.sysds.common.Types.OpOp3;
-import org.apache.sysds.common.Types.OpOp4;
-import org.apache.sysds.common.Types.OpOpDG;
-import org.apache.sysds.common.Types.OpOpN;
-import org.apache.sysds.common.Types.ParamBuiltinOp;
-import org.apache.sysds.common.Types.ReOrgOp;
 import org.apache.sysds.lops.MapMultChain.ChainType;
 import org.apache.sysds.parser.DataExpression;
-import org.apache.sysds.common.Types.DataType;
-import org.apache.sysds.common.Types.ValueType;
-import org.apache.sysds.conf.ConfigurationManager;
-import org.apache.sysds.conf.DMLConfig;
 
 /**
  * Rule: Algebraic Simplifications. Simplifies binary expressions
@@ -109,7 +109,6 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 	public Hop rewriteHopDAG(Hop root, ProgramRewriteStatus state) {
 		if( root == null )
 			return root;
-		
 		//one pass rewrite-descend (rewrite created pattern)
 		rule_AlgebraicSimplification( root, false );
 		
@@ -197,6 +196,7 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 			hi = simplifyNnzComputation(hop, hi, i);          //e.g., sum(ppred(X,0,"!=")) -> literal(nnz(X)), if nnz known
 			hi = simplifyNrowNcolComputation(hop, hi, i);     //e.g., nrow(X) -> literal(nrow(X)), if nrow known to remove data dependency
 			hi = simplifyTableSeqExpand(hop, hi, i);          //e.g., table(seq(1,nrow(v)), v, nrow(v), m) -> rexpand(v, max=m, dir=row, ignore=false, cast=true)
+			hi = simplyfyMMCBindZeroVector(hop, hi, i);       //e.g.. cbind((X %*% Y), matrix (0, nrow(X), 1)) -> X %*% (cbind(Y, matrix(0, nrow(Y), 1))) if nRows of x is larger than nCols of y
 			if( OptimizerUtils.ALLOW_OPERATOR_FUSION )
 				foldMultipleMinMaxOperations(hi);             //e.g., min(X,min(min(3,7),Y)) -> min(X,3,7,Y)
 			
@@ -2796,4 +2796,28 @@ public class RewriteAlgebraicSimplificationDynamic extends HopRewriteRule
 		
 		return hi;
 	}
+
+	private static Hop simplyfyMMCBindZeroVector(Hop parent, Hop hi, int pos) {
+
+		// cbind((X %*% Y), matrix(0, nrow(X), 1)) ->
+		// X %*% (cbind(Y, matrix(0, nrow(Y), 1)))
+		// if nRows of x is larger than nCols of y
+		// rewrite used in MLogReg first level loop.
+		
+		if(HopRewriteUtils.isBinary(hi, OpOp2.CBIND) && HopRewriteUtils.isMatrixMultiply(hi.getInput(0)) &&
+			HopRewriteUtils.isDataGenOpWithConstantValue(hi.getInput(1), 0) && hi.getDim1() > hi.getDim2() * 2) {
+			final Hop oldGen = hi.getInput(1);
+			final Hop y = hi.getInput(0).getInput(1);
+			final Hop x = hi.getInput(0).getInput(0);
+			final Hop newGen = HopRewriteUtils.createDataGenOp(y, oldGen, 0);
+			final Hop newCBind = HopRewriteUtils.createBinary(y, newGen, OpOp2.CBIND);
+			final Hop newMM = HopRewriteUtils.createMatrixMultiply(x, newCBind);
+
+			HopRewriteUtils.replaceChildReference(parent, hi, newMM, pos);
+			LOG.debug("Applied MMCBind Zero algebraic simplification (line " +hi.getBeginLine()+")." );
+			return newMM;
+
+		}
+		return hi;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
index 31b3714..c865507 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/workload/WorkloadAnalyzer.java
@@ -381,21 +381,21 @@ public class WorkloadAnalyzer {
 					transientCompressed.contains(in.get(1).getName());
 				OpSided ret = new OpSided(hop, left, right, transposedLeft, transposedRight);
 				if(ret.isRightMM()) {
-					HashSet<Long> overlapping2 = new HashSet<>();
-					overlapping2.add(hop.getHopID());
-					WorkloadAnalyzer overlappingAnalysis = new WorkloadAnalyzer(prog, overlapping2);
-					WTreeRoot r = overlappingAnalysis.createWorkloadTree(hop);
-
-					CostEstimatorBuilder b = new CostEstimatorBuilder(r);
-					if(LOG.isTraceEnabled())
-						LOG.trace("Workload for overlapping: " + r + "\n" + b);
-
-					if(b.shouldUseOverlap())
-						overlapping.add(hop.getHopID());
-					else {
-						decompressHops.add(hop);
-						ret.setOverlappingDecompression(true);
-					}
+					// HashSet<Long> overlapping2 = new HashSet<>();
+					// overlapping2.add(hop.getHopID());
+					// WorkloadAnalyzer overlappingAnalysis = new WorkloadAnalyzer(prog, overlapping2);
+					// WTreeRoot r = overlappingAnalysis.createWorkloadTree(hop);
+
+					// CostEstimatorBuilder b = new CostEstimatorBuilder(r);
+					// if(LOG.isTraceEnabled())
+					// 	LOG.trace("Workload for overlapping: " + r + "\n" + b);
+
+					// if(b.shouldUseOverlap())
+					overlapping.add(hop.getHopID());
+					// else {
+					// 	decompressHops.add(hop);
+					// 	ret.setOverlappingDecompression(true);
+					// }
 				}
 
 				return ret;
diff --git a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
index 5de8880..af05bdc 100644
--- a/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/compress/workload/WorkloadAlgorithmTest.java
@@ -83,7 +83,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 
 	@Test
 	public void testLmCP() {
-		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.HYBRID, 2, false);
+		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.SINGLE_NODE, 2, false);
 	}
 
 	@Test
@@ -93,7 +93,7 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 
 	@Test
 	public void testLmDSCP() {
-		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.HYBRID, 2, false);
+		runWorkloadAnalysisTest(TEST_NAME2, ExecMode.SINGLE_NODE, 2, false);
 	}
 
 	@Test
@@ -103,41 +103,42 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 
 	@Test
 	public void testPCACP() {
-		runWorkloadAnalysisTest(TEST_NAME3, ExecMode.HYBRID, 1, false);
+		runWorkloadAnalysisTest(TEST_NAME3, ExecMode.SINGLE_NODE, 1, false);
 	}
 
 	@Test
 	public void testSliceLineCP1() {
-		runWorkloadAnalysisTest(TEST_NAME4, ExecMode.HYBRID, 0, false);
+		runWorkloadAnalysisTest(TEST_NAME4, ExecMode.SINGLE_NODE, 0, false);
 	}
 
 	@Test
 	public void testSliceLineCP2() {
-		runWorkloadAnalysisTest(TEST_NAME4, ExecMode.HYBRID, 2, true);
+		runWorkloadAnalysisTest(TEST_NAME4, ExecMode.SINGLE_NODE, 2, true);
 	}
 
 	@Test
 	public void testLmCGSP() {
 		runWorkloadAnalysisTest(TEST_NAME6, ExecMode.SPARK, 2, false);
 	}
-	
+
 	@Test
 	public void testLmCGCP() {
-		runWorkloadAnalysisTest(TEST_NAME6, ExecMode.HYBRID, 2, false);
+		runWorkloadAnalysisTest(TEST_NAME6, ExecMode.SINGLE_NODE, 2, false);
 	}
-	
+
 	// private void runWorkloadAnalysisTest(String testname, ExecMode mode, int compressionCount) {
 	private void runWorkloadAnalysisTest(String testname, ExecMode mode, int compressionCount, boolean intermediates) {
 		ExecMode oldPlatform = setExecMode(mode);
 		boolean oldIntermediates = WorkloadAnalyzer.ALLOW_INTERMEDIATE_CANDIDATES;
-		
+
 		try {
 			loadTestConfiguration(getTestConfiguration(testname));
 			WorkloadAnalyzer.ALLOW_INTERMEDIATE_CANDIDATES = intermediates;
-			
+
 			String HOME = SCRIPT_DIR + TEST_DIR;
 			fullDMLScriptName = HOME + testname + ".dml";
-			programArgs = new String[] {"-stats", "20", "-args", input("X"), input("y"), output("B")};
+			programArgs = new String[] {"-stats", "20", "-args", input("X"), input("y"),
+				output("B")};
 
 			writeInputMatrixWithMTD("X", X, false);
 			writeInputMatrixWithMTD("y", y, false);
@@ -149,11 +150,12 @@ public class WorkloadAlgorithmTest extends AutomatedTestBase {
 			long actualCompressionCount = (mode == ExecMode.HYBRID || mode == ExecMode.SINGLE_NODE) ? Statistics
 				.getCPHeavyHitterCount("compress") : Statistics.getCPHeavyHitterCount("sp_compress");
 
-			Assert.assertEquals(compressionCount, actualCompressionCount);
-			if( compressionCount > 0 )
-				Assert.assertTrue( mode == ExecMode.HYBRID ?
-					heavyHittersContainsString("compress") : heavyHittersContainsString("sp_compress"));
-			if( !testname.equals(TEST_NAME4) )
+			Assert.assertEquals("Assert that the compression counts expeted matches actual: " + compressionCount
+				+ " vs " + actualCompressionCount, compressionCount, actualCompressionCount);
+			if(compressionCount > 0)
+				Assert.assertTrue(mode == ExecMode.SINGLE_NODE || mode == ExecMode.HYBRID ?  heavyHittersContainsString(
+					"compress") : heavyHittersContainsString("sp_compress"));
+			if(!testname.equals(TEST_NAME4))
 				Assert.assertFalse(heavyHittersContainsString("m_scale"));
 
 		}
diff --git a/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java b/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
new file mode 100644
index 0000000..1cc1cca
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/rewrite/RewriteMMCBindZeroVector.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.rewrite;
+
+import static org.junit.Assert.fail;
+
+import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.common.Types.ExecMode;
+import org.apache.sysds.common.Types.ExecType;
+import org.apache.sysds.hops.OptimizerUtils;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.sysds.test.TestUtils;
+import org.junit.Test;
+
+/**
+ * from:
+ * 
+ * res = cbind((X %*% Y), matrix (0, nrow(X), 1));
+ * 
+ * to:
+ * 
+ * res = X %*% (cbind(Y, matrix(0, nrow(Y), 1)))
+ * 
+ * 
+ * if the X has many rows, the allocation of x is expensive, to cbind. the case where this is applicable is mLogReg.
+ * 
+ */
+public class RewriteMMCBindZeroVector extends AutomatedTestBase {
+	// private static final Log LOG = LogFactory.getLog(RewriteMMCBindZeroVector.class.getName());
+
+	private static final String TEST_NAME1 = "RewritMMCBindZeroVectorOp";
+	private static final String TEST_DIR = "functions/rewrite/";
+	private static final String TEST_CLASS_DIR = TEST_DIR + RewriteMMCBindZeroVector.class.getSimpleName() + "/";
+
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1, new String[] {"R"}));
+	}
+
+	@Test
+	public void testNoRewritesCP() {
+		testRewrite(TEST_NAME1, false, ExecType.CP, 100, 3, 10);
+	}
+
+	@Test
+	public void testNoRewritesSP() {
+		testRewrite(TEST_NAME1, false, ExecType.SPARK, 100, 3, 10);
+	}
+
+	@Test
+	public void testRewritesCP() {
+		testRewrite(TEST_NAME1, true, ExecType.CP, 100, 3, 10);
+	}
+
+	@Test
+	public void testRewritesSP() {
+		testRewrite(TEST_NAME1, true, ExecType.SPARK, 100, 3, 10);
+	}
+
+	private void testRewrite(String testname, boolean rewrites, ExecType et, int leftRows, int rightCols, int shared) {
+		ExecMode platformOld = rtplatform;
+		switch(et) {
+			case SPARK:
+				rtplatform = ExecMode.SPARK;
+				break;
+			default:
+				rtplatform = ExecMode.HYBRID;
+				break;
+		}
+
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		if(rtplatform == ExecMode.SPARK || rtplatform == ExecMode.HYBRID)
+			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+
+		boolean rewritesOld = OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION;
+		OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewrites;
+
+		try {
+			TestConfiguration config = getTestConfiguration(testname);
+			loadTestConfiguration(config);
+
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = HOME + testname + ".dml";
+			programArgs = new String[] {"-explain", "-stats", "-args", input("X"), input("Y"),
+				output("R")};
+			fullRScriptName = HOME + testname + ".R";
+			rCmd = getRCmd(inputDir(), expectedDir());
+
+			double[][] X = getRandomMatrix(leftRows, shared, -1, 1, 0.97d, 7);
+			double[][] Y = getRandomMatrix(shared, rightCols, -1, 1, 0.9d, 3);
+			writeInputMatrixWithMTD("X", X, false);
+			writeInputMatrixWithMTD("Y", Y, false);
+
+			// execute tests
+			String out = runTest(null).toString();
+
+			for(String line : out.split("\n")) {
+				if(rewrites) {
+					if(line.contains("append"))
+						break;
+					else if(line.contains("ba+*"))
+						fail(
+							"invalid execution matrix multiplication is done before append, therefore the rewrite did not tricker.\n\n"
+								+ out);
+				}
+				else {
+					if(line.contains("ba+*"))
+						break;
+					else if(line.contains("append"))
+						fail(
+							"invalid execution append was done before multiplication, therefore the rewrite did tricker when not allowed.\n\n"
+								+ out);
+				}
+
+			}
+			// compare matrices
+			// HashMap<CellIndex, Double> dmlfile = readDMLMatrixFromOutputDir("R");
+
+		}
+		finally {
+			OptimizerUtils.ALLOW_ALGEBRAIC_SIMPLIFICATION = rewritesOld;
+			rtplatform = platformOld;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+		}
+	}
+}
diff --git a/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml b/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml
index 12d9dd5..d427506 100644
--- a/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml
+++ b/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml
@@ -22,21 +22,20 @@
 X = read($1);
 Y = read($2);
 
-
 print("")
 print("MLogReg")
 
 X = scale(X=X, scale=TRUE, center=TRUE);
-B = multiLogReg(X=X, Y=Y, verbose=FALSE, maxi=3, maxii=2);
+B = multiLogReg(X=X, Y=Y, verbose=FALSE, maxi=3, maxii=2, icpt=0);
 
 [nn, P, acc] = multiLogRegPredict(X=X, B=B, Y=Y)
-
 [nn, C] = confusionMatrix(P, Y)
-print("Confusion: ")
-print(toString(C))
 
+print("Confusion:")
+print(toString(C))
+print("")
 print(acc)
 
-if(acc < 50){
+if(acc < 50)
     stop("MLogReg Accuracy achieved is not high enough")
-}
+
diff --git a/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml b/src/test/scripts/functions/rewrite/RewritMMCBindZeroVectorOp.dml
similarity index 71%
copy from src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml
copy to src/test/scripts/functions/rewrite/RewritMMCBindZeroVectorOp.dml
index 12d9dd5..e6b0498 100644
--- a/src/test/scripts/functions/compress/workload/WorkloadAnalysisMLogReg.dml
+++ b/src/test/scripts/functions/rewrite/RewritMMCBindZeroVectorOp.dml
@@ -7,9 +7,9 @@
 # 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
@@ -19,24 +19,10 @@
 #
 #-------------------------------------------------------------
 
-X = read($1);
-Y = read($2);
-
-
-print("")
-print("MLogReg")
-
-X = scale(X=X, scale=TRUE, center=TRUE);
-B = multiLogReg(X=X, Y=Y, verbose=FALSE, maxi=3, maxii=2);
-
-[nn, P, acc] = multiLogRegPredict(X=X, B=B, Y=Y)
 
-[nn, C] = confusionMatrix(P, Y)
-print("Confusion: ")
-print(toString(C))
+X = read($1)
+Y = read($2)
 
-print(acc)
+res = cbind((X %*% Y), matrix (0, nrow(X), 1));
 
-if(acc < 50){
-    stop("MLogReg Accuracy achieved is not high enough")
-}
+print(sum(res))