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 2022/04/21 10:58:38 UTC

[systemds] branch main updated: [SYSTEMDS-2610] CLA Epic

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 8282474eda [SYSTEMDS-2610] CLA Epic
8282474eda is described below

commit 8282474edae2f34cb44d00cef559c748c6bdd76e
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Thu Apr 21 12:58:11 2022 +0200

    [SYSTEMDS-2610] CLA Epic
    
    This commit contains fixes for multiple JIRA tasks:
    
    [SYSTEMDS-3320] CLA optimize sparse combine
    [SYSTEMDS-3336] CLA Specialized maps
    - MapToZero for all zero mappings.
    - MapToCharPByte for 3 byte mappings (in between char and int).
    [SYSTEMDS-3356] CLA DDCFOR and SDCFOR
    [SYSTEMDS-3357] CLA Sparsify columnGroups
    
    Minor:
    
    - general specialization lmm and change cost model for lmm
    - fix cost estimator on unknown dimensions (set to 16)
    - fix sparse TSMM in full rows CSR
    - remove memorizer on Offsets
    - clear soft reference to uncompressed in case of spark compression
    - replace shortcut and compressed multiply cost minimum rows processed
    - lower criteria for transpose in compression
    - MM binary decompress into output
    - fix single col table on compressed colgroup
    - transpose size in memory if compressed is equal to compressed size
    
    Closes #1589
---
 src/main/java/org/apache/sysds/hops/ReorgOp.java   |   8 +
 .../ipa/IPAPassCompressionWorkloadAnalysis.java    |  11 +-
 .../runtime/compress/CompressedMatrixBlock.java    |  45 +-
 .../compress/CompressedMatrixBlockFactory.java     |  64 ++-
 .../compress/CompressionSettingsBuilder.java       |   2 +-
 .../runtime/compress/bitmap/BitmapEncoder.java     |  22 +-
 .../runtime/compress/cocode/CoCodeGreedy.java      | 168 +++---
 .../runtime/compress/cocode/CoCodePriorityQue.java |  25 +-
 .../sysds/runtime/compress/cocode/ColIndexes.java  |  10 +
 .../sysds/runtime/compress/cocode/Memorizer.java   |  29 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |   8 +-
 .../runtime/compress/colgroup/AColGroupValue.java  |  38 +-
 .../sysds/runtime/compress/colgroup/APreAgg.java   |  75 ++-
 .../sysds/runtime/compress/colgroup/ASDCZero.java  |  27 +-
 .../runtime/compress/colgroup/ColGroupDDC.java     |  66 ++-
 .../{ColGroupPFOR.java => ColGroupDDCFOR.java}     | 346 +++++++------
 .../runtime/compress/colgroup/ColGroupFactory.java | 557 ++++++++++++--------
 .../runtime/compress/colgroup/ColGroupIO.java      |  13 +-
 .../runtime/compress/colgroup/ColGroupSDC.java     |  31 +-
 .../{ColGroupPFOR.java => ColGroupSDCFOR.java}     | 105 ++--
 .../compress/colgroup/ColGroupSDCSingle.java       |  19 +-
 .../compress/colgroup/ColGroupSDCSingleZeros.java  | 121 +++--
 .../compress/colgroup/ColGroupSDCZeros.java        |  44 +-
 .../compress/colgroup/ColGroupUncompressed.java    |   4 +-
 .../sysds/runtime/compress/colgroup/FORUtil.java   |  75 +++
 .../compress/colgroup/dictionary/ADictionary.java  |  16 +-
 .../colgroup/dictionary/DictLibMatrixMult.java     |  72 ++-
 .../compress/colgroup/dictionary/Dictionary.java   |  27 +-
 .../colgroup/dictionary/DictionaryFactory.java     |  18 +-
 .../colgroup/dictionary/MatrixBlockDictionary.java |  63 ++-
 .../compress/colgroup/dictionary/QDictionary.java  |   9 +-
 .../compress/colgroup/mapping/AMapToData.java      | 112 +++-
 .../compress/colgroup/mapping/MapToBit.java        |  97 ++--
 .../compress/colgroup/mapping/MapToByte.java       |  55 +-
 .../compress/colgroup/mapping/MapToChar.java       |  46 +-
 .../compress/colgroup/mapping/MapToCharPByte.java  | 196 +++++++
 .../compress/colgroup/mapping/MapToFactory.java    |  99 ++--
 .../compress/colgroup/mapping/MapToInt.java        |  52 +-
 .../compress/colgroup/mapping/MapToUByte.java      |  22 +
 .../compress/colgroup/mapping/MapToZero.java       | 141 +++++
 .../runtime/compress/colgroup/offset/AOffset.java  | 530 +++++--------------
 .../compress/colgroup/offset/OffsetByte.java       |  10 +-
 .../compress/colgroup/offset/OffsetChar.java       |   4 +-
 .../compress/colgroup/offset/OffsetFactory.java    |  41 +-
 .../compress/colgroup/offset/OffsetSingle.java     | 132 +++++
 .../compress/colgroup/offset/OffsetTwo.java        | 141 +++++
 .../sysds/runtime/compress/cost/ACostEstimate.java |   8 +
 .../compress/cost/ComputationCostEstimator.java    |  36 +-
 .../compress/cost/CostEstimatorBuilder.java        |  22 +-
 .../compress/cost/DistinctCostEstimator.java       |   4 +
 .../runtime/compress/cost/HybridCostEstimator.java |   5 +
 .../compress/cost/InstructionTypeCounter.java      |  13 +
 .../runtime/compress/cost/MemoryCostEstimator.java |   5 +
 .../compress/estim/CompressedSizeEstimator.java    |  93 ++--
 .../estim/CompressedSizeEstimatorExact.java        |   3 +
 .../estim/CompressedSizeEstimatorFactory.java      |   2 +
 .../estim/CompressedSizeEstimatorSample.java       |  69 ++-
 .../runtime/compress/estim/CompressedSizeInfo.java |   5 +
 .../compress/estim/CompressedSizeInfoColGroup.java |  26 +-
 .../runtime/compress/estim/EstimationFactors.java  |  10 +-
 .../compress/estim/encoding/ConstEncoding.java     |  10 +-
 .../compress/estim/encoding/DenseEncoding.java     | 187 ++++---
 .../compress/estim/encoding/EmptyEncoding.java     |  17 +-
 .../runtime/compress/estim/encoding/IEncode.java   | 192 ++++---
 .../compress/estim/encoding/SparseEncoding.java    | 258 ++++++----
 .../sysds/runtime/compress/lib/CLALibAppend.java   | 108 ++--
 .../runtime/compress/lib/CLALibBinaryCellOp.java   | 323 ++++++++++--
 .../sysds/runtime/compress/lib/CLALibCMOps.java    |   2 +-
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |  37 +-
 .../runtime/compress/lib/CLALibDecompress.java     |  20 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |   9 +-
 .../runtime/compress/lib/CLALibRightMultBy.java    |   5 +-
 .../sysds/runtime/compress/lib/CLALibScalar.java   |   5 +-
 .../sysds/runtime/compress/lib/CLALibSlice.java    |   2 +-
 .../sysds/runtime/compress/lib/CLALibTSMM.java     |   4 +-
 .../sysds/runtime/compress/lib/CLALibUnary.java    |   6 +-
 .../compress/readers/ReaderColumnSelection.java    |  25 +-
 .../ReaderColumnSelectionDenseMultiBlock.java      |  18 +-
 ...erColumnSelectionDenseMultiBlockTransposed.java |  19 +-
 .../ReaderColumnSelectionDenseSingleBlock.java     |  26 +-
 ...rColumnSelectionDenseSingleBlockTransposed.java |  24 +-
 .../readers/ReaderColumnSelectionSparse.java       |  49 +-
 .../ReaderColumnSelectionSparseTransposed.java     | 137 +++--
 .../sysds/runtime/compress/utils/DCounts.java      |   6 +-
 .../compress/utils/DblArrayCountHashMap.java       |  22 +-
 .../runtime/compress/utils/DoubleCountHashMap.java | 161 ++++--
 .../apache/sysds/runtime/compress/utils/Util.java  |  11 +-
 .../apache/sysds/runtime/compress/workload/Op.java |   2 +
 .../sysds/runtime/compress/workload/OpSided.java   |   2 +
 .../compress/workload/WorkloadAnalyzer.java        |  11 +-
 .../sysds/runtime/matrix/data/LibMatrixReorg.java  |  48 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |  16 +-
 src/test/java/org/apache/sysds/test/TestUtils.java |  45 +-
 .../compress/AbstractCompressedUnaryTests.java     |   5 +-
 .../component/compress/CompressedMatrixTest.java   |  19 +-
 .../component/compress/CompressedTestBase.java     | 146 ++++--
 .../component/compress/CompressedVectorTest.java   |   2 +-
 .../compress/CompressibleInputGenerator.java       |  12 +-
 .../component/compress/ExtendedMatrixTests.java    |  15 +-
 .../compress/ParCompressedMatrixTest.java          |   5 +-
 .../test/component/compress/TestConstants.java     |   9 +-
 .../compress/colgroup/ColGroupDeltaDDCTest.java    | 132 +++--
 .../compress/colgroup/JolEstimateDeltaDDCTest.java |  11 +-
 .../compress/colgroup/JolEstimateSDCTest.java      |   8 +-
 .../compress/colgroup/JolEstimateTest.java         |   5 +-
 .../colgroup/JolEstimateUncompressedTest.java      |  11 +-
 .../test/component/compress/cost/ACostTest.java    |  19 +-
 .../estim/encoding/EncodeSampleCustom.java         |  29 +-
 .../encoding/EncodeSampleDenseNonUniform.java      |   4 +-
 .../estim/encoding/EncodeSampleMultiColTest.java   |   6 +-
 .../estim/encoding/EncodeSampleSingleColTest.java  |  47 +-
 .../compress/estim/encoding/EncodeSampleTest.java  |  37 +-
 .../estim/encoding/EncodeSampleUniformTest.java    |   4 +-
 .../compress/mapping/MappingPreAggregateTests.java |  14 +-
 .../compress/mapping/MappingTestUtil.java          |  40 +-
 .../component/compress/mapping/MappingTests.java   |  76 ++-
 .../compress/mapping/MappingTestsResize.java       |   2 +-
 .../compress/mapping/StandAloneTests.java          | 176 -------
 .../compress/offset/OffsetSingleTests.java         |  46 ++
 .../compress/offset/OffsetTestPreAggregate.java    |  17 +-
 .../compress/offset/OffsetTestPreAggregateBit.java | 128 -----
 .../offset/OffsetTestPreAggregateByte.java         | 132 -----
 .../offset/OffsetTestPreAggregateChar.java         | 150 ------
 .../offset/OffsetTestPreAggregateSparse.java       |  15 +-
 .../offset/OffsetTestPreAggregateSparseByte.java   |  57 ---
 .../offset/OffsetTestPreAggregateSparseChar.java   |  66 ---
 ...AggregateSparseBit.java => OffsetTestUtil.java} |  50 +-
 .../component/compress/offset/OffsetTests.java     |  81 ++-
 .../component/compress/readers/ReadersTest.java    |  18 +
 .../readers/ReadersTestCompareReaders.java         | 567 +++++++++++++++++++++
 .../compress/configuration/CompressForce.java      |  12 +-
 131 files changed, 4999 insertions(+), 3078 deletions(-)

diff --git a/src/main/java/org/apache/sysds/hops/ReorgOp.java b/src/main/java/org/apache/sysds/hops/ReorgOp.java
index 64fd364aee..e01fe4dead 100644
--- a/src/main/java/org/apache/sysds/hops/ReorgOp.java
+++ b/src/main/java/org/apache/sysds/hops/ReorgOp.java
@@ -225,6 +225,14 @@ public class ReorgOp extends MultiThreadedHop
 	}
 
 
+	@Override
+	public void computeMemEstimate(MemoTable memo){
+		if(_op == ReOrgOp.TRANS && getInput().get(0).isCompressedOutput() )
+			_outputMemEstimate = getInput().get(0).getCompressedSize();
+		else
+			super.computeMemEstimate(memo);
+	}
+
 	@Override
 	protected double computeOutputMemEstimate( long dim1, long dim2, long nnz ) {
 		//no dedicated mem estimation per op type, because always propagated via refreshSizeInformation
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 11c3ad64a8..9b689b8801 100644
--- a/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
+++ b/src/main/java/org/apache/sysds/hops/ipa/IPAPassCompressionWorkloadAnalysis.java
@@ -22,6 +22,8 @@ package org.apache.sysds.hops.ipa;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.conf.ConfigurationManager;
 import org.apache.sysds.conf.DMLConfig;
 import org.apache.sysds.hops.OptimizerUtils;
@@ -36,6 +38,7 @@ import org.apache.sysds.runtime.compress.workload.WorkloadAnalyzer;
  * workload-aware compression planning.
  */
 public class IPAPassCompressionWorkloadAnalysis extends IPAPass {
+	private static final Log LOG = LogFactory.getLog(IPAPassCompressionWorkloadAnalysis.class.getName());
 
 	@Override
 	public boolean isApplicable(FunctionCallGraph fgraph) {
@@ -57,13 +60,13 @@ public class IPAPassCompressionWorkloadAnalysis extends IPAPass {
 			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);
-
+			else if(LOG.isTraceEnabled())
+				LOG.trace("IPAPass Says no Compress:\n" + tree + "\n" + b);
+			else if(LOG.isDebugEnabled())
+				LOG.debug("IPApass Says no Compress:\n" + tree.getRoot() + "\n" + b);
 		}
 
 		return map != null;
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 648a3453d2..640df9c3bb 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -218,7 +218,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	 * @param k degree of parallelism
 	 * @return a new uncompressed matrix block containing the contents of this block
 	 */
-	public MatrixBlock decompress(int k) {
+	public synchronized MatrixBlock decompress(int k) {
 		// Early out if empty.
 		if(isEmpty())
 			return new MatrixBlock(rlen, clen, true, 0);
@@ -508,11 +508,16 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock replaceOperations(MatrixValue result, double pattern, double replacement) {
-		if(isOverlapping()) {
+		if(Double.isInfinite(pattern)) {
+			LOG.info("Ignoring replace infinite in compression since it does not contain this value");
+			return this;
+		}
+		else if(isOverlapping()) {
 			final String message = "replaceOperations " + pattern + " -> " + replacement;
 			return getUncompressed(message).replaceOperations(result, pattern, replacement);
 		}
 		else {
+
 			CompressedMatrixBlock ret = new CompressedMatrixBlock(getNumRows(), getNumColumns());
 			final List<AColGroup> prev = getColGroups();
 			final int colGroupsLength = prev.size();
@@ -726,14 +731,14 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	@Override
 	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that) {
 		MatrixBlock right = getUncompressed(that);
-		return getUncompressed("covOperations").covOperations(op, right);
+		return getUncompressed("covOperations", op.getNumThreads()).covOperations(op, right);
 	}
 
 	@Override
 	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that, MatrixBlock weights) {
 		MatrixBlock right1 = getUncompressed(that);
 		MatrixBlock right2 = getUncompressed(weights);
-		return getUncompressed("covOperations").covOperations(op, right1, right2);
+		return getUncompressed("covOperations", op.getNumThreads()).covOperations(op, right1, right2);
 	}
 
 	@Override
@@ -866,9 +871,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 
 		if(m2 instanceof CompressedMatrixBlock)
-			m2 = ((CompressedMatrixBlock) m2).getUncompressed("Ternary Operator arg2 " + op.fn.getClass().getSimpleName());
+			m2 = ((CompressedMatrixBlock) m2).getUncompressed("Ternary Operator arg2 " + op.fn.getClass().getSimpleName(),
+				op.getNumThreads());
 		if(m3 instanceof CompressedMatrixBlock)
-			m3 = ((CompressedMatrixBlock) m3).getUncompressed("Ternary Operator arg3 " + op.fn.getClass().getSimpleName());
+			m3 = ((CompressedMatrixBlock) m3).getUncompressed("Ternary Operator arg3 " + op.fn.getClass().getSimpleName(),
+				op.getNumThreads());
 
 		if(s2 != s3 && (op.fn instanceof PlusMultiply || op.fn instanceof MinusMultiply)) {
 			// SPECIAL CASE for sparse-dense combinations of common +* and -*
@@ -933,20 +940,26 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	public MatrixBlock getUncompressed() {
-		MatrixBlock d_compressed = getCachedDecompressed();
+		return getUncompressed((String) null);
+	}
+
+	public MatrixBlock getUncompressed(String operation) {
+		return getUncompressed(operation,
+			ConfigurationManager.isParallelMatrixOperations() ? InfrastructureAnalyzer.getLocalParallelism() : 1);
+	}
+
+	public MatrixBlock getUncompressed(String operation, int k) {
+		final MatrixBlock d_compressed = getCachedDecompressed();
 		if(d_compressed != null)
 			return d_compressed;
-		else if(isEmpty())
+		// Print warning if we do not have a cached decompressed version.
+		if(operation != null)
+			printDecompressWarning(operation);
+
+		if(isEmpty())
 			return new MatrixBlock(getNumRows(), getNumColumns(), true);
-		else if(ConfigurationManager.isParallelMatrixOperations())
-			return this.decompress(InfrastructureAnalyzer.getLocalParallelism());
-		else
-			return this.decompress(1);
-	}
 
-	public MatrixBlock getUncompressed(String operation) {
-		printDecompressWarning(operation);
-		return getUncompressed();
+		return this.decompress(k);
 	}
 
 	private static void printDecompressWarning(String operation) {
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 df9dacac86..f31d12f20d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -41,9 +41,7 @@ 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.lib.CLALibUtils;
-import org.apache.sysds.runtime.compress.utils.DblArrayIntListHashMap;
-import org.apache.sysds.runtime.compress.utils.DoubleCountHashMap;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.workload.WTreeRoot;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
@@ -108,6 +106,10 @@ public class CompressedMatrixBlockFactory {
 		return compress(mb, 1, new CompressionSettingsBuilder(), root);
 	}
 
+	public static Pair<MatrixBlock, CompressionStatistics> compress(MatrixBlock mb, CostEstimatorBuilder csb) {
+		return compress(mb, 1, new CompressionSettingsBuilder(), csb);
+	}
+
 	public static Pair<MatrixBlock, CompressionStatistics> compress(MatrixBlock mb,
 		CompressionSettingsBuilder customSettings) {
 		return compress(mb, 1, customSettings, (WTreeRoot) null);
@@ -121,6 +123,10 @@ public class CompressedMatrixBlockFactory {
 		return compress(mb, k, new CompressionSettingsBuilder(), root);
 	}
 
+	public static Pair<MatrixBlock, CompressionStatistics> compress(MatrixBlock mb, int k, CostEstimatorBuilder csb) {
+		return compress(mb, k, new CompressionSettingsBuilder(), csb);
+	}
+
 	public static Pair<MatrixBlock, CompressionStatistics> compress(MatrixBlock mb, ACostEstimate costEstimator) {
 		return compress(mb, 1, new CompressionSettingsBuilder(), costEstimator);
 	}
@@ -236,6 +242,10 @@ public class CompressedMatrixBlockFactory {
 		if(compressionGroups == null)
 			return abortCompression();
 
+		// clear extra data from analysis
+		compressionGroups.clearMaps();
+		informationExtractor.clearNNZ();
+
 		transposePhase();
 		compressPhase();
 		finalizePhase();
@@ -252,6 +262,13 @@ public class CompressedMatrixBlockFactory {
 		// Compute the individual columns cost information
 		compressionGroups = informationExtractor.computeCompressedSizeInfos(k);
 
+		if(LOG.isTraceEnabled()) {
+			LOG.trace("Logging all individual columns estimated cost:");
+			for(CompressedSizeInfoColGroup g : compressionGroups.getInfo())
+				LOG.trace(String.format("Cost: %8.0f Size: %16d %15s", costEstimator.getCost(g), g.getMinSize(),
+					Arrays.toString(g.getColumns())));
+		}
+
 		_stats.estimatedSizeCols = compressionGroups.memoryEstimate();
 		_stats.estimatedCostCols = costEstimator.getCost(compressionGroups);
 
@@ -309,7 +326,8 @@ public class CompressedMatrixBlockFactory {
 	}
 
 	private void transposePhase() {
-		if(!compSettings.transposed) {
+		final boolean haveMemory = Runtime.getRuntime().freeMemory() - (mb.estimateSizeInMemory() * 2) > 0;
+		if(!compSettings.transposed && haveMemory) {
 			transposeHeuristics();
 			if(compSettings.transposed) {
 				boolean sparse = mb.isInSparseFormat();
@@ -333,8 +351,8 @@ public class CompressedMatrixBlockFactory {
 				if(mb.isInSparseFormat()) {
 					boolean haveManyColumns = mb.getNumColumns() > 10000;
 					boolean isNnzLowAndVerySparse = mb.getNonZeros() < 1000 && mb.getSparsity() < 0.4;
-					boolean isAboveRowNumbers = mb.getNumRows() > 500000;
-					boolean isAboveThreadToColumnRatio = compressionGroups.getNumberColGroups() > mb.getNumColumns() / 4;
+					boolean isAboveRowNumbers = mb.getNumRows() > 500000 && mb.getSparsity() < 0.4;
+					boolean isAboveThreadToColumnRatio = compressionGroups.getNumberColGroups() > mb.getNumColumns() / 30;
 					compSettings.transposed = haveManyColumns || isNnzLowAndVerySparse ||
 						(isAboveRowNumbers && isAboveThreadToColumnRatio);
 				}
@@ -351,8 +369,6 @@ public class CompressedMatrixBlockFactory {
 	}
 
 	private void finalizePhase() {
-
-		CLALibUtils.combineConstColumns(res);
 		res.cleanupBlock(true, true);
 
 		_stats.compressedSize = res.getInMemorySize();
@@ -360,6 +376,7 @@ public class CompressedMatrixBlockFactory {
 
 		final double ratio = _stats.getRatio();
 		final double denseRatio = _stats.getDenseRatio();
+
 		if(ratio < 1 && denseRatio < 100.0) {
 			LOG.info("--dense size:        " + _stats.denseSize);
 			LOG.info("--original size:     " + _stats.originalSize);
@@ -374,14 +391,16 @@ public class CompressedMatrixBlockFactory {
 
 		_stats.setColGroupsCounts(res.getColGroups());
 
+		if(compSettings.isInSparkInstruction)
+			res.clearSoftReferenceToDecompressed();
+
 		final long oldNNZ = mb.getNonZeros();
-		if(oldNNZ <= 0)
-			res.setNonZeros(oldNNZ);
-		else
+		if(oldNNZ <= 0L)
 			res.recomputeNonZeros();
+		else
+			res.setNonZeros(oldNNZ);
 
 		logPhase();
-
 	}
 
 	private Pair<MatrixBlock, CompressionStatistics> abortCompression() {
@@ -428,10 +447,6 @@ public class CompressedMatrixBlockFactory {
 						break;
 					case 3:
 						LOG.debug("--compression phase " + phase + " Compress  : " + getLastTimePhase());
-						LOG.debug("--compression Hash collisions:" + "(" + DblArrayIntListHashMap.hashMissCount + ","
-							+ DoubleCountHashMap.hashMissCount + ")");
-						DblArrayIntListHashMap.hashMissCount = 0;
-						DoubleCountHashMap.hashMissCount = 0;
 						LOG.debug("--compressed initial actual size:" + _stats.compressedInitialSize);
 						break;
 					case 4:
@@ -439,17 +454,18 @@ public class CompressedMatrixBlockFactory {
 						LOG.debug("--compression phase  " + phase + " Cleanup   : " + getLastTimePhase());
 						LOG.debug("--col groups types   " + _stats.getGroupsTypesString());
 						LOG.debug("--col groups sizes   " + _stats.getGroupsSizesString());
-						LOG.debug(String.format("--dense size:        %16d" , _stats.denseSize));
-						LOG.debug(String.format("--original size:     %16d" , _stats.originalSize));
-						LOG.debug(String.format("--compressed size:   %16d" , _stats.compressedSize));
+						LOG.debug(String.format("--dense size:        %16d", _stats.denseSize));
+						LOG.debug(String.format("--original size:     %16d", _stats.originalSize));
+						LOG.debug(String.format("--compressed size:   %16d", _stats.compressedSize));
 						LOG.debug(String.format("--compression ratio: %4.3f", _stats.getRatio()));
 						LOG.debug(String.format("--Dense       ratio: %4.3f", _stats.getDenseRatio()));
 						if(!(costEstimator instanceof MemoryCostEstimator)) {
-							LOG.debug(String.format("--original cost:     %5.2E" , _stats.originalCost));
-							LOG.debug(String.format("--single col cost:   %5.2E" , _stats.estimatedCostCols));
-							LOG.debug(String.format("--cocode cost:       %5.2E" , _stats.estimatedCostCoCoded));
-							LOG.debug(String.format("--actual cost:       %5.2E" , _stats.compressedCost));
-							LOG.debug(String.format("--relative cost:     %1.4f" , (_stats.compressedCost / _stats.originalCost)));
+							LOG.debug(String.format("--original cost:     %5.2E", _stats.originalCost));
+							LOG.debug(String.format("--single col cost:   %5.2E", _stats.estimatedCostCols));
+							LOG.debug(String.format("--cocode cost:       %5.2E", _stats.estimatedCostCoCoded));
+							LOG.debug(String.format("--actual cost:       %5.2E", _stats.compressedCost));
+							LOG.debug(
+								String.format("--relative cost:     %1.4f", (_stats.compressedCost / _stats.originalCost)));
 						}
 						if(compressionGroups.getInfo().size() < 1000) {
 							int[] lengths = new int[res.getColGroups().size()];
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 ce2723e9bd..ec5512266e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -206,7 +206,7 @@ public class CompressionSettingsBuilder {
 	 * @return The CompressionSettingsBuilder
 	 */
 	public CompressionSettingsBuilder clearValidCompression() {
-		this.validCompressions = EnumSet.of(CompressionType.UNCOMPRESSED);
+		this.validCompressions = EnumSet.of(CompressionType.UNCOMPRESSED, CompressionType.EMPTY, CompressionType.CONST);
 		return this;
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/bitmap/BitmapEncoder.java b/src/main/java/org/apache/sysds/runtime/compress/bitmap/BitmapEncoder.java
index 9353744628..c81c5948ec 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/bitmap/BitmapEncoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/bitmap/BitmapEncoder.java
@@ -19,6 +19,7 @@
 
 package org.apache.sysds.runtime.compress.bitmap;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -72,7 +73,7 @@ public class BitmapEncoder {
 
 	private static ABitmap extractBitmapSingleColumn(int colIndex, MatrixBlock rawBlock, int numRows, boolean transposed,
 		int est, boolean sort) {
-		if(transposed){
+		if(transposed) {
 			if(rawBlock.isInSparseFormat() && rawBlock.getSparseBlock().isEmpty(colIndex))
 				return null;
 			return makeSingleColBitmap(extractSingleColT(colIndex, rawBlock, est), rawBlock.getNumColumns(), sort);
@@ -84,10 +85,23 @@ public class BitmapEncoder {
 	private static DoubleIntListHashMap extractSingleCol(int colIndex, MatrixBlock rawBlock, int estimatedUnique) {
 		final DoubleIntListHashMap distinctVals = new DoubleIntListHashMap(estimatedUnique);
 		final int nRows = rawBlock.getNumRows();
-		final boolean notSparse = !rawBlock.isInSparseFormat();
 		final int nCols = rawBlock.getNumColumns();
-
-		if(notSparse && rawBlock.getDenseBlock().isContiguous()) {
+		final boolean sparse = rawBlock.isInSparseFormat();
+
+		if(sparse) {
+			final SparseBlock sb = rawBlock.getSparseBlock();
+			for(int r = 0; r < nRows; r++) {
+				if(sb.isEmpty(r))
+					continue;
+				final int apos = sb.pos(r);
+				final int alen = sb.size(r) + apos;
+				final int[] aix = sb.indexes(r);
+				final int idx = Arrays.binarySearch(aix, apos, alen, colIndex);
+				if(idx >= 0)
+					distinctVals.appendValue(sb.values(r)[idx], r);
+			}
+		}
+		else if(rawBlock.getDenseBlock().isContiguous()) {
 			final double[] values = rawBlock.getDenseBlockValues();
 			if(nCols == 1)
 				// Since the only values contained is in this column index. simply extract it continuously.
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 3046caa4fd..ad7100f72c 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
@@ -20,17 +20,18 @@
 package org.apache.sysds.runtime.compress.cocode;
 
 import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
 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;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class CoCodeGreedy extends AColumnCoCoder {
@@ -64,23 +65,29 @@ public class CoCodeGreedy extends AColumnCoCoder {
 
 		final List<ColIndexes> workSet = new ArrayList<>(inputColumns.size());
 
-		// assume that we can at max reduce 90 % of cost if combined
-		final double costFilterThreshold = 0.9;
-
 		final ExecutorService pool = CommonThreadPool.get(k);
 		for(int i = 0; i < inputColumns.size(); i++) {
 			CompressedSizeInfoColGroup g = inputColumns.get(i);
 			workSet.add(new ColIndexes(g.getColumns()));
 		}
 
-		// if(k > 1)
-		// parallelFirstCombine(workSet, mem, cEst, pool);
+		if(k > 1)
+			parallelFirstCombine(workSet, pool);
+
+		// second layer to keep the second best combination
+		double secondChange = 0;
+		CompressedSizeInfoColGroup secondTmp = null;
+		ColIndexes secondSelectedJ = null, secondSelected1 = null, secondSelected2 = null;
 
 		// Process merging iterations until no more change
 		while(workSet.size() > 1) {
-			double changeInCost = 0;
-			CompressedSizeInfoColGroup tmp = null;
-			ColIndexes selected1 = null, selected2 = null;
+			if(secondChange != 0)
+				mem.incst4();
+			// maintain selected
+			double changeInCost = secondChange;
+			CompressedSizeInfoColGroup tmp = secondTmp;
+			ColIndexes selectedJ = secondSelectedJ, selected1 = secondSelected1, selected2 = secondSelected2;
+
 			for(int i = 0; i < workSet.size(); i++) {
 				for(int j = i + 1; j < workSet.size(); j++) {
 					final ColIndexes c1 = workSet.get(i);
@@ -94,54 +101,65 @@ public class CoCodeGreedy extends AColumnCoCoder {
 					// Since even if the entire size of one of the column lists is removed,
 					// it still does not improve compression.
 					// In the case of workload we relax the requirement for the filter.
-					// if(-Math.min(costC1, costC2) > changeInCost)
-					if(-Math.min(costC1, costC2) * costFilterThreshold > changeInCost)
+					if(-Math.min(costC1, costC2) > changeInCost)
 						continue;
 
-					// Join the two column groups.
-					// and Memorize the new join.
-					final CompressedSizeInfoColGroup c1c2Inf = mem.getOrCreate(c1, c2);
+					// Combine the two column groups.
+					// and Memorize the new Combine.
+					final int[] c = Util.combine(c1._indexes, c2._indexes);
+					final ColIndexes cI = new ColIndexes(c);
+					final CompressedSizeInfoColGroup c1c2Inf = mem.getOrCreate(cI, c1, c2);
 					final double costC1C2 = _cest.getCost(c1c2Inf);
 					final double newCostIfJoined = costC1C2 - costC1 - costC2;
 
-					// Select the best join of either the currently selected
+					// Select the best Combine of either the currently selected
 					// or keep the old one.
-					if((tmp == null && newCostIfJoined < changeInCost) || tmp != null && (newCostIfJoined < changeInCost ||
-						newCostIfJoined == changeInCost && c1c2Inf.getColumns().length < tmp.getColumns().length)) {
-						changeInCost = newCostIfJoined;
-						tmp = c1c2Inf;
-						selected1 = c1;
-						selected2 = c2;
+					if(newCostIfJoined < 0) {
+						if(tmp == null) {
+							changeInCost = newCostIfJoined;
+							tmp = c1c2Inf;
+							selectedJ = cI;
+							selected1 = c1;
+							selected2 = c2;
+						}
+						else if((newCostIfJoined < changeInCost ||
+							newCostIfJoined == changeInCost && c1c2Inf.getColumns().length < tmp.getColumns().length)) {
+
+							if(selected1 != secondSelected1 && selected2 != secondSelected2) {
+								secondTmp = tmp;
+								secondSelectedJ = selectedJ;
+								secondSelected1 = selected1;
+								secondSelected2 = selected2;
+								secondChange = changeInCost;
+							}
+
+							changeInCost = newCostIfJoined;
+							tmp = c1c2Inf;
+							selectedJ = cI;
+							selected1 = c1;
+							selected2 = c2;
+						}
 					}
 				}
 			}
 
 			if(tmp != null) {
+				// remove from workset
 				workSet.remove(selected1);
 				workSet.remove(selected2);
-				mem.remove(selected1, selected2);
-
-				Collections.sort(workSet, new CompareColumns());
-
-				final ColIndexes a = new ColIndexes(tmp.getColumns());
-				// if(k > 1) {
-				// final List<CombineTask> tasks = new ArrayList<>();
-				// final int size = workSet.size();
-				// try {
-				// // combine the first k columns...
-				// // just to parallelize at least the first couple of options.
-				// // This potentially filters out some of the options quickly.
-				// for(int j = 0; j < Math.min(k, size); j++)
-				// tasks.add(new CombineTask(a, workSet.get(j), mem));
-
-				// for(Future<Object> t : pool.invokeAll(tasks))
-				// t.get();
-				// }
-				// catch(Exception e) {
-				// throw new DMLCompressionException("Failed parallelize first level all join all", e);
-				// }
-				// }
-				workSet.add(a);
+				mem.remove(selected1, selected2); // remove all memorized values of the combined columns
+
+				// ColIndexes combined = new ColIndexes(tmp.getColumns());
+				mem.put(selectedJ, tmp); // add back the new combination to memorizer
+				workSet.add(selectedJ);
+				if(selectedJ.contains(secondSelected1, secondSelected2)) {
+					secondTmp = null;
+					secondSelectedJ = null;
+					secondSelected1 = null;
+					secondSelected2 = null;
+					secondChange = 0;
+				}
+
 			}
 			else
 				break;
@@ -159,54 +177,36 @@ public class CoCodeGreedy extends AColumnCoCoder {
 		return ret;
 	}
 
-	// protected static void parallelFirstCombine(List<ColIndexes> workSet, Memorizer mem, ACostEstimate cEst,
-	// ExecutorService pool) {
-	// try {
-	// final List<CombineTask> tasks = new ArrayList<>();
-	// final int size = workSet.size();
-	// for(int i = 0; i < size; i++)
-	// for(int j = i + 1; j < size; j++)
-	// tasks.add(new CombineTask(workSet.get(i), workSet.get(j), mem));
-
-	// for(Future<Object> t : pool.invokeAll(tasks))
-	// t.get();
-	// }
-	// catch(Exception e) {
-	// throw new DMLCompressionException("Failed parallelize first level all join all", e);
-	// }
-	// }
-
-	protected static class CombineTask implements Callable<Object> {
+	protected void parallelFirstCombine(List<ColIndexes> workSet, ExecutorService pool) {
+		try {
+			final List<CombineTask> tasks = new ArrayList<>();
+			final int size = workSet.size();
+			for(int i = 0; i < size; i++)
+				for(int j = i + 1; j < size; j++)
+					tasks.add(new CombineTask(workSet.get(i), workSet.get(j)));
+
+			for(Future<Object> t : pool.invokeAll(tasks))
+				t.get();
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException("Failed parallelize first level all join all", e);
+		}
+	}
+
+	protected class CombineTask implements Callable<Object> {
 		private final ColIndexes _c1, _c2;
-		private final Memorizer _m;
 
-		protected CombineTask(ColIndexes c1, ColIndexes c2, Memorizer m) {
+		protected CombineTask(ColIndexes c1, ColIndexes c2) {
 			_c1 = c1;
 			_c2 = c2;
-			_m = m;
 		}
 
 		@Override
 		public Object call() {
-			_m.getOrCreate(_c1, _c2);
+			final int[] c = Util.combine(_c1._indexes, _c2._indexes);
+			final ColIndexes cI = new ColIndexes(c);
+			mem.getOrCreate(cI, _c1, _c2);
 			return null;
-
 		}
 	}
-
-	private class CompareColumns implements Comparator<ColIndexes> {
-
-		@Override
-		public int compare(ColIndexes arg0, ColIndexes arg1) {
-			final double c1 = _cest.getCost(mem.get(arg0));
-			final double c2 = _cest.getCost(mem.get(arg1));
-			if(c1 > c2)
-				return -1;
-			else if(c1 == c2)
-				return 0;
-			else
-				return 1;
-		}
-
-	}
 }
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 a42017b163..aec327e5a7 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
@@ -46,7 +46,7 @@ import org.apache.sysds.runtime.util.CommonThreadPool;
  */
 public class CoCodePriorityQue extends AColumnCoCoder {
 
-	private static final int COL_COMBINE_THREASHOLD = 1024;
+	private static final int COL_COMBINE_THRESHOLD = 1024;
 
 	protected CoCodePriorityQue(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator,
 		CompressionSettings cs) {
@@ -62,13 +62,13 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 	protected static List<CompressedSizeInfoColGroup> join(List<CompressedSizeInfoColGroup> groups,
 		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups, int k) {
 
-		if(groups.size() > COL_COMBINE_THREASHOLD && k > 1)
-			return joinMultiThreaded(groups, sEst, cEst, minNumGroups, k);
+		if(groups.size() > COL_COMBINE_THRESHOLD && k > 1)
+			return combineMultiThreaded(groups, sEst, cEst, minNumGroups, k);
 		else
-			return joinSingleThreaded(groups, sEst, cEst, minNumGroups);
+			return combineSingleThreaded(groups, sEst, cEst, minNumGroups);
 	}
 
-	private static List<CompressedSizeInfoColGroup> joinMultiThreaded(List<CompressedSizeInfoColGroup> groups,
+	private static List<CompressedSizeInfoColGroup> combineMultiThreaded(List<CompressedSizeInfoColGroup> groups,
 		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups, int k) {
 		try {
 			final ExecutorService pool = CommonThreadPool.get(k);
@@ -93,12 +93,12 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 		}
 	}
 
-	private static List<CompressedSizeInfoColGroup> joinSingleThreaded(List<CompressedSizeInfoColGroup> groups,
+	private static List<CompressedSizeInfoColGroup> combineSingleThreaded(List<CompressedSizeInfoColGroup> groups,
 		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
-		return joinBlock(groups, 0, groups.size(), sEst, cEst, minNumGroups);
+		return combineBlock(groups, 0, groups.size(), sEst, cEst, minNumGroups);
 	}
 
-	private static List<CompressedSizeInfoColGroup> joinBlock(List<CompressedSizeInfoColGroup> groups, int start,
+	private static List<CompressedSizeInfoColGroup> combineBlock(List<CompressedSizeInfoColGroup> groups, int start,
 		int end, CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
 		Queue<CompressedSizeInfoColGroup> que = getQue(end - start, cEst);
 
@@ -108,10 +108,10 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 				que.add(g);
 		}
 
-		return joinBlock(que, sEst, cEst, minNumGroups);
+		return combineBlock(que, sEst, cEst, minNumGroups);
 	}
 
-	private static List<CompressedSizeInfoColGroup> joinBlock(Queue<CompressedSizeInfoColGroup> que,
+	private static List<CompressedSizeInfoColGroup> combineBlock(Queue<CompressedSizeInfoColGroup> que,
 		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
 
 		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
@@ -128,9 +128,6 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 				if(costOfJoin < costIndividual) {
 					que.poll();
 					int numColumns = g.getColumns().length;
-					// if(minNumGroups != 0 && numColumns > 8)
-						// ret.add(g); // Add this column group to ret, since it already is very CoCoded.
-					// else 
 					if(numColumns > 128)
 						ret.add(g);
 					else
@@ -183,7 +180,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 		@Override
 		public List<CompressedSizeInfoColGroup> call() {
 			try {
-				return joinBlock(_groups, _start, _end, _sEst, _cEst, _minNumGroups);
+				return combineBlock(_groups, _start, _end, _sEst, _cEst, _minNumGroups);
 			}
 			catch(Exception e) {
 				throw new DMLCompressionException("Falied PQTask ", e);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColIndexes.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColIndexes.java
index 0800a86e08..02c69ddd4e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColIndexes.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColIndexes.java
@@ -40,4 +40,14 @@ public class ColIndexes {
 		ColIndexes thatGrp = (ColIndexes) that;
 		return Arrays.equals(_indexes, thatGrp._indexes);
 	}
+
+	public boolean contains(ColIndexes a, ColIndexes b) {
+		if(a == null || b == null)
+			return false;
+		int id = Arrays.binarySearch(_indexes, a._indexes[0]);
+		if(id >= 0)
+			return true;
+		id = Arrays.binarySearch(_indexes, b._indexes[0]);
+		return id >= 0;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/Memorizer.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/Memorizer.java
index 6524ca491f..854de11716 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/Memorizer.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/Memorizer.java
@@ -20,16 +20,17 @@
 package org.apache.sysds.runtime.compress.cocode;
 
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
-import org.apache.sysds.runtime.compress.utils.Util;
 
 public class Memorizer {
 	private final CompressedSizeEstimator _sEst;
 	private final Map<ColIndexes, CompressedSizeInfoColGroup> mem;
-	private int st1 = 0, st2 = 0, st3 = 0;
+	private int st1 = 0, st2 = 0, st3 = 0, st4 = 0;
 
 	public Memorizer(CompressedSizeEstimator sEst) {
 		_sEst = sEst;
@@ -40,6 +41,10 @@ public class Memorizer {
 		mem.put(new ColIndexes(g.getColumns()), g);
 	}
 
+	public void put(ColIndexes key, CompressedSizeInfoColGroup val) {
+		mem.put(key, val);
+	}
+
 	public CompressedSizeInfoColGroup get(ColIndexes c) {
 		return mem.get(c);
 	}
@@ -47,20 +52,23 @@ public class Memorizer {
 	public void remove(ColIndexes c1, ColIndexes c2) {
 		mem.remove(c1);
 		mem.remove(c2);
+		Iterator<Entry<ColIndexes, CompressedSizeInfoColGroup>> i = mem.entrySet().iterator();
+		while(i.hasNext()) {
+			final ColIndexes eci = i.next().getKey();
+			if(eci.contains(c1, c2))
+				i.remove();
+		}
 	}
 
-	public CompressedSizeInfoColGroup getOrCreate(ColIndexes c1, ColIndexes c2) {
-		final int[] c = Util.combine(c1._indexes, c2._indexes);
-		final ColIndexes cI = new ColIndexes(c);
+	public CompressedSizeInfoColGroup getOrCreate(ColIndexes cI, ColIndexes c1, ColIndexes c2){
 		CompressedSizeInfoColGroup g = mem.get(cI);
 		st2++;
 		if(g == null) {
 			final CompressedSizeInfoColGroup left = mem.get(c1);
 			final CompressedSizeInfoColGroup right = mem.get(c2);
 			if(left != null && right != null) {
-
 				st3++;
-				g = _sEst.combine(c, left, right);
+				g = _sEst.combine(cI._indexes, left, right);
 
 				synchronized(this) {
 					mem.put(cI, g);
@@ -75,14 +83,19 @@ public class Memorizer {
 		st1++;
 	}
 
+	public void incst4() {
+		st4++;
+	}
+
 	public String stats() {
-		return " possible: " + st1 + " requests: " + st2 + " joined: " + st3;
+		return " possible: " + st1 + " requests: " + st2 + " combined: " + st3  + " outSecond: "+ st4;
 	}
 
 	public void resetStats() {
 		st1 = 0;
 		st2 = 0;
 		st3 = 0;
+		st4 = 0;
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
index e10c03120a..45a0d62df7 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
@@ -51,8 +51,8 @@ public abstract class AColGroup implements Serializable {
 	private static final long serialVersionUID = -1318908671481L;
 
 	/** Public super types of compression ColGroups supported */
-	public enum CompressionType {
-		UNCOMPRESSED, RLE, OLE, DDC, CONST, EMPTY, SDC, PFOR, DeltaDDC
+	public static enum CompressionType {
+		UNCOMPRESSED, RLE, OLE, DDC, CONST, EMPTY, SDC, SDCFOR, DDCFOR, DeltaDDC
 	}
 
 	/**
@@ -60,8 +60,8 @@ public abstract class AColGroup implements Serializable {
 	 * 
 	 * Protected such that outside the ColGroup package it should be unknown which specific subtype is used.
 	 */
-	protected enum ColGroupType {
-		UNCOMPRESSED, RLE, OLE, DDC, CONST, EMPTY, SDC, SDCSingle, SDCSingleZeros, SDCZeros, PFOR, DeltaDDC;
+	protected static enum ColGroupType {
+		UNCOMPRESSED, RLE, OLE, DDC, CONST, EMPTY, SDC, SDCSingle, SDCSingleZeros, SDCZeros, SDCFOR, DDCFOR, DeltaDDC;
 	}
 
 	/** The ColGroup Indexes contained in the ColGroup */
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupValue.java
index e3bf1c871f..0f6d433843 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupValue.java
@@ -33,6 +33,7 @@ import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.DictionaryFactory;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
+import org.apache.sysds.runtime.compress.utils.Util;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -82,8 +83,9 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		super(colIndices);
 		_numRows = numRows;
 		_dict = dict;
+		if(dict == null)
+			throw new NullPointerException("null dict is invalid");
 		if(cachedCounts != null)
-
 			counts = new SoftReference<>(cachedCounts);
 	}
 
@@ -403,21 +405,37 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 
 	@Override
 	protected AColGroup sliceSingleColumn(int idx) {
-		final AColGroupValue ret = (AColGroupValue) this.clone();
-		ret._colIndexes = new int[] {0};
-		if(_colIndexes.length == 1)
+		final int[] retIndexes = new int[] {0};
+		if(_colIndexes.length == 1) {
+			final AColGroupValue ret = (AColGroupValue) this.clone();
+			ret._colIndexes = retIndexes;
 			ret._dict = ret._dict.clone();
-		else
-			ret._dict = ret._dict.sliceOutColumnRange(idx, idx + 1, _colIndexes.length);
-
-		return ret;
+			ret._dict.getNumberOfValues(1);
+			return ret;
+		}
+		else {
+			final ADictionary retDict = _dict.sliceOutColumnRange(idx, idx + 1, _colIndexes.length);
+			if(retDict == null)
+				return new ColGroupEmpty(retIndexes);
+			else {
+				final AColGroupValue ret = (AColGroupValue) this.clone();
+				ret._colIndexes = retIndexes;
+				ret._dict = retDict;
+				ret._dict.getNumberOfValues(1);
+				return ret;
+			}
+		}
 	}
 
 	@Override
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
+		ADictionary retDict = _dict.sliceOutColumnRange(idStart, idEnd, _colIndexes.length);
+		if(retDict == null)
+			return new ColGroupEmpty(_colIndexes);
 		final AColGroupValue ret = (AColGroupValue) this.clone();
-		ret._dict = ret._dict.sliceOutColumnRange(idStart, idEnd, _colIndexes.length);
+		ret._dict = retDict;
 		ret._colIndexes = outputCols;
+		ret._dict.getNumberOfValues(outputCols.length);
 		return ret;
 	}
 
@@ -502,7 +520,7 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		if(d == null)
 			return ColGroupEmpty.create(max);
 		else
-			return copyAndSet(d);
+			return copyAndSet(Util.genColsIndices(max), d);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/APreAgg.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/APreAgg.java
index f90460b782..971f6470a1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/APreAgg.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/APreAgg.java
@@ -37,6 +37,8 @@ public abstract class APreAgg extends AColGroupValue {
 
 	private static final long serialVersionUID = 3250955207277128281L;
 
+	private static boolean loggedWarningForDirect = false;
+
 	/**
 	 * Constructor for serialization
 	 * 
@@ -79,7 +81,6 @@ public abstract class APreAgg extends AColGroupValue {
 		else if(lhs instanceof APreAgg)
 			leftMultByColGroupValue((APreAgg) lhs, result);
 		else if(lhs instanceof ColGroupUncompressed)
-			// throw new NotImplementedException();
 			leftMultByUncompressedColGroup((ColGroupUncompressed) lhs, result);
 		else
 			throw new DMLCompressionException(
@@ -110,7 +111,7 @@ public abstract class APreAgg extends AColGroupValue {
 	 * @param that the other column group whose indexes are used for aggregation.
 	 * @return A aggregate dictionary
 	 */
-	public final Dictionary preAggregateThatIndexStructure(APreAgg that) {
+	public final ADictionary preAggregateThatIndexStructure(APreAgg that) {
 		int outputLength = that._colIndexes.length * this.getNumValues();
 		Dictionary ret = new Dictionary(new double[outputLength]);
 
@@ -126,7 +127,7 @@ public abstract class APreAgg extends AColGroupValue {
 			throw new NotImplementedException(
 				"Not supported pre aggregate using index structure of :" + cThat + " in " + cThis);
 		}
-		return ret;
+		return ret.getMBDict(that._colIndexes.length);
 	}
 
 	/**
@@ -176,14 +177,54 @@ public abstract class APreAgg extends AColGroupValue {
 
 		if(sameIndexStructure(lg))
 			DictLibMatrixMult.TSMMToUpperTriangleScaling(lg._dict, _dict, leftIdx, rightIdx, getCounts(), result);
-		else if(shouldPreAggregateLeft(lg)) {
-			final ADictionary lpa = this.preAggregateThatIndexStructure(lg);
-			DictLibMatrixMult.TSMMToUpperTriangle(lpa, _dict, leftIdx, rightIdx, result);
+		else {
+			final boolean left = shouldPreAggregateLeft(lg);
+			if(!loggedWarningForDirect && shouldDirectMultiply(lg, leftIdx.length, rightIdx.length, left)) {
+				loggedWarningForDirect = true;
+				LOG.warn("Not implemented direct tsmm colgroup");
+			}
+
+			if(left) {
+				final ADictionary lpa = this.preAggregateThatIndexStructure(lg);
+				if(lpa != null)
+					DictLibMatrixMult.TSMMToUpperTriangle(lpa, _dict, leftIdx, rightIdx, result);
+			}
+			else {
+				final ADictionary rpa = lg.preAggregateThatIndexStructure(this);
+				if(rpa != null)
+					DictLibMatrixMult.TSMMToUpperTriangle(lg._dict, rpa, leftIdx, rightIdx, result);
+			}
+		}
+	}
+
+	private boolean shouldDirectMultiply(APreAgg lg, int nColL, int nColR, boolean leftPreAgg) {
+		int lMRows = lg.numRowsToMultiply();
+		int rMRows = this.numRowsToMultiply();
+		long commonDim = (long) Math.min(lMRows, rMRows);
+		long directFLOPS = commonDim * nColL * nColR * 2; // times 2 for first add then multiply
+
+		long preAggFLOPS = 0;
+
+		if(leftPreAgg) {
+			final int nVal = this.getNumValues();
+			// allocation
+			preAggFLOPS += nColL * nVal;
+			// preAgg
+			preAggFLOPS += nColL * commonDim; // worst case but okay
+			// multiply
+			preAggFLOPS += nColR * nColL * nVal;
 		}
 		else {
-			final ADictionary rpa = lg.preAggregateThatIndexStructure(this);
-			DictLibMatrixMult.TSMMToUpperTriangle(lg._dict, rpa, leftIdx, rightIdx, result);
+			final int nVal = lg.getNumValues();
+			// allocation
+			preAggFLOPS += nColR * nVal;
+			// preAgg
+			preAggFLOPS += nColR * commonDim; // worst case but okay
+			// multiply
+			preAggFLOPS += nColR * nColL * nVal;
 		}
+
+		return directFLOPS < preAggFLOPS;
 	}
 
 	private void leftMultByColGroupValue(APreAgg lhs, MatrixBlock result) {
@@ -196,17 +237,23 @@ public abstract class APreAgg extends AColGroupValue {
 			DictLibMatrixMult.TSMMDictionaryWithScaling(rDict, getCounts(), leftIdx, rightIdx, result);
 		else if(sameIdx)
 			DictLibMatrixMult.MMDictsWithScaling(lDict, rDict, leftIdx, rightIdx, result, getCounts());
-		else if(shouldPreAggregateLeft(lhs)) // left preAgg
-			DictLibMatrixMult.MMDicts(lDict, lhs.preAggregateThatIndexStructure(this), leftIdx, rightIdx, result);
-		else // right preAgg
-			DictLibMatrixMult.MMDicts(this.preAggregateThatIndexStructure(lhs), rDict, leftIdx, rightIdx, result);
+		else if(shouldPreAggregateLeft(lhs)) {// left preAgg
+			final ADictionary lhsPA = lhs.preAggregateThatIndexStructure(this);
+			if(lhsPA != null)
+				DictLibMatrixMult.MMDicts(lDict, lhsPA, leftIdx, rightIdx, result);
+		}
+		else {// right preAgg
+			final ADictionary rhsPA = preAggregateThatIndexStructure(lhs);
+			if(rhsPA != null)
+				DictLibMatrixMult.MMDicts(rhsPA, rDict, leftIdx, rightIdx, result);
+		}
 
 	}
 
 	private void leftMultByUncompressedColGroup(ColGroupUncompressed lhs, MatrixBlock result) {
 		if(lhs.getData().isEmpty())
 			return;
-		LOG.warn("Transpose of uncompressed to fit to template need t(a) %*% b support");
+		LOG.warn("Transpose of uncompressed to fit to template need t(a) %*% b");
 		final MatrixBlock tmp = LibMatrixReorg.transpose(lhs.getData(), InfrastructureAnalyzer.getLocalParallelism());
 		final int numVals = getNumValues();
 		final MatrixBlock preAgg = new MatrixBlock(tmp.getNumRows(), numVals, false);
@@ -328,4 +375,6 @@ public abstract class APreAgg extends AColGroupValue {
 			}
 		}
 	}
+
+	protected abstract int numRowsToMultiply();
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDCZero.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDCZero.java
index 2abe9f2427..3cdc0904f2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDCZero.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDCZero.java
@@ -21,6 +21,7 @@ package org.apache.sysds.runtime.compress.colgroup;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
+import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
 import org.apache.sysds.runtime.data.DenseBlock;
@@ -29,7 +30,7 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 public abstract class ASDCZero extends APreAgg {
 	private static final long serialVersionUID = -69266306137398807L;
-	
+
 	/** Sparse row indexes for the data */
 	protected AOffset _indexes;
 
@@ -127,9 +128,7 @@ public abstract class ASDCZero extends APreAgg {
 
 			if(last == aix[apos])
 				multiplyScalar(aval[apos], resV, offRet, it);
-
 		}
-
 	}
 
 	protected final void leftMultByMatrixNoPreAggRows(MatrixBlock mb, MatrixBlock result, int rl, int ru, int cl, int cu,
@@ -153,7 +152,6 @@ public abstract class ASDCZero extends APreAgg {
 			final int offRet = nCols * r;
 			leftMultByMatrixNoPreAggSingleRowSparse(sb, resV, offRet, r, it.clone());
 		}
-
 	}
 
 	protected final void leftMultByMatrixNoPreAggRowsDense(MatrixBlock mb, double[] resV, int nCols, int rl, int ru,
@@ -177,14 +175,31 @@ public abstract class ASDCZero extends APreAgg {
 	 */
 	protected abstract void multiplyScalar(double v, double[] resV, int offRet, AIterator it);
 
+	public void decompressToDenseBlock(DenseBlock db, int rl, int ru, int offR, int offC, AIterator it) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
+			final MatrixBlock mb = md.getMatrixBlock();
+			// The dictionary is never empty.
+			if(mb.isInSparseFormat())
+				// TODO make one where the iterator is known in argument
+				decompressToDenseBlockSparseDictionary(db, rl, ru, offR, offC, mb.getSparseBlock());
+			else
+				decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, mb.getDenseBlockValues(), it);
+		}
+		else
+			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, _dict.getValues(), it);
+	}
+
 	public void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC, AIterator it) {
-		decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, _dict.getValues(), it);
+		decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, _dict.getValues(), it);
 	}
 
-	public abstract void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+	public abstract void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values, AIterator it);
 
 	public AIterator getIterator(int row) {
 		return _indexes.getIterator(row);
 	}
+
+	protected abstract int getIndexesSize();
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
index df867b4326..c4e37fa907 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
@@ -27,6 +27,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
+import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
@@ -34,6 +35,8 @@ import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
+import org.apache.sysds.runtime.functionobjects.Minus;
+import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
@@ -80,7 +83,19 @@ public class ColGroupDDC extends APreAgg {
 	@Override
 	protected void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
 		SparseBlock sb) {
-		throw new NotImplementedException();
+		for(int r = rl, offT = rl + offR; r < ru; r++, offT++) {
+			final int vr = _data.getIndex(r);
+			if(sb.isEmpty(vr))
+				continue;
+			final double[] c = db.values(offT);
+			final int off = db.pos(offT) + offC;
+			final int apos = sb.pos(vr);
+			final int alen = sb.size(vr) + apos;
+			final int[] aix = sb.indexes(vr);
+			final double[] aval = sb.values(vr);
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]]] += aval[j];
+		}
 	}
 
 	@Override
@@ -94,9 +109,10 @@ public class ColGroupDDC extends APreAgg {
 		}
 		else if(db.isContiguous() && _colIndexes.length == db.getDim(1) && offC == 0)
 			decompressToDenseBlockDenseDictAllColumnsContiguous(db, rl, ru, offR, values);
+		else if(db.isContiguous() && offC == 0)
+			decompressToDenseBlockDenseDictNoColOffset(db, rl, ru, offR, values);
 		else
 			decompressToDenseBlockDenseDictGeneric(db, rl, ru, offR, offC, values);
-
 	}
 
 	private void decompressToDenseBlockDenseDictSingleColContiguous(DenseBlock db, int rl, int ru, int offR, int offC,
@@ -129,6 +145,19 @@ public class ColGroupDDC extends APreAgg {
 		}
 	}
 
+	private void decompressToDenseBlockDenseDictNoColOffset(DenseBlock db, int rl, int ru, int offR, double[] values) {
+		// generic
+		final int nCol = _colIndexes.length;
+		final int colOut = db.getDim(1);
+		int off = (rl + offR) * colOut;
+		for(int i = rl, offT = rl + offR; i < ru; i++, off += colOut) {
+			final double[] c = db.values(offT);
+			final int rowIndex = _data.getIndex(i) * nCol;
+			for(int j = 0; j < nCol; j++)
+				c[off + _colIndexes[j]] += values[rowIndex + j];
+		}
+	}
+
 	private void decompressToDenseBlockDenseDictGeneric(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values) {
 		// generic
@@ -201,7 +230,6 @@ public class ColGroupDDC extends APreAgg {
 			lmSparseMatrixNoPreAggSingleCol(matrix.getSparseBlock(), nColM, retV, nColRet, dictVals, rl, ru, cl, cu);
 		else
 			lmDenseMatrixNoPreAggSingleCol(matrix.getDenseBlockValues(), nColM, retV, nColRet, dictVals, rl, ru, cl, cu);
-
 	}
 
 	private void lmSparseMatrixNoPreAggSingleCol(SparseBlock sb, int nColM, double[] retV, int nColRet, double[] vals,
@@ -278,17 +306,7 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	public void preAggregateSparse(SparseBlock sb, double[] preAgg, int rl, int ru) {
-		if(rl == ru - 1)
-			for(int r = rl; r < ru; r++) {
-				final int apos = sb.pos(r);
-				final int alen = sb.size(r) + apos;
-				final int[] aix = sb.indexes(r);
-				final double[] avals = sb.values(r);
-				for(int j = apos; j < alen; j++)
-					preAgg[_data.getIndex(aix[j])] += avals[j];
-			}
-		else
-			throw new NotImplementedException();
+		_data.preAggregateSparse(sb, preAgg, rl, ru);
 	}
 
 	@Override
@@ -335,6 +353,14 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
+		if((op.fn instanceof Plus || op.fn instanceof Minus) && _dict instanceof MatrixBlockDictionary &&
+			((MatrixBlockDictionary) _dict).getMatrixBlock().isInSparseFormat()) {
+			final double v0 = op.executeScalar(0);
+			if(v0 == 0)
+				return this;
+			final double[] reference = FORUtil.createReference(_colIndexes.length, v0);
+			return ColGroupDDCFOR.create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), reference);
+		}
 		return create(_colIndexes, _numRows, _dict.applyScalarOp(op), _data, getCachedCounts());
 	}
 
@@ -351,7 +377,12 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	public AColGroup binaryRowOpRight(BinaryOperator op, double[] v, boolean isRowSafe) {
-		ADictionary ret = _dict.binOpRight(op, v, _colIndexes);
+		if((op.fn instanceof Plus || op.fn instanceof Minus) && _dict instanceof MatrixBlockDictionary &&
+			((MatrixBlockDictionary) _dict).getMatrixBlock().isInSparseFormat()) {
+			final double[] reference = ColGroupUtils.binaryDefRowRight(op, v, _colIndexes);
+			return ColGroupDDCFOR.create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), reference);
+		}
+		final ADictionary ret = _dict.binOpRight(op, v, _colIndexes);
 		return create(_colIndexes, _numRows, ret, _data, getCachedCounts());
 	}
 
@@ -381,6 +412,11 @@ public class ColGroupDDC extends APreAgg {
 		return e.getCost(nRows, nRows, nCols, nVals, _dict.getSparsity());
 	}
 
+	@Override
+	protected int numRowsToMultiply() {
+		return _numRows;
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
similarity index 55%
copy from src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java
copy to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
index 4567e4621b..8f004c9246 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
@@ -25,43 +25,33 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
-import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
-import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
-import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
 import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
+import org.apache.sysds.runtime.compress.utils.Util;
+import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Divide;
 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.instructions.cp.CM_COV_Object;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.CMOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
 
 /**
- * ColGroup for Patched Frame Of Reference.
- * 
- * This column group fits perfectly into the collection of compression groups
- * 
- * It can be constructed when a SDCZeros group get a non zero default value. Then a natural extension is to transform
- * the group into a PFOR group, since the default value is then treated as an offset, and the dictionary can be copied
- * with no modifications.
- * 
+ * Class to encapsulate information about a column group that is encoded with dense dictionary encoding (DDC).
  */
-public class ColGroupPFOR extends AMorphingMMColGroup {
+public class ColGroupDDCFOR extends AMorphingMMColGroup {
+	private static final long serialVersionUID = -5769772089913918987L;
 
-	private static final long serialVersionUID = 3883228464052204203L;
-
-	/** Sparse row indexes for the data that is nonZero */
-	protected AOffset _indexes;
-
-	/** Pointers to row indexes in the dictionary. */
+	/** Pointers to row indexes in the dictionary */
 	protected AMapToData _data;
 
 	/** Reference values in this column group */
@@ -70,57 +60,55 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	/**
 	 * Constructor for serialization
 	 * 
-	 * @param numRows Number of rows contained
+	 * @param numRows number of rows
 	 */
-	protected ColGroupPFOR(int numRows) {
+	protected ColGroupDDCFOR(int numRows) {
 		super(numRows);
 	}
 
-	private ColGroupPFOR(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
-		int[] cachedCounts, double[] reference) {
-		super(colIndices, numRows, dict, cachedCounts);
-		if(data.getUnique() != dict.getNumberOfValues(colIndices.length))
-			throw new DMLCompressionException("Invalid construction of SDCZero group");
+	private ColGroupDDCFOR(int[] colIndexes, int numRows, ADictionary dict, double[] reference, AMapToData data,
+		int[] cachedCounts) {
+		super(colIndexes, numRows, dict, cachedCounts);
+		if(data.getUnique() != dict.getNumberOfValues(colIndexes.length))
+			throw new DMLCompressionException("Invalid construction of DDC group " + data.getUnique() + " vs. "
+				+ dict.getNumberOfValues(colIndexes.length));
+		_zeros = false;
 		_data = data;
-		_indexes = indexes;
-		_zeros = allZero(reference);
 		_reference = reference;
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
+	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AMapToData data,
 		int[] cachedCounts, double[] reference) {
-		if(dict == null) {
-			// either ColGroupEmpty or const
-			boolean allZero = true;
-			for(double d : reference)
-				if(d != 0) {
-					allZero = false;
-					break;
-				}
-
-			if(allZero)
-				return new ColGroupEmpty(colIndices);
-			else
-				return ColGroupConst.create(colIndices, reference);
-		}
-		return new ColGroupPFOR(colIndices, numRows, dict, indexes, data, cachedCounts, reference);
+		final boolean allZero = FORUtil.allZero(reference);
+		if(dict == null && allZero)
+			return new ColGroupEmpty(colIndexes);
+		else if(dict == null)
+			return ColGroupConst.create(colIndexes, reference);
+		else if(allZero)
+			return ColGroupDDC.create(colIndexes, numRows, dict, data, cachedCounts);
+		else
+			return new ColGroupDDCFOR(colIndexes, numRows, dict, reference, data, cachedCounts);
 	}
 
-	private final static boolean allZero(double[] in) {
-		for(double v : in)
-			if(v != 0)
-				return false;
-		return true;
+	public CompressionType getCompType() {
+		return CompressionType.DDCFOR;
 	}
 
 	@Override
-	public CompressionType getCompType() {
-		return CompressionType.PFOR;
+	public double getIdx(int r, int colIdx) {
+		return _dict.getValue(_data.getIndex(r) * _colIndexes.length + colIdx) + _reference[colIdx];
 	}
 
 	@Override
-	public ColGroupType getColGroupType() {
-		return ColGroupType.PFOR;
+	protected void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
+		for(int rix = rl; rix < ru; rix++)
+			c[rix] += preAgg[_data.getIndex(rix)];
+	}
+
+	@Override
+	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
+		for(int i = rl; i < ru; i++)
+			c[i] = builtin.execute(c[i], preAgg[_data.getIndex(i)]);
 	}
 
 	@Override
@@ -128,38 +116,104 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		return _data.getCounts(counts);
 	}
 
-	private final double refSum() {
-		double ret = 0;
-		for(double d : _reference)
-			ret += d;
-		return ret;
+	@Override
+	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		if(_colIndexes.length == 1)
+			leftMultByMatrixNoPreAggSingleCol(matrix, result, rl, ru, cl, cu);
+		else
+			lmMatrixNoPreAggMultiCol(matrix, result, rl, ru, cl, cu);
 	}
 
-	private final double refSumSq() {
-		double ret = 0;
-		for(double d : _reference)
-			ret += d * d;
-		return ret;
+	private void leftMultByMatrixNoPreAggSingleCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl,
+		int cu) {
+		final double[] retV = result.getDenseBlockValues();
+		final int nColM = matrix.getNumColumns();
+		final int nColRet = result.getNumColumns();
+		final double[] dictVals = _dict.getValues(); // guaranteed dense double since we only have one column.
+
+		if(matrix.isInSparseFormat())
+			lmSparseMatrixNoPreAggSingleCol(matrix.getSparseBlock(), nColM, retV, nColRet, dictVals, rl, ru, cl, cu);
+		else
+			lmDenseMatrixNoPreAggSingleCol(matrix.getDenseBlockValues(), nColM, retV, nColRet, dictVals, rl, ru, cl, cu);
+	}
+
+	private void lmSparseMatrixNoPreAggSingleCol(SparseBlock sb, int nColM, double[] retV, int nColRet, double[] vals,
+		int rl, int ru, int cl, int cu) {
+		final int colOut = _colIndexes[0];
+
+		for(int r = rl; r < ru; r++) {
+			if(sb.isEmpty(r))
+				continue;
+			final int apos = sb.pos(r);
+			final int alen = sb.size(r) + apos;
+			final int[] aix = sb.indexes(r);
+			final double[] aval = sb.values(r);
+			final int offR = r * nColRet;
+			for(int i = apos; i < alen; i++)
+				retV[offR + colOut] += aval[i] * vals[_data.getIndex(aix[i])];
+		}
 	}
 
-	@Override
-	protected void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
-		ColGroupSDC.computeRowSums(c, rl, ru, preAgg, _data, _indexes, _numRows);
+	private void lmDenseMatrixNoPreAggSingleCol(double[] mV, int nColM, double[] retV, int nColRet, double[] vals,
+		int rl, int ru, int cl, int cu) {
+		final int colOut = _colIndexes[0];
+		for(int r = rl; r < ru; r++) {
+			final int offL = r * nColM;
+			final int offR = r * nColRet;
+			for(int c = cl; c < cu; c++)
+				retV[offR + colOut] += mV[offL + c] * vals[_data.getIndex(r)];
+		}
+	}
+
+	private void lmMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		if(matrix.isInSparseFormat())
+			lmSparseMatrixNoPreAggMultiCol(matrix, result, rl, ru, cl, cu);
+		else
+			lmDenseMatrixNoPreAggMultiCol(matrix, result, rl, ru, cl, cu);
+	}
+
+	private void lmSparseMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		final double[] retV = result.getDenseBlockValues();
+		final int nColRet = result.getNumColumns();
+		final SparseBlock sb = matrix.getSparseBlock();
+
+		for(int r = rl; r < ru; r++) {
+			if(sb.isEmpty(r))
+				continue;
+			final int apos = sb.pos(r);
+			final int alen = sb.size(r) + apos;
+			final int[] aix = sb.indexes(r);
+			final double[] aval = sb.values(r);
+			final int offR = r * nColRet;
+			for(int i = apos; i < alen; i++)
+				_dict.multiplyScalar(aval[i], retV, offR, _data.getIndex(aix[i]), _colIndexes);
+		}
+	}
+
+	private void lmDenseMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		final double[] retV = result.getDenseBlockValues();
+		final int nColM = matrix.getNumColumns();
+		final int nColRet = result.getNumColumns();
+		final double[] mV = matrix.getDenseBlockValues();
+		for(int r = rl; r < ru; r++) {
+			final int offL = r * nColM;
+			final int offR = r * nColRet;
+			for(int c = cl; c < cu; c++)
+				_dict.multiplyScalar(mV[offL + c], retV, offR, _data.getIndex(c), _colIndexes);
+		}
 	}
 
 	@Override
-	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
-		ColGroupSDC.computeRowMxx(c, builtin, rl, ru, preAgg, _data, _indexes, _numRows, preAgg[preAgg.length - 1]);
+	public ColGroupType getColGroupType() {
+		return ColGroupType.DDCFOR;
 	}
 
 	@Override
-	public double getIdx(int r, int colIdx) {
-		final AIterator it = _indexes.getIterator(r);
-		final int nCol = _colIndexes.length;
-		if(it == null || it.value() != r)
-			return _reference[colIdx];
-		final int rowOff = _data.getIndex(it.getDataIndex()) * nCol;
-		return _dict.getValue(rowOff + colIdx) + _reference[colIdx];
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		size += _data.getInMemorySize();
+		size += 8 * _colIndexes.length;
+		return size;
 	}
 
 	@Override
@@ -167,26 +221,23 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		final double[] newRef = new double[_reference.length];
 		for(int i = 0; i < _reference.length; i++)
 			newRef[i] = op.executeScalar(_reference[i]);
-		if(op.fn instanceof Plus || op.fn instanceof Minus) {
-			return create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
-		}
+		if(op.fn instanceof Plus || op.fn instanceof Minus)
+			return create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), newRef);
 		else if(op.fn instanceof Multiply || op.fn instanceof Divide) {
 			final ADictionary newDict = _dict.applyScalarOp(op);
-			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 		else {
 			final ADictionary newDict = _dict.applyScalarOpWithReference(op, _reference, newRef);
-			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		final double[] newRef = new double[_reference.length];
-		for(int i = 0; i < _reference.length; i++)
-			newRef[i] = op.fn.execute(_reference[i]);
+		final double[] newRef = FORUtil.unaryOperator(op, _reference);
 		final ADictionary newDict = _dict.applyUnaryOpWithReference(op, _reference, newRef);
-		return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+		return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 	}
 
 	@Override
@@ -196,15 +247,15 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 			newRef[i] = op.fn.execute(v[_colIndexes[i]], _reference[i]);
 
 		if(op.fn instanceof Plus || op.fn instanceof Minus) // only edit reference
-			return create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), newRef);
 		else if(op.fn instanceof Multiply || op.fn instanceof Divide) {
 			// possible to simply process on dict and keep reference
 			final ADictionary newDict = _dict.binOpLeft(op, v, _colIndexes);
-			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 		else { // have to apply reference while processing
 			final ADictionary newDict = _dict.binOpLeftWithReference(op, v, _colIndexes, _reference, newRef);
-			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
@@ -215,22 +266,21 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 			newRef[i] = op.fn.execute(_reference[i], v[_colIndexes[i]]);
 
 		if(op.fn instanceof Plus || op.fn instanceof Minus)// only edit reference
-			return new ColGroupPFOR(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), newRef);
 		else if(op.fn instanceof Multiply || op.fn instanceof Divide) {
 			// possible to simply process on dict and keep reference
 			final ADictionary newDict = _dict.binOpRight(op, v, _colIndexes);
-			return new ColGroupPFOR(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 		else { // have to apply reference while processing
 			final ADictionary newDict = _dict.binOpRightWithReference(op, v, _colIndexes, _reference, newRef);
-			return new ColGroupPFOR(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
 	@Override
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
-		_indexes.write(out);
 		_data.write(out);
 		for(double d : _reference)
 			out.writeDouble(d);
@@ -239,7 +289,6 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	@Override
 	public void readFields(DataInput in) throws IOException {
 		super.readFields(in);
-		_indexes = OffsetFactory.readIn(in);
 		_data = MapToFactory.readIn(in);
 		_reference = new double[_colIndexes.length];
 		for(int i = 0; i < _colIndexes.length; i++)
@@ -250,18 +299,15 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	public long getExactSizeOnDisk() {
 		long ret = super.getExactSizeOnDisk();
 		ret += _data.getExactSizeOnDisk();
-		ret += _indexes.getExactSizeOnDisk();
 		ret += 8 * _colIndexes.length; // reference values.
 		return ret;
 	}
 
 	@Override
-	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-		size += _indexes.getInMemorySize();
-		size += _data.getInMemorySize();
-		size += 8 * _colIndexes.length;
-		return size;
+	public double getCost(ComputationCostEstimator e, int nRows) {
+		final int nVals = getNumValues();
+		final int nCols = getNumCols();
+		return e.getCost(nRows, nRows, nCols, nVals, _dict.getSparsity());
 	}
 
 	@Override
@@ -273,39 +319,36 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 				break;
 			}
 
-		if(patternInReference) {
+		if(patternInReference)
 			throw new NotImplementedException("Not Implemented replace where a value in reference should be replaced");
-			// _dict.replace(pattern, replace, _reference, _newReplace);
-		}
 		else {
 			final ADictionary newDict = _dict.replaceWithReference(pattern, replace, _reference);
-			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), _reference);
+			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), _reference);
 		}
-
 	}
 
 	@Override
 	protected double computeMxx(double c, Builtin builtin) {
-		return _dict.aggregateWithReference(c, builtin, _reference);
+		return _dict.aggregateWithReference(c, builtin, _reference, false);
 	}
 
 	@Override
 	protected void computeColMxx(double[] c, Builtin builtin) {
-		_dict.aggregateColsWithReference(c, builtin, _colIndexes, _reference);
+		_dict.aggregateColsWithReference(c, builtin, _colIndexes, _reference, false);
 	}
 
 	@Override
 	protected void computeSum(double[] c, int nRows) {
-		// trick,use normal sum
+		// trick, use normal sum
 		super.computeSum(c, nRows);
 		// and add all sum of reference multiplied with nrows.
-		final double refSum = refSum();
+		final double refSum = FORUtil.refSum(_reference);
 		c[0] += refSum * nRows;
 	}
 
 	@Override
 	public void computeColSums(double[] c, int nRows) {
-		// trick, use the normal sum
+		// trick, use normal sum
 		super.computeColSums(c, nRows);
 		// and add reference multiplied with number of rows.
 		for(int i = 0; i < _colIndexes.length; i++)
@@ -316,7 +359,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	protected void computeSumSq(double[] c, int nRows) {
 		// square sum the dictionary.
 		c[0] += _dict.sumSqWithReference(getCounts(), _reference);
-		final double refSum = refSumSq();
+		final double refSum = FORUtil.refSumSq(_reference);
 		// Square sum of the reference values only for the rows that is not represented in the Offsets.
 		c[0] += refSum * (_numRows - _data.size());
 	}
@@ -369,7 +412,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected AColGroup sliceSingleColumn(int idx) {
-		ColGroupPFOR ret = (ColGroupPFOR) super.sliceSingleColumn(idx);
+		ColGroupDDCFOR ret = (ColGroupDDCFOR) super.sliceSingleColumn(idx);
 		// select values from double array.
 		ret._reference = new double[1];
 		ret._reference[0] = _reference[idx];
@@ -378,7 +421,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
-		ColGroupPFOR ret = (ColGroupPFOR) super.sliceMultiColumns(idStart, idEnd, outputCols);
+		ColGroupDDCFOR ret = (ColGroupDDCFOR) super.sliceMultiColumns(idStart, idEnd, outputCols);
 		final int len = idEnd - idStart;
 		ret._reference = new double[len];
 		for(int i = 0, ii = idStart; i < len; i++, ii++)
@@ -392,47 +435,67 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		if(pattern == 0 && _zeros)
 			return true;
 		else if(Double.isNaN(pattern) || Double.isInfinite(pattern))
-			return containsInfOrNan(pattern) || _dict.containsValue(pattern);
+			return FORUtil.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
 		else
 			return _dict.containsValueWithReference(pattern, _reference);
 	}
 
-	private boolean containsInfOrNan(double pattern) {
-		if(Double.isNaN(pattern)) {
-			for(double d : _reference)
-				if(Double.isNaN(d))
-					return true;
-			return false;
-		}
+	@Override
+	public long getNumberNonZeros(int nRows) {
+		long nnz = 0;
+		int refCount = 0;
+		for(int i = 0; i < _reference.length; i++)
+			if(_reference[i] != 0)
+				refCount++;
+
+		if(refCount == _colIndexes.length)
+			return (long) _colIndexes.length * nRows;
 		else {
-			for(double d : _reference)
-				if(Double.isInfinite(d))
-					return true;
-			return false;
+			nnz += _dict.getNumberNonZerosWithReference(getCounts(), _reference, nRows);
+			nnz += refCount * nRows;
 		}
-	}
 
-	@Override
-	public long getNumberNonZeros(int nRows) {
-		final int[] counts = getCounts();
-		final int count = _numRows - _data.size();
-		long c = _dict.getNumberNonZerosWithReference(counts, _reference, nRows);
-		for(int x = 0; x < _colIndexes.length; x++)
-			c += _reference[x] != 0 ? count : 0;
-		return c;
+		return Math.min((long) _colIndexes.length * nRows, nnz);
 	}
 
 	@Override
 	public AColGroup extractCommon(double[] constV) {
 		for(int i = 0; i < _colIndexes.length; i++)
 			constV[_colIndexes[i]] += _reference[i];
-		return ColGroupSDCZeros.create(_colIndexes, _numRows, _dict, _indexes, _data, getCounts());
+		return ColGroupDDC.create(_colIndexes, _numRows, _dict, _data, getCounts());
 	}
 
 	@Override
 	public AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows) {
-		ADictionary d = _dict.rexpandColsWithReference(max, ignore, cast, _reference[0]);
-		return ColGroupSDC.rexpandCols(max, ignore, cast, nRows, d, _indexes, _data, getCachedCounts(), _reference[0]);
+		final double def = _reference[0];
+		ADictionary d = _dict.rexpandColsWithReference(max, ignore, cast, def);
+
+		// return ColGroupDDC.rexpandCols(max, ignore, cast, nRows, d, _data, getCachedCounts(), _reference[0]);
+		if(d == null) {
+			if(def <= 0 || def > max)
+				return ColGroupEmpty.create(max);
+			else {
+				double[] retDef = new double[max];
+				retDef[((int) def) - 1] = 1;
+				return ColGroupConst.create(retDef);
+			}
+		}
+		else {
+			int[] outCols = Util.genColsIndices(max);
+			if(def <= 0) {
+				if(ignore)
+					return ColGroupDDC.create(outCols, nRows, d, _data, getCachedCounts());
+				else
+					throw new DMLRuntimeException("Invalid content of zero in rexpand");
+			}
+			else if(def > max)
+				return ColGroupDDC.create(outCols, nRows, d, _data, getCachedCounts());
+			else {
+				double[] retDef = new double[max];
+				retDef[((int) def) - 1] = 1;
+				return create(outCols, nRows, d, _data, getCachedCounts(), retDef);
+			}
+		}
 	}
 
 	@Override
@@ -444,25 +507,14 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		return ret;
 	}
 
-	@Override
-	public double getCost(ComputationCostEstimator e, int nRows) {
-		final int nVals = getNumValues();
-		final int nCols = getNumCols();
-		final int nRowsScanned = _data.size();
-		return e.getCost(nRows, nRowsScanned, nCols, nVals, _dict.getSparsity());
-	}
-
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append(super.toString());
-		sb.append(String.format("\n%15s", "Indexes: "));
-		sb.append(_indexes.toString());
-		sb.append(String.format("\n%15s", "Data: "));
+		sb.append(String.format("\n%15s ", "Data: "));
 		sb.append(_data);
 		sb.append(String.format("\n%15s", "Reference:"));
 		sb.append(Arrays.toString(_reference));
 		return sb.toString();
 	}
-
 }
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 f20823fac4..d40367de2e 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
@@ -21,7 +21,6 @@ package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -31,10 +30,9 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.DMLCompressionException;
@@ -44,11 +42,11 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.DictionaryFactory;
+import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.colgroup.insertionsort.AInsertionSorter;
 import org.apache.sysds.runtime.compress.colgroup.insertionsort.InsertionSorterFactory;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
-import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
 import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
@@ -62,10 +60,11 @@ import org.apache.sysds.runtime.compress.utils.DoubleCountHashMap;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
 import org.apache.sysds.runtime.data.SparseBlock;
+import org.apache.sysds.runtime.functionobjects.Minus;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.util.CommonThreadPool;
-import org.apache.sysds.runtime.util.DataConverter;
 
 /**
  * Factory class for constructing ColGroups.
@@ -82,6 +81,8 @@ public class ColGroupFactory {
 	private final int nRow;
 	private final int nCol;
 
+	private final ExecutorService pool;
+
 	private ColGroupFactory(MatrixBlock in, CompressedSizeInfo csi, CompressionSettings cs, ACostEstimate ce, int k) {
 		this.in = in;
 		this.csi = csi;
@@ -92,6 +93,7 @@ public class ColGroupFactory {
 		this.nRow = cs.transposed ? in.getNumColumns() : in.getNumRows();
 		this.nCol = cs.transposed ? in.getNumRows() : in.getNumColumns();
 
+		this.pool = (k > 1) ? CommonThreadPool.get(k) : null;
 	}
 
 	/**
@@ -127,6 +129,20 @@ public class ColGroupFactory {
 	}
 
 	private List<AColGroup> compress() {
+		try{
+			List<AColGroup> ret = compressExecute();
+			if(pool != null)
+				pool.shutdown();
+				return ret;
+		}
+		catch(Exception e ){
+			if(pool != null)
+				pool.shutdown();
+			throw new DMLCompressionException("Compression Failed", e);
+		}
+	}
+
+	private List<AColGroup> compressExecute() {
 		for(CompressedSizeInfoColGroup g : csi.getInfo())
 			g.clearMap();
 		if(in.isEmpty()) {
@@ -151,67 +167,185 @@ public class ColGroupFactory {
 
 	private List<AColGroup> compressColGroupsParallel() {
 		try {
-			ExecutorService pool = CommonThreadPool.get(k);
-			List<CompressTask> tasks = new ArrayList<>();
-
-			List<List<CompressedSizeInfoColGroup>> threadGroups = makeGroups();
-			for(List<CompressedSizeInfoColGroup> tg : threadGroups)
-				if(!tg.isEmpty())
-					tasks.add(new CompressTask(tg));
-
-			List<AColGroup> ret = new ArrayList<>();
-			for(Future<Collection<AColGroup>> t : pool.invokeAll(tasks))
-				ret.addAll(t.get());
-			pool.shutdown();
-			return ret;
+			final List<CompressedSizeInfoColGroup> groups = csi.getInfo();
+			final int nGroups = groups.size();
+			// final int blkz = nGroups * 10 / k;
+			final int skip = Math.min(k * 10, nGroups);
+			final List<CompressTask> tasks = new ArrayList<>(skip);
+
+			// sort to make the "assumed" big jobs first.
+			Collections.sort(groups, Comparator.comparing(g -> -g.getNumVals()));
+
+			final AColGroup[] ret = new AColGroup[nGroups];
+
+			for(int i = 0; i < skip; i++)
+				tasks.add(new CompressTask(groups, ret, i, skip));
+
+			for(Future<Object> t : pool.invokeAll(tasks))
+				t.get();
+
+			return Arrays.asList(ret);
 		}
 		catch(InterruptedException | ExecutionException e) {
 			throw new DMLRuntimeException("Failed compression ", e);
 		}
 	}
 
-	private List<List<CompressedSizeInfoColGroup>> makeGroups() {
-		// sort by number of distinct items
-		final List<CompressedSizeInfoColGroup> groups = csi.getInfo();
-		Collections.sort(groups, Comparator.comparing(g -> -g.getNumVals()));
-		List<List<CompressedSizeInfoColGroup>> ret = new ArrayList<>();
-		for(int i = 0; i < k; i++)
-			ret.add(new ArrayList<>());
-
-		for(int i = 0; i < groups.size(); i++)
-			ret.get(i % k).add(groups.get(i));
-
-		return ret;
-	}
-
 	protected AColGroup compressColGroup(CompressedSizeInfoColGroup cg) {
 		if(LOG.isDebugEnabled() && nCol < 1000 && ce != null) {
 			final Timing time = new Timing(true);
-			final AColGroup ret = compressColGroupForced(cg);
-			synchronized(this) {
+			final AColGroup ret = compressColGroupAllSteps(cg);
+			logEstVsActual(time.stop(), ret, cg);
+			return ret;
+		}
+		return compressColGroupAllSteps(cg);
+	}
+
+	private void logEstVsActual(double time, AColGroup act, CompressedSizeInfoColGroup est) {
+		final double estC = ce.getCost(est);
+		final double actC = ce.getCost(act, nRow);
+		final String retType = act.getClass().getSimpleName().toString();
+		final String cols = Arrays.toString(est.getColumns());
+		final String wanted = est.getBestCompressionType().toString();
+		if(estC < actC * 0.75) {
+			StringBuilder sb = new StringBuilder();
+			sb.append("The estimate cost is significantly off : distinct: ");
+			sb.append(est.getNumVals());
+			sb.append(" ");
+			sb.append(act.getNumValues());
+			sb.append(" estimate offsets:");
+			sb.append(est.getNumOffs());
+			if(act instanceof ColGroupSDCZeros)
+				sb.append("  act:" + ((ColGroupSDCZeros) act).getIndexesSize());
+			String warning = sb.toString();
+
+			LOG.debug(String.format("time[ms]: %10.2f %25s est %10.0f -- act %10.0f cols:%s wanted:%s\n%s", time, retType,
+				estC, actC, cols, wanted, warning));
+		}
+		else {
+			LOG.debug(String.format("time[ms]: %10.2f %25s est %10.0f -- act %10.0f cols:%s wanted:%s", time, retType,
+				estC, actC, cols, wanted));
+		}
+
+	}
+
+	private AColGroup compressColGroupAllSteps(CompressedSizeInfoColGroup cg) {
+		AColGroup g = compressColGroupInitial(cg);
+		final int nCol = g.getColIndices().length;
+		if(ce != null && ce.shouldSparsify() && nCol >= 4 && isSparsifyingColGroup(g)) {
+
+			double[] constV = null;
+			if(g instanceof ColGroupSDC) {
+				constV = ((ColGroupSDC) g)._defaultTuple;
+				g = ((ColGroupSDC) g).subtractDefaultTuple();
+			}
+
+			final AColGroupValue clg = (AColGroupValue) g;
+			final int nVal = g.getNumValues();
+			final MatrixBlockDictionary mbd = clg._dict.getMBDict(nCol);
+			final MatrixBlock mb = mbd.getMatrixBlock();
+
+			if(mb == null || mb.isEmpty())
+				return g;
+
+			final int[] nnz = LibMatrixReorg.countNnzPerColumn(mb);
+
+			double[] ref = new double[nCol];
+			boolean contains = false;
+			for(int i = 0; i < nCol; i++) {
+				if(nnz[i] > nVal / 2) {
+					contains = true;
+					ref[i] = 1;
+				}
+			}
+			if(contains)
+				getMostCommonValues(mb, ref, nnz);
+			contains = false;
+			for(int i = 0; i < nCol; i++)
+				if(ref[i] != 0) {
+					contains = true;
+					break;
+				}
+
+			if(contains) {
+				// minus overlap on dictionary
+				MatrixBlockDictionary mDict = mbd.binOpRight(new BinaryOperator(Minus.getMinusFnObject()), ref);
+				if(constV != null)
+					for(int i = 0; i < nCol; i++)
+						ref[i] += constV[i]; // plus reference on overlap
+
 				LOG.debug(
-					String.format("time[ms]: %10.2f %20s %s cols:%s wanted:%s", time.stop(), getColumnTypesString(ret),
-						getEstimateVsActualSize(ret, cg), Arrays.toString(cg.getColumns()), cg.getBestCompressionType()));
+					String.format("Sparsifying colgroup before %1.4f now %1.4f", mb.getSparsity(), mDict.getSparsity()));
+				if(g instanceof ColGroupDDC)
+					g = ColGroupDDCFOR.create(g.getColIndices(), nRow, mDict, ((ColGroupDDC) clg)._data,
+						clg.getCachedCounts(), ref);
+				else if(g instanceof ColGroupSDCZeros) {
+					g = ColGroupSDCFOR.create(g.getColIndices(), nRow, mDict, ((ColGroupSDCZeros) clg)._indexes,
+						((ColGroupSDCZeros) clg)._data, clg.getCachedCounts(), ref);
+
+				}
 			}
-			return ret;
+			else {
+				if(g instanceof ColGroupSDCZeros)
+					g = ColGroupSDCFOR.create(g.getColIndices(), nRow, mbd, ((ColGroupSDCZeros) clg)._indexes,
+						((ColGroupSDCZeros) clg)._data, clg.getCachedCounts(), ref);
+			}
+
 		}
-		return compressColGroupForced(cg);
+		return g;
 	}
 
-	private String getColumnTypesString(AColGroup ret) {
-		return ret.getClass().getSimpleName().toString();
+	private void getMostCommonValues(MatrixBlock mb, double[] ref, int[] nnzCols) {
+		// take each column marked by ref and find most common value in that and assign it to ref.
+		// if the columns are
+
+		DoubleCountHashMap[] counters = new DoubleCountHashMap[ref.length];
+
+		if(mb.isInSparseFormat()) {
+			// initialize the counters with zero count.
+			for(int i = 0; i < ref.length; i++) {
+				if(ref[i] != 0) {
+					counters[i] = new DoubleCountHashMap(8);
+					counters[i].increment(0, nnzCols[i]);
+				}
+			}
+			final SparseBlock sb = mb.getSparseBlock();
+			for(int r = 0; r < mb.getNumRows(); r++) {
+				if(sb.isEmpty(r))
+					continue;
+				final int apos = sb.pos(r);
+				final int alen = sb.size(r) + apos;
+				final int[] aix = sb.indexes(r);
+				final double[] aval = sb.values(r);
+				for(int j = apos; j < alen; j++)
+					if(ref[aix[j]] != 0)
+						counters[aix[j]].increment(aval[j]);
+			}
+		}
+		else {
+			for(int i = 0; i < ref.length; i++)
+				if(ref[i] != 0)
+					counters[i] = new DoubleCountHashMap(8);
+			double[] dv = mb.getDenseBlockValues();
+			final int nCol = ref.length;
+			for(int r = 0; r < mb.getNumRows(); r++) {
+				final int rOff = r * nCol;
+				for(int c = 0; c < nCol; c++)
+					if(ref[c] != 0)
+						counters[c].increment(dv[rOff + c]);
+
+			}
+		}
+		for(int i = 0; i < ref.length; i++)
+			if(ref[i] != 0)
+				ref[i] = counters[i].getMostFrequent();
 	}
 
-	private String getEstimateVsActualSize(AColGroup ret, CompressedSizeInfoColGroup cg) {
-		Level before = Logger.getLogger(ACostEstimate.class.getName()).getLevel();
-		Logger.getLogger(ACostEstimate.class.getName()).setLevel(Level.TRACE);
-		final double est = ce.getCost(cg);
-		final double act = ce.getCost(ret, nRow);
-		Logger.getLogger(ACostEstimate.class.getName()).setLevel(before);
-		return String.format("[B] %10.0f -- %10.0f", est, act);
+	private boolean isSparsifyingColGroup(AColGroup g) {
+		return g instanceof ColGroupDDC || g instanceof ColGroupSDC;
 	}
 
-	private AColGroup compressColGroupForced(CompressedSizeInfoColGroup cg) {
+	private AColGroup compressColGroupInitial(CompressedSizeInfoColGroup cg) {
 		final int[] colIndexes = cg.getColumns();
 		final int nrUniqueEstimate = cg.getNumVals();
 		CompressionType ct = cg.getBestCompressionType();
@@ -221,18 +355,12 @@ public class ColGroupFactory {
 		else if(ct == CompressionType.UNCOMPRESSED) // don't construct mapping if uncompressed
 			return ColGroupUncompressed.create(colIndexes, in, cs.transposed);
 		else if((ct == CompressionType.SDC || ct == CompressionType.CONST) && in.isInSparseFormat() && cs.transposed &&
-			((colIndexes.length > 1 && in.getSparsity() < 0.001) || colIndexes.length == 1))
-			// Leverage the Sparse matrix, to construct SDC group
+			((colIndexes.length > 1 && cg.getNumOffs() < 0.3 * nRow) || colIndexes.length == 1))
 			return compressSDCFromSparseTransposedBlock(colIndexes, nrUniqueEstimate, cg.getTupleSparsity());
-		else if(colIndexes.length > 1 && ct == CompressionType.DDC)
+		else if(ct == CompressionType.DDC)
 			return directCompressDDC(colIndexes, cg);
-		else if(ct == CompressionType.DeltaDDC) {
-			if(colIndexes.length > 1)
-				return directCompressDeltaDDC(colIndexes, in, cs, cg, k);
-			else
-				return compressDeltaDDC(colIndexes, in, cs, cg);
-		}
 		else {
+			LOG.debug("Default slow path: " + ct + "  " + cs.transposed + " " + Arrays.toString(colIndexes));
 			final int numRows = cs.transposed ? in.getNumColumns() : in.getNumRows();
 			final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, cs.transposed, nrUniqueEstimate,
 				cs.sortTuplesByFrequency);
@@ -271,32 +399,41 @@ public class ColGroupFactory {
 		}
 	}
 
-	private static AColGroup directCompressDeltaDDC(int[] colIndexes, MatrixBlock raw, CompressionSettings cs,
-		CompressedSizeInfoColGroup cg, int k) {
-		final int rlen = cs.transposed ? raw.getNumColumns() : raw.getNumRows();
-		// use a Map that is at least char size.
-		final AMapToData d = MapToFactory.create(rlen, MAP_TYPE.INT);
-		if(cs.transposed) {
-			LOG.warn("In-effecient transpose back of the input matrix to do delta encoding");
-			raw = LibMatrixReorg.transposeInPlace(raw, k);
-			cs.transposed = false;
-		}
-		// Delta encode the raw data
-		raw = deltaEncodeMatrixBlock(raw);
-		return directCompressDDCDeltaColGroup(colIndexes, raw, cs, cg, d, rlen, k);
+	private AColGroup directCompressDDC(int[] colIndexes, CompressedSizeInfoColGroup cg) {
+		if(colIndexes.length > 1)
+			return directCompressDDCMultiCol(colIndexes, cg);
+		else
+			return directCompressDDCSingleCol(colIndexes, cg);
 	}
 
-	private AColGroup directCompressDDC(int[] colIndexes, CompressedSizeInfoColGroup cg) {
-		final AMapToData d = MapToFactory.create(nRow, MAP_TYPE.INT);
+	private AColGroup directCompressDDCSingleCol(int[] colIndexes, CompressedSizeInfoColGroup cg) {
+		final int col = colIndexes[0];
+		final AMapToData d = MapToFactory.create(nRow, Math.max(Math.min(cg.getNumOffs() + 1, nRow), 126));
+		final DoubleCountHashMap map = new DoubleCountHashMap(cg.getNumVals());
+
+		// unlike multi-col no special handling of zero entries are needed.
+		if(cs.transposed)
+			readToMapDDCTransposed(col, map, d);
+		else
+			readToMapDDC(col, map, d);
+
+		ADictionary dict = DictionaryFactory.create(map);
+		final int nUnique = map.size();
+		final AMapToData resData = MapToFactory.resize(d, nUnique);
+		return ColGroupDDC.create(colIndexes, nRow, dict, resData, null);
+	}
+
+	private AColGroup directCompressDDCMultiCol(int[] colIndexes, CompressedSizeInfoColGroup cg) {
+		final AMapToData d = MapToFactory.create(nRow, Math.max(Math.min(cg.getNumOffs() + 1, nRow), 126));
 		final int fill = d.getUpperBoundValue();
 		d.fill(fill);
 
 		final DblArrayCountHashMap map = new DblArrayCountHashMap(cg.getNumVals(), colIndexes.length);
 		boolean extra;
 		if(nRow < CompressionSettings.PAR_DDC_THRESHOLD || k == 1)
-			extra = readToMapDDC(colIndexes, in, map, cs, d, 0, nRow, fill);
+			extra = readToMapDDC(colIndexes, map, d, 0, nRow, fill);
 		else
-			extra = parallelReadToMapDDC(colIndexes, in, map, cs, d, nRow, fill, k);
+			extra = parallelReadToMapDDC(colIndexes, map, d, nRow, fill, k);
 
 		if(map.size() == 0)
 			// If the column was empty.
@@ -308,47 +445,24 @@ public class ColGroupFactory {
 		if(dict == null)
 			// Again highly unlikely but possible.
 			return new ColGroupEmpty(colIndexes);
+		try{
+			if(extra)
+				d.replace(fill, map.size());
+	
+			final int nUnique = map.size() + (extra ? 1 : 0);
+	
+			final AMapToData resData = MapToFactory.resize(d, nUnique);
+			return ColGroupDDC.create(colIndexes, nRow, dict, resData, null);
 
-		if(extra)
-			d.replace(fill, map.size());
-
-		final int nUnique = map.size() + (extra ? 1 : 0);
-		final AMapToData resData = MapToFactory.resize(d, nUnique);
-		return ColGroupDDC.create(colIndexes, nRow, dict, resData, null);
-	}
-
-	private static AColGroup directCompressDDCDeltaColGroup(int[] colIndexes, MatrixBlock raw, CompressionSettings cs,
-		CompressedSizeInfoColGroup cg, AMapToData data, int rlen, int k) {
-		final int fill = data.getUpperBoundValue();
-		data.fill(fill);
-
-		DblArrayCountHashMap map = new DblArrayCountHashMap(cg.getNumVals(), colIndexes.length);
-		boolean extra;
-		if(rlen < CompressionSettings.PAR_DDC_THRESHOLD || k == 1)
-			extra = readToMapDDC(colIndexes, raw, map, cs, data, 0, rlen, fill);
-		else
-			extra = parallelReadToMapDDC(colIndexes, raw, map, cs, data, rlen, fill, k);
-
-		if(map.size() == 0)
-			// If the column was empty.
-			// This is highly unlikely but could happen if forced compression of
-			// not transposed column and the estimator says use DDC.
-			return new ColGroupEmpty(colIndexes);
-		ADictionary dict = DictionaryFactory.createDelta(map, colIndexes.length, extra);
-		if(extra) {
-			data.replace(fill, map.size());
-			data.setUnique(map.size() + 1);
 		}
-		else
-			data.setUnique(map.size());
-
-		AMapToData resData = MapToFactory.resize(data, map.size() + (extra ? 1 : 0));
-		return ColGroupDeltaDDC.create(colIndexes, rlen, dict, resData, null);
+		catch(Exception e ){
+			ReaderColumnSelection reader = ReaderColumnSelection.createReader(in, colIndexes, cs.transposed, 0, nRow);
+			throw new DMLCompressionException("direct compress DDC Multi col failed extra:" + extra + " with reader type:" + reader.getClass().getSimpleName(), e);
+		}
 	}
 
-	private static boolean readToMapDDC(final int[] colIndexes, final MatrixBlock raw, final DblArrayCountHashMap map,
-		final CompressionSettings cs, final AMapToData data, final int rl, final int ru, final int fill) {
-		ReaderColumnSelection reader = ReaderColumnSelection.createReader(raw, colIndexes, cs.transposed, rl, ru);
+	private boolean readToMapDDC(int[] colIndexes, DblArrayCountHashMap map, AMapToData data, int rl, int ru, int fill) {
+		ReaderColumnSelection reader = ReaderColumnSelection.createReader(in, colIndexes, cs.transposed, rl, ru);
 		DblArray cellVals = reader.nextRow();
 		boolean extra = false;
 		int r = rl;
@@ -372,24 +486,85 @@ public class ColGroupFactory {
 		return extra;
 	}
 
-	private static boolean parallelReadToMapDDC(final int[] colIndexes, final MatrixBlock raw,
-		final DblArrayCountHashMap map, final CompressionSettings cs, final AMapToData data, final int rlen,
-		final int fill, final int k) {
+	private void readToMapDDC(int col, DoubleCountHashMap map, AMapToData data) {
+		if(in.isInSparseFormat()) {
+			// not good but could happen
+			final SparseBlock sb = in.getSparseBlock();
+			for(int r = 0; r < nRow; r++) {
+				if(sb.isEmpty(r))
+					data.set(r, map.increment(0));
+				else {
+					final int apos = sb.pos(r);
+					final int alen = sb.size(r) + apos;
+					final int[] aix = sb.indexes(r);
+					final int idx = Arrays.binarySearch(aix, apos, alen, col);
+					if(idx < 0)
+						data.set(r, map.increment(0));
+					else
+						data.set(r, map.increment(sb.values(r)[idx]));
+				}
+			}
+		}
+		else if(in.getDenseBlock().isContiguous()) {
+			final double[] dv = in.getDenseBlockValues();
+			int off = col;
+			for(int r = 0; r < nRow; r++, off += nCol) {
+				final int id = map.increment(dv[off]);
+				data.set(r, id);
+			}
+		}
+		else {
+			throw new NotImplementedException();
+		}
+	}
+
+	private void readToMapDDCTransposed(int col, DoubleCountHashMap map, AMapToData data) {
+		if(in.isInSparseFormat()) {
+			// good
+			SparseBlock sb = in.getSparseBlock();
+			if(sb.isEmpty(col))
+				return;
+
+			final int apos = sb.pos(col);
+			final int alen = sb.size(col) + apos;
+			final int[] aix = sb.indexes(col);
+			final double[] aval = sb.values(col);
+			// count zeros
+			map.increment(0, nRow - apos - alen);
+			// insert all other counts
+			for(int j = apos; j < alen; j++) {
+				final int id = map.increment(aval[j]);
+				data.set(aix[j], id);
+			}
+		}
+		else if(in.getDenseBlock().isContiguous()) {
+			double[] dv = in.getDenseBlockValues();
+			int off = col * nRow;
+			for(int r = 0; r < nRow; r++, off++) {
+				final int id = map.increment(dv[off]);
+				data.set(r, id);
+			}
+		}
+		else {
+			throw new NotImplementedException();
+		}
+	}
+
+	private boolean parallelReadToMapDDC(int[] colIndexes, DblArrayCountHashMap map, AMapToData data, int rlen, int fill,
+		int k) {
 
 		try {
 			final int blk = Math.max(rlen / colIndexes.length / k, 64000 / colIndexes.length);
-			ExecutorService pool = CommonThreadPool.get(Math.min(Math.max(rlen / blk, 1), k));
-			List<readToMapDDCTask> tasks = new ArrayList<>();
 
+			List<readToMapDDCTask> tasks = new ArrayList<>();
 			for(int i = 0; i < rlen; i += blk) {
 				int end = Math.min(rlen, i + blk);
-				tasks.add(new readToMapDDCTask(colIndexes, raw, map, cs, data, i, end, fill));
+				tasks.add(new readToMapDDCTask(colIndexes, map, data, i, end, fill));
 			}
 			boolean extra = false;
 			for(Future<Boolean> t : pool.invokeAll(tasks))
 				extra |= t.get();
 
-			pool.shutdown();
 			return extra;
 		}
 		catch(Exception e) {
@@ -397,22 +572,6 @@ public class ColGroupFactory {
 		}
 	}
 
-	private static MatrixBlock deltaEncodeMatrixBlock(MatrixBlock mb) {
-		LOG.warn("Delta encoding entire matrix input!!");
-		int rows = mb.getNumRows();
-		int cols = mb.getNumColumns();
-		double[][] ret = new double[rows][cols];
-		double[] a = mb.getDenseBlockValues();
-		for(int i = 0, ix = 0; i < rows; i++) {
-			int prevRowOff = i > 0 ? ix - cols : 0;
-			for(int j = 0; j < cols; j++, ix++) {
-				double currentValue = a[ix];
-				ret[i][j] = i > 0 ? currentValue - a[prevRowOff + j] : currentValue;
-			}
-		}
-		return DataConverter.convertToMatrixBlock(ret);
-	}
-
 	private static AColGroup compressSDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
 		double tupleSparsity) {
 
@@ -446,11 +605,9 @@ public class ColGroupFactory {
 			ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
 			return compressSDCZero(colIndexes, rlen, ubm, dict, cs);
 		}
-		else {
-			double[] defaultTuple = new double[colIndexes.length];
-			ADictionary dict = DictionaryFactory.create(ubm, largestIndex, defaultTuple, tupleSparsity, numZeros > 0);
-			return compressSDCNormal(colIndexes, numZeros, rlen, ubm, largestIndex, dict, defaultTuple, cs);
-		}
+		else
+			return compressSDCNormal(colIndexes, numZeros, rlen, ubm, largestIndex, tupleSparsity, cs);
+
 	}
 
 	private static AColGroup compressSDCZero(int[] colIndexes, int rlen, ABitmap ubm, ADictionary dict,
@@ -464,9 +621,11 @@ public class ColGroupFactory {
 	}
 
 	private static AColGroup compressSDCNormal(int[] colIndexes, int numZeros, int rlen, ABitmap ubm, int largestIndex,
-		ADictionary dict, double[] defaultTuple, CompressionSettings cs) {
-		IntArrayList[] offsets = ubm.getOffsetList();
-		AInsertionSorter s = InsertionSorterFactory.createNegative(rlen, offsets, largestIndex, cs.sdcSortType);
+		double tupleSparsity, CompressionSettings cs) {
+		final double[] defaultTuple = new double[colIndexes.length];
+		final ADictionary dict = DictionaryFactory.create(ubm, largestIndex, defaultTuple, tupleSparsity, numZeros > 0);
+		AInsertionSorter s = InsertionSorterFactory.createNegative(rlen, ubm.getOffsetList(), largestIndex,
+			cs.sdcSortType);
 		AOffset indexes = OffsetFactory.createOffset(s.getIndexes());
 		AMapToData _data = s.getData();
 		_data = MapToFactory.resize(_data, dict.getNumberOfValues(colIndexes.length));
@@ -502,27 +661,6 @@ public class ColGroupFactory {
 		return ColGroupDDC.create(colIndexes, rlen, dict, data, null);
 	}
 
-	private static AColGroup compressDeltaDDC(int[] colIndexes, MatrixBlock in, CompressionSettings cs,
-		CompressedSizeInfoColGroup cg) {
-
-		LOG.warn("Multi column Delta encoding only supported if delta encoding is only compression");
-		if(cs.transposed) {
-			LibMatrixReorg.transposeInPlace(in, 1);
-			cs.transposed = false;
-		}
-		// Delta encode the raw data
-		in = deltaEncodeMatrixBlock(in);
-
-		final int rlen = in.getNumRows();
-		// TODO Add extractBitMap that is delta to not require delta encoding entire input matrix.
-		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, cs.transposed, cg.getNumVals(),
-			cs.sortTuplesByFrequency);
-		boolean zeros = ubm.getNumOffsets() < rlen;
-		ADictionary dict = DictionaryFactory.create(ubm, cg.getTupleSparsity(), zeros);
-		AMapToData data = MapToFactory.create(rlen, zeros, ubm.getOffsetList());
-		return ColGroupDeltaDDC.create(colIndexes, rlen, dict, data, null);
-	}
-
 	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
 		double tupleSparsity) {
 
@@ -530,15 +668,15 @@ public class ColGroupFactory {
 		ColGroupOLE ole = new ColGroupOLE(rlen);
 
 		final int numVals = ubm.getNumValues();
-		char[][] lbitmaps = new char[numVals][];
+		char[][] lBitMaps = new char[numVals][];
 		int totalLen = 0;
 		for(int i = 0; i < numVals; i++) {
-			lbitmaps[i] = ColGroupOLE.genOffsetBitmap(ubm.getOffsetsList(i).extractValues(), ubm.getNumOffsets(i));
-			totalLen += lbitmaps[i].length;
+			lBitMaps[i] = ColGroupOLE.genOffsetBitmap(ubm.getOffsetsList(i).extractValues(), ubm.getNumOffsets(i));
+			totalLen += lBitMaps[i].length;
 		}
 
 		// compact bitmaps to linearized representation
-		ole.createCompressedBitmaps(numVals, totalLen, lbitmaps);
+		ole.createCompressedBitmaps(numVals, totalLen, lBitMaps);
 		ole._dict = dict;
 		ole._zeros = ubm.getNumOffsets() < (long) rlen;
 		ole._colIndexes = colIndexes;
@@ -552,15 +690,15 @@ public class ColGroupFactory {
 		ColGroupRLE rle = new ColGroupRLE(rlen);
 		// compress the bitmaps
 		final int numVals = ubm.getNumValues();
-		char[][] lbitmaps = new char[numVals][];
+		char[][] lBitMaps = new char[numVals][];
 		int totalLen = 0;
 
 		for(int k = 0; k < numVals; k++) {
-			lbitmaps[k] = ColGroupRLE.genRLEBitmap(ubm.getOffsetsList(k).extractValues(), ubm.getNumOffsets(k));
-			totalLen += lbitmaps[k].length;
+			lBitMaps[k] = ColGroupRLE.genRLEBitmap(ubm.getOffsetsList(k).extractValues(), ubm.getNumOffsets(k));
+			totalLen += lBitMaps[k].length;
 		}
 		// compact bitmaps to linearized representation
-		rle.createCompressedBitmaps(numVals, totalLen, lbitmaps);
+		rle.createCompressedBitmaps(numVals, totalLen, lBitMaps);
 		rle._dict = dict;
 		rle._zeros = ubm.getNumOffsets() < (long) rlen;
 		rle._colIndexes = colIndexes;
@@ -675,6 +813,32 @@ public class ColGroupFactory {
 				return ColGroupSDCZeros.create(cols, nRow, new Dictionary(dict), offsets, mapToData, counts);
 			}
 		}
+		else if(entries.length == 1) {
+			// SDCSingle and we know all values are x or 0
+			final int nonZeros = nRow - entries[0].count;
+			final double x = entries[0].key;
+			final double[] defaultTuple = new double[] {x};
+			final ADictionary zeroDict = new Dictionary(new double[] {0});
+			final int[] counts = new int[] {nonZeros};
+			final int[] notZeroOffsets = new int[nonZeros];
+			final int[] aix = sb.indexes(sbRow);
+			int i = 0;
+			int r = 0;
+			for(int j = apos; r < aix[alen - 1]; r++) {
+				if(r == aix[j])
+					j++;
+				else
+					notZeroOffsets[i++] = r;
+			}
+			r++;
+
+			for(; r < nRow; r++, i++)
+				notZeroOffsets[i] = r;
+
+			final AOffset offsets = OffsetFactory.createOffset(notZeroOffsets);
+
+			return ColGroupSDCSingle.create(cols, nRow, zeroDict, defaultTuple, offsets, counts);
+		}
 		else {
 			final ABitmap ubm = BitmapEncoder.extractBitmap(cols, in, true, entries.length, true);
 			// zero is not the default value fall back to the standard compression path.
@@ -682,21 +846,26 @@ public class ColGroupFactory {
 		}
 	}
 
-	class CompressTask implements Callable<Collection<AColGroup>> {
+	private class CompressTask implements Callable<Object> {
 
 		private final List<CompressedSizeInfoColGroup> _groups;
+		private final AColGroup[] _ret;
+		private final int _off;
+		private final int _step;
 
-		protected CompressTask(List<CompressedSizeInfoColGroup> groups) {
+		protected CompressTask(List<CompressedSizeInfoColGroup> groups, AColGroup[] ret, int off, int step) {
 			_groups = groups;
+			_ret = ret;
+			_off = off;
+			_step = step;
 		}
 
 		@Override
-		public Collection<AColGroup> call() {
+		public Object call() {
 			try {
-				ArrayList<AColGroup> res = new ArrayList<>(_groups.size());
-				for(CompressedSizeInfoColGroup g : _groups)
-					res.add(compressColGroup(g));
-				return res;
+				for(int i = _off; i < _groups.size(); i += _step)
+					_ret[i] = compressColGroup(_groups.get(i));
+				return null;
 			}
 			catch(Exception e) {
 				e.printStackTrace();
@@ -705,22 +874,18 @@ public class ColGroupFactory {
 		}
 	}
 
-	static class readToMapDDCTask implements Callable<Boolean> {
+	private class readToMapDDCTask implements Callable<Boolean> {
 		private final int[] _colIndexes;
-		private final MatrixBlock _raw;
 		private final DblArrayCountHashMap _map;
-		private final CompressionSettings _cs;
 		private final AMapToData _data;
 		private final int _rl;
 		private final int _ru;
 		private final int _fill;
 
-		protected readToMapDDCTask(int[] colIndexes, MatrixBlock raw, DblArrayCountHashMap map, CompressionSettings cs,
-			AMapToData data, int rl, int ru, int fill) {
+		protected readToMapDDCTask(int[] colIndexes, DblArrayCountHashMap map, AMapToData data, int rl, int ru,
+			int fill) {
 			_colIndexes = colIndexes;
-			_raw = raw;
 			_map = map;
-			_cs = cs;
 			_data = data;
 			_rl = rl;
 			_ru = ru;
@@ -729,27 +894,7 @@ public class ColGroupFactory {
 
 		@Override
 		public Boolean call() {
-			return Boolean.valueOf(readToMapDDC(_colIndexes, _raw, _map, _cs, _data, _rl, _ru, _fill));
-		}
-	}
-
-	/**
-	 * Temp reuse object, to contain intermediates for compressing column groups that can be used by the same thread
-	 * again for subsequent compressions.
-	 */
-	static class Tmp {
-		private DoubleCountHashMap dblCountMap;
-
-		protected Tmp() {
-			dblCountMap = null;
-		}
-
-		protected DoubleCountHashMap getDblCountMap(int size) {
-			if(dblCountMap != null)
-				dblCountMap.reset(size);
-			else
-				dblCountMap = new DoubleCountHashMap(size);
-			return dblCountMap;
+			return Boolean.valueOf(readToMapDDC(_colIndexes, _map, _data, _rl, _ru, _fill));
 		}
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java
index bcb6a025ba..d4b5172705 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java
@@ -50,8 +50,9 @@ public class ColGroupIO {
 
 		// Read in how many colGroups there are
 		final int nColGroups = in.readInt();
-		if(LOG.isDebugEnabled())
-			LOG.debug("reading " + nColGroups + " ColGroups");
+		final boolean trace = LOG.isTraceEnabled();
+		if(trace)
+			LOG.trace("reading " + nColGroups + " ColGroups");
 		
 		// Allocate that amount into an ArrayList
 		final List<AColGroup> _colGroups = new ArrayList<>(nColGroups);
@@ -59,7 +60,7 @@ public class ColGroupIO {
 		// Read each ColGroup one at a time.
 		for(int i = 0; i < nColGroups; i++) {
 			ColGroupType ctype = ColGroupType.values()[in.readByte()];
-			if(LOG.isTraceEnabled())
+			if(trace)
 				LOG.trace("Reading in : " + ctype);
 			final AColGroup grp = constructColGroup(ctype, nRows);
 			grp.readFields(in);
@@ -120,8 +121,10 @@ public class ColGroupIO {
 				return new ColGroupSDCSingleZeros(nRows);
 			case SDCZeros:
 				return new ColGroupSDCZeros(nRows);
-			case PFOR:
-				return new ColGroupPFOR(nRows);
+			case SDCFOR:
+				return new ColGroupSDCFOR(nRows);
+			case DDCFOR:
+				return new ColGroupDDCFOR(nRows);
 			default:
 				throw new DMLRuntimeException("Unsupported ColGroup Type used:  " + ctype);
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDC.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDC.java
index ca72831f78..e0ebe986d5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDC.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDC.java
@@ -73,9 +73,13 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 	private ColGroupSDC(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple, AOffset offsets,
 		AMapToData data, int[] cachedCounts) {
 		super(colIndices, numRows, dict, cachedCounts);
-		if(data.getUnique() != dict.getNumberOfValues(colIndices.length))
+		if(data.getUnique() != dict.getNumberOfValues(colIndices.length)) {
+			if(data.getUnique() != data.getMax())
+				throw new DMLCompressionException(
+					"Invalid unique count compared to actual: " + data.getUnique() + " " + data.getMax());
 			throw new DMLCompressionException("Invalid construction of SDC group: number uniques: " + data.getUnique()
 				+ " vs." + dict.getNumberOfValues(colIndices.length));
+		}
 
 		_indexes = offsets;
 		_data = data;
@@ -88,17 +92,15 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 
 	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple,
 		AOffset offsets, AMapToData data, int[] cachedCounts) {
-		if(dict == null)
-			throw new NotImplementedException("Not implemented case where SDC ends up with empty dict");
-		else {
-			boolean allZero = true;
-			for(double d : defaultTuple)
-				allZero &= d == 0;
-			if(allZero)
-				return ColGroupSDCZeros.create(colIndices, numRows, dict, offsets, data, cachedCounts);
-			else
-				return new ColGroupSDC(colIndices, numRows, dict, defaultTuple, offsets, data, cachedCounts);
-		}
+		final boolean allZero = FORUtil.allZero(defaultTuple);
+		if(dict == null && allZero)
+			return new ColGroupEmpty(colIndices);
+		else if(dict == null)
+			return ColGroupSDCSingle.create(colIndices, numRows, null, defaultTuple, offsets, null);
+		else if(allZero)
+			return ColGroupSDCZeros.create(colIndices, numRows, dict, offsets, data, cachedCounts);
+		else
+			return new ColGroupSDC(colIndices, numRows, dict, defaultTuple, offsets, data, cachedCounts);
 	}
 
 	@Override
@@ -416,6 +418,11 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		return ColGroupSDCZeros.create(_colIndexes, _numRows, subtractedDict, _indexes, _data, getCounts());
 	}
 
+	public AColGroup subtractDefaultTuple() {
+		ADictionary subtractedDict = _dict.subtractTuple(_defaultTuple);
+		return ColGroupSDCZeros.create(_colIndexes, _numRows, subtractedDict, _indexes, _data, getCounts());
+	}
+
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
 		CM_COV_Object ret = super.centralMoment(op, nRows);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
similarity index 85%
rename from src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
index 4567e4621b..91b9ff22ae 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupPFOR.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
@@ -54,7 +54,7 @@ import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
  * with no modifications.
  * 
  */
-public class ColGroupPFOR extends AMorphingMMColGroup {
+public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	private static final long serialVersionUID = 3883228464052204203L;
 
@@ -72,55 +72,42 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	 * 
 	 * @param numRows Number of rows contained
 	 */
-	protected ColGroupPFOR(int numRows) {
+	protected ColGroupSDCFOR(int numRows) {
 		super(numRows);
 	}
 
-	private ColGroupPFOR(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
+	private ColGroupSDCFOR(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
 		int[] cachedCounts, double[] reference) {
 		super(colIndices, numRows, dict, cachedCounts);
 		if(data.getUnique() != dict.getNumberOfValues(colIndices.length))
 			throw new DMLCompressionException("Invalid construction of SDCZero group");
 		_data = data;
 		_indexes = indexes;
-		_zeros = allZero(reference);
+		_zeros = false;
 		_reference = reference;
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
+	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
 		int[] cachedCounts, double[] reference) {
-		if(dict == null) {
-			// either ColGroupEmpty or const
-			boolean allZero = true;
-			for(double d : reference)
-				if(d != 0) {
-					allZero = false;
-					break;
-				}
-
-			if(allZero)
-				return new ColGroupEmpty(colIndices);
-			else
-				return ColGroupConst.create(colIndices, reference);
-		}
-		return new ColGroupPFOR(colIndices, numRows, dict, indexes, data, cachedCounts, reference);
-	}
-
-	private final static boolean allZero(double[] in) {
-		for(double v : in)
-			if(v != 0)
-				return false;
-		return true;
+		final boolean allZero = FORUtil.allZero(reference);
+		if(allZero && dict == null)
+			return new ColGroupEmpty(colIndexes);
+		else if(dict == null)
+			return ColGroupConst.create(colIndexes, reference);
+		else if(allZero)
+			return ColGroupSDCZeros.create(colIndexes, numRows, dict, offsets, data, cachedCounts);
+		else
+			return new ColGroupSDCFOR(colIndexes, numRows, dict, offsets, data, cachedCounts, reference);
 	}
 
 	@Override
 	public CompressionType getCompType() {
-		return CompressionType.PFOR;
+		return CompressionType.SDCFOR;
 	}
 
 	@Override
 	public ColGroupType getColGroupType() {
-		return ColGroupType.PFOR;
+		return ColGroupType.SDCFOR;
 	}
 
 	@Override
@@ -128,20 +115,6 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		return _data.getCounts(counts);
 	}
 
-	private final double refSum() {
-		double ret = 0;
-		for(double d : _reference)
-			ret += d;
-		return ret;
-	}
-
-	private final double refSumSq() {
-		double ret = 0;
-		for(double d : _reference)
-			ret += d * d;
-		return ret;
-	}
-
 	@Override
 	protected void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
 		ColGroupSDC.computeRowSums(c, rl, ru, preAgg, _data, _indexes, _numRows);
@@ -167,9 +140,8 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		final double[] newRef = new double[_reference.length];
 		for(int i = 0; i < _reference.length; i++)
 			newRef[i] = op.executeScalar(_reference[i]);
-		if(op.fn instanceof Plus || op.fn instanceof Minus) {
+		if(op.fn instanceof Plus || op.fn instanceof Minus)
 			return create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
-		}
 		else if(op.fn instanceof Multiply || op.fn instanceof Divide) {
 			final ADictionary newDict = _dict.applyScalarOp(op);
 			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
@@ -182,9 +154,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		final double[] newRef = new double[_reference.length];
-		for(int i = 0; i < _reference.length; i++)
-			newRef[i] = op.fn.execute(_reference[i]);
+		final double[] newRef = FORUtil.unaryOperator(op, _reference);
 		final ADictionary newDict = _dict.applyUnaryOpWithReference(op, _reference, newRef);
 		return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
 	}
@@ -215,15 +185,15 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 			newRef[i] = op.fn.execute(_reference[i], v[_colIndexes[i]]);
 
 		if(op.fn instanceof Plus || op.fn instanceof Minus)// only edit reference
-			return new ColGroupPFOR(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), newRef);
 		else if(op.fn instanceof Multiply || op.fn instanceof Divide) {
 			// possible to simply process on dict and keep reference
 			final ADictionary newDict = _dict.binOpRight(op, v, _colIndexes);
-			return new ColGroupPFOR(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
 		}
 		else { // have to apply reference while processing
 			final ADictionary newDict = _dict.binOpRightWithReference(op, v, _colIndexes, _reference, newRef);
-			return new ColGroupPFOR(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
 		}
 	}
 
@@ -286,26 +256,26 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected double computeMxx(double c, Builtin builtin) {
-		return _dict.aggregateWithReference(c, builtin, _reference);
+		return _dict.aggregateWithReference(c, builtin, _reference, true);
 	}
 
 	@Override
 	protected void computeColMxx(double[] c, Builtin builtin) {
-		_dict.aggregateColsWithReference(c, builtin, _colIndexes, _reference);
+		_dict.aggregateColsWithReference(c, builtin, _colIndexes, _reference, true);
 	}
 
 	@Override
 	protected void computeSum(double[] c, int nRows) {
-		// trick,use normal sum
+		// trick, use normal sum
 		super.computeSum(c, nRows);
 		// and add all sum of reference multiplied with nrows.
-		final double refSum = refSum();
+		final double refSum = FORUtil.refSum(_reference);
 		c[0] += refSum * nRows;
 	}
 
 	@Override
 	public void computeColSums(double[] c, int nRows) {
-		// trick, use the normal sum
+		// trick, use normal sum
 		super.computeColSums(c, nRows);
 		// and add reference multiplied with number of rows.
 		for(int i = 0; i < _colIndexes.length; i++)
@@ -316,7 +286,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 	protected void computeSumSq(double[] c, int nRows) {
 		// square sum the dictionary.
 		c[0] += _dict.sumSqWithReference(getCounts(), _reference);
-		final double refSum = refSumSq();
+		final double refSum = FORUtil.refSumSq(_reference);
 		// Square sum of the reference values only for the rows that is not represented in the Offsets.
 		c[0] += refSum * (_numRows - _data.size());
 	}
@@ -369,7 +339,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected AColGroup sliceSingleColumn(int idx) {
-		ColGroupPFOR ret = (ColGroupPFOR) super.sliceSingleColumn(idx);
+		ColGroupSDCFOR ret = (ColGroupSDCFOR) super.sliceSingleColumn(idx);
 		// select values from double array.
 		ret._reference = new double[1];
 		ret._reference[0] = _reference[idx];
@@ -378,7 +348,7 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
-		ColGroupPFOR ret = (ColGroupPFOR) super.sliceMultiColumns(idStart, idEnd, outputCols);
+		ColGroupSDCFOR ret = (ColGroupSDCFOR) super.sliceMultiColumns(idStart, idEnd, outputCols);
 		final int len = idEnd - idStart;
 		ret._reference = new double[len];
 		for(int i = 0, ii = idStart; i < len; i++, ii++)
@@ -392,26 +362,11 @@ public class ColGroupPFOR extends AMorphingMMColGroup {
 		if(pattern == 0 && _zeros)
 			return true;
 		else if(Double.isNaN(pattern) || Double.isInfinite(pattern))
-			return containsInfOrNan(pattern) || _dict.containsValue(pattern);
+			return FORUtil.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
 		else
 			return _dict.containsValueWithReference(pattern, _reference);
 	}
 
-	private boolean containsInfOrNan(double pattern) {
-		if(Double.isNaN(pattern)) {
-			for(double d : _reference)
-				if(Double.isNaN(d))
-					return true;
-			return false;
-		}
-		else {
-			for(double d : _reference)
-				if(Double.isInfinite(d))
-					return true;
-			return false;
-		}
-	}
-
 	@Override
 	public long getNumberNonZeros(int nRows) {
 		final int[] counts = getCounts();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingle.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingle.java
index 7bfe370513..21bf950ddc 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingle.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingle.java
@@ -70,19 +70,24 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		_indexes = offsets;
 		_zeros = false;
 		_defaultTuple = defaultTuple;
+
+		if(_indexes == null)
+			throw new NullPointerException("null indexes is invalid for SDCSingle");
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple,
+	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, double[] defaultTuple,
 		AOffset offsets, int[] cachedCounts) {
-		boolean allZero = true;
-		for(double d : defaultTuple)
-			allZero &= d == 0;
+		final boolean allZero = FORUtil.allZero(defaultTuple);
 		if(dict == null && allZero)
-			return new ColGroupEmpty(colIndices);
+			return new ColGroupEmpty(colIndexes);
+		else if(dict == null) {
+			ADictionary ADict = new Dictionary(new double[colIndexes.length]);
+			return new ColGroupSDCSingle(colIndexes, numRows, ADict, defaultTuple, offsets, cachedCounts);
+		}
 		else if(allZero)
-			return ColGroupSDCSingleZeros.create(colIndices, numRows, dict, offsets, cachedCounts);
+			return ColGroupSDCSingleZeros.create(colIndexes, numRows, dict, offsets, cachedCounts);
 		else
-			return new ColGroupSDCSingle(colIndices, numRows, dict, defaultTuple, offsets, cachedCounts);
+			return new ColGroupSDCSingle(colIndexes, numRows, dict, defaultTuple, offsets, cachedCounts);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java
index eacb4e99b2..72259d25f1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java
@@ -93,11 +93,11 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
 		else
-			decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, values, it);
+			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, values, it);
 	}
 
 	@Override
-	public void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC, double[] values,
+	public void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC, double[] values,
 		AIterator it) {
 		final int last = _indexes.getOffsetToLast();
 		if(it == null || it.value() >= ru || rl > last)
@@ -341,42 +341,87 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 
 	@Override
 	public void preAggregateSparse(SparseBlock sb, double[] preAgg, int rl, int ru) {
-		final AIterator it = _indexes.getIterator();
-		if(rl == ru - 1) {
-			final int apos = sb.pos(rl);
-			final int alen = sb.size(rl) + apos;
-			final int[] aix = sb.indexes(rl);
-			final double[] avals = sb.values(rl);
-			final int offsetToLast = _indexes.getOffsetToLast();
+		final AOffsetIterator it = _indexes.getOffsetIterator();
+		if(rl == ru - 1)
+			preAggregateSparseSingleRow(sb, preAgg, rl, _indexes.getOffsetToLast(), it);
+		else
+			preAggregateSparseMultiRow(sb, preAgg, rl, ru, _indexes.getOffsetToLast(), it);
+	}
 
-			double ret = 0;
-			int j = apos;
+	private static void preAggregateSparseSingleRow(final SparseBlock sb, final double[] preAgg, final int r,
+		final int last, final AOffsetIterator it) {
+		if(sb.isEmpty(r))
+			return;
 
-			while(true) {
-				final int idx = aix[j];
-
-				if(idx == it.value()) {
-					ret += avals[j++];
-					if(j >= alen || it.value() >= offsetToLast)
-						break;
-					it.next();
-				}
-				else if(idx < it.value()) {
-					j++;
-					if(j >= alen)
-						break;
-				}
-				else {
-					if(it.value() >= offsetToLast)
-						break;
-					it.next();
-				}
+		final int apos = sb.pos(r);
+		final int alen = sb.size(r) + apos;
+		final int[] aix = sb.indexes(r);
+		final double[] avals = sb.values(r);
+
+		double ret = 0;
+		int i = it.value();
+		int j = apos;
+		while(i < last && j < alen) {
+			final int idx = aix[j];
+			if(idx == i) {
+				ret += avals[j++];
+				i = it.next();
+			}
+			else if(idx < i)
+				j++;
+			else
+				i = it.next();
+		}
+
+		while(j < alen && aix[j] < last)
+			j++;
+
+		if(j < alen && aix[j] == last)
+			ret += avals[j];
+
+		preAgg[0] = ret;
+	}
+
+	private static void preAggregateSparseMultiRow(final SparseBlock sb, final double[] preAgg, final int rl,
+		final int ru, final int last, final AOffsetIterator it) {
+
+		int i = it.value();
+		final int[] aOffs = new int[ru - rl];
+
+		// Initialize offsets for each row
+		for(int r = rl; r < ru; r++)
+			aOffs[r - rl] = sb.pos(r);
+
+		while(i < last) { // while we are not done iterating
+			for(int r = rl; r < ru; r++) {
+				final int off = r - rl;
+				int apos = aOffs[off]; // current offset
+				final int alen = sb.size(r) + sb.pos(r);
+				final int[] aix = sb.indexes(r);
+				while(apos < alen && aix[apos] < i)// increment all pointers to offset
+					apos++;
+
+				if(apos < alen && aix[apos] == i)
+					preAgg[off] += sb.values(r)[apos];
+				aOffs[off] = apos;
 			}
+			i = it.next();
+		}
 
-			preAgg[0] = ret;
+		// process final element
+		for(int r = rl; r < ru; r++) {
+			final int off = r - rl;
+			int apos = aOffs[off];
+			final int alen = sb.size(r) + sb.pos(r);
+			final int[] aix = sb.indexes(r);
+			while(apos < alen && aix[apos] < last)
+				apos++;
+
+			if(apos < alen && aix[apos] == last)
+				preAgg[off] += sb.values(r)[apos];
+			aOffs[off] = apos;
 		}
-		else
-			throw new NotImplementedException();
+
 	}
 
 	@Override
@@ -652,6 +697,16 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 		return e.getCost(nRows, nRowsScanned, nCols, nVals, _dict.getSparsity());
 	}
 
+	@Override
+	protected int getIndexesSize() {
+		return getCounts()[0];
+	}
+
+	@Override
+	protected int numRowsToMultiply() {
+		return getCounts()[0];
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCZeros.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCZeros.java
index f39c14a6af..52ee9efc52 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCZeros.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCZeros.java
@@ -36,9 +36,11 @@ import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
+import org.apache.sysds.runtime.functionobjects.Minus;
 import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
+import org.apache.sysds.runtime.matrix.operators.RightScalarOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
 
@@ -103,15 +105,17 @@ public class ColGroupSDCZeros extends ASDCZero {
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else
-			decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, values, it);
+		else{
+			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, values, it);
+			_indexes.cacheIterator(it, ru);
+		}
 	}
 
 	@Override
-	public final void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+	public final void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values, AIterator it) {
-			final int last = _indexes.getOffsetToLast();
-		if(it == null || it.value() >= ru  || rl > last)
+		final int last = _indexes.getOffsetToLast();
+		if(it == null || it.value() >= ru || rl > last)
 			return;
 		final boolean post = ru > last;
 		final boolean contiguous = db.isContiguous();
@@ -126,14 +130,12 @@ public class ColGroupSDCZeros extends ASDCZero {
 				decompressToDenseBlockDenseDictionaryPreSingleColOutContiguous(db, ru, offR, offC, values, it, _data);
 			else
 				decompressToDenseBlockDenseDictionaryPreSingleColContiguous(db, rl, ru, offR, offC, values, it);
-			_indexes.cacheIterator(it, ru);
 		}
 		else {
 			if(_colIndexes.length == db.getDim(1))
 				decompressToDenseBlockDenseDictionaryPreAllCols(db, rl, ru, offR, offC, values, it);
 			else
 				decompressToDenseBlockDenseDictionaryPreGeneric(db, rl, ru, offR, offC, values, it);
-			_indexes.cacheIterator(it, ru);
 		}
 	}
 
@@ -459,17 +461,13 @@ public class ColGroupSDCZeros extends ASDCZero {
 		boolean isSparseSafeOp = op.sparseSafe || val0 == 0;
 		if(isSparseSafeOp)
 			return create(_colIndexes, _numRows, _dict.applyScalarOp(op), _indexes, _data, getCachedCounts());
-		else if(op.fn instanceof Plus) {
-			double[] reference = new double[_colIndexes.length];
-			for(int i = 0; i < _colIndexes.length; i++)
-				reference[i] = val0;
-			return ColGroupPFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), reference);
+		else if(op.fn instanceof Plus || (op.fn instanceof Minus && op instanceof RightScalarOperator)) {
+			final double[] reference = FORUtil.createReference(_colIndexes.length, val0);
+			return ColGroupSDCFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), reference);
 		}
 		else {
-			ADictionary newDict = _dict.applyScalarOp(op);
-			double[] defaultTuple = new double[_colIndexes.length];
-			for(int i = 0; i < _colIndexes.length; i++)
-				defaultTuple[i] = val0;
+			final ADictionary newDict = _dict.applyScalarOp(op);
+			final double[] defaultTuple = FORUtil.createReference(_colIndexes.length, val0);
 			return ColGroupSDC.create(_colIndexes, _numRows, newDict, defaultTuple, _indexes, _data, getCachedCounts());
 		}
 	}
@@ -495,7 +493,7 @@ public class ColGroupSDCZeros extends ASDCZero {
 		}
 		else if(op.fn instanceof Plus) {
 			double[] reference = ColGroupUtils.binaryDefRowLeft(op, v, _colIndexes);
-			return ColGroupPFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), reference);
+			return ColGroupSDCFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), reference);
 		}
 		else {
 			ADictionary newDict = _dict.binOpLeft(op, v, _colIndexes);
@@ -514,7 +512,7 @@ public class ColGroupSDCZeros extends ASDCZero {
 		}
 		else if(op.fn instanceof Plus) {
 			double[] def = ColGroupUtils.binaryDefRowRight(op, v, _colIndexes);
-			return ColGroupPFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), def);
+			return ColGroupSDCFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), def);
 		}
 		else {
 			ADictionary newDict = _dict.binOpRight(op, v, _colIndexes);
@@ -640,6 +638,16 @@ public class ColGroupSDCZeros extends ASDCZero {
 		return e.getCost(nRows, nRowsScanned, nCols, nVals, _dict.getSparsity());
 	}
 
+	@Override
+	protected int getIndexesSize() {
+		return _data.size();
+	}
+
+	@Override
+	protected int numRowsToMultiply() {
+		return _data.size();
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
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 7adbbd4041..862ee09bb1 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
@@ -415,7 +415,7 @@ public class ColGroupUncompressed extends AColGroup {
 			return; // product
 		}
 
-		LOG.warn("Inefficient Unary Aggregate because of Uncompressed ColumnGroup");
+		// LOG.warn("Inefficient Unary Aggregate because of Uncompressed ColumnGroup");
 		// Since usually Uncompressed column groups are used in case of extreme sparsity, it is fine
 		// using a slice, since we dont allocate extra just extract the pointers to the sparse rows.
 		MatrixBlock tmpData = _data.slice(rl, ru - 1, false);
@@ -775,7 +775,7 @@ public class ColGroupUncompressed extends AColGroup {
 	@Override
 	public AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows) {
 		MatrixBlock nd = LibMatrixReorg.rexpand(_data, new MatrixBlock(), max, false, cast, ignore, 1);
-		return create(nd, _colIndexes);
+		return create(nd, Util.genColsIndices(max));
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/FORUtil.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/FORUtil.java
new file mode 100644
index 0000000000..1124c6b882
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/FORUtil.java
@@ -0,0 +1,75 @@
+/*
+ * 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.colgroup;
+
+import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
+
+public class FORUtil {
+
+	protected static final double refSum(double[] reference) {
+		double ret = 0;
+		for(double d : reference)
+			ret += d;
+		return ret;
+	}
+
+	protected static final double refSumSq(double[] reference) {
+		double ret = 0;
+		for(double d : reference)
+			ret += d * d;
+		return ret;
+	}
+
+	protected final static boolean allZero(double[] in) {
+		for(double v : in)
+			if(v != 0)
+				return false;
+		return true;
+	}
+
+	protected final static boolean containsInfOrNan(double pattern, double[] reference) {
+		if(Double.isNaN(pattern)) {
+			for(double d : reference)
+				if(Double.isNaN(d))
+					return true;
+			return false;
+		}
+		else {
+			for(double d : reference)
+				if(Double.isInfinite(d))
+					return true;
+			return false;
+		}
+	}
+
+	protected final static double[] createReference(int nCol, double val) {
+		double[] reference = new double[nCol];
+		for(int i = 0; i < nCol; i++)
+			reference[i] = val;
+		return reference;
+	}
+
+	protected final static double[] unaryOperator(UnaryOperator op, double[] reference) {
+		final double[] newRef = new double[reference.length];
+		for(int i = 0; i < reference.length; i++)
+			newRef[i] = op.fn.execute(reference[i]);
+		return newRef;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/ADictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/ADictionary.java
index 8b48820245..b3320fb277 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/ADictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/ADictionary.java
@@ -82,9 +82,10 @@ public abstract class ADictionary implements Serializable {
 	 * @param init      The initial value, in cases such as Max value this could be -infinity.
 	 * @param fn        The function to apply to the values
 	 * @param reference The reference offset to each value in the dictionary
+	 * @param def       If the reference should be treated as an instance of only as reference
 	 * @return The aggregated value as a double.
 	 */
-	public abstract double aggregateWithReference(double init, Builtin fn, double[] reference);
+	public abstract double aggregateWithReference(double init, Builtin fn, double[] reference, boolean def);
 
 	/**
 	 * Aggregate all entries in the rows.
@@ -132,8 +133,10 @@ public abstract class ADictionary implements Serializable {
 	 * @param fn         The function to apply to individual columns
 	 * @param reference  The reference offset values to add to each cell.
 	 * @param colIndexes The mapping to the target columns from the individual columns
+	 * @param def        If the reference should be treated as a tuple as well
 	 */
-	public abstract void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference);
+	public abstract void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference,
+		boolean def);
 
 	/**
 	 * Allocate a new dictionary and applies the scalar operation on each cell of the to then return the new dictionary.
@@ -221,6 +224,15 @@ public abstract class ADictionary implements Serializable {
 	 */
 	public abstract ADictionary binOpRight(BinaryOperator op, double[] v, int[] colIndexes);
 
+	/**
+	 * Apply binary row operation on the right side.
+	 * 
+	 * @param op The operation to this dictionary
+	 * @param v  The values to apply on the dictionary (same number of cols as the dictionary)
+	 * @return A new dictionary containing the updated values.
+	 */
+	public abstract ADictionary binOpRight(BinaryOperator op, double[] v);
+
 	/**
 	 * Apply the binary operator such that each value is offset by the reference before application. Then put the result
 	 * into the new dictionary, but offset it by the new reference.
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictLibMatrixMult.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictLibMatrixMult.java
index a788cf2f0c..2b2e41600a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictLibMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictLibMatrixMult.java
@@ -344,9 +344,65 @@ public class DictLibMatrixMult {
 
 	protected static void MMToUpperTriangleSparseDense(SparseBlock left, double[] right, int[] rowsLeft, int[] colsRight,
 		MatrixBlock result) {
+		final int loc = location(rowsLeft, colsRight);
+		if(loc < 0)
+			MMToUpperTriangleSparseDenseAllUpperTriangle(left, right, rowsLeft, colsRight, result);
+		else if(loc > 0)
+			MMToUpperTriangleSparseDenseAllLowerTriangle(left, right, rowsLeft, colsRight, result);
+		else
+			MMToUpperTriangleSparseDenseDiagonal(left, right, rowsLeft, colsRight, result);
+	}
+
+	protected static void MMToUpperTriangleSparseDenseAllUpperTriangle(SparseBlock left, double[] right, int[] rowsLeft,
+		int[] colsRight, MatrixBlock result) {
 		final double[] resV = result.getDenseBlockValues();
 		final int commonDim = Math.min(left.numRows(), right.length / colsRight.length);
 		final int resCols = result.getNumColumns();
+		for(int i = 0; i < commonDim; i++) {
+			if(left.isEmpty(i))
+				continue;
+			final int apos = left.pos(i);
+			final int alen = left.size(i) + apos;
+			final int[] aix = left.indexes(i);
+			final double[] leftVals = left.values(i);
+			final int offRight = i * colsRight.length;
+			for(int k = apos; k < alen; k++) {
+				final int rowOut = rowsLeft[aix[k]];
+				final double vl = leftVals[k];
+				for(int j = 0; j < colsRight.length; j++)
+					resV[colsRight[j] * resCols + rowOut] += vl * right[offRight + j];
+			}
+		}
+	}
+
+	protected static void MMToUpperTriangleSparseDenseAllLowerTriangle(SparseBlock left, double[] right, int[] rowsLeft,
+		int[] colsRight, MatrixBlock result) {
+		final double[] resV = result.getDenseBlockValues();
+		final int commonDim = Math.min(left.numRows(), right.length / colsRight.length);
+		final int resCols = result.getNumColumns();
+		for(int i = 0; i < commonDim; i++) {
+			if(left.isEmpty(i))
+				continue;
+			final int apos = left.pos(i);
+			final int alen = left.size(i) + apos;
+			final int[] aix = left.indexes(i);
+			final double[] leftVals = left.values(i);
+			final int offRight = i * colsRight.length;
+			for(int k = apos; k < alen; k++) {
+				final int rowOut = rowsLeft[aix[k]] * resCols;
+				final double vl = leftVals[k];
+				for(int j = 0; j < colsRight.length; j++)
+					resV[colsRight[j] + rowOut] += vl * right[offRight + j];
+			}
+		}
+	}
+
+	protected static void MMToUpperTriangleSparseDenseDiagonal(SparseBlock left, double[] right, int[] rowsLeft,
+		int[] colsRight, MatrixBlock result) {
+		final double[] resV = result.getDenseBlockValues();
+		final int commonDim = Math.min(left.numRows(), right.length / colsRight.length);
+		final int resCols = result.getNumColumns();
+		// generic
 		for(int i = 0; i < commonDim; i++) {
 			if(left.isEmpty(i))
 				continue;
@@ -409,11 +465,9 @@ public class DictLibMatrixMult {
 			for(int i = 0; i < rowsLeft.length; i++) {
 				final int rowOut = rowsLeft[i] * resCols;
 				final double vl = left[offL + i];
-				if(vl != 0) {
-					for(int j = 0; j < colsRight.length; j++)
-						resV[colsRight[j] + rowOut] += vl * right[offR + j];
+				for(int j = 0; j < colsRight.length; j++)
+					resV[colsRight[j] + rowOut] += vl * right[offR + j];
 
-				}
 			}
 		}
 	}
@@ -429,12 +483,10 @@ public class DictLibMatrixMult {
 			for(int i = 0; i < rowsLeft.length; i++) {
 				final int rowOut = rowsLeft[i];
 				final double vl = left[offL + i];
-				if(vl != 0) {
-					for(int j = 0; j < colsRight.length; j++) {
-						final double vr = right[offR + j];
-						final int colOut = colsRight[j];
-						addToUpperTriangle(resCols, rowOut, colOut, resV, vl * vr);
-					}
+				for(int j = 0; j < colsRight.length; j++) {
+					final double vr = right[offR + j];
+					final int colOut = colsRight[j];
+					addToUpperTriangle(resCols, rowOut, colOut, resV, vl * vr);
 				}
 			}
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/Dictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/Dictionary.java
index 6e056934ca..a44fd1cb19 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/Dictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/Dictionary.java
@@ -87,14 +87,16 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public double aggregateWithReference(double init, Builtin fn, double[] reference) {
+	public double aggregateWithReference(double init, Builtin fn, double[] reference, boolean def) {
 		final int nCol = reference.length;
 		double ret = init;
 		for(int i = 0; i < _values.length; i++)
 			ret = fn.execute(ret, _values[i] + reference[i % nCol]);
 
-		for(int i = 0; i < nCol; i++)
-			ret = fn.execute(ret, reference[i]);
+		if(def)
+			for(int i = 0; i < nCol; i++)
+				ret = fn.execute(ret, reference[i]);
+
 		return ret;
 	}
 
@@ -214,6 +216,18 @@ public class Dictionary extends ADictionary {
 		return new Dictionary(retVals);
 	}
 
+	@Override
+	public Dictionary binOpRight(BinaryOperator op, double[] v){
+		final ValueFunction fn = op.fn;
+		final double[] retVals = new double[_values.length];
+		final int len = size();
+		final int lenV = v.length;
+		for(int i = 0; i < len; i++)
+			retVals[i] = fn.execute(_values[i], v[i % lenV]);
+		return new Dictionary(retVals);
+	}
+
+
 	@Override
 	public Dictionary binOpRightWithReference(BinaryOperator op, double[] v, int[] colIndexes, double[] reference,
 		double[] newReference) {
@@ -653,14 +667,15 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference) {
+	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference, boolean def) {
 		final int nCol = reference.length;
 		final int rlen = _values.length / nCol;
 		for(int k = 0; k < rlen; k++)
 			for(int j = 0, valOff = k * nCol; j < nCol; j++)
 				c[colIndexes[j]] = fn.execute(c[colIndexes[j]], _values[valOff + j] + reference[j]);
-		for(int i = 0; i < nCol; i++)
-			c[colIndexes[i]] = fn.execute(c[colIndexes[i]], reference[i]);
+		if(def)
+			for(int i = 0; i < nCol; i++)
+				c[colIndexes[i]] = fn.execute(c[colIndexes[i]], reference[i]);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictionaryFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictionaryFactory.java
index 96c11191dd..be1b095679 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictionaryFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/DictionaryFactory.java
@@ -32,6 +32,7 @@ import org.apache.sysds.runtime.compress.bitmap.Bitmap;
 import org.apache.sysds.runtime.compress.bitmap.MultiColBitmap;
 import org.apache.sysds.runtime.compress.utils.DArrCounts;
 import org.apache.sysds.runtime.compress.utils.DblArrayCountHashMap;
+import org.apache.sysds.runtime.compress.utils.DoubleCountHashMap;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
@@ -99,18 +100,6 @@ public interface DictionaryFactory {
 			LOG.error("Failed to create dictionary: ", e);
 			return null;
 		}
-
-	}
-
-	public static ADictionary createDelta(DblArrayCountHashMap map, int nCols, boolean addZeroTuple) {
-		final ArrayList<DArrCounts> vals = map.extractValues();
-		final int nVals = vals.size();
-		final double[] resValues = new double[(nVals + (addZeroTuple ? 1 : 0)) * nCols];
-		for(int i = 0; i < nVals; i++) {
-			final DArrCounts dac = vals.get(i);
-			System.arraycopy(dac.key.getData(), 0, resValues, dac.id * nCols, nCols);
-		}
-		return new DeltaDictionary(resValues, nCols);
 	}
 
 	public static ADictionary create(ABitmap ubm) {
@@ -236,4 +225,9 @@ public interface DictionaryFactory {
 
 		return new Dictionary(resValues);
 	}
+
+	public static ADictionary create(DoubleCountHashMap map) {
+		final double[] resValues = map.getDictionary();
+		return new Dictionary(resValues);
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java
index 482342141b..b3baa7db56 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java
@@ -64,12 +64,11 @@ public class MatrixBlockDictionary extends ADictionary {
 	}
 
 	public MatrixBlockDictionary(MatrixBlock data, int nCol) {
-
-		if(data.isEmpty())
+		data.examSparsity(true);
+		if(data.getNumColumns() != nCol)
+			throw new DMLCompressionException("Invalid number of columns in dictionary");
+		else if(data.isEmpty())
 			throw new DMLCompressionException("Invalid construction of empty dictionary");
-		else if(data.getNumColumns() != nCol)
-			throw new DMLCompressionException(
-				"Invalid construction expected nCol: " + nCol + " but matrix block contains: " + data.getNumColumns());
 		else if(data.isInSparseFormat() && data.getSparseBlock() instanceof SparseBlockMCSR) {
 			SparseBlock csr = SparseBlockFactory.copySparseBlock(SparseBlock.Type.CSR, data.getSparseBlock(), false);
 			data.setSparseBlock(csr);
@@ -94,7 +93,8 @@ public class MatrixBlockDictionary extends ADictionary {
 	public double[] getValues() {
 		if(_data.isInSparseFormat()) {
 			LOG.warn("Inefficient call to getValues for a MatrixBlockDictionary because it was sparse");
-			_data.sparseToDense();
+			throw new DMLCompressionException("Should not call this function");
+			// _data.sparseToDense();
 		}
 		return _data.getDenseBlockValues();
 	}
@@ -131,13 +131,14 @@ public class MatrixBlockDictionary extends ADictionary {
 	}
 
 	@Override
-	public double aggregateWithReference(double init, Builtin fn, double[] reference) {
+	public double aggregateWithReference(double init, Builtin fn, double[] reference, boolean def) {
 		final int nCol = reference.length;
 		final int nRows = _data.getNumRows();
 		double ret = init;
 
-		for(int i = 0; i < nCol; i++)
-			ret = fn.execute(ret, reference[i]);
+		if(def)
+			for(int i = 0; i < nCol; i++)
+				ret = fn.execute(ret, reference[i]);
 
 		if(!_data.isEmpty() && _data.isInSparseFormat()) {
 			final SparseBlock sb = _data.getSparseBlock();
@@ -153,6 +154,13 @@ public class MatrixBlockDictionary extends ADictionary {
 					ret = fn.execute(ret, v);
 				}
 			}
+			if(!def) {
+				final int[] nnz = LibMatrixReorg.countNnzPerColumn(_data);
+				for(int i = 0; i < nnz.length; i++)
+					if(nnz[i] < nRows)
+						ret = fn.execute(ret, reference[i]);
+
+			}
 		}
 		else if(!_data.isEmpty()) {
 			final double[] values = _data.getDenseBlockValues();
@@ -300,15 +308,16 @@ public class MatrixBlockDictionary extends ADictionary {
 	}
 
 	@Override
-	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference) {
+	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference, boolean def) {
 		final int nCol = _data.getNumColumns();
 		final int nRow = _data.getNumRows();
 
-		for(int j = 0; j < colIndexes.length; j++) {
-			final int idx = colIndexes[j];
-			c[idx] = fn.execute(c[idx], reference[j]);
-		}
-		if(!_data.isEmpty() && _data.isInSparseFormat()) {
+		if(def)
+			for(int j = 0; j < colIndexes.length; j++) {
+				final int idx = colIndexes[j];
+				c[idx] = fn.execute(c[idx], reference[j]);
+			}
+		if(_data.isInSparseFormat()) {
 			final SparseBlock sb = _data.getSparseBlock();
 			for(int i = 0; i < nRow; i++) {
 				if(sb.isEmpty(i))
@@ -323,8 +332,16 @@ public class MatrixBlockDictionary extends ADictionary {
 					c[idx] = fn.execute(c[idx], avals[k] + reference[aix[k]]);
 				}
 			}
+			if(!def) {
+				final int[] nnz = LibMatrixReorg.countNnzPerColumn(_data);
+				for(int i = 0; i < nnz.length; i++)
+					if(nnz[i] < nRow) {
+						final int idx = colIndexes[i];
+						c[idx] = fn.execute(c[idx], reference[i]);
+					}
+			}
 		}
-		else if(!_data.isEmpty()) {
+		else {
 			final double[] values = _data.getDenseBlockValues();
 			int off = 0;
 			for(int k = 0; k < nRow; k++) {
@@ -468,11 +485,19 @@ public class MatrixBlockDictionary extends ADictionary {
 	}
 
 	@Override
-	public ADictionary binOpRight(BinaryOperator op, double[] v, int[] colIndexes) {
+	public MatrixBlockDictionary binOpRight(BinaryOperator op, double[] v, int[] colIndexes) {
 		MatrixBlock rowVector = Util.extractValues(v, colIndexes);
 		return new MatrixBlockDictionary(_data.binaryOperations(op, rowVector, null), _data.getNumColumns());
 	}
 
+	@Override
+	public MatrixBlockDictionary binOpRight(BinaryOperator op, double[] v) {
+		MatrixBlock rowVector = new MatrixBlock(1, v.length, v);
+		MatrixBlock ret = _data.binaryOperations(op, rowVector, null);
+		ret.examSparsity(true);
+		return new MatrixBlockDictionary(ret, _data.getNumColumns());
+	}
+
 	@Override
 	public Dictionary binOpRightWithReference(BinaryOperator op, double[] v, int[] colIndexes, double[] reference,
 		double[] newReference) {
@@ -897,7 +922,9 @@ public class MatrixBlockDictionary extends ADictionary {
 
 	@Override
 	public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
-		MatrixBlock retBlock = _data.slice(0, _data.getNumRows() - 1, idxStart, idxEnd - 1);
+		final MatrixBlock retBlock = _data.slice(0, _data.getNumRows() - 1, idxStart, idxEnd - 1);
+		if(retBlock.isEmpty())
+			return null;
 		return new MatrixBlockDictionary(retBlock, idxEnd - idxStart);
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index db15d6d76e..cc7e500ed4 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -105,7 +105,7 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public double aggregateWithReference(double init, Builtin fn, double[] reference) {
+	public double aggregateWithReference(double init, Builtin fn, double[] reference, boolean def) {
 		throw new NotImplementedException();
 	}
 
@@ -424,7 +424,7 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference) {
+	public void aggregateColsWithReference(double[] c, Builtin fn, int[] colIndexes, double[] reference, boolean def) {
 		throw new NotImplementedException();
 	}
 
@@ -474,6 +474,11 @@ public class QDictionary extends ADictionary {
 		throw new NotImplementedException();
 	}
 
+	@Override
+	public ADictionary binOpRight(BinaryOperator op, double[] v) {
+		throw new NotImplementedException();
+	}
+
 	@Override
 	public ADictionary binOpLeftWithReference(BinaryOperator op, double[] v, int[] colIndexes, double[] reference,
 		double[] newReference) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/AMapToData.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/AMapToData.java
index 7b841e46cc..189f901233 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/AMapToData.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/AMapToData.java
@@ -22,10 +22,12 @@ package org.apache.sysds.runtime.compress.colgroup.mapping;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.BitSet;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
@@ -305,8 +307,9 @@ public abstract class AMapToData implements Serializable {
 	 * @param cu      The column in m to end at (not inclusive)
 	 * @param indexes The Offset Indexes to iterate through
 	 */
-	public abstract void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu,
-		AOffset indexes);
+	public final void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, AOffset indexes) {
+		indexes.preAggregateDenseMap(m, preAV, rl, ru, cl, cu, getUnique(), this);
+	}
 
 	/**
 	 * PreAggregate the SparseBlock in the range of rows given.
@@ -317,7 +320,51 @@ public abstract class AMapToData implements Serializable {
 	 * @param ru      The row to end at (not inclusive)
 	 * @param indexes The Offset Indexes to iterate through
 	 */
-	public abstract void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes);
+	public final void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes) {
+		indexes.preAggregateSparseMap(sb, preAV, rl, ru, getUnique(), this);
+	}
+
+	/**
+	 * PreAggregate the sparseblock in the range of rows given.
+	 * 
+	 * @param sb    Sparse block to preAggregate from
+	 * @param preAV Pre aggregate target
+	 * @param rl    row index in sb
+	 * @param ru    upper row index in sp (not inclusive)
+	 */
+	public final void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru) {
+		if(rl == ru - 1)
+			preAggregateSparseSingleRow(sb, preAV, rl);
+		else
+			preAggregateSparseMultiRow(sb, preAV, rl, ru);
+	}
+
+	private final void preAggregateSparseSingleRow(final SparseBlock sb, final double[] preAV, final int r) {
+		if(sb.isEmpty(r))
+			return;
+		final int apos = sb.pos(r);
+		final int alen = sb.size(r) + apos;
+		final int[] aix = sb.indexes(r);
+		final double[] avals = sb.values(r);
+		for(int j = apos; j < alen; j++)
+			preAV[getIndex(aix[j])] += avals[j];
+	}
+
+	private final void preAggregateSparseMultiRow(final SparseBlock sb, final double[] preAV, final int rl,
+		final int ru) {
+		final int unique = getUnique();
+		for(int r = rl; r < ru; r++) {
+			if(sb.isEmpty(r))
+				continue;
+			final int apos = sb.pos(r);
+			final int alen = sb.size(r) + apos;
+			final int[] aix = sb.indexes(r);
+			final double[] avals = sb.values(r);
+			final int off = unique * (r - rl);
+			for(int j = apos; j < alen; j++)
+				preAV[off + getIndex(aix[j])] += avals[j];
+		}
+	}
 
 	/**
 	 * Get the number of counts of each unique value contained in this map. Note that in the case the mapping is shorter
@@ -326,13 +373,26 @@ public abstract class AMapToData implements Serializable {
 	 * @param counts The object to return.
 	 * @return the Counts
 	 */
-	public int[] getCounts(int[] counts) {
-		final int sz = size();
-		for(int i = 0; i < sz; i++)
-			counts[getIndex(i)]++;
+	public final int[] getCounts(int[] counts) {
+		count(counts);
+
+		if(counts[counts.length - 1] == 0) {
+			int actualUnique = counts.length;
+			for(; actualUnique > 1; actualUnique--) {
+				if(counts[actualUnique - 1] > 0)
+					break;
+			}
+			throw new DMLCompressionException("Invalid number unique expected: " + counts.length + " but is actually: "
+				+ actualUnique + " type: " + getType());
+		}
 		return counts;
 	}
 
+	protected void count(int[] ret) {
+		for(int i = 0; i < size(); i++)
+			ret[getIndex(i)]++;
+	}
+
 	/**
 	 * PreAggregate into dictionary with two sides of DDC.
 	 * 
@@ -372,7 +432,6 @@ public abstract class AMapToData implements Serializable {
 	protected void preAggregateDDC_DDCMultiCol(AMapToData tm, ADictionary td, double[] v, int nCol) {
 		final int sz = size();
 		final int h = sz % 8;
-
 		for(int r = 0; r < h; r++)
 			td.addToEntry(v, tm.getIndex(r), getIndex(r), nCol);
 
@@ -646,11 +705,42 @@ public abstract class AMapToData implements Serializable {
 	 * @param d Map to copy all values into.
 	 */
 	public void copy(AMapToData d) {
-		final int sz = size();
-		for(int i = 0; i < sz; i++)
-			set(i, d.getIndex(i));
+		if(d.nUnique == 1)
+			return;
+		else if(d instanceof MapToBit)
+			copyBit((MapToBit) d);
+		else if(d instanceof MapToInt)
+			copyInt((MapToInt) d);
+		else {
+			final int sz = size();
+			for(int i = 0; i < sz; i++)
+				set(i, d.getIndex(i));
+		}
+	}
+
+	protected void copyInt(MapToInt d) {
+		copyInt(d.getData());
 	}
 
+	protected void copyBit(MapToBit d) {
+		copyBit(d.getData());
+	}
+
+	public abstract void copyInt(int[] d);
+
+	public abstract void copyBit(BitSet d);
+
+	public int getMax() {
+		int m = -1;
+		for(int i = 0; i < size(); i++) {
+			int v = getIndex(i);
+			m = v > m ? v : m;
+		}
+		return m;
+	}
+
+	public abstract AMapToData resize(int unique);
+
 	@Override
 	public String toString() {
 		final int sz = size();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToBit.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToBit.java
index fa4091d5d2..270eaf3533 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToBit.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToBit.java
@@ -26,9 +26,6 @@ import java.util.BitSet;
 
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
-import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
-import org.apache.sysds.runtime.data.SparseBlock;
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
 public class MapToBit extends AMapToData {
@@ -38,7 +35,7 @@ public class MapToBit extends AMapToData {
 	private final BitSet _data;
 	private final int _size;
 
-	protected MapToBit(int size){
+	protected MapToBit(int size) {
 		this(2, size);
 	}
 
@@ -52,6 +49,14 @@ public class MapToBit extends AMapToData {
 		super(unique);
 		_data = d;
 		_size = size;
+		if(_data.isEmpty()) {
+			unique = 1;
+			LOG.warn("Empty bit set should not happen");
+		}
+	}
+
+	protected BitSet getData() {
+		return _data;
 	}
 
 	@Override
@@ -80,22 +85,13 @@ public class MapToBit extends AMapToData {
 		return size;
 	}
 
-	@Override
-	public long getExactSizeOnDisk() {
-		final int dSize = _data.size();
-		long size = 1 + 4 + 4 + 4; // base variables
-		size += (dSize / 64) * 8; // all longs except last
-		// size += (dSize % 64 == 0 ? 0 : 8); // last long
-		return size;
-	}
-
 	@Override
 	public void set(int n, int v) {
 		_data.set(n, v == 1);
 	}
 
 	@Override
-	public int setAndGet(int n, int v){
+	public int setAndGet(int n, int v) {
 		_data.set(n, v == 1);
 		return 1;
 	}
@@ -114,11 +110,17 @@ public class MapToBit extends AMapToData {
 			_data.clear();
 	}
 
+	@Override
+	public long getExactSizeOnDisk() {
+		long size = 1 + 4 + 4; // base variables
+		size += _data.toLongArray().length * 8;
+		return size;
+	}
+
 	@Override
 	public void write(DataOutput out) throws IOException {
 		long[] internals = _data.toLongArray();
 		out.writeByte(MAP_TYPE.BIT.ordinal());
-		out.writeInt(getUnique());
 		out.writeInt(_size);
 		out.writeInt(internals.length);
 		for(int i = 0; i < internals.length; i++)
@@ -126,23 +128,11 @@ public class MapToBit extends AMapToData {
 	}
 
 	protected static MapToBit readFields(DataInput in) throws IOException {
-		int unique = in.readInt();
 		int size = in.readInt();
 		long[] internalLong = new long[in.readInt()];
 		for(int i = 0; i < internalLong.length; i++)
 			internalLong[i] = in.readLong();
-
-		return new MapToBit(unique, BitSet.valueOf(internalLong), size);
-	}
-
-	@Override
-	public void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, AOffset indexes) {
-		indexes.preAggregateDenseMap(m, preAV, rl, ru, cl, cu, getUnique(), _data);
-	}
-
-	@Override
-	public void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes) {
-		indexes.preAggregateSparseMap(sb, preAV, rl, ru, getUnique(), _data);
+		return new MapToBit(2, BitSet.valueOf(internalLong), size);
 	}
 
 	@Override
@@ -151,17 +141,10 @@ public class MapToBit extends AMapToData {
 	}
 
 	@Override
-	public int[] getCounts(int[] counts) {
+	protected void count(int[] ret) {
 		final int sz = size();
-
-		if(counts.length == 1)
-			counts[0] = sz;
-		else {
-			counts[1] = _data.cardinality();
-			counts[0] = sz - counts[1];
-		}
-
-		return counts;
+		ret[1] = _data.cardinality();
+		ret[0] = sz - ret[1];
 	}
 
 	@Override
@@ -207,10 +190,38 @@ public class MapToBit extends AMapToData {
 		}
 	}
 
-	public boolean isEmpty(){
+	public boolean isEmpty() {
 		return _data.isEmpty();
 	}
 
+	@Override
+	public void copy(AMapToData d) {
+		if(d instanceof MapToBit)
+			copyBit((MapToBit) d);
+		else if(d instanceof MapToInt)
+			copyInt((MapToInt) d);
+		else {
+			final int sz = size();
+			for(int i = 0; i < sz; i++)
+				if(d.getIndex(i) != 0)
+					_data.set(i);
+		}
+	}
+
+	@Override
+	public void copyInt(int[] d) {
+		// start from end because bitset is allocating based on last bit set.
+		for(int i = d.length - 1; i > -1; i--)
+			if(d[i] != 0)
+				_data.set(i);
+	}
+
+	@Override
+	public void copyBit(BitSet d) {
+		_data.clear();
+		_data.or(d);
+	}
+
 	private static class JoinBitSets {
 		int tt = 0;
 		int ft = 0;
@@ -253,4 +264,12 @@ public class MapToBit extends AMapToData {
 			ff += size - (longest * 64); // remainder
 		}
 	}
+
+	@Override
+	public AMapToData resize(int unique) {
+		if(unique <= 1)
+			return new MapToZero(size());
+		else
+			return this;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToByte.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToByte.java
index 17eef52f74..deba11577a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToByte.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToByte.java
@@ -23,11 +23,9 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.BitSet;
 
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
-import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
-import org.apache.sysds.runtime.data.SparseBlock;
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
 public class MapToByte extends AMapToData {
@@ -133,14 +131,22 @@ public class MapToByte extends AMapToData {
 	}
 
 	@Override
-	public void copy(AMapToData d) {
-		if(d instanceof MapToChar) {
-			char[] dd = ((MapToChar) d).getChars();
-			for(int i = 0; i < size(); i++)
-				_data[i] = (byte) dd[i];
+	public void copyInt(int[] d) {
+		for(int i = 0; i < _data.length; i++)
+			_data[i] = (byte) d[i];
+	}
+
+	@Override
+	public void copyBit(BitSet d) {
+		for(int i = d.nextSetBit(0); i >= 0; i = d.nextSetBit(i + 1)) {
+			_data[i] = 1;
 		}
-		else
-			super.copy(d);
+	}
+
+	@Override
+	public void count(int[] ret) {
+		for(int i = 0; i < _data.length; i++)
+			ret[_data[i] & 0xFF]++;
 	}
 
 	@Override
@@ -162,17 +168,28 @@ public class MapToByte extends AMapToData {
 	}
 
 	@Override
-	public final void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, AOffset indexes) {
-		indexes.preAggregateDenseMap(m, preAV, rl, ru, cl, cu, getUnique(), _data);
-	}
-
-	@Override
-	public void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes) {
-		indexes.preAggregateSparseMap(sb, preAV, rl, ru, getUnique(), _data);
+	public int getUpperBoundValue() {
+		return 255;
 	}
 
 	@Override
-	public int getUpperBoundValue() {
-		return 255;
+	public AMapToData resize(int unique){
+		final int size = _data.length;
+		AMapToData ret;
+		if(unique <= 1)
+			return new MapToZero(size);
+		else if(unique == 2 && size > 32)
+			ret = new MapToBit(unique, size);
+		else if (unique <= 127){
+			ret = toUByte();
+			ret.setUnique(unique);
+			return ret;
+		}
+		else{
+			setUnique(unique);
+			return this;
+		}
+		ret.copy(this);
+		return ret;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToChar.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToChar.java
index a877abdaa3..4d56ca871c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToChar.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToChar.java
@@ -23,11 +23,9 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.BitSet;
 
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
-import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
-import org.apache.sysds.runtime.data.SparseBlock;
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
 public class MapToChar extends AMapToData {
@@ -173,18 +171,48 @@ public class MapToChar extends AMapToData {
 		}
 	}
 
+
 	@Override
-	public void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, AOffset indexes) {
-		indexes.preAggregateDenseMap(m, preAV, rl, ru, cl, cu, getUnique(), _data);
+	public int getUpperBoundValue() {
+		return Character.MAX_VALUE;
 	}
 
 	@Override
-	public void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes) {
-		indexes.preAggregateSparseMap(sb, preAV, rl, ru, getUnique(), _data);
+	public void copyInt(int[] d){
+		for(int i = 0; i < _data.length; i++)
+			_data[i] = (char)d[i];
 	}
 
 	@Override
-	public int getUpperBoundValue() {
-		return Character.MAX_VALUE;
+	public void copyBit(BitSet d) {
+		for(int i = d.nextSetBit(0); i >= 0; i = d.nextSetBit(i + 1)) {
+			_data[i] = 1;
+		}
+	}
+
+	@Override
+	public void count(int[] ret){
+		for(int i = 0; i < _data.length; i++)
+			ret[_data[i]]++; 
+	}
+
+	@Override
+	public AMapToData resize(int unique){
+		final int size = _data.length;
+		AMapToData ret;
+		if(unique <= 1)
+			return new MapToZero(size);
+		else if(unique == 2 && size > 32)
+			ret = new MapToBit(unique, size);
+		else if (unique <= 127)
+			ret = new MapToUByte(unique, size);
+		else if(unique < 256)
+			ret = new MapToByte(unique, size);
+		else{
+			setUnique(unique);
+			return this;
+		}
+		ret.copy(this);
+		return ret;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToCharPByte.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToCharPByte.java
new file mode 100644
index 0000000000..1783e22231
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToCharPByte.java
@@ -0,0 +1,196 @@
+/*
+ * 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.colgroup.mapping;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.BitSet;
+
+import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
+import org.apache.sysds.utils.MemoryEstimates;
+
+public class MapToCharPByte extends AMapToData {
+
+	private static final long serialVersionUID = 6315708056775476541L;
+
+	// 8323073
+	public static final int max = 0xFFFF * 127;
+	private final char[] _data_c;
+	private final byte[] _data_b; // next byte after the char
+
+	protected MapToCharPByte(int size) {
+		this(Character.MAX_VALUE, size);
+	}
+
+	public MapToCharPByte(int unique, int size) {
+		super(Math.min(unique, max));
+		_data_c = new char[size];
+		_data_b = new byte[size];
+	}
+
+	public MapToCharPByte(int unique, char[] data_c, byte[] data_b) {
+		super(unique);
+		_data_c = data_c;
+		_data_b = data_b;
+	}
+
+	@Override
+	public MAP_TYPE getType() {
+		return MapToFactory.MAP_TYPE.CHAR_BYTE;
+	}
+
+	@Override
+	public int getIndex(int n) {
+		return _data_c[n] + ((int) _data_b[n] << 16);
+	}
+
+	@Override
+	public void fill(int v) {
+		int m = v & 0xffffff;
+		Arrays.fill(_data_c, (char) m);
+		Arrays.fill(_data_b, (byte) (m >> 16));
+	}
+
+	@Override
+	public long getInMemorySize() {
+		return getInMemorySize(_data_c.length);
+	}
+
+	public static long getInMemorySize(int dataLength) {
+		long size = 16 + 8 + 8; // object header + object reference
+		size += MemoryEstimates.charArrayCost(dataLength);
+		size += MemoryEstimates.byteArrayCost(dataLength);
+		return size;
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		return 1 + 4 + 4 + _data_c.length * 3;
+	}
+
+	@Override
+	public void set(int n, int v) {
+		int m = v & 0xffffff;
+		_data_c[n] = (char) m;
+		_data_b[n] = (byte) (m >> 16);
+	}
+
+	@Override
+	public int setAndGet(int n, int v) {
+		int m = v & 0xffffff;
+		_data_c[n] = (char) m;
+		_data_b[n] = (byte) (m >> 16);
+		return m;
+	}
+
+	@Override
+	public int size() {
+		return _data_c.length;
+	}
+
+	@Override
+	public void replace(int v, int r) {
+		int m = v & 0xffffff;
+		int mr = r & 0xffffff;
+		char c = (char) m;
+		char cr = (char) mr;
+		byte b = (byte) (m >> 16);
+		byte br = (byte) (mr >> 16);
+
+		for(int i = 0; i < _data_c.length; i++)
+			if(_data_b[i] == b && _data_c[i] == c) {
+				_data_b[i] = br;
+				_data_c[i] = cr;
+			}
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(MAP_TYPE.CHAR_BYTE.ordinal());
+		out.writeInt(getUnique());
+		out.writeInt(_data_c.length);
+		for(int i = 0; i < _data_c.length; i++)
+			out.writeChar(_data_c[i]);
+		for(int i = 0; i < _data_c.length; i++)
+			out.writeByte(_data_b[i]);
+	}
+
+	protected static MapToCharPByte readFields(DataInput in) throws IOException {
+		int unique = in.readInt();
+		final int length = in.readInt();
+		final char[] data_c = new char[length];
+		for(int i = 0; i < length; i++)
+			data_c[i] = in.readChar();
+		final byte[] data_b = new byte[length];
+		for(int i = 0; i < length; i++)
+			data_b[i] = in.readByte();
+		return new MapToCharPByte(unique, data_c, data_b);
+	}
+
+	protected char[] getChars() {
+		return _data_c;
+	}
+
+	protected byte[] getBytes() {
+		return _data_b;
+	}
+
+	@Override
+	public int getUpperBoundValue() {
+		return max;
+	}
+
+	@Override
+	public void copyInt(int[] d) {
+		for(int i = 0; i < d.length; i++)
+			set(i, d[i]);
+	}
+
+	@Override
+	public void copyBit(BitSet d) {
+		for(int i = d.nextSetBit(0); i >= 0; i = d.nextSetBit(i + 1))
+			_data_c[i] = 1;
+	}
+
+	@Override
+	public AMapToData resize(int unique) {
+		final int size = _data_c.length;
+		AMapToData ret;
+		if(unique <= 1)
+			return new MapToZero(size);
+		else if(unique == 2 && size > 32)
+			ret = new MapToBit(unique, size);
+		else if(unique <= 127)
+			ret = new MapToUByte(unique, size);
+		else if(unique < 256)
+			ret = new MapToByte(unique, size);
+		else if(unique < Character.MAX_VALUE - 1)
+			ret = new MapToChar(unique, size);
+		else {
+			setUnique(unique);
+			return this;
+		}
+		ret.copy(this);
+		return ret;
+	}
+
+}
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 341bbd153a..a68a870307 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
@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.bitmap.ABitmap;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
 
@@ -31,7 +32,7 @@ public interface MapToFactory {
 	static final Log LOG = LogFactory.getLog(MapToFactory.class.getName());
 
 	public enum MAP_TYPE {
-		BIT, UBYTE, BYTE, CHAR, INT;
+		ZERO, BIT, UBYTE, BYTE, CHAR, CHAR_BYTE, INT;
 	}
 
 	public static AMapToData create(int size, ABitmap ubm) {
@@ -41,7 +42,7 @@ public interface MapToFactory {
 	}
 
 	public static AMapToData create(int size, boolean zeros, IntArrayList[] values) {
-		AMapToData _data = MapToFactory.create(size, values.length + (zeros ? 1 : 0));
+		AMapToData _data = create(size, values.length + (zeros ? 1 : 0));
 
 		if(zeros)
 			_data.fill(values.length);
@@ -56,9 +57,8 @@ public interface MapToFactory {
 	}
 
 	public static AMapToData create(int size, int[] values, int nUnique) {
-		AMapToData _data = MapToFactory.create(size, nUnique);
-		for(int i = 0; i < size; i++)
-			_data.set(i, values[i]);
+		AMapToData _data = create(size, nUnique);
+		_data.copyInt(values);
 		return _data;
 	}
 
@@ -69,8 +69,10 @@ public interface MapToFactory {
 	 * @param numTuples The maximum value to be able to represent inside the map.
 	 * @return A new map
 	 */
-	public static AMapToData create(int size, int numTuples) {
-		if(numTuples <= 2 && size > 32)
+	public static AMapToData create(final int size, final int numTuples) {
+		if(numTuples <= 1)
+			return new MapToZero(size);
+		else if(numTuples == 2 && size > 32)
 			return new MapToBit(numTuples, size);
 		else if(numTuples <= 127)
 			return new MapToUByte(numTuples, size);
@@ -78,12 +80,16 @@ public interface MapToFactory {
 			return new MapToByte(numTuples, size);
 		else if(numTuples <= ((int) Character.MAX_VALUE) + 1)
 			return new MapToChar(numTuples, size);
+		else if(numTuples <= MapToCharPByte.max)
+			return new MapToCharPByte(numTuples, size);
 		else
 			return new MapToInt(numTuples, size);
 	}
 
 	public static AMapToData create(int size, MAP_TYPE t) {
 		switch(t) {
+			case ZERO:
+				return new MapToZero(size);
 			case BIT:
 				return new MapToBit(size);
 			case UBYTE:
@@ -92,9 +98,12 @@ public interface MapToFactory {
 				return new MapToByte(size);
 			case CHAR:
 				return new MapToChar(size);
+			case CHAR_BYTE:
+				return new MapToCharPByte(size);
 			case INT:
-			default:
 				return new MapToInt(size);
+			default:
+				throw new DMLCompressionException("Unsupported type " + t);
 		}
 	}
 
@@ -109,42 +118,7 @@ public interface MapToFactory {
 	 * @return The returned hopefully reduced map.
 	 */
 	public static AMapToData resize(AMapToData d, int numTuples) {
-		final int size = d.size();
-		AMapToData ret;
-		if(d instanceof MapToBit) {
-			d.setUnique(numTuples);
-			return d;
-		}
-		else if(numTuples <= 2 && size > 32)
-			ret = new MapToBit(numTuples, size);
-		else if(d instanceof MapToUByte) {
-			d.setUnique(numTuples);
-			return d;
-		}
-		else if(numTuples <= 127) {
-			if(d instanceof MapToByte)
-				return ((MapToByte) d).toUByte();
-			ret = new MapToUByte(numTuples, size);
-		}
-		else if(d instanceof MapToByte) {
-			d.setUnique(numTuples);
-			return d;
-		}
-		else if(numTuples <= 256)
-			ret = new MapToByte(numTuples, size);
-		else if(d instanceof MapToChar) {
-			d.setUnique(numTuples);
-			return d;
-		}
-		else if(numTuples <= (int) Character.MAX_VALUE + 1)
-			ret = new MapToChar(numTuples, size);
-		else {// then the input was int and reshapes to int
-			d.setUnique(numTuples);
-			return d;
-		}
-
-		ret.copy(d);
-		return ret;
+		return d.resize(numTuples);
 	}
 
 	/**
@@ -160,6 +134,8 @@ public interface MapToFactory {
 		final int numTuples = d.getUnique();
 		AMapToData ret;
 		switch(t) {
+			case ZERO:
+				return new MapToZero(size);
 			case BIT:
 				ret = new MapToBit(numTuples, size);
 				break;
@@ -172,24 +148,30 @@ public interface MapToFactory {
 			case CHAR:
 				ret = new MapToChar(numTuples, size);
 				break;
+			case CHAR_BYTE:
+				ret = new MapToCharPByte(numTuples, size);
+				break;
 			case INT:
-			default:
 				ret = new MapToInt(numTuples, size);
 				break;
+			default:
+				throw new DMLCompressionException("Unsupported type of map " + t);
 		}
 		ret.copy(d);
 		return ret;
 	}
 
 	public static long estimateInMemorySize(int size, int numTuples) {
-		if(numTuples <= 2 && size > 32)
+		if(numTuples <= 1)
+			return MapToZero.getInMemorySize(size);
+		else if(numTuples == 2 && size > 32)
 			return MapToBit.getInMemorySize(size);
-		else if(numTuples <= 127)
-			return MapToByte.getInMemorySize(size);
 		else if(numTuples <= 256)
 			return MapToByte.getInMemorySize(size);
 		else if(numTuples <= ((int) Character.MAX_VALUE) + 1)
 			return MapToChar.getInMemorySize(size);
+		else if(numTuples <= MapToCharPByte.max)
+			return MapToCharPByte.getInMemorySize(size);
 		else
 			return MapToInt.getInMemorySize(size);
 	}
@@ -197,6 +179,8 @@ public interface MapToFactory {
 	public static AMapToData readIn(DataInput in) throws IOException {
 		MAP_TYPE t = MAP_TYPE.values()[in.readByte()];
 		switch(t) {
+			case ZERO:
+				return MapToZero.readFields(in);
 			case BIT:
 				return MapToBit.readFields(in);
 			case UBYTE:
@@ -205,25 +189,12 @@ public interface MapToFactory {
 				return MapToByte.readFields(in);
 			case CHAR:
 				return MapToChar.readFields(in);
+			case CHAR_BYTE:
+				return MapToCharPByte.readFields(in);
 			case INT:
-			default:
 				return MapToInt.readFields(in);
-		}
-	}
-
-	public static int getUpperBoundValue(MAP_TYPE t) {
-		switch(t) {
-			case BIT:
-				return 1;
-			case UBYTE:
-				return 127;
-			case BYTE:
-				return 255;
-			case CHAR:
-				return Character.MAX_VALUE;
-			case INT:
 			default:
-				return Integer.MAX_VALUE;
+				throw new DMLCompressionException("unsupported type " + t);
 		}
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToInt.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToInt.java
index 6ac550e628..f78b618f51 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToInt.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToInt.java
@@ -23,11 +23,9 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.BitSet;
 
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
-import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
-import org.apache.sysds.runtime.data.SparseBlock;
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
 public class MapToInt extends AMapToData {
@@ -50,6 +48,10 @@ public class MapToInt extends AMapToData {
 		_data = data;
 	}
 
+	protected int[] getData() {
+		return _data;
+	}
+
 	@Override
 	public MAP_TYPE getType() {
 		return MapToFactory.MAP_TYPE.INT;
@@ -168,18 +170,50 @@ public class MapToInt extends AMapToData {
 	}
 
 	@Override
-	public void preAggregateDense(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, AOffset indexes) {
-		indexes.preAggregateDenseMap(m, preAV, rl, ru, cl, cu, getUnique(), _data);
+	public int getUpperBoundValue() {
+		return Integer.MAX_VALUE;
 	}
 
 	@Override
-	public void preAggregateSparse(SparseBlock sb, double[] preAV, int rl, int ru, AOffset indexes) {
-		indexes.preAggregateSparseMap(sb, preAV, rl, ru, getUnique(), _data);
+	public void copyInt(int[] d) {
+		for(int i = 0; i < _data.length; i++)
+			_data[i] = d[i];
 	}
 
 	@Override
-	public int getUpperBoundValue() {
-		return Integer.MAX_VALUE;
+	public void copyBit(BitSet d) {
+		for(int i = d.nextSetBit(0); i >= 0; i = d.nextSetBit(i + 1))
+			_data[i] = 1;
 	}
 
+	@Override
+	public void count(int[] ret) {
+		for(int i = 0; i < _data.length; i++)
+			ret[_data[i]]++;
+	}
+
+
+	@Override
+	public AMapToData resize(int unique){
+		final int size = _data.length;
+		AMapToData ret;
+		if(unique <= 1)
+			return new MapToZero(size);
+		else if(unique == 2 && size > 32)
+			ret = new MapToBit(unique, size);
+		else if (unique <= 127)
+			ret = new MapToUByte(unique, size);
+		else if(unique < 256)
+			ret = new MapToByte(unique, size);
+		else if(unique < Character.MAX_VALUE -1)
+			ret = new MapToChar(unique, size);
+		else if(unique < MapToCharPByte.max)
+			ret = new MapToCharPByte(unique, size);
+		else{
+			setUnique(unique);
+			return this;
+		}
+		ret.copyInt(_data);
+		return ret;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToUByte.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToUByte.java
index 679118e5aa..b80c91d977 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToUByte.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToUByte.java
@@ -112,4 +112,26 @@ public class MapToUByte extends MapToByte {
 	public int getUpperBoundValue() {
 		return 127;
 	}
+
+	@Override
+	public void count(int[] ret) {
+		for(int i = 0; i < _data.length; i++)
+			ret[_data[i]]++;
+	}
+
+	@Override
+	public AMapToData resize(int unique) {
+		final int size = _data.length;
+		if(unique <= 1)
+			return new MapToZero(size);
+		else if(unique == 2 && size > 32) {
+			AMapToData ret = new MapToBit(unique, size);
+			ret.copy(this);
+			return ret;
+		}
+		else {
+			setUnique(unique);
+			return this;
+		}
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToZero.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToZero.java
new file mode 100644
index 0000000000..dab63dc013
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/mapping/MapToZero.java
@@ -0,0 +1,141 @@
+/*
+ * 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.colgroup.mapping;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.BitSet;
+
+import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
+import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory.MAP_TYPE;
+
+public class MapToZero extends AMapToData {
+
+	private static final long serialVersionUID = -8065234231282619923L;
+
+	private final int _size;
+
+	public MapToZero(int size) {
+		super(1);
+		_size = size;
+	}
+
+	@Override
+	public MAP_TYPE getType() {
+		return MapToFactory.MAP_TYPE.ZERO;
+	}
+
+	@Override
+	public int getIndex(int n) {
+		return 0;
+	}
+
+	@Override
+	public void fill(int v) {
+		// do nothing
+	}
+
+	@Override
+	public long getInMemorySize() {
+		return getInMemorySize(0);
+	}
+
+	public static long getInMemorySize(int dataLength) {
+		return 16 + 4;
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		return 1 + 4;
+	}
+
+	@Override
+	public void set(int n, int v) {
+		// do nothing
+	}
+
+	@Override
+	public int setAndGet(int n, int v) {
+		return 0;
+	}
+
+	@Override
+	public int size() {
+		return _size;
+	}
+
+	@Override
+	public void replace(int v, int r) {
+		// do nothing
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(MAP_TYPE.ZERO.ordinal());
+		out.writeInt(_size);
+	}
+
+	protected static MapToZero readFields(DataInput in) throws IOException {
+		return new MapToZero(in.readInt());
+	}
+
+	@Override
+	public int getUpperBoundValue() {
+		return 0;
+	}
+
+	@Override
+	protected void count(int[] ret) {
+		final int sz = size();
+		ret[0] = sz;
+	}
+
+	@Override
+	public void preAggregateDDC_DDCSingleCol(AMapToData tm, double[] td, double[] v) {
+		final int sz = size();
+		for(int r = 0; r < sz; r++)
+			v[0] += td[tm.getIndex(r)];
+
+	}
+
+	@Override
+	public void preAggregateDDC_DDCMultiCol(AMapToData tm, ADictionary td, double[] v, int nCol) {
+		final int sz = size();
+		for(int r = 0; r < sz; r++)
+			td.addToEntry(v, tm.getIndex(r), 0, nCol);
+	}
+
+	@Override
+	public void copyInt(int[] d) {
+		// do nothing
+	}
+
+	@Override
+	public void copyBit(BitSet d) {
+		// do nothing
+	}
+
+	@Override
+	public AMapToData resize(int unique) {
+		// do nothing
+		return this;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/AOffset.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/AOffset.java
index 4ef82128ac..685c88a39e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/AOffset.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/AOffset.java
@@ -21,13 +21,11 @@ package org.apache.sysds.runtime.compress.colgroup.offset;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.Map;
 
-import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
+import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -54,9 +52,6 @@ public abstract class AOffset implements Serializable {
 		}
 	};
 
-	/** Memorizer for the row indexes mostly used for when we parallelize across rows */
-	private Map<Integer, AIterator> memorizer = null;
-
 	/**
 	 * Get an iterator of the offsets while also maintaining the data index pointer.
 	 * 
@@ -90,18 +85,11 @@ public abstract class AOffset implements Serializable {
 			return c.it.clone();
 		else {
 			AIterator it = null;
-			if(memorizer != null) {
-				it = memorizer.getOrDefault(row, null);
-
-				if(it != null)
-					return it.clone();
-			}
 			// Use the cached iterator if it is closer to the queried row.
 			it = c != null && c.row < row ? c.it.clone() : getIterator();
 			it.skipTo(row);
 			// cache this new iterator.
 			cacheIterator(it.clone(), row);
-			memorizeIterator(it.clone(), row);
 			return it;
 		}
 
@@ -119,14 +107,6 @@ public abstract class AOffset implements Serializable {
 		cacheRow.set(new OffsetCache(it, row));
 	}
 
-	private void memorizeIterator(AIterator it, int row) {
-		if(it == null)
-			return;
-		else if(memorizer == null)
-			memorizer = new HashMap<>();
-		memorizer.put(row, it);
-	}
-
 	/**
 	 * Write the offsets to disk.
 	 * 
@@ -181,67 +161,7 @@ public abstract class AOffset implements Serializable {
 	public abstract int getOffsetsLength();
 
 	public final void preAggregateDenseMap(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		int[] data) {
-		// multi row iterator.
-		final AIterator it = getIterator(cl);
-		if(it == null)
-			return;
-		else if(it.offset > cu)
-			cacheIterator(it, cu); // cache this iterator.
-		else if(rl == ru - 1) {
-			final DenseBlock db = m.getDenseBlock();
-			final double[] mV = db.values(rl);
-			final int off = db.pos(rl);
-			preAggregateDenseMapRowInt(mV, off, preAV, cu, nVal, data, it);
-		}
-		else {
-			final DenseBlock db = m.getDenseBlock();
-			preAggregateDenseMapRowsInt(db, preAV, rl, ru, cl, cu, nVal, data, it);
-		}
-	}
-
-	public final void preAggregateDenseMap(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		char[] data) {
-		// multi row iterator.
-		final AIterator it = getIterator(cl);
-		if(it == null)
-			return;
-		else if(it.offset > cu)
-			cacheIterator(it, cu); // cache this iterator.
-		else if(rl == ru - 1) {
-			final DenseBlock db = m.getDenseBlock();
-			final double[] mV = db.values(rl);
-			final int off = db.pos(rl);
-			preAggregateDenseMapRowChar(mV, off, preAV, cu, nVal, data, it);
-		}
-		else {
-			final DenseBlock db = m.getDenseBlock();
-			preAggregateDenseMapRowsChar(db, preAV, rl, ru, cl, cu, nVal, data, it);
-		}
-	}
-
-	public final void preAggregateDenseMap(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		byte[] data) {
-		// multi row iterator.
-		final AIterator it = getIterator(cl);
-		if(it == null)
-			return;
-		else if(it.offset > cu)
-			cacheIterator(it, cu); // cache this iterator.
-		else if(rl == ru - 1) {
-			final DenseBlock db = m.getDenseBlock();
-			final double[] mV = db.values(rl);
-			final int off = db.pos(rl);
-			preAggregateDenseMapRowByte(mV, off, preAV, cu, nVal, data, it);
-		}
-		else {
-			final DenseBlock db = m.getDenseBlock();
-			preAggregateDenseMapRowsByte(db, preAV, rl, ru, cl, cu, nVal, data, it);
-		}
-	}
-
-	public final void preAggregateDenseMap(MatrixBlock m, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		BitSet data) {
+		AMapToData data) {
 		// multi row iterator.
 		final AIterator it = getIterator(cl);
 		if(it == null)
@@ -252,233 +172,83 @@ public abstract class AOffset implements Serializable {
 			final DenseBlock db = m.getDenseBlock();
 			final double[] mV = db.values(rl);
 			final int off = db.pos(rl);
-			preAggregateDenseMapRowBit(mV, off, preAV, cu, nVal, data, it);
+			preAggregateDenseMapRow(mV, off, preAV, cu, nVal, data, it);
 		}
 		else {
 			final DenseBlock db = m.getDenseBlock();
-			preAggregateDenseMapRowsBit(db, preAV, rl, ru, cl, cu, nVal, data, it);
-		}
-	}
-
-	protected void preAggregateDenseMapRowInt(double[] mV, int off, double[] preAV, int cu, int nVal, int[] data,
-		AIterator it) {
-		final int maxId = data.length - 1;
-		while(it.isNotOver(cu)) {
-			final int dx = it.getDataIndex();
-			preAV[data[dx]] += mV[off + it.value()];
-			if(dx < maxId)
-				it.next();
-			else
-				break;
+			preAggregateDenseMapRows(db, preAV, rl, ru, cl, cu, nVal, data, it);
 		}
-		cacheIterator(it, cu);
 	}
 
-	protected void preAggregateDenseMapRowByte(double[] mV, int off, double[] preAV, int cu, int nVal, byte[] data,
+	protected final void preAggregateDenseMapRow(double[] mV, int off, double[] preAV, int cu, int nVal, AMapToData data,
 		AIterator it) {
 		final int last = getOffsetToLast();
-		while(it.isNotOver(cu)) {
-			final int dx = it.getDataIndex();
-			preAV[data[dx] & 0xFF] += mV[off + it.value()];
-			if(it.value() < last)
-				it.next();
-			else
-				break;
-		}
-		cacheIterator(it, cu);
-	}
-
-	protected void preAggregateDenseMapRowChar(double[] mV, int off, double[] preAV, int cu, int nVal, char[] data,
-		AIterator it) {
-		final int last = getOffsetToLast();
-		while(it.isNotOver(cu)) {
-			final int dx = it.getDataIndex();
-			preAV[data[dx]] += mV[off + it.value()];
-			if(it.value() < last)
-				it.next();
-			else
-				break;
-		}
-		cacheIterator(it, cu);
-	}
-
-	protected void preAggregateDenseMapRowBit(double[] mV, int off, double[] preAV, int cu, int nVal, BitSet data,
-		AIterator it) {
-		final int last = getOffsetToLast();
-		while(it.isNotOver(cu)) {
-			final int dx = it.getDataIndex();
-			preAV[data.get(dx) ? 1 : 0] += mV[off + it.value()];
-			if(it.value() < last)
-				it.next();
-			else
-				break;
-		}
-		cacheIterator(it, cu);
-	}
-
-	protected void preAggregateDenseMapRowsInt(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		int[] data, AIterator it) {
-		final AIterator sIt = it.clone();
-		if(cu <= getOffsetToLast()) {
-			// inside offsets
-			for(int r = rl; r < ru; r++) {
-				final int offOut = (r - rl) * nVal;
-				final double[] vals = db.values(r);
-				final int off = db.pos(r);
-				final int cur = cu + off;
-				it = sIt.clone();
-				it.offset += off;
-				while(it.offset < cur) {
-					preAV[offOut + data[it.getDataIndex()] & 0xFF] += vals[it.offset];
-					it.next();
-				}
-				it.offset -= off;
-			}
-			cacheIterator(it, cu);
-		}
-		else {
-			final int maxId = data.length - 1;
-			// all the way to the end of offsets.
-			for(int r = rl; r < ru; r++) {
-				final int offOut = (r - rl) * nVal;
-				final int off = db.pos(r);
-				final double[] vals = db.values(r);
-				it = sIt.clone();
-				it.offset = it.offset + off;
-				preAV[offOut + data[it.getDataIndex()] & 0xFF] += vals[it.offset];
-				while(it.getDataIndex() < maxId) {
-					it.next();
-					preAV[offOut + data[it.getDataIndex()] & 0xFF] += vals[it.offset];
-				}
-			}
-		}
-	}
-
-	protected void preAggregateDenseMapRowsChar(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		char[] data, AIterator it) {
-		if(cu <= getOffsetToLast())
-			preAggregateDenseMapRowsCharBelowEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
+		if(cu <= last)
+			preAggregateDenseMapRowBellowEnd(mV, off, preAV, cu, nVal, data, it);
 		else
-			preAggregateDenseMapRowsCharEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
+			preAggregateDenseMapRowEnd(mV, off, preAV, last, nVal, data, it);
 	}
 
-	private void preAggregateDenseMapRowsCharBelowEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
-		int nVal, char[] data, AIterator it) {
-		final double[] vals = db.values(rl);
-		final int nCol = db.getCumODims(0);
+	protected final void preAggregateDenseMapRowBellowEnd(final double[] mV, final int off, final double[] preAV, int cu,
+		final int nVal, final AMapToData data, final AIterator it) {
+		it.offset += off;
+		cu += off;
 		while(it.offset < cu) {
-			final int dataOffset = data[it.getDataIndex()];
-			final int start = it.offset + nCol * rl;
-			final int end = it.offset + nCol * ru;
-			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-				preAV[offOut] += vals[off];
+			preAV[data.getIndex(it.getDataIndex())] += mV[it.offset];
 			it.next();
 		}
+		it.offset -= off;
+		cu -= off;
 		cacheIterator(it, cu);
 	}
 
-	private void preAggregateDenseMapRowsCharEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		char[] data, AIterator it) {
-		final double[] vals = db.values(rl);
-		final int nCol = db.getCumODims(0);
-		final int last = getOffsetToLast();
-		int dataOffset = data[it.getDataIndex()];
-		int start = it.offset + nCol * rl;
-		int end = it.offset + nCol * ru;
-		for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-			preAV[offOut] += vals[off];
-		while(it.offset < last) {
-			it.next();
-			dataOffset = data[it.getDataIndex()];
-			start = it.offset + nCol * rl;
-			end = it.offset + nCol * ru;
-			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-				preAV[offOut] += vals[off];
-		}
-	}
-
-	protected void preAggregateDenseMapRowsByte(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		byte[] data, AIterator it) {
-		if(cu <= getOffsetToLast())
-			preAggregateDenseMapRowsByteBelowEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
-		else
-			preAggregateDenseMapRowsByteEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
-	}
+	protected final void preAggregateDenseMapRowEnd(final double[] mV, final int off, final double[] preAV,
+		final int last, final int nVal, final AMapToData data, final AIterator it) {
 
-	protected void preAggregateDenseMapRowsByteBelowEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
-		int nVal, byte[] data, AIterator it) {
-		final double[] vals = db.values(rl);
-		final int nCol = db.getCumODims(0);
-		while(it.offset < cu) {
-			final int dataOffset = data[it.getDataIndex()] & 0xFF;
-			final int start = it.offset + nCol * rl;
-			final int end = it.offset + nCol * ru;
-			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-				preAV[offOut] += vals[off];
-			it.next();
-		}
-
-		cacheIterator(it, cu);
-	}
-
-	protected void preAggregateDenseMapRowsByteEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
-		int nVal, byte[] data, AIterator it) {
-		final double[] vals = db.values(rl);
-		final int nCol = db.getCumODims(0);
-		final int last = getOffsetToLast();
-		int dataOffset = data[it.getDataIndex()] & 0xFF;
-		int start = it.offset + nCol * rl;
-		int end = it.offset + nCol * ru;
-		for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-			preAV[offOut] += vals[off];
 		while(it.offset < last) {
+			final int dx = it.getDataIndex();
+			preAV[data.getIndex(dx)] += mV[off + it.offset];
 			it.next();
-			dataOffset = data[it.getDataIndex()] & 0xFF;
-			start = it.offset + nCol * rl;
-			end = it.offset + nCol * ru;
-			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
-				preAV[offOut] += vals[off];
 		}
+		preAV[data.getIndex(it.getDataIndex())] += mV[off + last];
 	}
 
-	protected void preAggregateDenseMapRowsBit(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
-		BitSet data, AIterator it) {
+	protected final void preAggregateDenseMapRows(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
+		int nVal, AMapToData data, AIterator it) {
 		if(cu <= getOffsetToLast())
-			preAggregateDenseMapRowsBitBelowEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
+			preAggregateDenseMapRowsBelowEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
 		else
-			preAggregateDenseMapRowsBitEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
+			preAggregateDenseMapRowsEnd(db, preAV, rl, ru, cl, cu, nVal, data, it);
 	}
 
-	protected void preAggregateDenseMapRowsBitBelowEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
-		int nVal, BitSet data, AIterator it) {
+	private void preAggregateDenseMapRowsBelowEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
+		int nVal, AMapToData data, AIterator it) {
 		final double[] vals = db.values(rl);
 		final int nCol = db.getCumODims(0);
-
 		while(it.offset < cu) {
-			final int dataOffset = data.get(it.getDataIndex()) ? 1 : 0;
+			final int dataOffset = data.getIndex(it.getDataIndex());
 			final int start = it.offset + nCol * rl;
 			final int end = it.offset + nCol * ru;
 			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
 				preAV[offOut] += vals[off];
 			it.next();
 		}
-
 		cacheIterator(it, cu);
 	}
 
-	protected void preAggregateDenseMapRowsBitEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu,
-		int nVal, BitSet data, AIterator it) {
+	private void preAggregateDenseMapRowsEnd(DenseBlock db, double[] preAV, int rl, int ru, int cl, int cu, int nVal,
+		AMapToData data, AIterator it) {
 		final double[] vals = db.values(rl);
 		final int nCol = db.getCumODims(0);
 		final int last = getOffsetToLast();
-		int dataOffset = data.get(it.getDataIndex()) ? 1 : 0;
+		int dataOffset = data.getIndex(it.getDataIndex());
 		int start = it.offset + nCol * rl;
 		int end = it.offset + nCol * ru;
 		for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
 			preAV[offOut] += vals[off];
 		while(it.offset < last) {
 			it.next();
-			dataOffset = data.get(it.getDataIndex()) ? 1 : 0;
+			dataOffset = data.getIndex(it.getDataIndex());
 			start = it.offset + nCol * rl;
 			end = it.offset + nCol * ru;
 			for(int offOut = dataOffset, off = start; off < end; offOut += nVal, off += nCol)
@@ -486,196 +256,125 @@ public abstract class AOffset implements Serializable {
 		}
 	}
 
-	public final void preAggregateSparseMap(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, int[] data) {
-		final AIterator it = getIterator();
-		if(rl == ru - 1)
-			preAggregateSparseMapRow(sb, preAV, rl, nVal, data, it);
-		else
-			throw new NotImplementedException("MultiRow Preaggregation not supported yet");
-	}
-
-	public final void preAggregateSparseMap(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, char[] data) {
-		final AIterator it = getIterator();
-		if(rl == ru - 1)
-			preAggregateSparseMapRow(sb, preAV, rl, nVal, data, it);
-		else
-			throw new NotImplementedException("MultiRow Preaggregation not supported yet");
-	}
-
-	public final void preAggregateSparseMap(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, byte[] data) {
+	public final void preAggregateSparseMap(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, AMapToData data) {
 		final AIterator it = getIterator();
 		if(rl == ru - 1)
 			preAggregateSparseMapRow(sb, preAV, rl, nVal, data, it);
 		else
-			throw new NotImplementedException("MultiRow Preaggregation not supported yet");
+			preAggregateSparseMapRows(sb, preAV, rl, ru, nVal, data, it);
 	}
 
-	public final void preAggregateSparseMap(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, BitSet data) {
-		final AIterator it = getIterator();
-		if(rl == ru - 1)
-			preAggregateSparseMapRow(sb, preAV, rl, nVal, data, it);
+	private void preAggregateSparseMapRow(SparseBlock sb, double[] preAV, int r, int nVal, AMapToData data,
+		AIterator it) {
+		if(sb.isEmpty(r))
+			return;
+		final int alen = sb.size(r) + sb.pos(r);
+		final int[] aix = sb.indexes(r);
+		final int last = getOffsetToLast();
+		if(aix[alen - 1] < last)
+			preAggregateSparseMapRowBellowEnd(sb, preAV, r, nVal, data, it);
 		else
-			throw new NotImplementedException("MultiRow Preaggregation not supported yet");
+			preAggregateSparseMapRowEnd(sb, preAV, r, nVal, data, it);
 	}
 
-	private void preAggregateSparseMapRow(SparseBlock sb, double[] preAV, int r, int nVal, byte[] data, AIterator it) {
+	private final void preAggregateSparseMapRowBellowEnd(SparseBlock sb, double[] preAV, int r, int nVal,
+		AMapToData data, AIterator it) {
 		int apos = sb.pos(r);
 		final int alen = sb.size(r) + apos;
 		final int[] aix = sb.indexes(r);
 		final double[] avals = sb.values(r);
-
-		final int last = getOffsetToLast();
-
-		if(aix[alen - 1] < last) {
-			int v = it.value();
-			while(apos < alen) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()] & 0xFF] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
+		int v = it.value();
+		while(apos < alen) {
+			if(aix[apos] == v) {
+				preAV[data.getIndex(it.getDataIndex())] += avals[apos++];
+				v = it.next();
 			}
-		}
-		else {
-			int v = it.value();
-			while(v < last) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()] & 0xFF] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
-			}
-			while(aix[apos] < last && apos < alen)
+			else if(aix[apos] < v)
 				apos++;
-			if(v == aix[apos]) // process last element
-				preAV[data[it.getDataIndex()] & 0xFF] += avals[apos];
+			else
+				v = it.next();
 		}
 	}
 
-	private void preAggregateSparseMapRow(final SparseBlock sb, final double[] preAV, final int r, final int nVal,
-		final char[] data, final AIterator it) {
+	private final void preAggregateSparseMapRowEnd(SparseBlock sb, double[] preAV, int r, int nVal, AMapToData data,
+		AIterator it) {
 		int apos = sb.pos(r);
 		final int alen = sb.size(r) + apos;
 		final int[] aix = sb.indexes(r);
 		final double[] avals = sb.values(r);
 		final int last = getOffsetToLast();
-
-		if(aix[alen - 1] < last) {
-			int v = it.value();
-			while(apos < alen) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()]] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
+		int v = it.value();
+		while(v < last) {
+			if(aix[apos] == v) {
+				preAV[data.getIndex(it.getDataIndex())] += avals[apos++];
+				v = it.next();
 			}
-		}
-		else {
-			int v = it.value();
-			while(v < last) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()]] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
-			}
-			while(aix[apos] < last && apos < alen)
+			else if(aix[apos] < v)
 				apos++;
-			if(v == aix[apos]) // process last element
-				preAV[data[it.getDataIndex()]] += avals[apos];
+			else
+				v = it.next();
 		}
+		while(aix[apos] < last && apos < alen)
+			apos++;
+		if(v == aix[apos]) // process last element
+			preAV[data.getIndex(it.getDataIndex())] += avals[apos];
 	}
 
-	private void preAggregateSparseMapRow(SparseBlock sb, double[] preAV, int r, int nVal, int[] data, AIterator it) {
-		int apos = sb.pos(r);
-		final int alen = sb.size(r) + apos;
-		final int[] aix = sb.indexes(r);
-		final double[] avals = sb.values(r);
+	private void preAggregateSparseMapRows(SparseBlock sb, double[] preAV, int rl, int ru, int nVal, AMapToData data,
+		AIterator it) {
+		int i = it.value();
 		final int last = getOffsetToLast();
+		final int[] aOffs = new int[ru - rl];
+		for(int r = rl; r < ru; r++)
+			aOffs[r - rl] = sb.pos(r);
 
-		if(aix[alen - 1] < last) {
-			int v = it.value();
-			while(apos < alen) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()]] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
+		while(i < last) { // while we are not done iterating
+			for(int r = rl; r < ru; r++) {
+				final int off = r - rl;
+				int apos = aOffs[off]; // current offset
+				final int alen = sb.size(r) + sb.pos(r);
+				final int[] aix = sb.indexes(r);
+				while(apos < alen && aix[apos] < i)// increment all pointers to offset
 					apos++;
-				else
-					v = it.next();
+
+				if(apos < alen && aix[apos] == i)
+					preAV[off * nVal + data.getIndex(it.getDataIndex())] += sb.values(r)[apos];
+				aOffs[off] = apos;
 			}
+			i = it.next();
 		}
-		else {
-			int v = it.value();
-			while(v < last) {
-				if(aix[apos] == v) {
-					preAV[data[it.getDataIndex()]] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
-			}
-			while(aix[apos] < last && apos < alen)
+
+		// process final element
+		for(int r = rl; r < ru; r++) {
+			final int off = r - rl;
+			int apos = aOffs[off];
+			final int alen = sb.size(r) + sb.pos(r);
+			final int[] aix = sb.indexes(r);
+			while(apos < alen && aix[apos] < last)
 				apos++;
-			if(v == aix[apos]) // process last element
-				preAV[data[it.getDataIndex()]] += avals[apos];
+
+			if(apos < alen && aix[apos] == last)
+				preAV[off* nVal + data.getIndex(it.getDataIndex())] += sb.values(r)[apos];
+			aOffs[off] = apos;
 		}
 	}
 
-	private void preAggregateSparseMapRow(SparseBlock sb, double[] preAV, int r, int nVal, BitSet data, AIterator it) {
-
-		int apos = sb.pos(r);
-		final int alen = sb.size(r) + apos;
-		final int[] aix = sb.indexes(r);
-		final double[] avals = sb.values(r);
-		final int last = getOffsetToLast();
-
-		if(aix[alen - 1] < last) {
-			int v = it.value();
-			while(apos < alen) {
-				if(aix[apos] == v) {
-					preAV[data.get(it.getDataIndex()) ? 1 : 0] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
+	public boolean equals(AOffset b) {
+		if(getOffsetToLast() == b.getOffsetToLast()) {
+			int last = getOffsetToLast();
+			AOffsetIterator ia = getOffsetIterator();
+			AOffsetIterator ib = b.getOffsetIterator();
+			while(ia.value() < last) {
+				if(ia.value() != ib.value())
+					return false;
+				ia.next();
+				ib.next();
+				if(ib.value() == last && ia.value() != last)
+					return false;
 			}
+			return true;
 		}
-		else {
-			int v = it.value();
-			while(v < last) {
-				if(aix[apos] == v) {
-					preAV[data.get(it.getDataIndex()) ? 1 : 0] += avals[apos++];
-					v = it.next();
-				}
-				else if(aix[apos] < v)
-					apos++;
-				else
-					v = it.next();
-			}
-			while(aix[apos] < last && apos < alen)
-				apos++;
-			if(v == aix[apos]) // process last element
-				preAV[data.get(it.getDataIndex()) ? 1 : 0] += avals[apos];
-		}
-
+		return false;
 	}
 
 	@Override
@@ -692,6 +391,11 @@ public abstract class AOffset implements Serializable {
 		}
 		sb.append(it.offset);
 		sb.append("]");
+
+		if(it.offset != last)
+			throw new DMLCompressionException(
+				"Invalid iteration of offset when making string, the last offset is not equal to a iteration: "
+					+ getOffsetToLast() + " String: " + sb.toString());
 		return sb.toString();
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetByte.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetByte.java
index 86d999c610..0cb98728ad 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetByte.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetByte.java
@@ -50,6 +50,8 @@ public class OffsetByte extends AOffset {
 			endSize += 1 + (nv - ov - 1) / maxV;
 			ov = nv;
 		}
+
+		this.noZero = endSize == alen - apos - 1;
 		offsets = new byte[endSize];
 		ov = offsetToFirst;
 		int p = 0;
@@ -73,7 +75,7 @@ public class OffsetByte extends AOffset {
 		}
 
 		this.noOverHalf = getNoOverHalf();
-		this.noZero = getNoZero();
+
 	}
 
 	protected OffsetByte(byte[] offsets, int offsetToFirst, int offsetToLast) {
@@ -306,14 +308,14 @@ public class OffsetByte extends AOffset {
 		}
 
 		@Override
-		public int next() {
+		public final int next() {
 			offset += offsets[index];
 			index++;
 			return offset;
 		}
 
 		@Override
-		public int skipTo(int idx) {
+		public final int skipTo(int idx) {
 			while(offset < idx && index < offsets.length) {
 				offset += offsets[index];
 				index++;
@@ -323,7 +325,7 @@ public class OffsetByte extends AOffset {
 		}
 
 		@Override
-		public IterateByteOffsetNoOverHalf clone() {
+		public final IterateByteOffsetNoOverHalf clone() {
 			return new IterateByteOffsetNoOverHalf(index, offset);
 		}
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetChar.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetChar.java
index fdeac48029..bdf21d6d70 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetChar.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetChar.java
@@ -48,10 +48,10 @@ public class OffsetChar extends AOffset {
 			endSize += 1 + (nv - ov - 1) / maxV;
 			ov = nv;
 		}
+		this.noZero = endSize == alen - apos - 1;
 		offsets = new char[endSize];
 		ov = offsetToFirst;
 		int p = 0;
-
 		for(int i = apos + 1; i < alen; i++) {
 			final int nv = indexes[i];
 			final int offsetSize = (nv - ov);
@@ -65,10 +65,8 @@ public class OffsetChar extends AOffset {
 				p += div; // skip values
 				offsets[p++] = (char) (mod);
 			}
-
 			ov = nv;
 		}
-		this.noZero = getNoZero();
 	}
 
 	private OffsetChar(char[] offsets, int offsetToFirst, int offsetToLast) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetFactory.java
index 28d7bb0ab3..5e76e1c2a5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetFactory.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
 
 public interface OffsetFactory {
@@ -32,7 +33,7 @@ public interface OffsetFactory {
 
 	/** The specific underlying types of offsets. */
 	public enum OFF_TYPE {
-		BYTE, CHAR
+		BYTE, CHAR, SINGLE_OFFSET, TWO_OFFSET
 	}
 
 	/**
@@ -76,15 +77,23 @@ public interface OffsetFactory {
 	 * @return A new Offset.
 	 */
 	public static AOffset createOffset(int[] indexes, int apos, int alen) {
+
+		final int endLength = alen - apos - 1;
+		if(endLength < 0)
+			throw new DMLCompressionException("Invalid empty offset to create");
+		else if(endLength == 0) // means size of 1 since we store the first offset outside the list
+			return new OffsetSingle(indexes[apos]);
+		else if(endLength == 1)
+			return new OffsetTwo(indexes[apos], indexes[apos + 1]);
+
 		final int minValue = indexes[apos];
 		final int maxValue = indexes[alen - 1];
 		final int range = maxValue - minValue;
-		final int endLength = alen - apos - 1;
 		// -1 because one index is skipped using a first idex allocated as a int.
 
 		final int correctionByte = correctionByte(range, endLength);
 		final int correctionChar = correctionChar(range, endLength);
-	
+
 		final long byteSize = OffsetByte.estimateInMemorySize(endLength + correctionByte);
 		final long charSize = OffsetChar.estimateInMemorySize(endLength + correctionChar);
 
@@ -104,6 +113,10 @@ public interface OffsetFactory {
 	public static AOffset readIn(DataInput in) throws IOException {
 		OFF_TYPE t = OFF_TYPE.values()[in.readByte()];
 		switch(t) {
+			case SINGLE_OFFSET:
+				return OffsetSingle.readFields(in);
+			case TWO_OFFSET:
+				return OffsetTwo.readFields(in);
 			case BYTE:
 				return OffsetByte.readFields(in);
 			case CHAR:
@@ -127,17 +140,21 @@ public interface OffsetFactory {
 	public static long estimateInMemorySize(int size, int nRows) {
 		if(size == 0)
 			return 8; // If this is the case, then the compression results in constant col groups
+		else if(size == 1)
+			return OffsetSingle.estimateInMemorySize();
+		else if(size == 2)
+			return OffsetTwo.estimateInMemorySize();
+
+		final int avgDiff = nRows / size;
+		if(avgDiff < 256) {
+			final int correctionByte = correctionByte(nRows, size);
+			return OffsetByte.estimateInMemorySize(size - 1 + correctionByte);
+		}
 		else {
-			final int avgDiff = nRows / size;
-			if(avgDiff < 256) {
-				final int correctionByte = correctionByte(nRows, size);
-				return OffsetByte.estimateInMemorySize(size - 1 + correctionByte);
-			}
-			else {
-				final int correctionChar = correctionChar(nRows, size);
-				return OffsetChar.estimateInMemorySize(size - 1 + correctionChar);
-			}
+			final int correctionChar = correctionChar(nRows, size);
+			return OffsetChar.estimateInMemorySize(size - 1 + correctionChar);
 		}
+
 	}
 
 	public static int correctionByte(int nRows, int size) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetSingle.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetSingle.java
new file mode 100644
index 0000000000..b01bbb0e4c
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetSingle.java
@@ -0,0 +1,132 @@
+/*
+ * 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.colgroup.offset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class OffsetSingle extends AOffset {
+
+	private final int off;
+
+	public OffsetSingle(int off) {
+		this.off = off;
+	}
+
+	@Override
+	public AIterator getIterator() {
+		return new IterateSingle();
+	}
+
+	@Override
+	public AOffsetIterator getOffsetIterator() {
+		return new IterateOffsetSingle();
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		return 1 + 4;
+	}
+
+	@Override
+	public int getSize() {
+		return 1;
+	}
+
+	@Override
+	public int getOffsetToFirst() {
+		return off;
+	}
+
+	@Override
+	public int getOffsetToLast() {
+		return off;
+	}
+
+	@Override
+	public int getOffsetsLength() {
+		return 0;
+	}
+
+	@Override
+	public long getInMemorySize() {
+		return estimateInMemorySize();
+	}
+
+	public static long estimateInMemorySize() {
+		return 16 + 4; // object header plus int
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(OffsetFactory.OFF_TYPE.SINGLE_OFFSET.ordinal());
+		out.writeInt(off);
+	}
+
+	public static OffsetSingle readFields(DataInput in) throws IOException {
+		return new OffsetSingle(in.readInt());
+	}
+
+	private class IterateSingle extends AIterator {
+
+		private IterateSingle() {
+			super(off);
+		}
+
+		@Override
+		public int next() {
+			return off;
+		}
+
+		@Override
+		public int skipTo(int idx) {
+			return off;
+		}
+
+		@Override
+		public IterateSingle clone() {
+			return this;
+		}
+
+		@Override
+		public int getDataIndex() {
+			return 0;
+		}
+
+		@Override
+		public int getOffsetsIndex() {
+			return 0;
+		}
+	}
+
+	private class IterateOffsetSingle extends AOffsetIterator {
+
+		private IterateOffsetSingle() {
+			super(off);
+		}
+
+		@Override
+		public int next() {
+			return off;
+		}
+	}
+
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetTwo.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetTwo.java
new file mode 100644
index 0000000000..ac2f6338f0
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/offset/OffsetTwo.java
@@ -0,0 +1,141 @@
+/*
+ * 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.colgroup.offset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class OffsetTwo extends AOffset {
+
+	private final int first;
+	private final int last;
+
+	public OffsetTwo(int first, int last) {
+		this.first = first;
+		this.last = last;
+	}
+
+	@Override
+	public AIterator getIterator() {
+		return new IterateTwo();
+	}
+
+	@Override
+	public AOffsetIterator getOffsetIterator() {
+		return new IterateOffsetTwo();
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		return 1 + 4 + 4;
+	}
+
+	@Override
+	public int getSize() {
+		return 2;
+	}
+
+	@Override
+	public int getOffsetToFirst() {
+		return first;
+	}
+
+	@Override
+	public int getOffsetToLast() {
+		return last;
+	}
+
+	@Override
+	public int getOffsetsLength() {
+		return 1;
+	}
+
+	@Override
+	public long getInMemorySize() {
+		return estimateInMemorySize();
+	}
+
+	public static long estimateInMemorySize() {
+		return 16 + 4 + 4; // object header plus int
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		out.writeByte(OffsetFactory.OFF_TYPE.TWO_OFFSET.ordinal());
+		out.writeInt(first);
+		out.writeInt(last);
+	}
+
+	public static OffsetTwo readFields(DataInput in) throws IOException {
+		return new OffsetTwo(in.readInt(), in.readInt());
+	}
+
+	private class IterateTwo extends AIterator {
+
+		private IterateTwo() {
+			super(first);
+		}
+
+		@Override
+		public int next() {
+			offset = last;
+			return last;
+		}
+
+		@Override
+		public int skipTo(int idx) {
+			if(idx > first ){
+				offset = last;
+				return last;
+			}
+			return first;
+		}
+
+		@Override
+		public IterateTwo clone() {
+			IterateTwo ret = new IterateTwo();
+			ret.offset = offset;
+			return ret;
+		}
+
+		@Override
+		public int getDataIndex() {
+			return offset == first ? 0 : 1;
+		}
+
+		@Override
+		public int getOffsetsIndex() {
+			return offset == first ? 0 : 1;
+		}
+	}
+
+	private class IterateOffsetTwo extends AOffsetIterator {
+
+		private IterateOffsetTwo() {
+			super(first);
+		}
+
+		@Override
+		public int next() {
+			return offset = last;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/ACostEstimate.java b/src/main/java/org/apache/sysds/runtime/compress/cost/ACostEstimate.java
index e188cb3da9..258c438f1d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/ACostEstimate.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/ACostEstimate.java
@@ -123,6 +123,14 @@ public abstract class ACostEstimate implements Serializable {
 	 */
 	protected abstract double getCostSafe(CompressedSizeInfoColGroup g);
 
+	/**
+	 * Ask the cost estimator if it is a good idea to try to sparsify a column group. It is the same as asking if it is a
+	 * good idea to make FOR on top of the column group.
+	 * 
+	 * @return true if yes
+	 */
+	public abstract boolean shouldSparsify();
+
 	@Override
 	public String toString() {
 		return this.getClass().getSimpleName();
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 309ff09cfb..a92e0ecf7a 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
@@ -19,6 +19,7 @@
 
 package org.apache.sysds.runtime.compress.cost;
 
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -110,20 +111,19 @@ public class ComputationCostEstimator extends ACostEstimate {
 	 * @return A cost
 	 */
 	public double getCost(int nRows, int nRowsScanned, int nCols, int nVals, double sparsity) {
-		if(LOG.isTraceEnabled())
-			LOG.trace(nRows + " " + nRowsScanned + " " + nCols + " " + nVals + " " + sparsity);
-		sparsity = (nCols < 3 || _isDensifying || sparsity > 0.4) ? 1 : sparsity;
+		sparsity = (nCols < 3 || sparsity > 0.4) ? 1 : sparsity;
 
-		if((double) nRowsScanned / nRows > 0.6)
-			nRowsScanned = nRows;
 		double cost = 0;
-		cost += leftMultCost(nRowsScanned, nCols, nVals, sparsity);
+		cost += leftMultCost(nRowsScanned, nRows, nCols, nVals, sparsity);
 		cost += scanCost(nRowsScanned, nCols, nVals, sparsity);
 		cost += dictionaryOpsCost(nVals, nCols, sparsity);
 		cost += rightMultCost(nCols, nVals, sparsity);
 		cost += decompressionCost(nVals, nCols, nRowsScanned, sparsity);
 		cost += overlappingDecompressionCost(nRowsScanned);
-		cost += compressedMultiplicationCost(nRowsScanned, nVals, nCols, sparsity);
+		cost += compressedMultiplicationCost(nRowsScanned, nRows, nVals, nCols, sparsity);
+		cost += 100; // base cost
+		if(cost < 0)
+			throw new DMLCompressionException("Ivalid negative cost: " + cost);
 		return cost;
 	}
 
@@ -134,8 +134,8 @@ public class ComputationCostEstimator extends ACostEstimate {
 	@Override
 	public double getCost(MatrixBlock mb) {
 		double cost = 0;
-		final int nCols = mb.getNumColumns();
-		final int nRows = mb.getNumRows();
+		final double nCols = mb.getNumColumns();
+		final double nRows = mb.getNumRows();
 		final double sparsity = (nCols < 3 || _isDensifying) ? 1 : mb.getSparsity();
 
 		cost += dictionaryOpsCost(nRows, nCols, sparsity);
@@ -145,10 +145,11 @@ public class ComputationCostEstimator extends ACostEstimate {
 		// Scan cost we set the rows scanned to zero, since they
 		// are not indirectly scanned like in compression
 		cost += scanCost(0, nRows, nCols, sparsity);
-		cost += compressedMultiplicationCost(0, nRows, nCols, sparsity);
-
+		cost += compressedMultiplicationCost(0, 0, nRows, nCols, sparsity);
 		// decompression cost ... 0 for both overlapping and normal decompression
 
+		if(cost < 0)
+			throw new DMLCompressionException("Invalid negative cost : " + cost);
 		return cost;
 	}
 
@@ -157,6 +158,11 @@ public class ComputationCostEstimator extends ACostEstimate {
 		return cg.getCost(this, nRows);
 	}
 
+	@Override
+	public boolean shouldSparsify() {
+		return _leftMultiplications > 0 || _compressedMultiplication > 0 || _rightMultiplications > 0;
+	}
+
 	private double dictionaryOpsCost(double nVals, double nCols, double sparsity) {
 		// Dictionary ops simply goes through dictionary and modify all values.
 		// Therefore the cost is in number of cells in the dictionary.
@@ -164,9 +170,9 @@ public class ComputationCostEstimator extends ACostEstimate {
 		return _dictionaryOps * sparsity * nVals * nCols * 2;
 	}
 
-	private double leftMultCost(double nRows, double nCols, double nVals, double sparsity) {
+	private double leftMultCost(double nRowsScanned, double nRows, double nCols, double nVals, double sparsity) {
 		// Plus nVals * 2 because of allocation of nVals array and scan of that
-		final double preScalingCost = nRows + nVals * 2;
+		final double preScalingCost = Math.max(nRowsScanned, nRows / 10) + nVals * 2;
 		final double postScalingCost = sparsity * nVals * nCols;
 		return leftMultCost(preScalingCost, postScalingCost);
 	}
@@ -197,9 +203,9 @@ public class ComputationCostEstimator extends ACostEstimate {
 		return _scans * (nRowsScanned + nVals * nCols * sparsity);
 	}
 
-	private double compressedMultiplicationCost(double nRowsScanned, double nVals, double nCols, double sparsity) {
+	private double compressedMultiplicationCost(double nRowsScanned, double nRows, double nVals, double nCols, double sparsity) {
 		// return _compressedMultiplication * Math.max(nRowsScanned * nCols ,nVals * nCols * sparsity );
-		return _compressedMultiplication * (nRowsScanned + nVals * nCols * sparsity);
+		return _compressedMultiplication * (Math.max(nRowsScanned, nRows / 10) + nVals * nCols * sparsity);
 	}
 
 	@Override
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 43b5f993fc..2a9bdac1b2 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
@@ -50,6 +50,10 @@ public final class CostEstimatorBuilder implements Serializable {
 			addNode(1, n, counter);
 	}
 
+	public CostEstimatorBuilder(InstructionTypeCounter counter) {
+		this.counter = counter;
+	}
+
 	protected ACostEstimate create(boolean isInSpark) {
 		return new ComputationCostEstimator(counter);
 	}
@@ -121,26 +125,16 @@ public final class CostEstimatorBuilder implements Serializable {
 
 	public boolean shouldTryToCompress() {
 		int numberOps = 0;
-		numberOps += counter.scans + counter.leftMultiplications * 2 + counter.rightMultiplications * 2 +
-			counter.compressedMultiplications * 4 + counter.dictionaryOps;
-		numberOps -= counter.decompressions + counter.overlappingDecompressions * 2;
-
-		final int nrMultiplications = counter.leftMultiplications + counter.rightMultiplications +
-			counter.compressedMultiplications;
-		final int nrDecompressions = counter.decompressions + counter.overlappingDecompressions * 2;
-		if(counter.decompressions == 0 && counter.rightMultiplications == counter.overlappingDecompressions &&
-			numberOps > 10)
-			return true;
-		if(nrDecompressions > nrMultiplications || (nrDecompressions > 1 && nrMultiplications < 1))
-			// This condition is added for l2svm and mLogReg y dataset, that is compressing while it should not.
-			return false;
+		numberOps += counter.scans + counter.leftMultiplications + counter.rightMultiplications +
+			counter.compressedMultiplications + counter.dictionaryOps;
+		numberOps -= counter.decompressions + counter.overlappingDecompressions;
 		return numberOps > 4;
-
 	}
 
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
+		sb.append("CostVector: ");
 		sb.append(counter);
 		return sb.toString();
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/DistinctCostEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/cost/DistinctCostEstimator.java
index b8d8978d52..0de6195169 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/DistinctCostEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/DistinctCostEstimator.java
@@ -57,4 +57,8 @@ public class DistinctCostEstimator extends ACostEstimate {
 		throw new NotImplementedException();
 	}
 
+	@Override
+	public boolean shouldSparsify() {
+		return false;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/HybridCostEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/cost/HybridCostEstimator.java
index c8669dd805..7eb43015bf 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/HybridCostEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/HybridCostEstimator.java
@@ -60,4 +60,9 @@ public class HybridCostEstimator extends ACostEstimate {
 	public double getCost(AColGroup cg, int nRows) {
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public boolean shouldSparsify() {
+		return false;
+	}
 }
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 40b8856b42..b475b0ea1a 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
@@ -38,6 +38,19 @@ public final class InstructionTypeCounter implements Serializable {
 	protected InstructionTypeCounter() {
 	}
 
+	public InstructionTypeCounter(int scans, int decompressions, int overlappingDecompressions, int leftMultiplications,
+		int rightMultiplications, int compressedMultiplications, int dictionaryOps, int indexing, boolean isDensifying) {
+		this.scans = scans;
+		this.decompressions = decompressions;
+		this.overlappingDecompressions = overlappingDecompressions;
+		this.leftMultiplications = leftMultiplications;
+		this.rightMultiplications = rightMultiplications;
+		this.compressedMultiplications = compressedMultiplications;
+		this.dictionaryOps = dictionaryOps;
+		this.indexing = indexing;
+		this.isDensifying = isDensifying;
+	}
+
 	public int getScans() {
 		return scans;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cost/MemoryCostEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/cost/MemoryCostEstimator.java
index 3258fba4e2..65430d4a50 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cost/MemoryCostEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cost/MemoryCostEstimator.java
@@ -43,4 +43,9 @@ public class MemoryCostEstimator extends ACostEstimate {
 	public double getCost(AColGroup cg, int nRows) {
 		return cg.estimateInMemorySize();
 	}
+
+	@Override
+	public boolean shouldSparsify() {
+		return false;
+	}
 }
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 ea1abb842b..8eadac5be8 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
@@ -20,11 +20,11 @@
 package org.apache.sysds.runtime.compress.estim;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
-import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -71,12 +71,12 @@ public abstract class CompressedSizeEstimator {
 	}
 
 	/**
-	 * Multi threaded version of extracting Compression Size info
+	 * Multi threaded version of extracting compression size info
 	 * 
 	 * @param k The concurrency degree.
 	 * @return The Compression Size info of each Column compressed isolated.
 	 */
-	public CompressedSizeInfo computeCompressedSizeInfos(int k) {
+	public final CompressedSizeInfo computeCompressedSizeInfos(int k) {
 		final int _numCols = getNumColumns();
 		if(LOG.isDebugEnabled()) {
 			Timing time = new Timing(true);
@@ -94,7 +94,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 getColGroupInfo(int[] colIndexes) {
+	public final CompressedSizeInfoColGroup getColGroupInfo(int[] colIndexes) {
 		return getColGroupInfo(colIndexes, 8, worstCaseUpperBound(colIndexes));
 	}
 
@@ -120,7 +120,7 @@ public abstract class CompressedSizeEstimator {
 	 * @param colIndexes The columns to group together inside a ColGroup
 	 * @return The CompressedSizeInformation assuming delta encoding of the column.
 	 */
-	public CompressedSizeInfoColGroup getDeltaColGroupInfo(int[] colIndexes) {
+	public final CompressedSizeInfoColGroup getDeltaColGroupInfo(int[] colIndexes) {
 		return getDeltaColGroupInfo(colIndexes, 8, worstCaseUpperBound(colIndexes));
 	}
 
@@ -192,6 +192,11 @@ public abstract class CompressedSizeEstimator {
 			return combine(combinedColumns, g1, g2, (int) max);
 	}
 
+	/** Clear the pointer to the materialized list of nnz in columns */
+	public void clearNNZ() {
+		nnzCols = null;
+	}
+
 	/**
 	 * Extract the worst case upper bound of unique tuples in specified columns.
 	 * 
@@ -215,61 +220,75 @@ public abstract class CompressedSizeEstimator {
 	protected abstract CompressedSizeInfoColGroup combine(int[] combinedColumns, CompressedSizeInfoColGroup g1,
 		CompressedSizeInfoColGroup g2, int maxDistinct);
 
-	protected List<CompressedSizeInfoColGroup> CompressedSizeInfoColGroup(int clen) {
-		List<CompressedSizeInfoColGroup> ret = new ArrayList<CompressedSizeInfoColGroup>(clen);
+	private List<CompressedSizeInfoColGroup> CompressedSizeInfoColGroup(int clen, int k) {
+		if(k <= 1)
+			return CompressedSizeInfoColGroupSingleThread(clen);
+		else
+			return CompressedSizeInfoColGroupParallel(clen, k);
+	}
+
+	private List<CompressedSizeInfoColGroup> CompressedSizeInfoColGroupSingleThread(int clen) {
+		List<CompressedSizeInfoColGroup> ret = new ArrayList<>(clen);
+		if(!_cs.transposed && !_data.isEmpty() && _data.isInSparseFormat())
+			nnzCols = LibMatrixReorg.countNnzPerColumn(_data);
 		for(int col = 0; col < clen; col++)
 			ret.add(getColGroupInfo(new int[] {col}));
 		return ret;
 	}
 
-	protected List<CompressedSizeInfoColGroup> CompressedSizeInfoColGroup(int clen, int k) {
-		if(k <= 1)
-			return CompressedSizeInfoColGroup(clen);
+	private List<CompressedSizeInfoColGroup> CompressedSizeInfoColGroupParallel(int clen, int k) {
 		try {
 			final ExecutorService pool = CommonThreadPool.get(k);
-			final ArrayList<SizeEstimationTask> tasks = new ArrayList<>(clen);
-			for(int col = 0; col < clen; col++)
-				tasks.add(new SizeEstimationTask(col));
-
-			if(!_cs.transposed && _data.isInSparseFormat() && getNumColumns() < 1000) {
+			if(!_cs.transposed && !_data.isEmpty() && _data.isInSparseFormat()) {
 				LOG.debug("Extracting number of nonzeros in each column");
-				nnzCols = null;
 				List<Future<int[]>> nnzFutures = LibMatrixReorg.countNNZColumnsFuture(_data, k, pool);
-				List<Future<CompressedSizeInfoColGroup>> analysisFutures = pool.invokeAll(tasks);
 				for(Future<int[]> t : nnzFutures)
 					nnzCols = LibMatrixReorg.mergeNnzCounts(nnzCols, t.get());
-				return analysisFutures.stream().map(x -> getT(x)).collect(Collectors.toList());
 			}
-			else
-				return pool.invokeAll(tasks).stream().map(x -> getT(x)).collect(Collectors.toList());
 
-		}
-		catch(Exception e) {
-			LOG.error("Fallback to single threaded column info extraction", e);
-			return CompressedSizeInfoColGroup(clen);
-		}
-	}
+			CompressedSizeInfoColGroup[] res = new CompressedSizeInfoColGroup[clen];
+			final int blkz = Math.max(1, clen / (k * 10));
+			final ArrayList<SizeEstimationTask> tasks = new ArrayList<>(clen / blkz + 1);
+
+			if(blkz != 1)
+				LOG.debug("Extracting column samples in blocks of " + blkz);
+
+			for(int col = 0; col < clen; col += blkz)
+				tasks.add(new SizeEstimationTask(res, col, Math.min(clen, col + blkz)));
+
+			for(Future<Object> f : pool.invokeAll(tasks))
+				f.get();
+
+			pool.shutdown();
+			return Arrays.asList(res);
 
-	private <T> T getT(Future<T> x) {
-		try {
-			return x.get();
 		}
 		catch(Exception e) {
-			throw new DMLCompressionException("failed getting future colgroup info extraction", e);
+			throw new DMLCompressionException("Multithreaded first extraction failed", e);
 		}
 	}
 
-	private class SizeEstimationTask implements Callable<CompressedSizeInfoColGroup> {
-
-		private final int[] _cols;
+	private class SizeEstimationTask implements Callable<Object> {
+		final CompressedSizeInfoColGroup[] _res;
+		final int _cs;
+		final int _ce;
 
-		private SizeEstimationTask(int col) {
-			_cols = new int[] {col};
+		private SizeEstimationTask(CompressedSizeInfoColGroup[] res, int cs, int ce) {
+			_res = res;
+			_cs = cs;
+			_ce = ce;
 		}
 
 		@Override
-		public CompressedSizeInfoColGroup call() {
-			return getColGroupInfo(_cols);
+		public Object call() {
+			try {
+				for(int c = _cs; c < _ce; c++)
+					_res[c] = getColGroupInfo(new int[] {c});
+				return null;
+			}
+			catch(Exception e) {
+				throw new DMLCompressionException("ColGroup extraction failed", e);
+			}
 		}
 	}
 }
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 6dcf2e37b9..5eb33c735d 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
@@ -20,6 +20,7 @@
 package org.apache.sysds.runtime.compress.estim;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.estim.encoding.EmptyEncoding;
 import org.apache.sysds.runtime.compress.estim.encoding.IEncode;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
@@ -35,6 +36,8 @@ public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
 	@Override
 	public CompressedSizeInfoColGroup getColGroupInfo(int[] colIndexes, int estimate, int nrUniqueUpperBound) {
 		final IEncode map = IEncode.createFromMatrixBlock(_data, _cs.transposed, colIndexes);
+		if(map instanceof EmptyEncoding)
+			return new CompressedSizeInfoColGroup(colIndexes, getNumRows());
 		return getFacts(map, 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 1f35f5a290..a1160a7aca 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
@@ -40,6 +40,8 @@ public interface CompressedSizeEstimatorFactory {
 		final int nCols = cs.transposed ? data.getNumRows() : data.getNumColumns();
 		final double sparsity = data.getSparsity();
 		final int sampleSize = getSampleSize(cs, nRows, nCols, sparsity);
+		if(data.isEmpty())
+			return createExactEstimator(data, cs);
 		return createEstimator(data, cs, sampleSize, k, nRows);
 	}
 
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 fb0ab4b789..484f90e63f 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
@@ -24,6 +24,7 @@ import java.util.Random;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.estim.encoding.IEncode;
 import org.apache.sysds.runtime.compress.estim.sample.SampleEstimatorFactory;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
@@ -67,7 +68,8 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 
 	@Override
 	public CompressedSizeInfoColGroup getColGroupInfo(int[] colIndexes, int estimate, int maxDistinct) {
-		if(nnzCols != null && colIndexes.length == 1 && nnzCols[colIndexes[0]] == 0)
+		if(_data.isEmpty() || (nnzCols != null && colIndexes.length == 1 && nnzCols[colIndexes[0]] == 0) || (_cs.transposed &&
+			colIndexes.length == 1 && _data.isInSparseFormat() && _data.getSparseBlock().isEmpty(colIndexes[0])))
 			return new CompressedSizeInfoColGroup(colIndexes, getNumRows());
 
 		final IEncode map = IEncode.createFromMatrixBlock(_sample, _transposed, colIndexes);
@@ -99,54 +101,75 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	private CompressedSizeInfoColGroup extractInfo(IEncode map, int[] colIndexes, int maxDistinct) {
 		final EstimationFactors sampleFacts = map.extractFacts(colIndexes, _sampleSize, _data.getSparsity(),
 			_data.getSparsity());
-		final EstimationFactors em = scaleFactors(sampleFacts, colIndexes, maxDistinct);
+		final EstimationFactors em = scaleFactors(sampleFacts, colIndexes, maxDistinct, map.isDense());
 		return new CompressedSizeInfoColGroup(colIndexes, em, _cs.validCompressions, map);
 	}
 
-	private EstimationFactors scaleFactors(EstimationFactors sampleFacts, int[] colIndexes, int maxDistinct) {
+	private EstimationFactors scaleFactors(EstimationFactors sampleFacts, int[] colIndexes, int maxDistinct, boolean dense) {
 		final int numRows = getNumRows();
+		final int nCol = colIndexes.length;
 
 		final double scalingFactor = (double) numRows / _sampleSize;
 
 		final long nnz = calculateNNZ(colIndexes, scalingFactor);
-		final int numOffs = calculateOffs(sampleFacts, numRows, scalingFactor, colIndexes, nnz);
-		final int estDistinct = distinctCountScale(sampleFacts, numOffs, maxDistinct);
+		final int numOffs = calculateOffs(sampleFacts, numRows, scalingFactor, colIndexes, (int) nnz);
+		final int estDistinct = distinctCountScale(sampleFacts, numOffs, numRows, maxDistinct, dense, nCol);
 
 		// calculate the largest instance count.
 		final int maxLargestInstanceCount = numRows - estDistinct + 1;
-		final int scaledLargestInstanceCount = (int) Math.floor(sampleFacts.largestOff * scalingFactor);
-		final int largestInstanceCount = Math.min(maxLargestInstanceCount, scaledLargestInstanceCount);
+		final int scaledLargestInstanceCount = sampleFacts.largestOff < 0 ? numOffs /
+			estDistinct : (int) Math.floor(sampleFacts.largestOff * scalingFactor);
+		final int mostFrequentOffsetCount = Math.max(Math.min(maxLargestInstanceCount, scaledLargestInstanceCount),
+			numRows - numOffs);
 
 		final double overallSparsity = calculateSparsity(colIndexes, nnz, scalingFactor, sampleFacts.overAllSparsity);
-
-		// For safety add 10 percent more tuple sparsity to estimate since it can have a big impact
-		// on workload
-		final double tupleSparsity = Math.min(overallSparsity + 0.1, 1.0);
-
-		return new EstimationFactors(colIndexes.length, estDistinct, numOffs, largestInstanceCount,
-			sampleFacts.frequencies, sampleFacts.numSingle, numRows, sampleFacts.lossy, sampleFacts.zeroIsMostFrequent,
-			overallSparsity, tupleSparsity);
+		// For robustness safety add 10 percent more tuple sparsity
+		final double tupleSparsity = Math.min(overallSparsity * 1.3, 1.0); // increase sparsity by 30%.
+		try {
+			return new EstimationFactors(colIndexes.length, estDistinct, numOffs, mostFrequentOffsetCount,
+				sampleFacts.frequencies, sampleFacts.numSingle, numRows, sampleFacts.lossy, sampleFacts.zeroIsMostFrequent,
+				overallSparsity, tupleSparsity);
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException("Invalid construction of estimation factors with observed values:\n"
+				+ Arrays.toString(colIndexes) + " " + nnz + " " + numOffs + "  " + estDistinct + "  "
+				+ maxLargestInstanceCount + "  " + scaledLargestInstanceCount + " " + mostFrequentOffsetCount + " "
+				+ overallSparsity + " " + tupleSparsity + "\n" + nnzCols[colIndexes[0]], e);
+		}
 
 	}
 
-	private int distinctCountScale(EstimationFactors sampleFacts, int numOffs, int maxDistinct) {
+	private int distinctCountScale(EstimationFactors sampleFacts, int numOffs, int numRows, int maxDistinct, boolean dense, int nCol) {
 		// the frequencies of non empty entries.
 		final int[] freq = sampleFacts.frequencies;
 		if(freq == null || freq.length == 0)
-			return numOffs > 0 ? 1 : 0;
+			return numOffs; // very aggressive number of distinct
 		// sampled size is smaller than actual if there was empty rows.
 		// and the more we can reduce this value the more accurate the estimation will become.
 		final int sampledSize = sampleFacts.numOffs;
-		final int est = SampleEstimatorFactory.distinctCount(freq, numOffs, sampledSize, _cs.estimationType);
+		int est = SampleEstimatorFactory.distinctCount(freq, dense ? numRows : numOffs, sampledSize, _cs.estimationType);
+		if(est > 10000)
+			est += est * 0.5;
+		if(nCol > 4) // Increase estimate if we get into many columns cocoding to be safe
+			est += ((double)est) * ((double)nCol) / 10;
 		// Bound the estimate with the maxDistinct.
-		return Math.min(est, maxDistinct);
+		return Math.max(Math.min(est, Math.min(maxDistinct, numOffs)), 1);
 	}
 
 	private int calculateOffs(EstimationFactors sampleFacts, int numRows, double scalingFactor, int[] colIndexes,
-		long nnz) {
-		final int numCols = getNumColumns();
-		if(numCols == 1 || (nnzCols != null && colIndexes.length == 1))
-			return (int) nnz;
+		int nnz) {
+
+		if(getNumColumns() == 1)
+			return nnz;
+		else if(nnzCols != null) {
+			if(colIndexes.length == 1)
+				return nnzCols[colIndexes[0]];
+			else {
+				final int emptyTuples = sampleFacts.numRows - sampleFacts.numOffs;
+				final int estOffs = numRows - (int) Math.floor(emptyTuples * scalingFactor);
+				return Math.min(nnz, estOffs);
+			}
+		}
 		else {
 			final int emptyTuples = sampleFacts.numRows - sampleFacts.numOffs;
 			return numRows - (int) Math.floor(emptyTuples * scalingFactor);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
index f79067ac61..0c9e23f40e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
@@ -57,6 +57,11 @@ public class CompressedSizeInfo {
 		return this;
 	}
 
+	public void clearMaps(){
+		for(CompressedSizeInfoColGroup g : compressionInfo)
+			g.clearMap();
+	}
+
 	/**
 	 * Method for returning the calculated memory usage from this specific compression plan.
 	 * 
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 08fca55e84..7c330c1a1e 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
@@ -20,7 +20,7 @@
 package org.apache.sysds.runtime.compress.estim;
 
 import java.util.Arrays;
-import java.util.HashMap;
+import java.util.EnumMap;
 import java.util.Map;
 import java.util.Set;
 
@@ -45,7 +45,7 @@ public class CompressedSizeInfoColGroup {
 	private final double _cardinalityRatio;
 	private final long _minSize;
 	private final CompressionType _bestCompressionType;
-	private final Map<CompressionType, Long> _sizes;
+	private final EnumMap<CompressionType, Long> _sizes;
 
 	/**
 	 * Map containing a mapping to unique values, but not necessarily the actual values contained in this column group
@@ -81,9 +81,9 @@ public class CompressedSizeInfoColGroup {
 		_cols = columns;
 		_facts = new EstimationFactors(columns.length, 0, nRows);
 		_cardinalityRatio = 0;
-		_sizes = new HashMap<>();
+		_sizes = new EnumMap<>(CompressionType.class);
 		final CompressionType ct = CompressionType.EMPTY;
-		_sizes.put(ct, getCompressionSize(columns.length, ct, _facts));
+		_sizes.put(ct,  ColGroupSizes.estimateInMemorySizeEMPTY(columns.length));
 		_bestCompressionType = ct;
 		_minSize = _sizes.get(ct);
 		_map = null;
@@ -94,7 +94,7 @@ public class CompressedSizeInfoColGroup {
 		if(_sizes != null) {
 			Long s = _sizes.get(ct);
 			if(s == null)
-				throw new DMLCompressionException("Asked fro valid " + ct + " but got null. contains:" + _sizes);
+				throw new DMLCompressionException("Asked for valid " + ct + " but got null. contains:" + _sizes);
 			return _sizes.get(ct);
 		}
 		else
@@ -167,9 +167,9 @@ public class CompressedSizeInfoColGroup {
 		return _facts.numOffs < _facts.numRows;
 	}
 
-	private static Map<CompressionType, Long> calculateCompressionSizes(int numCols, EstimationFactors fact,
+	private static EnumMap<CompressionType, Long> calculateCompressionSizes(int numCols, EstimationFactors fact,
 		Set<CompressionType> validCompressionTypes) {
-		Map<CompressionType, Long> res = new HashMap<>();
+		EnumMap<CompressionType, Long> res = new EnumMap<CompressionType, Long>(CompressionType.class);
 		for(CompressionType ct : validCompressionTypes) {
 			long compSize = getCompressionSize(numCols, ct, fact);
 			if(compSize > 0)
@@ -191,7 +191,7 @@ public class CompressedSizeInfoColGroup {
 		switch(ct) {
 			case DeltaDDC: // TODO add proper extraction
 			case DDC:
-				nv = fact.numVals + (fact.zeroIsMostFrequent ? 1 : 0);
+				nv = fact.numVals + (fact.numOffs < fact.numRows ? 1 : 0);
 				// + 1 if the column contains zero
 				return ColGroupSizes.estimateInMemorySizeDDC(numCols, nv, fact.numRows, fact.tupleSparsity, fact.lossy);
 			case RLE:
@@ -209,9 +209,7 @@ public class CompressedSizeInfoColGroup {
 				return ColGroupSizes.estimateInMemorySizeSDC(numCols, fact.numVals, fact.numRows, fact.largestOff,
 					fact.tupleSparsity, fact.zeroIsMostFrequent, fact.lossy);
 			case CONST:
-				if(fact.numOffs == 0)
-					return -1;
-				else if(fact.numOffs == fact.numRows && fact.numVals == 1)
+				if(fact.numOffs == fact.numRows && fact.numVals == 1)
 					return ColGroupSizes.estimateInMemorySizeCONST(numCols, fact.tupleSparsity, fact.lossy);
 				else
 					return -1;
@@ -234,11 +232,7 @@ public class CompressedSizeInfoColGroup {
 		StringBuilder sb = new StringBuilder();
 		sb.append(this.getClass().getSimpleName());
 		sb.append("cols: " + Arrays.toString(_cols));
-		sb.append(" Best Type: " + _bestCompressionType);
-		sb.append(" Cardinality: ");
-		sb.append(_cardinalityRatio);
-		sb.append(" mostCommonFraction: ");
-		sb.append(getMostCommonFraction());
+		sb.append(String.format(" common: %4.3f", getMostCommonFraction()));
 		sb.append(" Sizes: ");
 		sb.append(_sizes);
 		sb.append(" facts: " + _facts);
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 f8b711b750..bfaa583d45 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
@@ -50,7 +50,7 @@ public class EstimationFactors {
 	/** The sparsity of the tuples them selves in isolation */
 	protected final double tupleSparsity;
 
-	protected EstimationFactors(int nCols, int numVals, int numRows) {
+	public EstimationFactors(int nCols, int numVals, int numRows) {
 		this.numVals = numVals;
 		this.numRows = numRows;
 		this.frequencies = null;
@@ -88,6 +88,14 @@ public class EstimationFactors {
 			throw new DMLCompressionException(
 				"Invalid number of instance of most common element should be lower than number of rows. " + largestOff
 					+ " > numRows: " + numRows);
+		if(numVals <= 0)
+			throw new DMLCompressionException("Should not use this constructor if empty");
+		if(numOffs <= 0)
+			throw new DMLCompressionException("Num offs are to low for this constructor");
+		if(numVals > numOffs)
+			throw new DMLCompressionException("Num vals cannot be greater than num offs");
+		if(largestOff < 0)
+			throw new DMLCompressionException("Invalid number of offset, should be greater than one");
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/ConstEncoding.java b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/ConstEncoding.java
index 4175be2ebb..b6c131eb40 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/ConstEncoding.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/ConstEncoding.java
@@ -40,11 +40,6 @@ public class ConstEncoding implements IEncode {
 		return 1;
 	}
 
-	@Override
-	public int size() {
-		return 1;
-	}
-
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -57,4 +52,9 @@ public class ConstEncoding implements IEncode {
 		return new EstimationFactors(cols.length, 1, nRows, nRows, counts, 0, nRows, false, false, matrixSparsity,
 			tupleSparsity);
 	}
+
+	@Override
+	public boolean isDense() {
+		return true;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/DenseEncoding.java b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/DenseEncoding.java
index 844ca95149..43b005bbc3 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/DenseEncoding.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/DenseEncoding.java
@@ -19,7 +19,8 @@
 
 package org.apache.sysds.runtime.compress.estim.encoding;
 
-import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
@@ -30,23 +31,14 @@ import org.apache.sysds.runtime.compress.estim.EstimationFactors;
 public class DenseEncoding implements IEncode {
 
 	private final AMapToData map;
-	private final int[] counts;
-
-	protected DenseEncoding(AMapToData map, int[] counts) {
-		this.map = map;
-		this.counts = counts;
-	}
 
 	public DenseEncoding(AMapToData map) {
 		this.map = map;
-		this.counts = map.getCounts(new int[map.getUnique()]);
 	}
 
 	@Override
 	public DenseEncoding combine(IEncode e) {
-		if((long) (getUnique()) * e.getUnique() > Integer.MAX_VALUE)
-			throw new DMLCompressionException("Invalid input to combine.");
-		else if(e instanceof EmptyEncoding || e instanceof ConstEncoding)
+		if(e instanceof EmptyEncoding || e instanceof ConstEncoding)
 			return this;
 		else if(e instanceof SparseEncoding)
 			return combineSparse((SparseEncoding) e);
@@ -56,107 +48,149 @@ public class DenseEncoding implements IEncode {
 
 	protected DenseEncoding combineSparse(SparseEncoding e) {
 		final int maxUnique = e.getUnique() * getUnique();
-		final int nRows = size();
+		final int size = map.size();
 		final int nVl = getUnique();
-		final int defR = (e.getUnique() - 1) * nVl;
-		final AMapToData m = MapToFactory.create(maxUnique, maxUnique + 1);
-		final AMapToData d = MapToFactory.create(nRows, maxUnique);
 
-		// iterate through indexes that are in the sparse encoding
+		// temp result
+		final AMapToData ret = MapToFactory.create(size, maxUnique);
+
+		// Iteration 1 copy dense data.
+		ret.copy(map);
+
+		// Iterate through indexes that are in the sparse encoding
 		final AIterator itr = e.off.getIterator();
 		final int fr = e.off.getOffsetToLast();
 
-		int newUID = 1;
-		int r = 0;
-		for(; r <= fr; r++) {
-			final int ir = itr.value();
-			if(ir == r) {
-				final int nv = map.getIndex(ir) + e.map.getIndex(itr.getDataIndex()) * nVl;
-				newUID = addVal(nv, r, m, newUID, d);
-				if(ir >= fr) {
-					r++;
-					break;
-				}
-				else
-					itr.next();
-			}
-			else {
-				final int nv = map.getIndex(r) + defR;
-				newUID = addVal(nv, r, m, newUID, d);
-			}
+		int ir = itr.value();
+		while(ir < fr) {
+			ret.set(ir, ret.getIndex(ir) + ((e.map.getIndex(itr.getDataIndex()) + 1) * nVl));
+			ir = itr.next();
 		}
+		ret.set(fr, ret.getIndex(fr) + ((e.map.getIndex(itr.getDataIndex()) + 1) * nVl));
 
-		for(; r < nRows; r++) {
-			final int nv = map.getIndex(r) + defR;
-			newUID = addVal(nv, r, m, newUID, d);
-		}
+		// Iteration 2 reassign indexes.
+		if(maxUnique + nVl > size)
+			return combineSparseHashMap(ret);
+		else
+			return combineSparseMapToData(ret, maxUnique, nVl);
+	}
 
-		// set unique.
-		d.setUnique(newUID - 1);
-		return new DenseEncoding(d);
+	private final DenseEncoding combineSparseHashMap(final AMapToData ret) {
+		final int size = ret.size();
+		final Map<Integer, Integer> m = new HashMap<>(size);
+		for(int r = 0; r < size; r++) {
+			final int prev = ret.getIndex(r);
+			final int v = m.size();
+			final Integer mv = m.putIfAbsent(prev, v);
+			if(mv == null)
+				ret.set(r, v);
+			else
+				ret.set(r, mv);
+		}
+		return new DenseEncoding(MapToFactory.resize(ret, m.size()));
 	}
 
-	private static int addVal(int nv, int r, AMapToData map, int newId, AMapToData d) {
-		int mv = map.getIndex(nv);
-		if(mv == 0)
-			mv = map.setAndGet(nv, newId++);
-		d.set(r, mv - 1);
-		return newId;
+	private final DenseEncoding combineSparseMapToData(final AMapToData ret, final int maxUnique, final int nVl) {
+		final int size = ret.size();
+		final AMapToData m = MapToFactory.create(maxUnique, maxUnique + nVl);
+		int newUID = 1;
+		for(int r = 0; r < size; r++) {
+			final int prev = ret.getIndex(r);
+			int mv = m.getIndex(prev);
+			if(mv == 0)
+				mv = m.setAndGet(prev, newUID++);
+			ret.set(r, mv - 1);
+		}
+		// Potential iteration 3 of resize
+		return new DenseEncoding(MapToFactory.resize(ret, newUID - 1));
 	}
 
-	protected DenseEncoding combineDense(DenseEncoding other) {
-		if(map == other.map)
-			return this; // unlikely to happen but cheap to compute
-		final AMapToData d = combine(map, other.map);
-		return new DenseEncoding(d);
+	protected DenseEncoding combineDense(final DenseEncoding other) {
+		try {
+
+			if(map == other.map) // same object
+				return this; // unlikely to happen but cheap to compute
+
+			final AMapToData lm = map;
+			final AMapToData rm = other.map;
+
+			final int nVL = lm.getUnique();
+			final int nVR = rm.getUnique();
+			final int size = map.size();
+			final int maxUnique = nVL * nVR;
+
+			final AMapToData ret = MapToFactory.create(size, maxUnique);
+
+			if(maxUnique > size)
+				return combineDenseWithHashMap(lm, rm, size, nVL, ret);
+			else
+				return combineDenseWithMapToData(lm, rm, size, nVL, ret, maxUnique);
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException("Failed to combine two dense\n" + this + "\n" + other, e);
+		}
 	}
 
-	public static AMapToData combine(AMapToData left, AMapToData right) {
-		if(left == null)
-			return right;
-		else if(right == null)
-			return left;
+	protected final DenseEncoding combineDenseWithHashMap(final AMapToData lm, final AMapToData rm, final int size,
+		final int nVL, final AMapToData ret) {
+		final Map<Integer, Integer> m = new HashMap<>(size);
 
-		final int nVL = left.getUnique();
-		final int nVR = right.getUnique();
-		final int size = left.size();
-		final int maxUnique = nVL * nVR;
+		for(int r = 0; r < size; r++)
+			addValHashMap(lm.getIndex(r) + rm.getIndex(r) * nVL, r, m, ret);
+		return new DenseEncoding(MapToFactory.resize(ret, m.size()));
 
-		final AMapToData ret = MapToFactory.create(size, maxUnique);
-		final AMapToData map = MapToFactory.create(maxUnique, maxUnique + 1);
+	}
 
+	protected final DenseEncoding combineDenseWithMapToData(final AMapToData lm, final AMapToData rm, final int size,
+		final int nVL, final AMapToData ret, final int maxUnique) {
+		final AMapToData m = MapToFactory.create(maxUnique, maxUnique + 1);
 		int newUID = 1;
-		for(int i = 0; i < size; i++) {
-			final int nv = left.getIndex(i) + right.getIndex(i) * nVL;
-			newUID = addVal(nv, i, map, newUID, ret);
-		}
+		for(int r = 0; r < size; r++)
+			newUID = addValMapToData(lm.getIndex(r) + rm.getIndex(r) * nVL, r, m, newUID, ret);
+		return new DenseEncoding(MapToFactory.resize(ret, newUID - 1));
+	}
 
-		ret.setUnique(newUID - 1);
-		return ret;
+	protected static int addValMapToData(final int nv, final int r, final AMapToData map, int newId,
+		final AMapToData d) {
+		int mv = map.getIndex(nv);
+		if(mv == 0)
+			mv = map.setAndGet(nv, newId++);
+		d.set(r, mv - 1);
+		return newId;
 	}
 
-	@Override
-	public int getUnique() {
-		return counts.length;
+	protected static void addValHashMap(final int nv, final int r, final Map<Integer, Integer> map, final AMapToData d) {
+		final int v = map.size();
+		final Integer mv = map.putIfAbsent(nv, v);
+		if(mv == null)
+			d.set(r, v);
+		else
+			d.set(r, mv);
 	}
 
 	@Override
-	public int size() {
-		return map.size();
+	public int getUnique() {
+		return map.getUnique();
 	}
 
 	@Override
 	public EstimationFactors extractFacts(int[] cols, int nRows, double tupleSparsity, double matrixSparsity) {
 		int largestOffs = 0;
 
+		int[] counts = map.getCounts(new int[map.getUnique()]);
 		for(int i = 0; i < counts.length; i++)
 			if(counts[i] > largestOffs)
 				largestOffs = counts[i];
 
-		return new EstimationFactors(cols.length, counts.length, nRows, largestOffs, counts, 0, nRows, false, false,
+		return new EstimationFactors(cols.length, map.getUnique(), nRows, largestOffs, counts, 0, nRows, false, false,
 			matrixSparsity, tupleSparsity);
 	}
 
+	@Override
+	public boolean isDense() {
+		return true;
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -164,9 +198,6 @@ public class DenseEncoding implements IEncode {
 		sb.append("\n");
 		sb.append("mapping: ");
 		sb.append(map);
-		sb.append("\n");
-		sb.append("counts:  ");
-		sb.append(Arrays.toString(counts));
 		return sb.toString();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/EmptyEncoding.java b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/EmptyEncoding.java
index a437c58e99..30801c2f8e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/EmptyEncoding.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/EmptyEncoding.java
@@ -23,10 +23,7 @@ import org.apache.sysds.runtime.compress.estim.EstimationFactors;
 
 /** Empty encoding for cases where the entire group of columns is zero */
 public class EmptyEncoding implements IEncode {
-
-	/** always a empty int array */
-	private static final int[] counts = new int[] {};
-
+	
 	// empty constructor
 	public EmptyEncoding() {
 	}
@@ -41,11 +38,6 @@ public class EmptyEncoding implements IEncode {
 		return 1;
 	}
 
-	@Override
-	public int size() {
-		return 0;
-	}
-
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -55,6 +47,11 @@ public class EmptyEncoding implements IEncode {
 
 	@Override
 	public EstimationFactors extractFacts(int[] cols, int nRows, double tupleSparsity, double matrixSparsity) {
-		return new EstimationFactors(cols.length, 0, 0, nRows, counts, 0, nRows, false, true, 0, 0);
+		return new EstimationFactors(cols.length, 0, 0);
+	}
+	
+	@Override
+	public  boolean isDense(){
+		return false;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/IEncode.java b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/IEncode.java
index 712fd39ce8..d8303c8256 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/IEncode.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/IEncode.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
@@ -54,11 +55,11 @@ public interface IEncode {
 			return createWithReader(m, rowCols, transposed);
 	}
 
-	public static IEncode createFromMatrixBlockDelta(MatrixBlock m, boolean transposed, int[] rowCols){
+	public static IEncode createFromMatrixBlockDelta(MatrixBlock m, boolean transposed, int[] rowCols) {
 		return createFromMatrixBlockDelta(m, transposed, rowCols, transposed ? m.getNumColumns() : m.getNumRows());
 	}
 
-	public static IEncode createFromMatrixBlockDelta(MatrixBlock m, boolean transposed, int[] rowCols, int nVals){
+	public static IEncode createFromMatrixBlockDelta(MatrixBlock m, boolean transposed, int[] rowCols, int nVals) {
 		throw new NotImplementedException();
 	}
 
@@ -77,7 +78,7 @@ public interface IEncode {
 			return createFromDense(m, rowCol);
 	}
 
-	public static IEncode createFromDenseTransposed(MatrixBlock m, int row) {
+	private static IEncode createFromDenseTransposed(MatrixBlock m, int row) {
 		final DoubleCountHashMap map = new DoubleCountHashMap(16);
 		final DenseBlock db = m.getDenseBlock();
 		final int off = db.pos(row);
@@ -94,8 +95,8 @@ public interface IEncode {
 		if(nUnique == 1)
 			return new ConstEncoding(m.getNumColumns());
 
-		if(map.getOrDefault(0, -1) * 10 > nCol * 4) { // 40 %
-			final int[] counts = map.getUnorderedCountsAndReplaceWithUIDsWithout0(); // map.getUnorderedCountsAndReplaceWithUIDs();
+		if(map.getOrDefault(0, -1) > nCol / 4) {
+			map.replaceWithUIDsNoZero();
 			final int zeroCount = map.get(0);
 			final int nV = nCol - zeroCount;
 			final IntArrayList offsets = new IntArrayList(nV);
@@ -106,17 +107,15 @@ public interface IEncode {
 			for(int i = off, r = 0, di = 0; i < end; i++, r++) {
 				if(vals[i] != 0) {
 					offsets.appendValue(r);
-					d.set(di++, map.get(vals[i]) );
+					d.set(di++, map.get(vals[i]));
 				}
 			}
 
 			final AOffset o = OffsetFactory.createOffset(offsets);
-			return new SparseEncoding(d, o, zeroCount, counts, nCol);
+			return new SparseEncoding(d, o, zeroCount, nCol);
 		}
 		else {
-			// Allocate counts, and iterate once to replace counts with u ids
-			final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
-
+			map.replaceWithUIDs();
 			// Create output map
 			final AMapToData d = MapToFactory.create(nCol, nUnique);
 
@@ -124,11 +123,11 @@ public interface IEncode {
 			for(int i = off, r = 0; i < end; i++, r++)
 				d.set(r, map.get(vals[i]));
 
-			return new DenseEncoding(d, counts);
+			return new DenseEncoding(d);
 		}
 	}
 
-	public static IEncode createFromSparseTransposed(MatrixBlock m, int row) {
+	private static IEncode createFromSparseTransposed(MatrixBlock m, int row) {
 		final DoubleCountHashMap map = new DoubleCountHashMap(16);
 		final SparseBlock sb = m.getSparseBlock();
 		if(sb.isEmpty(row))
@@ -136,6 +135,7 @@ public interface IEncode {
 		final int apos = sb.pos(row);
 		final int alen = sb.size(row) + apos;
 		final double[] avals = sb.values(row);
+		final int[] aix = sb.indexes(row);
 
 		// Iteration 1 of non zero values, make Count HashMap.
 		for(int i = apos; i < alen; i++) // sequential of non zero cells.
@@ -143,25 +143,42 @@ public interface IEncode {
 
 		final int nUnique = map.size();
 
-		// Allocate counts
-		final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
-
-		// Create output map
-		final AMapToData d = MapToFactory.create(alen - apos, nUnique);
+		map.replaceWithUIDs();
 
-		// Iteration 2 of non zero values, make either a IEncode Dense or sparse map.
-		for(int i = apos, j = 0; i < alen; i++, j++)
-			d.set(j, map.get(avals[i]));
-
-		// Iteration 3 of non zero indexes, make a Offset Encoding to know what cells are zero and not.
-		// not done yet
-		AOffset o = OffsetFactory.createOffset(sb.indexes(row), apos, alen);
+		final int nCol = m.getNumColumns();
+		if(alen - apos > nCol / 4) { // return a dense encoding
+			final AMapToData d = MapToFactory.create(nCol, nUnique + 1);
+			// Since the dictionary is allocated with zero then we exploit that here and
+			// only iterate through non zero entries.
+			for(int i = apos; i < alen; i++)
+				// plus one to assign unique IDs.
+				d.set(aix[i], map.get(avals[i]) + 1);
+
+			return new DenseEncoding(d);
+		}
+		else { // return a sparse encoding
+			// Create output map
+			final AMapToData d = MapToFactory.create(alen - apos, nUnique);
+
+			// Iteration 2 of non zero values, make either a IEncode Dense or sparse map.
+			for(int i = apos, j = 0; i < alen; i++, j++)
+				d.set(j, map.get(avals[i]));
+
+			// Iteration 3 of non zero indexes, make a Offset Encoding to know what cells are zero and not.
+			// not done yet
+			AOffset o = OffsetFactory.createOffset(aix, apos, alen);
+			final int zero = m.getNumColumns() - o.getSize();
+			try {
+				return new SparseEncoding(d, o, zero, m.getNumColumns());
+			}
+			catch(Exception e) {
+				throw new DMLCompressionException(Arrays.toString(aix), e);
+			}
 
-		final int zero = m.getNumColumns() - o.getSize();
-		return new SparseEncoding(d, o, zero, counts, m.getNumColumns());
+		}
 	}
 
-	public static IEncode createFromDense(MatrixBlock m, int col) {
+	private static IEncode createFromDense(MatrixBlock m, int col) {
 		final DenseBlock db = m.getDenseBlock();
 		if(!db.isContiguous())
 			throw new NotImplementedException("Not Implemented non contiguous dense matrix encoding for sample");
@@ -180,13 +197,13 @@ public interface IEncode {
 		if(nUnique == 1)
 			return new ConstEncoding(m.getNumColumns());
 
-		if(map.getOrDefault(0, -1) * 10 > nRow * 4) { // 40 %
-			final int[] counts = map.getUnorderedCountsAndReplaceWithUIDsWithout0();
+		if(map.getOrDefault(0, -1) > nRow / 4) {
+			map.replaceWithUIDsNoZero();
 			final int zeroCount = map.get(0);
 			final int nV = m.getNumRows() - zeroCount;
 			final IntArrayList offsets = new IntArrayList(nV);
 
-			final AMapToData d = MapToFactory.create(nV, nUnique);
+			final AMapToData d = MapToFactory.create(nV, nUnique - 1);
 
 			for(int i = off, r = 0, di = 0; i < end; i += nCol, r++) {
 				if(vals[i] != 0) {
@@ -197,27 +214,26 @@ public interface IEncode {
 
 			final AOffset o = OffsetFactory.createOffset(offsets);
 
-			return new SparseEncoding(d, o, zeroCount, counts, nRow);
+			return new SparseEncoding(d, o, zeroCount, nRow);
 		}
 		else {
 			// Allocate counts, and iterate once to replace counts with u ids
-			final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
-
+			map.replaceWithUIDs();
 			final AMapToData d = MapToFactory.create(nRow, nUnique);
 			// Iteration 2, make final map
 			for(int i = off, r = 0; i < end; i += nCol, r++)
 				d.set(r, map.get(vals[i]));
-			return new DenseEncoding(d, counts);
+			return new DenseEncoding(d);
 		}
 	}
 
-	public static IEncode createFromSparse(MatrixBlock m, int col) {
+	private static IEncode createFromSparse(MatrixBlock m, int col) {
 
 		final DoubleCountHashMap map = new DoubleCountHashMap(16);
 		final SparseBlock sb = m.getSparseBlock();
 
-		final double guessedNumberOfNonZero = Math.min(4, Math.ceil((double)m.getNumRows() * m.getSparsity()));
-		final IntArrayList offsets = new IntArrayList((int)guessedNumberOfNonZero);
+		final double guessedNumberOfNonZero = Math.min(4, Math.ceil((double) m.getNumRows() * m.getSparsity()));
+		final IntArrayList offsets = new IntArrayList((int) guessedNumberOfNonZero);
 
 		// Iteration 1 of non zero values, make Count HashMap.
 		for(int r = 0; r < m.getNumRows(); r++) { // Horrible performance but ... it works.
@@ -236,16 +252,12 @@ public interface IEncode {
 			return new EmptyEncoding();
 
 		final int nUnique = map.size();
-		final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
+		map.replaceWithUIDs();
 
-		int sumCounts = 0;
-		for(int c : counts)
-			sumCounts += c;
-
-		final AMapToData d = MapToFactory.create(sumCounts, nUnique);
+		final AMapToData d = MapToFactory.create(offsets.size(), nUnique);
 
 		// Iteration 2 of non zero values, make either a IEncode Dense or sparse map.
-		for(int off = 0, r = 0; off < sumCounts; r++) {
+		for(int off = 0, r = 0; off < offsets.size(); r++) {
 			if(sb.isEmpty(r))
 				continue;
 			final int apos = sb.pos(r);
@@ -260,11 +272,11 @@ public interface IEncode {
 		// Iteration 3 of non zero indexes, make a Offset Encoding to know what cells are zero and not.
 		AOffset o = OffsetFactory.createOffset(offsets);
 
-		final int zero = m.getNumRows() - sumCounts;
-		return new SparseEncoding(d, o, zero, counts, m.getNumRows());
+		final int zero = m.getNumRows() - offsets.size();
+		return new SparseEncoding(d, o, zero, m.getNumRows());
 	}
 
-	public static IEncode createWithReader(MatrixBlock m, int[] rowCols, boolean transposed) {
+	private static IEncode createWithReader(MatrixBlock m, int[] rowCols, boolean transposed) {
 		final ReaderColumnSelection reader1 = ReaderColumnSelection.createReader(m, rowCols, transposed);
 		final int nRows = transposed ? m.getNumColumns() : m.getNumRows();
 		final DblArrayCountHashMap map = new DblArrayCountHashMap(16, rowCols.length);
@@ -280,62 +292,40 @@ public interface IEncode {
 
 		if(offsets.size() == 0)
 			return new EmptyEncoding();
-
-		if(map.size() == 1 && offsets.size() == nRows)
+		else if(map.size() == 1 && offsets.size() == nRows)
 			return new ConstEncoding(nRows);
 
-		if(offsets.size() < nRows) {
-			// there was fewer offsets than rows.
-			if(offsets.size() < nRows / 2) {
-				// Output encoded Sparse since there is more than half empty.
-				final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
-				final int zeros = nRows - offsets.size();
-				return createWithReaderSparse(m, map, zeros, counts, rowCols, offsets, nRows, transposed);
-			}
-			else {
-				// Output Encoded dense since there is not enough common values.
-				// TODO add Common group, that allows to now allocate this extra cell
-				final int[] counts = map.getUnorderedCountsAndReplaceWithUIDsWithExtraCell();
-				counts[counts.length - 1] = nRows - offsets.size();
-				return createWithReaderDense(m, map, counts, rowCols, nRows, transposed);
-			}
+		map.replaceWithUIDs();
+		if(offsets.size() < nRows / 4) {
+			// Output encoded sparse since there is very empty.
+			final int zeros = nRows - offsets.size();
+			return createWithReaderSparse(m, map, zeros, rowCols, offsets, nRows, transposed);
 		}
-		else {
-			// TODO add Common group, that allows to allocate with one of the map entries as the common value.
-			// the input was fully dense.
+		else
+			return createWithReaderDense(m, map, rowCols, nRows, transposed, offsets.size() < nRows);
 
-			final int[] counts = map.getUnorderedCountsAndReplaceWithUIDs();
-			return createWithReaderDense(m, map, counts, rowCols, nRows, transposed);
-		}
 	}
 
-	public static IEncode createWithReaderDense(MatrixBlock m, DblArrayCountHashMap map, int[] counts, int[] rowCols,
-		int nRows, boolean transposed) {
+	private static IEncode createWithReaderDense(MatrixBlock m, DblArrayCountHashMap map, int[] rowCols, int nRows,
+		boolean transposed, boolean zero) {
 		// Iteration 2,
+		final int unique = map.size() + (zero ? 1 : 0);
 		final ReaderColumnSelection reader2 = ReaderColumnSelection.createReader(m, rowCols, transposed);
-		final AMapToData d = MapToFactory.create(nRows, counts.length);
-		final int def = counts.length - 1;
+		final AMapToData d = MapToFactory.create(nRows, unique);
 
-		DblArray cellVals = reader2.nextRow();
-		int r = 0;
-		while(r < nRows && cellVals != null) {
-			final int row = reader2.getCurrentRowIndex();
-			if(row == r) {
-				d.set(row, map.get(cellVals));
-				cellVals = reader2.nextRow();
-			}
-			else
-				d.set(r, def);
-			r++;
-		}
+		DblArray cellVals;
+		if(zero)
+			while((cellVals = reader2.nextRow()) != null)
+				d.set(reader2.getCurrentRowIndex(), map.get(cellVals) + 1);
+		else
+			while((cellVals = reader2.nextRow()) != null)
+				d.set(reader2.getCurrentRowIndex(), map.get(cellVals));
 
-		while(r < nRows)
-			d.set(r++, def);
-		return new DenseEncoding(d, counts);
+		return new DenseEncoding(d);
 	}
 
-	public static IEncode createWithReaderSparse(MatrixBlock m, DblArrayCountHashMap map, int zeros, int[] counts,
-		int[] rowCols, IntArrayList offsets, int nRows, boolean transposed) {
+	private static IEncode createWithReaderSparse(MatrixBlock m, DblArrayCountHashMap map, int zeros, int[] rowCols,
+		IntArrayList offsets, int nRows, boolean transposed) {
 		final ReaderColumnSelection reader2 = ReaderColumnSelection.createReader(m, rowCols, transposed);
 		DblArray cellVals = reader2.nextRow();
 
@@ -348,20 +338,28 @@ public interface IEncode {
 			cellVals = reader2.nextRow();
 		}
 
-		// iteration 3 of non zero indexes,
 		final AOffset o = OffsetFactory.createOffset(offsets);
 
-		return new SparseEncoding(d, o, zeros, counts, nRows);
+		return new SparseEncoding(d, o, zeros, nRows);
 	}
 
+	/**
+	 * Combine two encodings, note it should be guaranteed by the caller that the number of unique multiplied does not
+	 * overflow Integer.
+	 * 
+	 * @param e The other side to combine with
+	 * @return The combined encoding
+	 */
 	public IEncode combine(IEncode e);
 
 	public int getUnique();
 
-	public int size();
-
-	// public int[] getCounts();
-
 	public EstimationFactors extractFacts(int[] cols, int nRows, double tupleSparsity, double matrixSparsity);
 
+
+	/**
+	 * Signify if the counts are including zero or without zero.
+	 * @return is dense
+	 */
+	public abstract boolean isDense();
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/SparseEncoding.java b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/SparseEncoding.java
index 439bd12988..31f970f5c2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/SparseEncoding.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/encoding/SparseEncoding.java
@@ -19,8 +19,6 @@
 
 package org.apache.sysds.runtime.compress.estim.encoding;
 
-import java.util.Arrays;
-
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
@@ -45,40 +43,35 @@ public class SparseEncoding implements IEncode {
 	/** Count of Zero tuples in this encoding */
 	protected final int zeroCount;
 
-	/** Count of non zero tuples in this encoding */
-	protected final int[] counts;
-
-	protected SparseEncoding(AMapToData map, AOffset off, int zeroCount, int[] counts, int nRows) {
+	protected SparseEncoding(AMapToData map, AOffset off, int zeroCount, int nRows) {
 		this.map = map;
 		this.off = off;
 		this.zeroCount = zeroCount;
-		this.counts = counts;
 		this.nRows = nRows;
+		if(off.getOffsetToLast() > nRows)
+			throw new DMLCompressionException("Invalid Sparse Encoding because offsets are calculated incorrectly");
 	}
 
 	@Override
 	public IEncode combine(IEncode e) {
-		if((long) (getUnique()) * e.getUnique() > Integer.MAX_VALUE)
-			throw new DMLCompressionException("Invalid input to combine.");
-		else if(e instanceof EmptyEncoding || e instanceof ConstEncoding)
+		if(e instanceof EmptyEncoding || e instanceof ConstEncoding)
 			return this;
-		else if(e instanceof SparseEncoding)
+		else if(e instanceof SparseEncoding) {
+			SparseEncoding es = (SparseEncoding) e;
+			if(es.off == off && es.map == map)
+				return this;
 			return combineSparse((SparseEncoding) e);
+		}
 		else
 			return e.combine(this);
 	}
 
 	protected IEncode combineSparse(SparseEncoding e) {
-		if(e.map == map && e.off == off)
-			return this; // unlikely to happen but cheap to compute therefore this skip is included.
-
+		if(e.nRows != nRows)
+			throw new DMLCompressionException("invalid number of rows");
 		final int maxUnique = e.getUnique() * getUnique();
 		final int[] d = new int[maxUnique - 1];
 
-		// We need at least this size of offsets, but i don't know if i need more.
-		final IntArrayList retOff = new IntArrayList(Math.max(e.size(), this.size()));
-		final IntArrayList tmpVals = new IntArrayList(Math.max(e.size(), this.size()));
-
 		final int fl = off.getOffsetToLast();
 		final int fr = e.off.getOffsetToLast();
 		final AIterator itl = off.getIterator();
@@ -87,19 +80,33 @@ public class SparseEncoding implements IEncode {
 		final int nVl = getUnique();
 		final int nVr = e.getUnique();
 
+		final int sl = map.size();
+		final int sr = e.map.size();
+
+		if(sl + sr > nRows / 2)
+			return combineSparseToDense(map, e.map, itl, itr, fl, fr, nVl, nVr, d, nRows, maxUnique);
+
+		final IntArrayList retOff = new IntArrayList(Math.max(sr, sl));
+		final IntArrayList tmpVals = new IntArrayList(Math.max(sr, sl));
+
 		final int unique = combineSparse(map, e.map, itl, itr, retOff, tmpVals, fl, fr, nVl, nVr, d);
 
-		if(retOff.size() < nRows * 0.2) {
-			final AOffset o = OffsetFactory.createOffset(retOff);
-			final AMapToData retMap = MapToFactory.create(tmpVals.size(), tmpVals.extractValues(), unique);
-			return new SparseEncoding(retMap, o, nRows - retOff.size(), retMap.getCounts(new int[unique - 1]), nRows);
+		if(retOff.size() < nRows / 4) {
+			try {
+				final AOffset o = OffsetFactory.createOffset(retOff);
+				final AMapToData retMap = MapToFactory.create(tmpVals.size(), tmpVals.extractValues(), unique - 1);
+				return new SparseEncoding(retMap, o, nRows - retOff.size(), nRows);
+			}
+			catch(Exception ex) {
+				throw new DMLCompressionException("Failed combining sparse " + retOff + " " +this + "  " + e, ex);
+			}
 		}
 		else {
+			// there will always be a zero therefore unique is not subtracted one.
+			// if there is no zeros this will not be valid and crash.
 			final AMapToData retMap = MapToFactory.create(nRows, unique);
-			retMap.fill(unique - 1);
 			for(int i = 0; i < retOff.size(); i++)
-				retMap.set(retOff.get(i), tmpVals.get(i));
-
+				retMap.set(retOff.get(i), tmpVals.get(i) + 1);
 			return new DenseEncoding(retMap);
 		}
 	}
@@ -109,84 +116,104 @@ public class SparseEncoding implements IEncode {
 
 		final int defR = (nVr - 1) * nVl;
 		final int defL = nVl - 1;
-
-		boolean doneL = false;
-		boolean doneR = false;
 		int newUID = 1;
-		while(true) {
-			final int il = itl.value();
-			final int ir = itr.value();
-			if(il == ir) {
-				// Both sides have a value.
+		int il = itl.value();
+		int ir = itr.value();
+
+		if(il == fl && ir == fr) { // easy both only have one value
+			if(fl == fr) {// both on same row
+				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
+				return addVal(nv, il, d, newUID, tmpVals, retOff);
+			}
+			else if(fl < fr) {// fl is first
+				newUID = addVal(lMap.getIndex(itl.getDataIndex()) + defR, il, d, newUID, tmpVals, retOff);
+				return addVal(rMap.getIndex(itr.getDataIndex()) * nVl + defL, ir, d, newUID, tmpVals, retOff);
+			}
+			else {// fl is last
+				newUID = addVal(rMap.getIndex(itr.getDataIndex()) * nVl + defL, ir, d, newUID, tmpVals, retOff);
+				return addVal(lMap.getIndex(itl.getDataIndex()) + defR, il, d, newUID, tmpVals, retOff);
+			}
+		}
+
+		while(il < fl && ir < fr) {
+			if(il == ir) {// Both sides have a value same row.
 				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
 				newUID = addVal(nv, il, d, newUID, tmpVals, retOff);
-				if(il >= fl || ir >= fr) {
-					if(il < fl)
-						itl.next();
-					else
-						doneL = true;
-					if(ir < fr)
-						itr.next();
-					else
-						doneR = true;
-					break;
-				}
-				// both sides.
-				itl.next();
-				itr.next();
-			}
-			else if(il < ir) {
-				// left side have a value before right
+				il = itl.next();
+				ir = itr.next();
+			}
+			else if(il < ir) { // left side have a value before right
 				final int nv = lMap.getIndex(itl.getDataIndex()) + defR;
 				newUID = addVal(nv, il, d, newUID, tmpVals, retOff);
-				if(il >= fl) {
-					doneL = true;
-					break;
-				}
-				itl.next();
+				il = itl.next();
 			}
-			else {
-				// right side have a value before left
+			else {// right side have a value before left
 				final int nv = rMap.getIndex(itr.getDataIndex()) * nVl + defL;
 				newUID = addVal(nv, ir, d, newUID, tmpVals, retOff);
-				if(ir >= fr) {
-					doneR = true;
-					break;
-				}
-				itr.next();
+				ir = itr.next();
 			}
 		}
 
-		// process stragglers
-		if(!doneL) { // If there is stragglers in the left side
-			while(true) {
-				final int il = itl.value();
-				final int ir = itr.value();
-				int nv;
-				if(ir == il)
-					nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
-				else
-					nv = lMap.getIndex(itl.getDataIndex()) + defR;
+		newUID = combineSparseTail(lMap, rMap, itl, itr, retOff, tmpVals, fl, fr, nVl, nVr, d, newUID);
+
+		return newUID;
+	}
+
+	private static int combineSparseTail(AMapToData lMap, AMapToData rMap, AIterator itl, AIterator itr,
+		IntArrayList retOff, IntArrayList tmpVals, int fl, int fr, int nVl, int nVr, int[] d, int newUID) {
+		final int defR = (nVr - 1) * nVl;
+		final int defL = nVl - 1;
+		int il = itl.value();
+		int ir = itr.value();
+
+		if(il < fl) {
+			while(il < fr && il < fl) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + defR;
+				newUID = addVal(nv, il, d, newUID, tmpVals, retOff);
+				il = itl.next();
+			}
+			if(fl == fr) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
+				return addVal(nv, il, d, newUID, tmpVals, retOff);
+			}
+			else if(il == fr) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
 				newUID = addVal(nv, il, d, newUID, tmpVals, retOff);
-				if(il >= fl)
-					break;
-				itl.next();
+				il = itl.next();
+			}
+			else {
+				final int nv = rMap.getIndex(itr.getDataIndex()) * nVl + defL;
+				newUID = addVal(nv, fr, d, newUID, tmpVals, retOff);
+			}
+			while(il < fl) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + defR;
+				newUID = addVal(nv, il, d, newUID, tmpVals, retOff);
+				il = itl.next();
 			}
 		}
-		else if(!doneR) {// If there is stragglers in the right side
-			while(true) {
-				final int il = itl.value();
-				final int ir = itr.value();
-				int nv;
-				if(ir == il)
-					nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
-				else
-					nv = rMap.getIndex(itr.getDataIndex()) * nVl + defL;
-
+		else if(ir < fr) {
+			while(ir < fl && ir < fr) {
+				final int nv = rMap.getIndex(itr.getDataIndex()) * nVl + defL;
+				newUID = addVal(nv, ir, d, newUID, tmpVals, retOff);
+				ir = itr.next();
+			}
+			if(fr == fl) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
+				return addVal(nv, ir, d, newUID, tmpVals, retOff);
+			}
+			else if(ir == fl) {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + rMap.getIndex(itr.getDataIndex()) * nVl;
+				newUID = addVal(nv, ir, d, newUID, tmpVals, retOff);
+				ir = itr.next();
+			}
+			else {
+				final int nv = lMap.getIndex(itl.getDataIndex()) + defR;
+				newUID = addVal(nv, fl, d, newUID, tmpVals, retOff);
+			}
+			while(ir < fr) {
+				final int nv = rMap.getIndex(itr.getDataIndex()) * nVl + defL;
 				newUID = addVal(nv, ir, d, newUID, tmpVals, retOff);
-				if(ir >= fr)
-					break;
-				itr.next();
+				ir = itr.next();
 			}
 		}
 
@@ -202,24 +229,64 @@ public class SparseEncoding implements IEncode {
 		return newUID;
 	}
 
-	@Override
-	public int getUnique() {
-		return counts.length + 1;
+	private static DenseEncoding combineSparseToDense(AMapToData lMap, AMapToData rMap, AIterator itl, AIterator itr,
+		int fl, int fr, int nVl, int nVr, int[] d, int nRows, int maxUnique) {
+
+		final AMapToData retMap = MapToFactory.create(nRows, (nVl + 1) * (nVr + 1));
+		int il = itl.value();
+		// parse through one side set all values into the dense.
+		while(il < fl) {
+			retMap.set(il, lMap.getIndex(itl.getDataIndex()) + 1);
+			il = itl.next();
+		}
+		retMap.set(fl, lMap.getIndex(itl.getDataIndex()) + 1);
+
+		int ir = itr.value();
+		// parse through other side set all values with offset based on what already is there.
+		while(ir < fr) {
+			final int vl = retMap.getIndex(ir); // probably 0
+			final int vr = rMap.getIndex(itr.getDataIndex()) + 1;
+			retMap.set(ir, vl + vr * nVl);
+			ir = itr.next();
+		}
+		retMap.set(fr, retMap.getIndex(fr) + (rMap.getIndex(itr.getDataIndex()) + 1) * nVl);
+
+		// parse through entire output reducing number of unique.
+		final AMapToData tmpMap = MapToFactory.create(maxUnique, maxUnique + 1);
+		int newUID = 1;
+		for(int r = 0; r < retMap.size(); r++) {
+			int nv = retMap.getIndex(r);
+			int mv = tmpMap.getIndex(nv);
+			if(mv == 0)
+				mv = tmpMap.setAndGet(nv, newUID++);
+			retMap.set(r, mv - 1);
+		}
+		// parse though other side and use all ret to set correctly.
+		retMap.setUnique(newUID - 1);
+
+		return new DenseEncoding(retMap);
 	}
 
 	@Override
-	public int size() {
-		return map.size();
+	public int getUnique() {
+		return map.getUnique() + 1;
 	}
 
 	@Override
 	public EstimationFactors extractFacts(int[] cols, int nRows, double tupleSparsity, double matrixSparsity) {
 		final int largestOffs = nRows - map.size(); // known largest off is zero tuples
 		tupleSparsity = Math.min((double) map.size() / (double) nRows, tupleSparsity);
-		return new EstimationFactors(cols.length, counts.length, map.size(), largestOffs, counts, 0, nRows, false, true,
+		final int[] counts = map.getCounts(new int[map.getUnique()]);
+		return new EstimationFactors(cols.length, map.getUnique(), map.size(), largestOffs, counts, 0, nRows, false, true,
 			matrixSparsity, tupleSparsity);
 	}
 
+	
+	@Override
+	public  boolean isDense(){
+		return false;
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -230,9 +297,6 @@ public class SparseEncoding implements IEncode {
 		sb.append("\n");
 		sb.append("offsets: ");
 		sb.append(off);
-		sb.append("\n");
-		sb.append("counts:  ");
-		sb.append(Arrays.toString(counts));
 		return sb.toString();
 	}
 }
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 f9986eb141..b9c91184b6 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
@@ -28,6 +28,8 @@ import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupEmpty;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
+import org.apache.sysds.runtime.compress.utils.Util;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 public class CLALibAppend {
@@ -44,41 +46,85 @@ public class CLALibAppend {
 		else if(right.isEmpty() && left instanceof CompressedMatrixBlock)
 			return appendRightEmpty((CompressedMatrixBlock) left, right, m, n);
 
-		if(!(left instanceof CompressedMatrixBlock)) {
+		if(!(left instanceof CompressedMatrixBlock) && left.getInMemorySize() < 1000) {
 			LOG.info("Trying to compress left side of append");
 			left = CompressedMatrixBlockFactory.compress(left, k).getLeft();
 		}
 
-		if(!(right instanceof CompressedMatrixBlock)) {
+		if(!(right instanceof CompressedMatrixBlock) && left.getInMemorySize() < 1000) {
 			LOG.info("Trying to compress right side of append");
 			right = CompressedMatrixBlockFactory.compress(right, k).getLeft();
 		}
 
 		// if compression failed then use default append method.
-		if(!(left instanceof CompressedMatrixBlock && right instanceof CompressedMatrixBlock))
-			return uc(left).append(uc(right), null);
-
-		return append((CompressedMatrixBlock) left, (CompressedMatrixBlock) right, true);
+		if(!(left instanceof CompressedMatrixBlock && right instanceof CompressedMatrixBlock)) {
+			final double spar = (double) (left.getNonZeros() + right.getNonZeros()) / ((double) m * n);
+			final double estSizeUncompressed = MatrixBlock.estimateSizeInMemory(m, n, spar);
+			final double estSizeCompressed = left.getInMemorySize() + right.getInMemorySize();
+			if(estSizeUncompressed < estSizeCompressed)
+				return uc(left).append(uc(right), null);
+			else if(left instanceof CompressedMatrixBlock)
+				return appendRightUncompressed((CompressedMatrixBlock) left, right, m, n);
+			else
+				return appendLeftUncompressed(left, (CompressedMatrixBlock) right, m, n);
+		}
 
+		return append((CompressedMatrixBlock) left, (CompressedMatrixBlock) right, m, n);
 	}
 
-	public static MatrixBlock append(CompressedMatrixBlock left, CompressedMatrixBlock right, boolean check) {
-		final int m = left.getNumRows();
-		final int n = left.getNumColumns() + right.getNumColumns();
-		// init result matrix
-		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+	private static MatrixBlock appendLeftUncompressed(MatrixBlock left, CompressedMatrixBlock right, final int m,
+		final int n) {
+
+		final CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+
+		final List<AColGroup> prev = right.getColGroups();
+		final List<AColGroup> newGroup = new ArrayList<>(prev.size() + 1);
+		final int nColL = left.getNumColumns();
+		final int[] colIdx = Util.genColsIndices(nColL);
+		final AColGroup g = ColGroupUncompressed.create(colIdx, left, false);
+		newGroup.add(g);
+		for(AColGroup group : prev) {
+			final AColGroup tmp = group.copy();
+			tmp.shiftColIndices(nColL);
+			newGroup.add(tmp);
+		}
+
+		ret.allocateColGroupList(newGroup);
+		ret.setNonZeros(left.getNonZeros() + right.getNonZeros());
+		return ret;
 
-		ret = appendColGroups(ret, left.getColGroups(), right.getColGroups(), left.getNumColumns());
+	}
+
+	private static MatrixBlock appendRightUncompressed(CompressedMatrixBlock left, MatrixBlock right, final int m,
+		final int n) {
+
+		final CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+		final List<AColGroup> prev = left.getColGroups();
+		final List<AColGroup> newGroup = new ArrayList<>(prev.size() + 1);
+		newGroup.addAll(prev);
+		final int[] colIdx = Util.genColsIndicesOffset(right.getNumColumns(), left.getNumColumns());
+		final AColGroup g = ColGroupUncompressed.create(colIdx, right, false);
+		newGroup.add(g);
+		ret.allocateColGroupList(newGroup);
+		ret.setNonZeros(left.getNonZeros() + right.getNonZeros());
+		return ret;
+	}
 
+	private static MatrixBlock append(CompressedMatrixBlock left, CompressedMatrixBlock right, final int m,
+		final int n) {
+		// init result matrix
+		final CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+		appendColGroups(ret, left.getColGroups(), right.getColGroups(), left.getNumColumns());
+		ret.setNonZeros(left.getNonZeros() + right.getNonZeros());
 		ret.setOverlapping(left.isOverlapping() || right.isOverlapping());
-		double compressedSize = ret.getInMemorySize();
-		double uncompressedSize = MatrixBlock.estimateSizeInMemory(m, n, ret.getSparsity());
 
-		double ratio = uncompressedSize / compressedSize;
+		final double compressedSize = ret.getInMemorySize();
+		final double uncompressedSize = MatrixBlock.estimateSizeInMemory(m, n, ret.getSparsity());
 
-		if(!check || compressedSize < uncompressedSize)
+		if(compressedSize < uncompressedSize)
 			return ret;
 		else {
+			final double ratio = uncompressedSize / compressedSize;
 			String message = String.format("Decompressing c bind matrix because it had to small compression ratio: %2.3f",
 				ratio);
 			return ret.getUncompressed(message);
@@ -86,52 +132,46 @@ public class CLALibAppend {
 	}
 
 	private static MatrixBlock appendRightEmpty(CompressedMatrixBlock left, MatrixBlock right, int m, int n) {
-		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+		final CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
 		List<AColGroup> newGroup = new ArrayList<>(1);
 		newGroup.add(ColGroupEmpty.create(right.getNumColumns()));
-		ret = appendColGroups(ret, left.getColGroups(), newGroup, left.getNumColumns());
+		appendColGroups(ret, left.getColGroups(), newGroup, left.getNumColumns());
+		ret.setNonZeros(left.getNonZeros() + right.getNonZeros());
 		ret.setOverlapping(left.isOverlapping());
 		return ret;
 	}
 
 	private static MatrixBlock appendLeftEmpty(MatrixBlock left, CompressedMatrixBlock right, int m, int n) {
-		CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
+		final CompressedMatrixBlock ret = new CompressedMatrixBlock(m, n);
 		List<AColGroup> newGroup = new ArrayList<>(1);
 		newGroup.add(ColGroupEmpty.create(left.getNumColumns()));
-		ret = appendColGroups(ret, newGroup, right.getColGroups(), left.getNumColumns());
+		appendColGroups(ret, newGroup, right.getColGroups(), left.getNumColumns());
+		ret.setNonZeros(left.getNonZeros() + right.getNonZeros());
 		ret.setOverlapping(right.isOverlapping());
 		return ret;
 	}
 
-	private static CompressedMatrixBlock appendColGroups(CompressedMatrixBlock ret, List<AColGroup> left,
-		List<AColGroup> right, int leftNumCols) {
+	private static void appendColGroups(CompressedMatrixBlock ret, List<AColGroup> left, List<AColGroup> right,
+		int leftNumCols) {
 
 		// shallow copy of lhs column groups
 		ret.allocateColGroupList(new ArrayList<AColGroup>(left.size() + right.size()));
 
-		final int nRows = ret.getNumRows();
-		long nnz = 0;
-		for(AColGroup group : left) {
-			AColGroup tmp = group.copy();
-			ret.getColGroups().add(tmp);
-			nnz += group.getNumberNonZeros(nRows);
-		}
+		for(AColGroup group : left)
+			ret.getColGroups().add(group.copy());
 
 		for(AColGroup group : right) {
-			AColGroup tmp = group.copy();
+			final AColGroup tmp = group.copy();
 			tmp.shiftColIndices(leftNumCols);
 			ret.getColGroups().add(tmp);
-			nnz += group.getNumberNonZeros(nRows);
 		}
 
 		// meta data maintenance
-		ret.setNonZeros(nnz);
 		CLALibUtils.combineConstColumns(ret);
-		return ret;
 	}
 
 	private static MatrixBlock uc(MatrixBlock mb) {
 		// get uncompressed
-		return CompressedMatrixBlock.getUncompressed(mb);
+		return CompressedMatrixBlock.getUncompressed(mb, "append");
 	}
 }
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 a919a2fb85..31608d722a 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
@@ -32,12 +32,13 @@ 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.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ASDCZero;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
+import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
+import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Divide;
 import org.apache.sysds.runtime.functionobjects.Minus;
@@ -70,10 +71,18 @@ public class CLALibBinaryCellOp {
 		}
 		if(that.isEmpty())
 			return binaryOperationsEmpty(op, m1, that, result);
-		that = CompressedMatrixBlock.getUncompressed(that, "Decompressing right side in BinaryOps");
+
 		LibMatrixBincell.isValidDimensionsBinaryExtended(m1, that);
+
 		BinaryAccessType atype = LibMatrixBincell.getBinaryAccessTypeExtended(m1, that);
-		return selectProcessingBasedOnAccessType(op, m1, that, result, atype, false);
+		if(that instanceof CompressedMatrixBlock && that.getInMemorySize() < m1.getInMemorySize()) {
+			MatrixBlock m1uc = CompressedMatrixBlock.getUncompressed(m1, "Decompressing left side in BinaryOps");
+			return selectProcessingBasedOnAccessType(op, (CompressedMatrixBlock) that, m1uc, result, atype, true);
+		}
+		else {
+			that = CompressedMatrixBlock.getUncompressed(that, "Decompressing right side in BinaryOps");
+			return selectProcessingBasedOnAccessType(op, m1, that, result, atype, false);
+		}
 	}
 
 	public static MatrixBlock binaryOperationsLeft(BinaryOperator op, CompressedMatrixBlock m1, MatrixBlock that,
@@ -131,11 +140,14 @@ public class CLALibBinaryCellOp {
 		}
 		else if(atype == BinaryAccessType.MATRIX_MATRIX) {
 			// Full matrix access.
-			MatrixBlock d_compressed = m1.getUncompressed("MatrixMatrix " + op);
-			if(left)
-				return that.binaryOperations(op, d_compressed);
-			else
-				return d_compressed.binaryOperations(op, that);
+			MatrixBlock d_compressed = m1.getCachedDecompressed();// m1.getUncompressed("MatrixMatrix " + op);
+			if(d_compressed != null) {
+				if(left)
+					return that.binaryOperations(op, d_compressed);
+				else
+					return d_compressed.binaryOperations(op, that);
+			}
+			return binaryMM(m1, that, op, left);
 		}
 		else if(isSupportedBinaryCellOp(op.fn) && atype == BinaryAccessType.MATRIX_ROW_VECTOR ||
 			atype == BinaryAccessType.ROW_VECTOR_MATRIX)
@@ -242,11 +254,12 @@ public class CLALibBinaryCellOp {
 			for(Future<AColGroup> f : pool.invokeAll(tasks))
 				newColGroups.add(f.get());
 
-			pool.shutdown();
 		}
 		catch(InterruptedException | ExecutionException e) {
+			pool.shutdown();
 			throw new DMLRuntimeException(e);
 		}
+		pool.shutdown();
 	}
 
 	private static CompressedMatrixBlock binaryMVRow(CompressedMatrixBlock m1, MatrixBlock m2, CompressedMatrixBlock ret,
@@ -325,54 +338,112 @@ public class CLALibBinaryCellOp {
 
 		final int nCols = m1.getNumColumns();
 		final int nRows = m1.getNumRows();
-		// Pre filter.
-		final List<AColGroup> groups = m1.getColGroups();
-		final boolean shouldFilter = CLALibUtils.shouldPreFilter(groups);
-		if(shouldFilter) {
-			CompressedMatrixBlock mf1 = new CompressedMatrixBlock(m1);
-			double[] constV = new double[nCols];
-			final List<AColGroup> filteredGroups = CLALibUtils.filterGroups(groups, constV);
-			filteredGroups.add(ColGroupConst.create(constV));
-			mf1.allocateColGroupList(filteredGroups);
-			m1 = mf1;
-		}
+		m1 = morph(m1);
+
 		MatrixBlock ret = new MatrixBlock(nRows, nCols, false, -1).allocateBlock();
 
-		final int blkz = CompressionSettings.BITMAP_BLOCK_SZ / nCols * 5;
 		final int k = op.getNumThreads();
 		long nnz = 0;
 
-		if(k <= 1) {
+		if(k <= 1)
+			nnz = binaryMVColSingleThread(m1, m2, op, left, ret);
+		else
+			nnz = binaryMVColMultiThread(m1, m2, op, left, ret);
+
+		ret.setNonZeros(nnz);
+		ret.examSparsity();
+		return ret;
+	}
+
+	private static long binaryMVColSingleThread(CompressedMatrixBlock m1, MatrixBlock m2, BinaryOperator op,
+		boolean left, MatrixBlock ret) {
+		final int nRows = m1.getNumRows();
+		long nnz = 0;
+		if(left)
+			nnz += new BinaryMVColLeftTask(m1, m2, ret, 0, nRows, op).call();
+		else
+			nnz += new BinaryMVColTask(m1, m2, ret, 0, nRows, op).call();
+		return nnz;
+	}
+
+	private static long binaryMVColMultiThread(CompressedMatrixBlock m1, MatrixBlock m2, BinaryOperator op, boolean left,
+		MatrixBlock ret) {
+		final int nRows = m1.getNumRows();
+		final int k = op.getNumThreads();
+		final int blkz = ret.getNumRows() / k;
+		long nnz = 0;
+		final ExecutorService pool = CommonThreadPool.get(op.getNumThreads());
+		final ArrayList<Callable<Integer>> tasks = new ArrayList<>();
+		try {
 			for(int i = 0; i < nRows; i += blkz) {
 				if(left)
-					nnz += new BinaryMVColLeftTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op).call();
+					tasks.add(new BinaryMVColLeftTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op));
 				else
-					nnz += new BinaryMVColTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op).call();
+					tasks.add(new BinaryMVColTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op));
 			}
+			for(Future<Integer> f : pool.invokeAll(tasks))
+				nnz += f.get();
+			pool.shutdown();
 		}
-		else {
-			ExecutorService pool = CommonThreadPool.get(op.getNumThreads());
-			ArrayList<Callable<Integer>> tasks = new ArrayList<>();
-			try {
-				for(int i = 0; i < nRows; i += blkz) {
-					if(left)
-						tasks.add(new BinaryMVColLeftTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op));
-					else
-						tasks.add(new BinaryMVColTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op));
-				}
-				for(Future<Integer> f : pool.invokeAll(tasks))
-					nnz += f.get();
-				pool.shutdown();
-			}
-			catch(InterruptedException | ExecutionException e) {
-				throw new DMLRuntimeException(e);
-			}
+		catch(InterruptedException | ExecutionException e) {
+			throw new DMLRuntimeException(e);
 		}
-		ret.setNonZeros(nnz);
+		return nnz;
+	}
 
+	private static MatrixBlock binaryMM(CompressedMatrixBlock m1, MatrixBlock m2, BinaryOperator op, boolean left) {
+		final int nCols = m1.getNumColumns();
+		final int nRows = m1.getNumRows();
+		m1 = morph(m1);
+
+		MatrixBlock ret = new MatrixBlock(nRows, nCols, false, -1).allocateBlock();
+
+		// final int k = op.getNumThreads();
+		long nnz = binaryMMMultiThread(m1, m2, op, left, ret);
+
+		ret.setNonZeros(nnz);
+		ret.examSparsity();
 		return ret;
 	}
 
+	private static long binaryMMMultiThread(CompressedMatrixBlock m1, MatrixBlock m2, BinaryOperator op, boolean left,
+		MatrixBlock ret) {
+		final int nRows = m1.getNumRows();
+		final int k = op.getNumThreads();
+		final int blkz = ret.getNumRows() / k;
+		long nnz = 0;
+		final ExecutorService pool = CommonThreadPool.get(op.getNumThreads());
+		final ArrayList<Callable<Long>> tasks = new ArrayList<>();
+		try {
+			for(int i = 0; i < nRows; i += blkz)
+				tasks.add(new BinaryMMTask(m1, m2, ret, i, Math.min(nRows, i + blkz), op, left));
+
+			for(Future<Long> f : pool.invokeAll(tasks))
+				nnz += f.get();
+			pool.shutdown();
+		}
+		catch(InterruptedException | ExecutionException e) {
+			throw new DMLRuntimeException(e);
+		}
+		return nnz;
+	}
+
+	private static CompressedMatrixBlock morph(CompressedMatrixBlock m) {
+		final List<AColGroup> groups = m.getColGroups();
+		final boolean shouldFilter = CLALibUtils.shouldPreFilter(groups);
+		if(shouldFilter) {
+			CompressedMatrixBlock mf1 = new CompressedMatrixBlock(m);
+			final int nCols = m.getNumColumns();
+			double[] constV = new double[nCols];
+			final List<AColGroup> filteredGroups = CLALibUtils.filterGroups(groups, constV);
+			filteredGroups.add(ColGroupConst.create(constV));
+			mf1.allocateColGroupList(filteredGroups);
+			return mf1;
+		}
+		else
+			return m;
+	}
+
 	private static class BinaryMVColTask implements Callable<Integer> {
 		private final int _rl;
 		private final int _ru;
@@ -393,17 +464,40 @@ public class CLALibBinaryCellOp {
 
 		@Override
 		public Integer call() {
+			final int _blklen = 32768 / _ret.getNumColumns();
+			final List<AColGroup> groups = _m1.getColGroups();
+
+			final AIterator[] its = new AIterator[groups.size()];
+
+			for(int i = 0; i < groups.size(); i++)
+				if(groups.get(i) instanceof ASDCZero)
+					its[i] = ((ASDCZero) groups.get(i)).getIterator(_rl);
+
+			for(int r = _rl; r < _ru; r += _blklen)
+				processBlock(r, Math.min(r + _blklen, _ru), groups, its);
+
+			return _ret.getNumColumns() * _ret.getNumRows();
+		}
+
+		private final void processBlock(final int rl, final int ru, final List<AColGroup> groups, final AIterator[] its) {
 			// unsafe decompress, since we count nonzeros afterwards.
-			for(AColGroup g : _m1.getColGroups())
-				g.decompressToDenseBlock(_ret.getDenseBlock(), _rl, _ru);
+			final DenseBlock db = _ret.getDenseBlock();
+			for(int i = 0; i < groups.size(); i++) {
+				final AColGroup g = groups.get(i);
+				// AColGroup g = _groups.get(i);
+				if(g instanceof ASDCZero)
+					((ASDCZero) g).decompressToDenseBlock(db, rl, ru, 0, 0, its[i]);
+				else
+					g.decompressToDenseBlock(db, rl, ru, 0, 0);
+			}
 
 			if(_m2.isInSparseFormat())
 				throw new NotImplementedException("Not Implemented sparse Format execution for MM.");
 			else {
-				int offset = _rl * _m1.getNumColumns();
+				int offset = rl * _m1.getNumColumns();
 				double[] _retDense = _ret.getDenseBlockValues();
 				double[] _m2Dense = _m2.getDenseBlockValues();
-				for(int row = _rl; row < _ru; row++) {
+				for(int row = rl; row < ru; row++) {
 					double vr = _m2Dense[row];
 					for(int col = 0; col < _m1.getNumColumns(); col++) {
 						double v = _op.fn.execute(_retDense[offset], vr);
@@ -412,7 +506,142 @@ public class CLALibBinaryCellOp {
 					}
 				}
 
-				return _ret.getNumColumns() * _ret.getNumRows();
+			}
+		}
+	}
+
+	private static class BinaryMMTask implements Callable<Long> {
+		private final int _rl;
+		private final int _ru;
+		private final CompressedMatrixBlock _m1;
+		private final MatrixBlock _m2;
+		private final MatrixBlock _ret;
+		private final boolean _left;
+		private final BinaryOperator _op;
+
+		protected BinaryMMTask(CompressedMatrixBlock m1, MatrixBlock m2, MatrixBlock ret, int rl, int ru,
+			BinaryOperator op, boolean left) {
+			_m1 = m1;
+			_m2 = m2;
+			_ret = ret;
+			_op = op;
+			_rl = rl;
+			_ru = ru;
+			_left = left;
+		}
+
+		@Override
+		public Long call() {
+			final List<AColGroup> groups = _m1.getColGroups();
+			final int _blklen = Math.max(65536 * 2 / _ret.getNumColumns() / groups.size(), 64);
+
+			final AIterator[] its = new AIterator[groups.size()];
+
+			for(int i = 0; i < groups.size(); i++)
+				if(groups.get(i) instanceof ASDCZero)
+					its[i] = ((ASDCZero) groups.get(i)).getIterator(_rl);
+
+			long nnz = 0;
+			for(int r = _rl; r < _ru; r += _blklen) {
+				final int re = Math.min(r + _blklen, _ru);
+				processBlock(r, re, groups, its);
+				nnz += _ret.recomputeNonZeros(r, re - 1);
+			}
+
+			return nnz;
+		}
+
+		private final void processBlock(final int rl, final int ru, final List<AColGroup> groups, final AIterator[] its) {
+			// unsafe decompress, since we count nonzeros afterwards.
+			final DenseBlock db = _ret.getDenseBlock();
+			for(int i = 0; i < groups.size(); i++) {
+				final AColGroup g = groups.get(i);
+				// AColGroup g = _groups.get(i);
+				if(g instanceof ASDCZero)
+					((ASDCZero) g).decompressToDenseBlock(db, rl, ru, 0, 0, its[i]);
+				else
+					g.decompressToDenseBlock(db, rl, ru, 0, 0);
+			}
+
+			final DenseBlock rv = _ret.getDenseBlock();
+			final int cols = _ret.getNumColumns();
+			if(_left) {
+				// all exec should have ret on right side
+				if(_m2.isInSparseFormat()) {
+					final SparseBlock m2sb = _m2.getSparseBlock();
+					for(int r = rl; r < ru; r++) {
+						final double[] retV = rv.values(r);
+						int off = rv.pos(r);
+						if(m2sb.isEmpty(r)) {
+							for(int c = off; c < cols + off; c++)
+								retV[c] = _op.fn.execute(retV[c], 0);
+						}
+						else {
+							final int apos = m2sb.pos(r);
+							final int alen = m2sb.size(r) + apos;
+							final int[] aix = m2sb.indexes(r);
+							final double[] avals = m2sb.values(r);
+							int j = 0;
+							for(int k = apos; j < cols && k < alen; j++, off++) {
+								final double v = aix[k] == j ? avals[k++] : 0;
+								retV[off] = _op.fn.execute(v, retV[off]);
+							}
+
+							for(; j < cols; j++)
+								retV[off] = _op.fn.execute(0, retV[off]);
+						}
+					}
+				}
+				else {
+					DenseBlock m2db = _m2.getDenseBlock();
+					for(int r = rl; r < ru; r++) {
+						double[] retV = rv.values(r);
+						double[] m2V = m2db.values(r);
+
+						int off = rv.pos(r);
+						for(int c = off; c < cols + off; c++)
+							retV[c] = _op.fn.execute(m2V[c], retV[c]);
+					}
+				}
+			}
+			else {
+				// all exec should have ret on left side
+				if(_m2.isInSparseFormat()) {
+					final SparseBlock m2sb = _m2.getSparseBlock();
+					for(int r = rl; r < ru; r++) {
+						final double[] retV = rv.values(r);
+						int off = rv.pos(r);
+						if(m2sb.isEmpty(r)) {
+							for(int c = off; c < cols + off; c++)
+								retV[c] = _op.fn.execute(retV[c], 0);
+						}
+						else {
+							final int apos = m2sb.pos(r);
+							final int alen = m2sb.size(r) + apos;
+							final int[] aix = m2sb.indexes(r);
+							final double[] avals = m2sb.values(r);
+							int j = 0;
+							for(int k = apos; j < cols && k < alen; j++, off++) {
+								final double v = aix[k] == j ? avals[k++] : 0;
+								retV[off] = _op.fn.execute(retV[off], v);
+							}
+
+							for(; j < cols; j++)
+								retV[off] = _op.fn.execute(retV[off], 0);
+						}
+					}
+				}
+				else {
+					final DenseBlock m2db = _m2.getDenseBlock();
+					for(int r = rl; r < ru; r++) {
+						final double[] retV = rv.values(r);
+						final double[] m2V = m2db.values(r);
+
+						int off = rv.pos(r);
+						for(int c = off; c < cols + off; c++)
+							retV[c] = _op.fn.execute(retV[c], m2V[c]);
+					}
+				}
 			}
 		}
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCMOps.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCMOps.java
index a400e6ad96..a926e96738 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCMOps.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCMOps.java
@@ -35,7 +35,7 @@ public class CLALibCMOps {
 		if(cmb.isEmpty())
 			return LibMatrixAgg.aggregateCmCov(cmb, null, null, op.fn);
 		else if(cmb.isOverlapping())
-			return cmb.getUncompressed("cmOperations on overlapping state").cmOperations(op);
+			return cmb.getUncompressed("cmOperations on overlapping state", op.getNumThreads()).cmOperations(op);
 		else {
 			final List<AColGroup> groups = cmb.getColGroups();
 			if(groups.size() == 1)
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 b90e077722..683506720b 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
@@ -71,13 +71,11 @@ public class CLALibCompAgg {
 	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>();
-
 	public static MatrixBlock aggregateUnary(CompressedMatrixBlock inputMatrix, MatrixBlock result,
 		AggregateUnaryOperator op, int blen, MatrixIndexes indexesIn, boolean inCP) {
 
 		if(!supported(op) || inputMatrix.isEmpty()) {
-			return inputMatrix.getUncompressed("Unary aggregate " + op + " not supported yet.")
+			return inputMatrix.getUncompressed("Unary aggregate " + op + " not supported yet.", op.getNumThreads())
 				.aggregateUnaryOperations(op, result, blen, indexesIn, inCP);
 		}
 
@@ -93,9 +91,9 @@ public class CLALibCompAgg {
 			// final double denseSize = MatrixBlock.estimateSizeDenseInMemory(r, c);
 			// final double localMaxMemory = InfrastructureAnalyzer.getLocalMaxMemory();
 
-			if(inputMatrix.getCachedDecompressed() != null) {
+			if(inputMatrix.getCachedDecompressed() != null)
 				return inputMatrix.getCachedDecompressed().aggregateUnaryOperations(op, result, blen, indexesIn, inCP);
-			}
+
 			// else if(colGroups.size() > 5 && denseSize <= localMaxMemory / 2) {
 			// MatrixBlock uc = inputMatrix.getUncompressed(
 			// op.indexFn.getClass().getSimpleName() + " " + op.aggOp.increOp.fn.getClass().getSimpleName()
@@ -270,13 +268,14 @@ public class CLALibCompAgg {
 					tasks.add(new UnaryAggregateTask(grp, ret, r, 0, r, op, c, m1.isOverlapping(), null));
 
 			List<Future<MatrixBlock>> futures = pool.invokeAll(tasks);
-			pool.shutdown();
 
 			reduceFutures(futures, ret, op, m1.isOverlapping());
 		}
 		catch(InterruptedException | ExecutionException e) {
+			pool.shutdown();
 			throw new DMLRuntimeException("Aggregate In parallel failed.", e);
 		}
+		pool.shutdown();
 	}
 
 	private static double[][] getPreAgg(AggregateUnaryOperator opm, List<AColGroup> groups) {
@@ -565,7 +564,7 @@ public class CLALibCompAgg {
 			_op = op;
 			_rl = rl;
 			_ru = ru;
-			_blklen = 32768 / ret.getNumColumns();
+			_blklen = Math.max(65536 * 2 / ret.getNumColumns() / filteredGroups.size(), 64);
 			_ret = ret;
 			_nCol = nCol;
 		}
@@ -583,7 +582,7 @@ public class CLALibCompAgg {
 			for(int i = 0; i < _groups.size(); i++) {
 				AColGroup g = _groups.get(i);
 				if(g instanceof ASDCZero)
-					((ASDCZero) g).decompressToDenseBlockDenseDictionary(db, rl, ru, -rl, 0, its[i]);
+					((ASDCZero) g).decompressToDenseBlock(db, rl, ru, -rl, 0, its[i]);
 				else
 					g.decompressToDenseBlock(db, rl, ru, -rl, 0);
 
@@ -621,10 +620,11 @@ public class CLALibCompAgg {
 					final int rbu = Math.min(r + _blklen, _ru);
 					tmp.reset(rbu - r, tmp.getNumColumns(), false);
 					decompressToTemp(tmp, r, rbu, its);
-					MatrixBlock outputBlock = tmp.prepareAggregateUnaryOutput(_op, null, 1000);
-					LibMatrixAgg.aggregateUnaryMatrix(tmp, outputBlock, _op);
-					outputBlock.dropLastRowsOrColumns(_op.aggOp.correction);
-					if(outputBlock.isEmpty()) {
+					final MatrixBlock tmpR = tmp.prepareAggregateUnaryOutput(_op, null, 1000);
+					LibMatrixAgg.aggregateUnaryMatrix(tmp, tmpR, _op);
+
+					tmpR.dropLastRowsOrColumns(_op.aggOp.correction);
+					if(tmpR.isEmpty()) {
 						if(isBinaryOp) {
 							final double[] retValues = _ret.getDenseBlockValues();
 							final int s = r * _ret.getNumColumns();
@@ -632,14 +632,17 @@ public class CLALibCompAgg {
 							Arrays.fill(retValues, s, e, 0);
 						}
 					}
-					else if(outputBlock.isInSparseFormat())
-						throw new DMLCompressionException("Output block should never be sparse");
+					else if(tmpR.isInSparseFormat()) {
+						throw new NotImplementedException(
+							"Not supported Sparse yet and it should be extremely unlikely/not happen. because we work with a single column here");
+					}
 					else {
+						// tmpR.sparseToDense();
 						final double[] retValues = _ret.getDenseBlockValues();
-						final double[] outputBlockValues = outputBlock.getDenseBlockValues();
+						final double[] tmpRValues = tmpR.getDenseBlockValues();
 						final int currentIndex = r * _ret.getNumColumns();
-						final int length = Math.min(outputBlockValues.length, retValues.length - currentIndex);
-						System.arraycopy(outputBlockValues, 0, retValues, currentIndex, length);
+						final int length = rbu - r;
+						System.arraycopy(tmpRValues, 0, retValues, currentIndex, length);
 					}
 				}
 				return null;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibDecompress.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibDecompress.java
index 9c011a168e..30ea5de820 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibDecompress.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibDecompress.java
@@ -77,6 +77,9 @@ public class CLALibDecompress {
 			if(LOG.isTraceEnabled())
 				LOG.trace("decompressed block w/ k=" + k + " in " + t + "ms.");
 		}
+
+		if(ret.getNonZeros() <= 0)
+			ret.setNonZeros(cmb.getNonZeros());
 	}
 
 	private static void decompressToSparseBlock(CompressedMatrixBlock cmb, MatrixBlock ret, int rowOffset,
@@ -247,8 +250,8 @@ public class CLALibDecompress {
 
 	private static void decompressDenseMultiThread(MatrixBlock ret, List<AColGroup> filteredGroups, int rlen, int blklen,
 		double[] constV, double eps, int k) {
+		final ExecutorService pool = CommonThreadPool.get(k);
 		try {
-			final ExecutorService pool = CommonThreadPool.get(k);
 			final ArrayList<DecompressDenseTask> tasks = new ArrayList<>();
 			for(int i = 0; i < rlen; i += blklen)
 				tasks.add(new DecompressDenseTask(filteredGroups, ret, eps, i, Math.min(i + blklen, rlen), constV));
@@ -256,29 +259,30 @@ public class CLALibDecompress {
 			long nnz = 0;
 			for(Future<Long> rt : pool.invokeAll(tasks))
 				nnz += rt.get();
-			pool.shutdown();
 			ret.setNonZeros(nnz);
 		}
 		catch(InterruptedException | ExecutionException ex) {
 			throw new DMLCompressionException("Parallel decompression failed", ex);
 		}
+		pool.shutdown();
 	}
 
 	private static void decompressSparseMultiThread(MatrixBlock ret, List<AColGroup> filteredGroups, int rlen,
 		int blklen, int k) {
+		final ExecutorService pool = CommonThreadPool.get(k);
 		try {
-			final ExecutorService pool = CommonThreadPool.get(k);
 			final ArrayList<DecompressSparseTask> tasks = new ArrayList<>();
 			for(int i = 0; i < rlen; i += blklen)
 				tasks.add(new DecompressSparseTask(filteredGroups, ret, i, Math.min(i + blklen, rlen)));
 
 			for(Future<Object> rt : pool.invokeAll(tasks))
 				rt.get();
-			pool.shutdown();
 		}
 		catch(InterruptedException | ExecutionException ex) {
+			pool.shutdown();
 			throw new DMLCompressionException("Parallel decompression failed", ex);
 		}
+		pool.shutdown();
 	}
 
 	/**
@@ -326,22 +330,22 @@ public class CLALibDecompress {
 
 		@Override
 		public Long call() {
-			try{
+			try {
 
 				long nnz = 0;
 				for(int b = _rl; b < _ru; b += _blklen) {
 					final int e = Math.min(b + _blklen, _ru);
 					for(AColGroup grp : _colGroups)
 						grp.decompressToDenseBlock(_ret.getDenseBlock(), b, e);
-	
+
 					if(_constV != null)
 						addVector(_ret, _constV, _eps, b, e);
 					nnz += _ret.recomputeNonZeros(b, e - 1);
 				}
-	
+
 				return nnz;
 			}
-			catch(Exception e){
+			catch(Exception e) {
 				e.printStackTrace();
 				throw new DMLCompressionException("Failed dense decompression", 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 dcfa22bd5d..daa6ea0bcf 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
@@ -221,8 +221,8 @@ public class CLALibLeftMultBy {
 
 	private static void LMMParallel(List<AColGroup> npa, List<APreAgg> pa, MatrixBlock that, MatrixBlock ret,
 		double[] rowSums, boolean overlapping, int k) {
+		final ExecutorService pool = CommonThreadPool.get(k);
 		try {
-			final ExecutorService pool = CommonThreadPool.get(k);
 			final ArrayList<Callable<MatrixBlock>> tasks = new ArrayList<>();
 
 			final int rl = that.getNumRows();
@@ -291,13 +291,12 @@ public class CLALibLeftMultBy {
 				}
 			}
 
-			pool.shutdown();
 		}
-		catch(InterruptedException |
-
-			ExecutionException e) {
+		catch(InterruptedException | ExecutionException e) {
+			pool.shutdown();
 			throw new DMLRuntimeException(e);
 		}
+		pool.shutdown();
 	}
 
 	private static void LMMTaskExec(List<AColGroup> npa, List<APreAgg> pa, MatrixBlock that, MatrixBlock ret, int rl,
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 922ee569be..9778236239 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
@@ -72,7 +72,7 @@ public class CLALibRightMultBy {
 		}
 		else {
 			if(m2 instanceof CompressedMatrixBlock)
-				m2 = ((CompressedMatrixBlock) m2).getUncompressed("Uncompressed right side of right MM");
+				m2 = ((CompressedMatrixBlock) m2).getUncompressed("Uncompressed right side of right MM", k);
 
 			if(!allowOverlap) {
 				LOG.trace("Overlapping output not allowed in call to Right MM");
@@ -224,7 +224,7 @@ public class CLALibRightMultBy {
 	}
 
 	private static boolean RMMParallel(List<AColGroup> filteredGroups, MatrixBlock that, List<AColGroup> retCg, int k) {
-		ExecutorService pool = CommonThreadPool.get(k);
+		final ExecutorService pool = CommonThreadPool.get(k);
 		boolean containsNull = false;
 		try {
 			List<Callable<AColGroup>> tasks = new ArrayList<>(filteredGroups.size());
@@ -241,6 +241,7 @@ public class CLALibRightMultBy {
 		catch(InterruptedException | ExecutionException e) {
 			throw new DMLRuntimeException(e);
 		}
+		pool.shutdown();
 		return containsNull;
 	}
 
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 d1a1428486..dcb8c62155 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
@@ -162,11 +162,10 @@ public class CLALibScalar {
 		CompressedMatrixBlock ret, int k) {
 		if(colGroups == null)
 			return;
-		ExecutorService pool = CommonThreadPool.get(k);
+		final ExecutorService pool = CommonThreadPool.get(k);
 		List<ScalarTask> tasks = partition(sop, colGroups);
 		try {
 			List<Future<List<AColGroup>>> rtasks = pool.invokeAll(tasks);
-			pool.shutdown();
 			List<AColGroup> newColGroups = new ArrayList<>();
 			for(Future<List<AColGroup>> f : rtasks) {
 				newColGroups.addAll(f.get());
@@ -174,8 +173,10 @@ public class CLALibScalar {
 			ret.allocateColGroupList(newColGroups);
 		}
 		catch(InterruptedException | ExecutionException e) {
+			pool.shutdown();
 			throw new DMLRuntimeException(e);
 		}
+		pool.shutdown();
 	}
 
 	private static List<ScalarTask> partition(ScalarOperator sop, List<AColGroup> colGroups) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSlice.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSlice.java
index 51077d427c..9bba812ec0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSlice.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSlice.java
@@ -81,7 +81,7 @@ public class CLALibSlice {
 	private static MatrixBlock sliceSingle(CompressedMatrixBlock cmb, int row, int col) {
 		// get a single index, and return in a matrixBlock
 		MatrixBlock tmp = new MatrixBlock(1, 1, 0);
-		tmp.appendValue(0, 0, cmb.getValue(row, col));
+		tmp.setValue(0, 0, cmb.getValue(row, col));
 		return tmp;
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibTSMM.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibTSMM.java
index 82d6c2e1e4..85e43099ff 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibTSMM.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibTSMM.java
@@ -61,6 +61,7 @@ public class CLALibTSMM {
 		}
 		else
 			tsmmColGroups(groups, ret, numRows, overlapping, k);
+
 		ret.setNonZeros(LibMatrixMult.copyUpperToLowerTriangle(ret));
 		ret.examSparsity();
 	}
@@ -111,11 +112,12 @@ public class CLALibTSMM {
 		try {
 			for(Future<MatrixBlock> future : pool.invokeAll(tasks))
 				future.get();
-			pool.shutdown();
 		}
 		catch(InterruptedException | ExecutionException e) {
+			pool.shutdown();
 			throw new DMLRuntimeException(e);
 		}
+		pool.shutdown();
 	}
 
 	private static void outerProductUpperTriangle(final double[] leftRowSum, final double[] rightColumnSum,
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibUnary.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibUnary.java
index 287d6d74ae..dee1b9022b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibUnary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibUnary.java
@@ -44,15 +44,17 @@ public class CLALibUnary {
 			// when in overlapping state it is guaranteed that there is no infinites, NA, or NANs.
 			if(Builtin.isBuiltinCode(op.fn, BuiltinCode.ISINF, BuiltinCode.ISNA, BuiltinCode.ISNAN))
 				return new MatrixBlock(r, c, 0);
+			if(op.fn instanceof Builtin)
+			return m.getUncompressed("Unary Op not supported Overlapping builtin: " + ((Builtin)(op.fn)).getBuiltinCode(), op.getNumThreads()).unaryOperations(op, null);
 			else
-				return m.getUncompressed(op.toString()).unaryOperations(op, null);
+				return m.getUncompressed("Unary Op not supported Overlapping: " + op.fn.getClass().getSimpleName(), op.getNumThreads()).unaryOperations(op, null);
 		}
 		else if(Builtin.isBuiltinCode(op.fn, BuiltinCode.ISINF, BuiltinCode.ISNAN, BuiltinCode.ISNA) &&
 			!m.containsValue(op.getPattern()))
 			return new MatrixBlock(r, c, 0); // avoid unnecessary allocation
 		else if(LibMatrixAgg.isSupportedUnaryOperator(op)) {
 			// e.g., cumsum/cumprod/cummin/cumax/cumsumprod
-			return m.getUncompressed(op.toString()).unaryOperations(op, null);
+			return m.getUncompressed("Unary Op not supported: " + op.fn.getClass().getSimpleName(), op.getNumThreads()).unaryOperations(op, null);
 		}
 		else {
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelection.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelection.java
index 1217c22c89..d02e73536b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelection.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelection.java
@@ -29,7 +29,6 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 public abstract class ReaderColumnSelection {
 
 	protected static final Log LOG = LogFactory.getLog(ReaderColumnSelection.class.getName());
-	protected static final DblArray emptyReturn = new DblArray();
 
 	protected final int[] _colIndexes;
 	protected final DblArray reusableReturn;
@@ -52,17 +51,14 @@ public abstract class ReaderColumnSelection {
 	 * 
 	 * @return next row
 	 */
-	public DblArray nextRow() {
-		DblArray ret = getNextRow();
-		while(ret != null && ret.isEmpty())
-			ret = getNextRow();
-
-		if(ret == null)
+	public final DblArray nextRow() {
+		if(_rl >= _ru)
 			return null;
-		else {
+		final DblArray ret = getNextRow();
+
+		if(ret != null)
 			ret.resetHash();
-			return ret;
-		}
+		return ret;
 	}
 
 	protected abstract DblArray getNextRow();
@@ -79,7 +75,7 @@ public abstract class ReaderColumnSelection {
 
 	public static ReaderColumnSelection createReader(MatrixBlock rawBlock, int[] colIndices, boolean transposed, int rl,
 		int ru) {
-		checkInput(rawBlock, colIndices);
+		checkInput(rawBlock, colIndices, rl, ru);
 		rl = rl - 1;
 
 		if(transposed) {
@@ -90,7 +86,6 @@ public abstract class ReaderColumnSelection {
 			else
 				return new ReaderColumnSelectionDenseSingleBlockTransposed(rawBlock, colIndices, rl, ru);
 		}
-
 		if(rawBlock.isInSparseFormat())
 			return new ReaderColumnSelectionSparse(rawBlock, colIndices, rl, ru);
 		else if(rawBlock.getDenseBlock().numBlocks() > 1)
@@ -98,10 +93,12 @@ public abstract class ReaderColumnSelection {
 		return new ReaderColumnSelectionDenseSingleBlock(rawBlock, colIndices, rl, ru);
 	}
 
-	private static void checkInput(MatrixBlock rawBlock, int[] colIndices) {
+	private static void checkInput(final MatrixBlock rawBlock, final int[] colIndices, final int rl, final int ru) {
 		if(colIndices.length <= 1)
 			throw new DMLCompressionException("Column selection reader should not be done on single column groups");
-		if(rawBlock.getSparseBlock() == null && rawBlock.getDenseBlock() == null)
+		else if(rawBlock.getSparseBlock() == null && rawBlock.getDenseBlock() == null)
 			throw new DMLCompressionException("Input Block was null");
+		else if(rl >= ru)
+			throw new DMLCompressionException("Invalid inverse range for reader " + rl + " to " + ru);
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlock.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlock.java
index 06f46b3e55..9619b08b2d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlock.java
@@ -27,20 +27,20 @@ public class ReaderColumnSelectionDenseMultiBlock extends ReaderColumnSelection
 	private DenseBlock _data;
 
 	protected ReaderColumnSelectionDenseMultiBlock(MatrixBlock data, int[] colIndices, int rl, int ru) {
-		super(colIndices, rl, Math.min(ru, data.getNumRows()));
+		super(colIndices, rl, Math.min(ru, data.getNumRows()) - 1);
 		_data = data.getDenseBlock();
 	}
 
 	protected DblArray getNextRow() {
-		if(_rl == _ru - 1)
-			return null;
-		_rl++;
 		boolean empty = true;
-		for(int i = 0; i < _colIndexes.length; i++) {
-			double v = _data.get(_rl, _colIndexes[i]);
-			empty &= v == 0;
-			reusableArr[i] = v;
+		while(empty && _rl < _ru) {
+			_rl++;
+			for(int i = 0; i < _colIndexes.length; i++) {
+				final double v = _data.get(_rl, _colIndexes[i]);
+				empty &= v == 0;
+				reusableArr[i] = v;
+			}
 		}
-		return empty ? emptyReturn : reusableReturn;
+		return empty ? null : reusableReturn;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlockTransposed.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlockTransposed.java
index 974d9a8aae..507fa03191 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlockTransposed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseMultiBlockTransposed.java
@@ -27,21 +27,20 @@ public class ReaderColumnSelectionDenseMultiBlockTransposed extends ReaderColumn
 	private DenseBlock _data;
 
 	protected ReaderColumnSelectionDenseMultiBlockTransposed(MatrixBlock data, int[] colIndices, int rl, int ru) {
-		super(colIndices.clone(), rl, Math.min(ru, data.getNumColumns()));
+		super(colIndices.clone(), rl, Math.min(ru, data.getNumColumns()) - 1);
 		_data = data.getDenseBlock();
 	}
 
 	protected DblArray getNextRow() {
-		if(_rl == _ru - 1)
-			return null;
-		_rl++;
-
 		boolean empty = true;
-		for(int i = 0; i < _colIndexes.length; i++) {
-			double v = _data.get(_colIndexes[i], _rl);
-			empty &= v == 0;
-			reusableArr[i] = v;
+		while(empty && _rl < _ru) {
+			_rl++;
+			for(int i = 0; i < _colIndexes.length; i++) {
+				double v = _data.get(_colIndexes[i], _rl);
+				empty &= v == 0;
+				reusableArr[i] = v;
+			}
 		}
-		return empty ? emptyReturn : reusableReturn;
+		return empty ? null : reusableReturn;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlock.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlock.java
index 9aeee02962..6a073a7532 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlock.java
@@ -19,7 +19,6 @@
 
 package org.apache.sysds.runtime.compress.readers;
 
-import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
@@ -28,29 +27,24 @@ public class ReaderColumnSelectionDenseSingleBlock extends ReaderColumnSelection
 	private final int _numCols;
 
 	protected ReaderColumnSelectionDenseSingleBlock(MatrixBlock data, int[] colIndices, int rl, int ru) {
-		super(colIndices, rl, Math.min(ru, data.getNumRows()));
+		super(colIndices, rl, Math.min(ru, data.getNumRows()) -1);
 		_data = data.getDenseBlockValues();
-
-		if(data.getDenseBlock().numBlocks() > 1)
-			throw new DMLCompressionException("Not handling multi block data reading in dense reader");
-
 		_numCols = data.getNumColumns();
 	}
 
 	protected DblArray getNextRow() {
-		if(_rl == _ru - 1)
-			return null;
-		_rl++;
 
-		final int indexOff = _rl * _numCols;
 		boolean empty = true;
-		for(int i = 0; i < _colIndexes.length; i++) {
-			double v = _data[indexOff + _colIndexes[i]];
-			empty &= v == 0;
-			reusableArr[i] = v;
+		while(empty && _rl < _ru) {
+			_rl++;
+			final int indexOff = _rl * _numCols;
+			for(int i = 0; i < _colIndexes.length; i++) {
+				double v = _data[indexOff + _colIndexes[i]];
+				empty &= v == 0;
+				reusableArr[i] = v;
+			}
 		}
 
-
-		return empty ? emptyReturn : reusableReturn;
+		return empty ? null : reusableReturn;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlockTransposed.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlockTransposed.java
index de647b9093..291a3de2c9 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlockTransposed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionDenseSingleBlockTransposed.java
@@ -19,7 +19,6 @@
 
 package org.apache.sysds.runtime.compress.readers;
 
-import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
@@ -27,26 +26,23 @@ public class ReaderColumnSelectionDenseSingleBlockTransposed extends ReaderColum
 	private final double[] _data;
 
 	protected ReaderColumnSelectionDenseSingleBlockTransposed(MatrixBlock data, int[] colIndexes, int rl, int ru) {
-		super(colIndexes.clone(), rl, Math.min(ru, data.getNumColumns()));
+		super(colIndexes.clone(), rl, Math.min(ru, data.getNumColumns()) -1 );
 		_data = data.getDenseBlockValues();
-		if(data.getDenseBlock().numBlocks() > 1)
-			throw new DMLCompressionException("Not handling multi block data reading in dense transposed reader");
 		for(int i = 0; i < _colIndexes.length; i++)
 			_colIndexes[i] = _colIndexes[i] * data.getNumColumns();
 	}
 
 	protected DblArray getNextRow() {
-		if(_rl == _ru - 1)
-			return null;
-		_rl++;
-
 		boolean empty = true;
-		for(int i = 0; i < _colIndexes.length; i++) {
-			final double v = _data[_colIndexes[i] + _rl];
-			empty &= v == 0;
-			reusableArr[i] = v;
-		}
+		while(empty && _rl < _ru ) {
+			_rl++;
+			for(int i = 0; i < _colIndexes.length; i++) {
+				final double v = _data[_colIndexes[i] + _rl];
+				empty &= v == 0;
+				reusableArr[i] = v;
+			}
 
-		return empty ? emptyReturn : reusableReturn;
+		}
+		return empty ? null : reusableReturn;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparse.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparse.java
index 71b71e4df5..508233c498 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparse.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparse.java
@@ -19,6 +19,8 @@
 
 package org.apache.sysds.runtime.compress.readers;
 
+import java.util.Arrays;
+
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -31,40 +33,42 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
  */
 public class ReaderColumnSelectionSparse extends ReaderColumnSelection {
 
-	private SparseBlock a;
+	private final SparseBlock a;
 
 	protected ReaderColumnSelectionSparse(MatrixBlock data, int[] colIndexes, int rl, int ru) {
-		super(colIndexes, rl, Math.min(ru, data.getNumRows()));
+		super(colIndexes, rl, Math.min(ru, data.getNumRows()) - 1);
 		a = data.getSparseBlock();
 	}
 
 	protected final DblArray getNextRow() {
-		if(_rl == _ru - 1) {
-			return null;
-		}
+		boolean empty = true;
+		while(empty && _rl < _ru) {
+			_rl++;
+			if(a.isEmpty(_rl))
+				continue; // if empty easy skip
 
-		_rl++;
+			final boolean zeroResult = processInRange(_rl);
 
-		if(a.isEmpty(_rl))
-			return emptyReturn;
+			if(zeroResult)
+				continue; // skip if no values found were in my cols
 
-		final int apos = a.pos(_rl);
-		final int alen = a.size(_rl) + apos;
-		final int[] aix = a.indexes(_rl);
-
-		if(aix[alen - 1] < _colIndexes[0] || aix[apos] > _colIndexes[_colIndexes.length - 1])
-			return emptyReturn;
+			return reusableReturn;
+		}
+		return null;
 
-		return nextRow(apos, alen, aix, a.values(_rl));
 	}
 
-	private final DblArray nextRow(final int apos, final int alen, final int[] aix, final double[] avals) {
+	final boolean processInRange(final int r) {
 		boolean zeroResult = true;
-
+		final int apos = a.pos(r);
+		final int alen = a.size(r) + apos;
+		final int[] aix = a.indexes(r);
+		final double[] avals = a.values(r);
 		int skip = 0;
-		int j = apos;
-		while(aix[j] < _colIndexes[0])
-			j++;
+		int j = Arrays.binarySearch(aix, apos, alen, _colIndexes[0]);
+		if(j < 0)
+			j = Math.abs(j+1);
+
 		while(skip < _colIndexes.length && j < alen) {
 			if(_colIndexes[skip] == aix[j]) {
 				reusableArr[skip] = avals[j];
@@ -77,9 +81,12 @@ public class ReaderColumnSelectionSparse extends ReaderColumnSelection {
 			else
 				reusableArr[skip++] = 0;
 		}
+		if(zeroResult)
+			return true; // skip if no values found were in my cols
+
 		while(skip < _colIndexes.length)
 			reusableArr[skip++] = 0;
 
-		return zeroResult ? emptyReturn : reusableReturn;
+		return false;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparseTransposed.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparseTransposed.java
index 62e0299753..fb162f10b7 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparseTransposed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderColumnSelectionSparseTransposed.java
@@ -19,6 +19,8 @@
 
 package org.apache.sysds.runtime.compress.readers;
 
+import java.util.Arrays;
+
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -31,59 +33,132 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
  */
 public class ReaderColumnSelectionSparseTransposed extends ReaderColumnSelection {
 
-	private SparseBlock a;
+	// sparse block to iterate through
+	private final SparseBlock a;
 	// current sparse skip positions.
-	private int[] sparsePos = null;
+	private final int[] sparsePos;
 
+	private boolean atEnd = false;
 
 	protected ReaderColumnSelectionSparseTransposed(MatrixBlock data, int[] colIndexes, int rl, int ru) {
 		super(colIndexes, rl, Math.min(ru, data.getNumColumns()));
 		sparsePos = new int[colIndexes.length];
-
 		a = data.getSparseBlock();
-		// Use -1 to indicate that this column is done.
+		_rl = _rl + 1; // correct row since this iterator use the exact row
+
 		for(int i = 0; i < colIndexes.length; i++) {
-			if(a.isEmpty(_colIndexes[i]))
+			final int c = _colIndexes[i];
+			if(a.isEmpty(c)) {
+				atEnd = true;
 				sparsePos[i] = -1;
-			else
-				sparsePos[i] = a.pos(_colIndexes[i]);
+			}
+			else {
+				final int[] aIdx = a.indexes(c);
+				final int pos = a.pos(c);
+				final int len = a.size(c) + pos;
+				final int spa = Arrays.binarySearch(aIdx, pos, len, _rl);
+				if(spa >= 0) {
+					if(aIdx[spa] < _ru)
+						sparsePos[i] = spa;
+					else {
+						sparsePos[i] = -1;
+						atEnd = true;
+					}
+				}
+				else { // spa < 0 or larger.
+					final int spaC = Math.abs(spa + 1);
+					if(spaC < len && aIdx[spaC] < _ru)
+						sparsePos[i] = spaC;
+					else {
+						atEnd = true;
+						sparsePos[i] = -1;
+					}
+				}
+			}
 		}
 	}
 
 	protected DblArray getNextRow() {
-		if(_rl == _ru - 1)
-			return null;
-
-		_rl++;
+		if(!atEnd)
+			return getNextRowBeforeEnd();
+		else
+			return getNextRowAtEnd();
+	}
 
-		boolean zeroResult = true;
-		boolean allDone = true;
+	protected DblArray getNextRowBeforeEnd() {
+		skipToRow();
+		if(_rl >= _ru) { // if done return null
+			_rl = _ru;
+			return null;
+		}
 		for(int i = 0; i < _colIndexes.length; i++) {
-			int colidx = _colIndexes[i];
-			if(sparsePos[i] != -1) {
-				allDone = false;
-				final int alen = a.size(colidx) + a.pos(colidx);
-				final int[] aix = a.indexes(colidx);
-				final double[] avals = a.values(colidx);
-				while(sparsePos[i] < alen && aix[sparsePos[i]] < _rl)
-					sparsePos[i] += 1;
-
-				if(sparsePos[i] >= alen) {
-					// Mark this column as done.
+			final int c = _colIndexes[i];
+			final int sp = sparsePos[i];
+			final int[] aix = a.indexes(c);
+			if(aix[sp] == _rl) {
+				final double[] avals = a.values(c);
+				reusableArr[i] = avals[sp];
+				final int spa = sparsePos[i]++;
+				final int len = a.size(c) + a.pos(c) - 1;
+				if(spa >= len || aix[spa] >= _ru) {
 					sparsePos[i] = -1;
-					reusableArr[i] = 0;
+					atEnd = true;
 				}
-				else if(aix[sparsePos[i]] == _rl) {
-					reusableArr[i] = avals[sparsePos[i]];
-					zeroResult = false;
+			}
+			else
+				reusableArr[i] = 0;
+		}
+
+		return reusableReturn;
+	}
+
+	private void skipToRow() {
+		_rl = a.indexes(_colIndexes[0])[sparsePos[0]];
+		for(int i = 1; i < _colIndexes.length; i++)
+			_rl = Math.min(a.indexes(_colIndexes[i])[sparsePos[i]], _rl);
+	}
+
+	protected DblArray getNextRowAtEnd() {
+		// at end
+		skipToRowAtEnd();
+
+		if(_rl == _ru) { // if done return null
+			_rl = _ru;
+			return null;
+		}
+
+		for(int i = 0; i < _colIndexes.length; i++) {
+			int c = _colIndexes[i];
+			final int sp = sparsePos[i];
+			if(sp != -1) {
+				final int[] aix = a.indexes(c);
+				if(aix[sp] == _rl) {
+					final double[] avals = a.values(c);
+					reusableArr[i] = avals[sp];
+					if(++sparsePos[i] >= a.size(c) + a.pos(c))
+						sparsePos[i] = -1;
 				}
 				else
 					reusableArr[i] = 0;
 			}
 		}
-		if(allDone)
-			_rl = _ru - 1;
-		return zeroResult ? emptyReturn : reusableReturn;
+		return reusableReturn;
+	}
 
+	private void skipToRowAtEnd() {
+		boolean allDone = true;
+		int mr = _ru;
+		for(int i = 0; i < _colIndexes.length; i++) {
+			final int sp = sparsePos[i];
+			if(sp != -1) {
+				allDone = false;
+				mr = Math.min(a.indexes(_colIndexes[i])[sp], mr);
+			}
+			else
+				reusableArr[i] = 0;
+		}
+		_rl = mr;
+		if(allDone)
+			_rl = _ru;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DCounts.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DCounts.java
index ab165fc630..316dd30ed3 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DCounts.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DCounts.java
@@ -19,11 +19,13 @@
 package org.apache.sysds.runtime.compress.utils;
 
 public class DCounts {
-	public double key = Double.MAX_VALUE;
+	final public double key;
 	public int count;
+	public int id;
 
-	public DCounts(double key) {
+	public DCounts(double key, int id) {
 		this.key = key;
+		this.id = id;
 		count = 1;
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayCountHashMap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayCountHashMap.java
index 5aed080b5d..bb086ba8c1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayCountHashMap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayCountHashMap.java
@@ -92,8 +92,7 @@ public class DblArrayCountHashMap {
 		Bucket ob = _data[ix];
 		_data[ix] = new Bucket(new DArrCounts(new DblArray(key), _size));
 		_data[ix].n = ob;
-		final int id = _size;
-		_size++;
+		final int id = _size++;
 		if(_size >= LOAD_FACTOR * _data.length)
 			resize();
 		return id;
@@ -127,6 +126,25 @@ public class DblArrayCountHashMap {
 		return ret;
 	}
 
+	public void replaceWithUIDs() {
+		int i = 0;
+		for(Bucket e : _data)
+			while(e != null) {
+				e.v.count = i++;
+				e = e.n;
+			}
+	}
+
+	public int getSumCounts(){
+		int c = 0;
+		for(Bucket e : _data)
+			while(e != null) {
+				c += e.v.count;
+				e = e.n;
+			}
+		return c;
+	}
+
 	public int[] getUnorderedCountsAndReplaceWithUIDs() {
 		final int[] counts = new int[_size];
 		int i = 0;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleCountHashMap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleCountHashMap.java
index 4a83b42d57..4785c65cbf 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleCountHashMap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleCountHashMap.java
@@ -29,13 +29,13 @@ public class DoubleCountHashMap {
 	protected static final Log LOG = LogFactory.getLog(DoubleCountHashMap.class.getName());
 	protected static final int RESIZE_FACTOR = 2;
 	protected static final float LOAD_FACTOR = 0.80f;
-	public static int hashMissCount = 0;
 
 	protected int _size = -1;
 	private Bucket[] _data = null;
 
 	public DoubleCountHashMap(int init_capacity) {
-		_data = new Bucket[Util.getPow2(init_capacity)];
+		_data = new Bucket[(Util.getPow2(init_capacity)/2) + 7];
+		// _data = new Bucket[(Util.getPow2(init_capacity)) ];
 		_size = 0;
 	}
 
@@ -45,8 +45,7 @@ public class DoubleCountHashMap {
 
 	private void appendValue(DCounts ent) {
 		// compute entry index position
-		int hash = hash(ent.key);
-		int ix = indexFor(hash, _data.length);
+		int ix = hashIndex(ent.key);
 		Bucket l = _data[ix];
 		if(l == null)
 			_data[ix] = new Bucket(ent);
@@ -61,26 +60,42 @@ public class DoubleCountHashMap {
 
 	}
 
-	public void increment(double key) {
-		int hash = hash(key);
-		int ix = indexFor(hash, _data.length);
+	public final int increment(final double key) {
+		final int ix = hashIndex(key);
 		Bucket l = _data[ix];
-		while(l != null && !(l.v.key == key)) {
-			hashMissCount++;
+		while(l != null) {
+			if(l.v.key == key) {
+				l.v.count++;
+				return l.v.id;
+			}
+			else
 			l = l.n;
 		}
+		return addNewBucket(ix, key);
+	}
 
-		if(l == null) {
-			Bucket ob = _data[ix];
-			_data[ix] = new Bucket(new DCounts(key));
-			_data[ix].n = ob;
-			_size++;
+	public final int increment(final double key, final int count) {
+		final int ix = hashIndex(key);
+		Bucket l = _data[ix];
+		while(l != null) {
+			if(l.v.key == key) {
+				l.v.count += count;
+				return l.v.id;
+			}
+			else
+			l = l.n;
 		}
-		else
-			l.v.count++;
+		return addNewBucket(ix, key);
+	}
 
+	private int addNewBucket(final int ix, final double key) {
+		Bucket ob = _data[ix];
+		_data[ix] = new Bucket(new DCounts(key, _size));
+		_data[ix].n = ob;
+		final int id = _size++;
 		if(_size >= LOAD_FACTOR * _data.length)
 			resize();
+		return id;
 	}
 
 	/**
@@ -90,8 +105,7 @@ public class DoubleCountHashMap {
 	 * @return count on key
 	 */
 	public int get(double key) {
-		int hash = hash(key);
-		int ix = indexFor(hash, _data.length);
+		int ix = hashIndex(key);
 		Bucket l = _data[ix];
 		while(!(l.v.key == key))
 			l = l.n;
@@ -99,13 +113,12 @@ public class DoubleCountHashMap {
 		return l.v.count;
 	}
 
-	public int getOrDefault(double key, int def){
-		int hash = hash(key);
-		int ix = indexFor(hash, _data.length);
+	public int getOrDefault(double key, int def) {
+		int ix = hashIndex(key);
 		Bucket l = _data[ix];
 		while(l != null && !(l.v.key == key))
 			l = l.n;
-		if (l == null)
+		if(l == null)
 			return def;
 		return l.v.count;
 	}
@@ -123,6 +136,27 @@ public class DoubleCountHashMap {
 		return ret;
 	}
 
+	public void replaceWithUIDs() {
+		int i = 0;
+		for(Bucket e : _data)
+			while(e != null) {
+				e.v.count = i++;
+				e = e.n;
+			}
+	}
+
+
+	public void replaceWithUIDsNoZero() {
+		int i = 0;
+		for(Bucket e : _data) {
+			while(e != null) {
+				if(e.v.key != 0) 
+					e.v.count = i++;
+				e = e.n;
+			}
+		}
+	}
+
 	public int[] getUnorderedCountsAndReplaceWithUIDs() {
 		final int[] counts = new int[_size];
 		int i = 0;
@@ -136,12 +170,12 @@ public class DoubleCountHashMap {
 		return counts;
 	}
 
-	public int[] getUnorderedCountsAndReplaceWithUIDsWithout0(){
-		final int[] counts = new int[_size - 1];
+	public int[] getUnorderedCountsAndReplaceWithUIDsWithout0() {
+		final int[] counts = new int[_size];
 		int i = 0;
-		for(Bucket e : _data){
+		for(Bucket e : _data) {
 			while(e != null) {
-				if(e.v.key != 0){
+				if(e.v.key != 0) {
 					counts[i] = e.v.count;
 					e.v.count = i++;
 				}
@@ -152,18 +186,20 @@ public class DoubleCountHashMap {
 		return counts;
 	}
 
-	// public int[] getUnorderedCountsAndReplaceWithUIDsWithExtraCell() {
-	// 	final int[] counts = new int[_size + 1];
-	// 	int i = 0;
-	// 	for(Bucket e : _data)
-	// 		while(e != null) {
-	// 			counts[i] = e.v.count;
-	// 			e.v.count = i++;
-	// 			e = e.n;
-	// 		}
-
-	// 	return counts;
-	// }
+	public double getMostFrequent(){
+		double f = 0;
+		int fq = 0;
+		for(Bucket e: _data){
+			while(e != null){
+				if(e.v.count > fq){
+					fq = e.v.count;
+					f = e.v.key;
+				}
+				e = e.n;
+			}
+		}
+		return f;
+	}
 
 	private void resize() {
 		// check for integer overflow on resize
@@ -182,26 +218,57 @@ public class DoubleCountHashMap {
 				e = e.n;
 			}
 		}
+	}
 
+	public double[] getDictionary() {
+		final double[] ret = new double[_size];
+		for(Bucket e : _data)
+			while(e != null) {
+				ret[e.v.id] = e.v.key;
+				e = e.n;
+			}
+		return ret;
 	}
 
-	private static int hash(double key) {
-		// return (int) key;
+	private final int hashIndex(final double key) {
+		
+		// previous require pow2 size.:
+		// long bits = Double.doubleToRawLongBits(key);
+		// int h =(int)( bits ^ (bits >>> 32));
+		// h = h ^ (h >>> 20) ^ (h >>> 12);
+		// h = h ^ (h >>> 7) ^ (h >>> 4);
+		// return h & (_data.length - 1);
+		// 100.809.414.955      instructions
+
+		// Option 1 ... conflict on 1 vs -1
+		long bits = Double.doubleToLongBits(key);
+		return Math.abs((int)(bits ^ (bits >>> 32)) % _data.length);
+		// 102.356.926.448      instructions
+
+		// Option 2
+		// long bits = Double.doubleToRawLongBits(key);
+		// return (int) ((bits ^ (bits >> 32) % _data.length));
+		
 
 		// basic double hash code (w/o object creation)
-		long bits = Double.doubleToRawLongBits(key);
-		int h = (int) (bits ^ (bits >>> 32));
+		// return Double.hashCode(key) % _data.length;
+		// return (int) ((bits ^ (bits >>> 32)) % _data.length);
+		// long bits = Double.doubleToLongBits(key);
+		// return (int) Long.remainderUnsigned(bits, (long) _data.length);
+		// long bits = Double.doubleToLongBits(key);
+		// long bits = Double.doubleToRawLongBits(key);
+		// return (int) (bits % (long) _data.length);
+		
+		// return h;
 
 		// This function ensures that hashCodes that differ only by
 		// constant multiples at each bit position have a bounded
 		// number of collisions (approximately 8 at default load factor).
-		h ^= (h >>> 20) ^ (h >>> 12);
-		return h ^ (h >>> 7) ^ (h >>> 4);
 	}
 
-	private static int indexFor(int h, int length) {
-		return h & (length - 1);
-	}
+	// private static int indexFor(int h, int length) {
+	// return h & (length - 1);
+	// }
 
 	protected static class Bucket {
 		protected DCounts v;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/Util.java b/src/main/java/org/apache/sysds/runtime/compress/utils/Util.java
index b3ed964731..0f8778ea81 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/Util.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/Util.java
@@ -56,13 +56,20 @@ public interface Util {
 		return Math.max(v, 4);
 	}
 
-	public static int[] genColsIndices(int numCols) {
-		int[] colIndices = new int[numCols];
+	public static int[] genColsIndices(final int numCols) {
+		final int[] colIndices = new int[numCols];
 		for(int i = 0; i < numCols; i++)
 			colIndices[i] = i;
 		return colIndices;
 	}
 
+	public static int[] genColsIndicesOffset(final int numCols, final int start) {
+		final int[] colIndices = new int[numCols];
+		for(int i = 0, j = start; i < numCols; i++, j++)
+			colIndices[i] = j;
+		return colIndices;
+	}
+
 	public static MatrixBlock matrixBlockFromDenseArray(double[] values, int nCol) {
 		final int nRow = values.length / nCol;
 		DenseBlock dictV = new DenseBlockFP64(new int[] {nRow, nCol}, values);
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 d16c0144d8..83ff6e5184 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
@@ -34,6 +34,8 @@ public abstract class Op {
 	public Op(Hop op) {
 		_op = op;
 		_dim =(int) op.getDim2();
+		if(_dim < 0)
+			_dim = 16;
 	}
 
 	public Hop getHop() {
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 b6e79df959..63ea7c04c5 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
@@ -41,6 +41,8 @@ public class OpSided extends Op {
 		_tLeft = tLeft;
 		_tRight = tRight;
 		_dim = (int) (cLeft ? op.getDim2() : op.getDim1());
+		if(_dim < 0)
+			_dim = 16;
 	}
 
 	public boolean getLeft() {
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 61b9aaa937..68b60438fa 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
@@ -43,6 +43,7 @@ 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.NaryOp;
 import org.apache.sysds.hops.ParameterizedBuiltinOp;
 import org.apache.sysds.hops.ReorgOp;
 import org.apache.sysds.hops.UnaryOp;
@@ -140,7 +141,7 @@ public class WorkloadAnalyzer {
 		List<Hop> candidates = new ArrayList<>();
 		for(StatementBlock sb : prog.getStatementBlocks())
 			getCandidates(sb, prog, candidates, new HashSet<>());
-		
+
 		return candidates;
 	}
 
@@ -445,7 +446,7 @@ public class WorkloadAnalyzer {
 						return;
 					}
 					else {
-						String ex = "Setting decompressed because input Binary Op is unknown, please add the case to WorkloadAnalyzer:\n"
+						String ex = "Setting decompressed because input Binary Op dimensions is unknown:\n"
 							+ Explain.explain(hop);
 						LOG.warn(ex);
 						setDecompressionOnAllInputs(hop, parent);
@@ -503,8 +504,12 @@ public class WorkloadAnalyzer {
 				setDecompressionOnAllInputs(hop, parent);
 				return;
 			}
+			else if(hop instanceof NaryOp){
+				setDecompressionOnAllInputs(hop, parent);
+				return;
+			}
 			else
-				throw new DMLCompressionException("Unknown Hop: " + Explain.explain(hop));
+				throw new DMLCompressionException("Unknown Hop:" +hop.getClass().getSimpleName() +"\n" + Explain.explain(hop));
 
 			o = o != null ? o : new OpNormal(hop, RewriteCompressedReblock.satisfiesSizeConstraintsForCompression(hop));
 			treeLookup.put(hop.getHopID(), o);
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixReorg.java b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixReorg.java
index 9f07754837..7c78f8f1a6 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixReorg.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixReorg.java
@@ -36,6 +36,8 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.controlprogram.caching.MatrixObject.UpdateType;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFactory;
@@ -148,6 +150,8 @@ public class LibMatrixReorg {
 	}
 
 	public static MatrixBlock transpose( MatrixBlock in, MatrixBlock out ) {
+		if(in instanceof CompressedMatrixBlock)
+			throw new DMLCompressionException("Invalid call to transposed with a compressed matrix block");
 		//sparse-safe operation
 		if( in.isEmptyBlock(false) )
 			return out;
@@ -180,7 +184,7 @@ public class LibMatrixReorg {
 			transposeUltraSparse(in, out);
 		else if( in.sparse && out.sparse )
 			transposeSparseToSparse(in, out, 0, in.rlen, 0, in.clen, 
-				countNnzPerColumn(in, 0, in.rlen));
+				countNnzPerColumn(in, 4096));
 		else if( in.sparse )
 			transposeSparseToDense(in, out, 0, in.rlen, 0, in.clen);
 		else
@@ -1818,11 +1822,15 @@ public class LibMatrixReorg {
 		}
 	}
 
-	private static int[] countNnzPerColumn(MatrixBlock in, int rl, int ru) {
+	private static int[] countNnzPerColumn(MatrixBlock in, int maxCol) {
+		return countNnzPerColumn(in, 0, in.getNumRows(), maxCol);
+	}
+
+	private static int[] countNnzPerColumn(MatrixBlock in, int rl, int ru, int maxCol) {
 		//initial pass to determine capacity (this helps to prevent
 		//sparse row reallocations and mem inefficiency w/ skew
 		int[] cnt = null;
-		if( in.sparse && in.clen <= 4096 ) { //16KB
+		if(in.clen <= maxCol) {
 			SparseBlock a = in.sparseBlock;
 			cnt = new int[in.clen];
 			for( int i=rl; i<ru; i++ ) {
@@ -1833,6 +1841,40 @@ public class LibMatrixReorg {
 		return cnt;
 	}
 
+	public static int[] countNnzPerColumn(MatrixBlock in) {
+		return countNnzPerColumn(in, 0, in.getNumRows());
+	}
+
+	public static int[] countNnzPerColumn(MatrixBlock in, int rl, int ru) {
+		if(in.isInSparseFormat())
+			return countNnzPerColumnSparse(in, rl, ru);
+		else
+			return countNnzPerColumnDense(in, rl, ru);
+	}
+
+	private static int[] countNnzPerColumnSparse(MatrixBlock in, int rl, int ru) {
+		final int[] cnt = new int[in.clen];
+		final SparseBlock a = in.sparseBlock;
+		for(int i = rl; i < ru; i++) {
+			if(!a.isEmpty(i))
+				countAgg(cnt, a.indexes(i), a.pos(i), a.size(i));
+		}
+		return cnt;
+	}
+
+
+	private static int[] countNnzPerColumnDense(MatrixBlock in, int rl, int ru) {
+		final int[] cnt = new int[in.clen];
+		final double[] dV = in.getDenseBlockValues();
+		int off = rl * in.clen;
+		for(int i = rl; i < ru; i++)
+			for(int j = 0; j < in.clen; j++)
+				if(dV[off++] != 0)
+					cnt[j]++;
+			
+		return cnt;
+	}
+
 	public static int[] mergeNnzCounts(int[] cnt, int[] cnt2) {
 		if( cnt == null )
 			return cnt2;
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 d65a6d548b..182b771138 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
@@ -3090,10 +3090,10 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 
 			if(m2 instanceof CompressedMatrixBlock)
 				m2 = ((CompressedMatrixBlock) m2)
-					.getUncompressed("Ternary Operator arg2 " + op.fn.getClass().getSimpleName());
+					.getUncompressed("Ternary Operator arg2 " + op.fn.getClass().getSimpleName(), op.getNumThreads());
 			if(m3 instanceof CompressedMatrixBlock)
 				m3 = ((CompressedMatrixBlock) m3)
-					.getUncompressed("Ternary Operator arg3 " + op.fn.getClass().getSimpleName());
+					.getUncompressed("Ternary Operator arg3 " + op.fn.getClass().getSimpleName(), op.getNumThreads());
 
 			ret.reset(m, n, sparseOutput);
 
@@ -3766,6 +3766,10 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		boolean plus = fn instanceof Plus;
 		Builtin bfn = !plus ? (Builtin)((SimpleOperator)op).fn : null;
 		
+		for(int i = 0; i < matrices.length; i++)
+			if(matrices[i] instanceof CompressedMatrixBlock)
+				matrices[i] = CompressedMatrixBlock.getUncompressed(matrices[i], "Nary operation process add row");
+
 		//process all scalars
 		double init = plus ? 0 :(bfn.getBuiltinCode() == BuiltinCode.MIN) ?
 			Double.POSITIVE_INFINITY : Double.NEGATIVE_INFINITY;
@@ -3834,8 +3838,6 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		for( MatrixBlock in : inputs ) {
 			if( in.isEmptyBlock(false) )
 				continue;
-			if(in instanceof CompressedMatrixBlock)
-				in = CompressedMatrixBlock.getUncompressed(in, "ProcessAddRow");
 			
 			if( in.isInSparseFormat() ) {
 				SparseBlock a = in.getSparseBlock();
@@ -5058,11 +5060,11 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	public MatrixBlock aggregateTernaryOperations(MatrixBlock m1, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret,
 			AggregateTernaryOperator op, boolean inCP) {
 		if(m1 instanceof CompressedMatrixBlock)
-			m1 = ((CompressedMatrixBlock) m1).getUncompressed("Aggregate Ternary Operator arg1 " + op.getClass().getSimpleName());
+			m1 = ((CompressedMatrixBlock) m1).getUncompressed("Aggregate Ternary Operator arg1 " + op.getClass().getSimpleName(), op.getNumThreads());
 		if(m2 instanceof CompressedMatrixBlock)
-			m2 = ((CompressedMatrixBlock) m2).getUncompressed("Aggregate Ternary Operator arg2 " + op.getClass().getSimpleName());
+			m2 = ((CompressedMatrixBlock) m2).getUncompressed("Aggregate Ternary Operator arg2 " + op.getClass().getSimpleName(), op.getNumThreads());
 		if(m3 instanceof CompressedMatrixBlock)
-			m3 = ((CompressedMatrixBlock) m3).getUncompressed("Aggregate Ternary Operator arg3 " + op.getClass().getSimpleName());
+			m3 = ((CompressedMatrixBlock) m3).getUncompressed("Aggregate Ternary Operator arg3 " + op.getClass().getSimpleName(), op.getNumThreads());
 
 		//create output matrix block w/ corrections
 		int rl = (op.indexFn instanceof ReduceRow) ? 2 : 1;
diff --git a/src/test/java/org/apache/sysds/test/TestUtils.java b/src/test/java/org/apache/sysds/test/TestUtils.java
index f367032467..0ca3976a43 100644
--- a/src/test/java/org/apache/sysds/test/TestUtils.java
+++ b/src/test/java/org/apache/sysds/test/TestUtils.java
@@ -1140,23 +1140,29 @@ public class TestUtils
 			if(sbe.isEmpty(i))
 				continue;
... 3335 lines suppressed ...