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/08/17 13:49:37 UTC

[systemds] branch main updated: [SYSTEMDS-3395] ColGroup Equivalence Tests

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 22475320d7 [SYSTEMDS-3395] ColGroup Equivalence Tests
22475320d7 is described below

commit 22475320d7ff6cdaeb2f7dc521fcbd275553a7a0
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Fri May 20 19:04:42 2022 +0200

    [SYSTEMDS-3395] ColGroup Equivalence Tests
    
    - Add RLE col group
    - Add morphing test - experiment verifying morphing is good.
    
    Closes #1619
---
 src/main/java/org/apache/sysds/conf/DMLConfig.java |    5 +-
 .../runtime/compress/CompressedMatrixBlock.java    |    6 +-
 .../compress/CompressedMatrixBlockFactory.java     |   13 +-
 .../runtime/compress/CompressionSettings.java      |    4 +
 .../sysds/runtime/compress/bitmap/ABitmap.java     |    4 +-
 .../sysds/runtime/compress/bitmap/Bitmap.java      |    6 -
 .../runtime/compress/bitmap/BitmapEncoder.java     |    6 +
 .../runtime/compress/bitmap/MultiColBitmap.java    |    6 -
 .../runtime/compress/cocode/AColumnCoCoder.java    |    6 +-
 .../runtime/compress/cocode/CoCodeBinPacking.java  |    4 +-
 .../runtime/compress/cocode/CoCodeGreedy.java      |    6 +-
 .../runtime/compress/cocode/CoCodeHybrid.java      |    4 +-
 .../runtime/compress/cocode/CoCodePriorityQue.java |   18 +-
 .../runtime/compress/cocode/CoCodeStatic.java      |    4 +-
 .../runtime/compress/cocode/CoCoderFactory.java    |    6 +-
 .../sysds/runtime/compress/cocode/Memorizer.java   |    6 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |   42 +-
 .../compress/colgroup/AColGroupCompressed.java     |  135 +-
 .../runtime/compress/colgroup/AColGroupOffset.java |  123 +-
 .../runtime/compress/colgroup/AColGroupValue.java  |  322 +---
 .../compress/colgroup/ADictBasedColGroup.java      |  287 +++
 .../compress/colgroup/AMorphingMMColGroup.java     |  154 +-
 .../sysds/runtime/compress/colgroup/APreAgg.java   |  113 +-
 .../sysds/runtime/compress/colgroup/ASDC.java      |   67 +
 .../sysds/runtime/compress/colgroup/ASDCZero.java  |   65 +-
 .../runtime/compress/colgroup/ColGroupConst.java   |  237 ++-
 .../runtime/compress/colgroup/ColGroupDDC.java     |  154 +-
 .../runtime/compress/colgroup/ColGroupDDCFOR.java  |  248 +--
 .../compress/colgroup/ColGroupDeltaDDC.java        |   21 +-
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   35 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |  381 ++--
 .../runtime/compress/colgroup/ColGroupIO.java      |   26 +-
 .../colgroup/ColGroupLinearFunctional.java         |   11 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     |  268 ++-
 .../runtime/compress/colgroup/ColGroupRLE.java     | 1210 ++++++++----
 .../runtime/compress/colgroup/ColGroupSDC.java     |  254 ++-
 .../runtime/compress/colgroup/ColGroupSDCFOR.java  |   99 +-
 .../compress/colgroup/ColGroupSDCSingle.java       |  308 +--
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  181 +-
 .../compress/colgroup/ColGroupSDCZeros.java        |  182 +-
 .../runtime/compress/colgroup/ColGroupSizes.java   |   29 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  387 ++--
 .../runtime/compress/colgroup/ColGroupUtils.java   |  219 ++-
 .../sysds/runtime/compress/colgroup/FORUtil.java   |   75 -
 .../compress/colgroup/dictionary/ADictionary.java  |  149 +-
 .../compress/colgroup/dictionary/Dictionary.java   |  244 ++-
 .../colgroup/dictionary/DictionaryFactory.java     |   46 +-
 .../colgroup/dictionary/MatrixBlockDictionary.java |  784 ++++++--
 .../compress/colgroup/dictionary/QDictionary.java  |  100 +-
 .../colgroup/functional/LinearRegression.java      |   34 +-
 .../compress/colgroup/mapping/AMapToData.java      |   76 +-
 .../compress/colgroup/mapping/MapToBit.java        |   48 +-
 .../compress/colgroup/mapping/MapToByte.java       |   17 +-
 .../compress/colgroup/mapping/MapToChar.java       |   11 +
 .../compress/colgroup/mapping/MapToCharPByte.java  |   12 +
 .../compress/colgroup/mapping/MapToFactory.java    |    3 +-
 .../compress/colgroup/mapping/MapToInt.java        |   18 +-
 .../compress/colgroup/mapping/MapToZero.java       |    5 +
 .../compress/colgroup/offset/AIterator.java        |    8 +
 .../runtime/compress/colgroup/offset/AOffset.java  |   21 +-
 .../compress/colgroup/offset/OffsetByte.java       |   25 +-
 .../{CompressedSizeEstimator.java => AComEst.java} |    8 +-
 ...sedSizeEstimatorExact.java => ComEstExact.java} |    6 +-
 ...izeEstimatorFactory.java => ComEstFactory.java} |   18 +-
 ...dSizeEstimatorSample.java => ComEstSample.java} |  114 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |   78 +-
 .../runtime/compress/estim/EstimationFactors.java  |   51 +-
 .../compress/estim/encoding/ConstEncoding.java     |    7 +-
 .../compress/estim/encoding/DenseEncoding.java     |   12 +-
 .../compress/estim/encoding/EmptyEncoding.java     |   12 +-
 .../runtime/compress/estim/encoding/IEncode.java   |   18 +-
 .../compress/estim/encoding/SparseEncoding.java    |   19 +-
 .../runtime/compress/lib/CLALibBinaryCellOp.java   |    2 +-
 .../sysds/runtime/compress/lib/CLALibCMOps.java    |   17 +-
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |    4 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |   41 +-
 .../runtime/compress/lib/CLALibRightMultBy.java    |   54 +-
 .../sysds/runtime/compress/lib/CLALibTSMM.java     |    8 +-
 .../sysds/runtime/compress/lib/CLALibUtils.java    |   32 +-
 .../readers/ReaderColumnSelectionSparse.java       |    3 +-
 .../ReaderColumnSelectionSparseTransposed.java     |    8 +-
 .../apache/sysds/runtime/compress/utils/Util.java  |   20 +-
 .../org/apache/sysds/runtime/data/DenseBlock.java  |    2 +-
 .../runtime/functionobjects/IndexFunction.java     |    5 +
 .../runtime/instructions/InstructionUtils.java     |   10 +-
 .../sysds/runtime/matrix/data/LibMatrixAgg.java    |   85 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |   13 +-
 .../matrix/operators/AggregateOperator.java        |   14 +
 .../matrix/operators/AggregateUnaryOperator.java   |   12 +
 .../transform/encode/MultiColumnEncoder.java       |    4 +-
 src/test/java/org/apache/sysds/test/TestUtils.java |  157 +-
 .../component/compress/CompressedTestBase.java     |   58 +-
 .../component/compress/CompressedVectorTest.java   |   13 +-
 .../compress/CompressibleInputGenerator.java       |    2 +-
 .../component/compress/colgroup/ColGroupBase.java  |  476 +++++
 .../colgroup/ColGroupLinearFunctionalBase.java     |    4 +-
 .../colgroup/ColGroupLinearFunctionalTest.java     |   26 +-
 .../ColGroupMorphingPerformanceCompare.java        |  241 +++
 .../compress/colgroup/ColGroupNegativeTests.java   |  549 ++++++
 .../component/compress/colgroup/ColGroupTest.java  | 2017 ++++++++++++++++++++
 .../compress/colgroup/ColGroupTestColShift.java    |   61 +
 .../ColGroupUtilsTest.java}                        |   30 +-
 .../compress/colgroup/JolEstimateRLETest.java      |  219 ++-
 .../compress/colgroup/JolEstimateTest.java         |   71 +-
 .../compress/colgroup/NegativeConstTests.java      |    6 +-
 .../test/component/compress/cost/ACostTest.java    |    6 +-
 .../compress/dictionary/DictionaryTest.java        |    6 +-
 .../compress/estim/JoinCompressionInfoTest.java    |    6 +-
 .../compress/estim/SampleEstimatorTest.java        |    8 +-
 .../compress/estim/encoding/EncodeSampleTest.java  |    4 +-
 .../compress/functional/LinearRegressionTests.java |   22 +-
 .../compress/mapping/MappingPreAggregateTests.java |    6 +-
 .../compress/mapping/PreAggregateDDC_DDCTest.java  |   10 +-
 .../compress/mapping/PreAggregateDDC_SDCZTest.java |    6 +-
 .../compress/mapping/PreAggregateSDCZ_DDCTest.java |    6 +-
 .../mapping/PreAggregateSDCZ_SDCZTest.java         |    6 +-
 .../component/compress/readers/ReadersTest.java    |    1 -
 .../readers/ReadersTestCompareReaders.java         |   30 +-
 118 files changed, 8953 insertions(+), 3109 deletions(-)

diff --git a/src/main/java/org/apache/sysds/conf/DMLConfig.java b/src/main/java/org/apache/sysds/conf/DMLConfig.java
index 357e43c495..5364299415 100644
--- a/src/main/java/org/apache/sysds/conf/DMLConfig.java
+++ b/src/main/java/org/apache/sysds/conf/DMLConfig.java
@@ -82,6 +82,7 @@ public class DMLConfig
 	public static final String COMPRESSED_VALID_COMPRESSIONS = "sysds.compressed.valid.compressions";
 	public static final String COMPRESSED_OVERLAPPING = "sysds.compressed.overlapping"; 
 	public static final String COMPRESSED_SAMPLING_RATIO = "sysds.compressed.sampling.ratio"; 
+	public static final String COMPRESSED_SOFT_REFERENCE_COUNT = "sysds.compressed.softreferencecount"; 
 	public static final String COMPRESSED_COCODE    = "sysds.compressed.cocode"; 
 	public static final String COMPRESSED_COST_MODEL= "sysds.compressed.costmodel";
 	public static final String COMPRESSED_TRANSPOSE = "sysds.compressed.transpose";
@@ -156,6 +157,7 @@ public class DMLConfig
 		_defaultVals.put(COMPRESSED_VALID_COMPRESSIONS, "SDC,DDC");
 		_defaultVals.put(COMPRESSED_OVERLAPPING, "true" );
 		_defaultVals.put(COMPRESSED_SAMPLING_RATIO, "0.01");
+		_defaultVals.put(COMPRESSED_SOFT_REFERENCE_COUNT, "true");
 		_defaultVals.put(COMPRESSED_COCODE,      "AUTO");
 		_defaultVals.put(COMPRESSED_COST_MODEL,  "AUTO");
 		_defaultVals.put(COMPRESSED_TRANSPOSE,   "auto");
@@ -436,7 +438,8 @@ public class DMLConfig
 			LOCAL_TMP_DIR,SCRATCH_SPACE,OPTIMIZATION_LEVEL, DEFAULT_BLOCK_SIZE,
 			CP_PARALLEL_OPS, CP_PARALLEL_IO, PARALLEL_ENCODE, NATIVE_BLAS, NATIVE_BLAS_DIR,
 			COMPRESSED_LINALG, COMPRESSED_LOSSY, COMPRESSED_VALID_COMPRESSIONS, COMPRESSED_OVERLAPPING,
-			COMPRESSED_SAMPLING_RATIO, COMPRESSED_COCODE, COMPRESSED_TRANSPOSE, DAG_LINEARIZATION,
+			COMPRESSED_SAMPLING_RATIO, COMPRESSED_SOFT_REFERENCE_COUNT,
+			COMPRESSED_COCODE, COMPRESSED_TRANSPOSE, DAG_LINEARIZATION,
 			CODEGEN, CODEGEN_API, CODEGEN_COMPILER, CODEGEN_OPTIMIZER, CODEGEN_PLANCACHE, CODEGEN_LITERALS,
 			STATS_MAX_WRAP_LEN, LINEAGECACHESPILL, COMPILERASSISTED_RW, BUFFERPOOL_LIMIT, MEMORY_MANAGER,
 			PRINT_GPU_MEMORY_INFO, AVAILABLE_GPUS, SYNCHRONIZE_GPU, EAGER_CUDA_FREE, FLOATING_POINT_PRECISION,
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 640df9c3bb..57ab75cae4 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -372,11 +372,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public void write(DataOutput out) throws IOException {
-		if(getExactSizeOnDisk() > MatrixBlock.estimateSizeOnDisk(rlen, clen, nonZeros)) {
+		if(nonZeros > 0 && getExactSizeOnDisk() > MatrixBlock.estimateSizeOnDisk(rlen, clen, nonZeros)) {
 			// If the size of this matrixBlock is smaller in uncompressed format, then
 			// decompress and save inside an uncompressed column group.
 			MatrixBlock uncompressed = getUncompressed("for smaller serialization");
-			ColGroupUncompressed cg = new ColGroupUncompressed(uncompressed);
+			ColGroupUncompressed cg = (ColGroupUncompressed) ColGroupUncompressed.create(uncompressed);
 			allocateColGroup(cg);
 			nonZeros = cg.getNumberNonZeros(rlen);
 			// clear the soft reference to the decompressed version, since the one column group is perfectly,
@@ -724,7 +724,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return super.cmOperations(op, right);
 		AColGroup grp = _colGroups.get(0);
 		if(grp instanceof ColGroupUncompressed)
-			return ((ColGroupUncompressed) grp).getData().cmOperations(op);
+			return ((ColGroupUncompressed) grp).getData().cmOperations(op, right);
 		return getUncompressed().cmOperations(op, right);
 	}
 
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 2b8d6a8467..0d4a4b473e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -40,8 +40,8 @@ import org.apache.sysds.runtime.compress.cost.CostEstimatorBuilder;
 import org.apache.sysds.runtime.compress.cost.CostEstimatorFactory;
 import org.apache.sysds.runtime.compress.cost.InstructionTypeCounter;
 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.AComEst;
+import org.apache.sysds.runtime.compress.estim.ComEstFactory;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.workload.WTreeRoot;
@@ -80,7 +80,7 @@ public class CompressedMatrixBlockFactory {
 	/** The current Phase ID */
 	private int phase = 0;
 	/** Object to extract statistics from columns to make decisions based on */
-	private CompressedSizeEstimator informationExtractor;
+	private AComEst informationExtractor;
 	/** Compression information gathered through the sampling, used for the actual compression decided */
 	private CompressedSizeInfo compressionGroups;
 
@@ -228,14 +228,15 @@ public class CompressedMatrixBlockFactory {
 	}
 
 	/**
-	 * Generate a CompressedMatrixBlock Object that contains a single uncompressed matrix block column group.
+	 * Generate a CompressedMatrixBlock Object that contains a single uncompressed matrix block column group. Note this
+	 * could be an empty colgroup if the input is empty.
 	 * 
 	 * @param mb The matrix block to be contained in the uncompressed matrix block column,
 	 * @return a CompressedMatrixBlock
 	 */
 	public static CompressedMatrixBlock genUncompressedCompressedMatrixBlock(MatrixBlock mb) {
 		CompressedMatrixBlock ret = new CompressedMatrixBlock(mb.getNumRows(), mb.getNumColumns());
-		AColGroup cg = new ColGroupUncompressed(mb);
+		AColGroup cg = ColGroupUncompressed.create(mb);
 		ret.allocateColGroup(cg);
 		ret.setNonZeros(mb.getNonZeros());
 		return ret;
@@ -296,7 +297,7 @@ public class CompressedMatrixBlockFactory {
 
 	private void classifyPhase() {
 		// Create the extractor for column statistics
-		informationExtractor = CompressedSizeEstimatorFactory.createEstimator(mb, compSettings, k);
+		informationExtractor = ComEstFactory.createEstimator(mb, compSettings, k);
 		// Compute the individual columns cost information
 		compressionGroups = informationExtractor.computeCompressedSizeInfos(k);
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
index 137c531a2d..062ccfc120 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -155,6 +155,10 @@ public class CompressionSettings {
 			LOG.debug(this.toString());
 	}
 
+	public boolean isRLEAllowed(){
+		return this.validCompressions.contains(CompressionType.RLE);
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/bitmap/ABitmap.java b/src/main/java/org/apache/sysds/runtime/compress/bitmap/ABitmap.java
index 51bff628a5..88522d4190 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/bitmap/ABitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/bitmap/ABitmap.java
@@ -114,7 +114,9 @@ public abstract class ABitmap {
 	 * 
 	 * @return number of distinct value groups in the column;
 	 */
-	public abstract int getNumValues();
+	public final int getNumValues(){
+		return _offsetsLists.length;
+	}
 
 	/**
 	 * Get the number of non zeros in a specific offset's tuple value.
diff --git a/src/main/java/org/apache/sysds/runtime/compress/bitmap/Bitmap.java b/src/main/java/org/apache/sysds/runtime/compress/bitmap/Bitmap.java
index f0e0a6954b..24506ec551 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/bitmap/Bitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/bitmap/Bitmap.java
@@ -61,12 +61,6 @@ public final class Bitmap extends ABitmap {
 		return 1;
 	}
 
-	@Override
-	public final int getNumValues() {
-		// Values are guaranteed not to be null.
-		return _values.length;
-	}
-
 	@Override
 	public int getNumColumns() {
 		return 1;
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 c81c5948ec..cc244d50f5 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
@@ -26,6 +26,7 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.readers.ReaderColumnSelection;
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.compress.utils.DblArrayIntListHashMap;
@@ -44,6 +45,11 @@ public class BitmapEncoder {
 
 	static Log LOG = LogFactory.getLog(BitmapEncoder.class.getName());
 
+	public static ABitmap extractBitmap(int[] colIndices, MatrixBlock rawBlock, int estimatedNumberOfUniques,
+		CompressionSettings cs) {
+		return extractBitmap(colIndices, rawBlock, cs.transposed, estimatedNumberOfUniques, cs.sortTuplesByFrequency);
+	}
+
 	/**
 	 * Generate uncompressed bitmaps for a set of columns in an uncompressed matrix block.
 	 * 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/bitmap/MultiColBitmap.java b/src/main/java/org/apache/sysds/runtime/compress/bitmap/MultiColBitmap.java
index 9b47143066..b35a172d8c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/bitmap/MultiColBitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/bitmap/MultiColBitmap.java
@@ -63,12 +63,6 @@ public final class MultiColBitmap extends ABitmap {
 		return nz;
 	}
 
-	@Override
-	public int getNumValues() {
-		// values are always guaranteed to be allocated
-		return _values.length;
-	}
-
 	@Override
 	public int getNumColumns() {
 		// values are always guaranteed to be allocated
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
index 7a5efa0421..fc13e16f65 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/AColumnCoCoder.java
@@ -23,18 +23,18 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 
 public abstract class AColumnCoCoder {
 
 	protected static final Log LOG = LogFactory.getLog(AColumnCoCoder.class.getName());
 
-	protected final CompressedSizeEstimator _sest;
+	protected final AComEst _sest;
 	protected final ACostEstimate _cest;
 	protected final CompressionSettings _cs;
 
-	protected AColumnCoCoder(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator,
+	protected AColumnCoCoder(AComEst sizeEstimator, ACostEstimate costEstimator,
 		CompressionSettings cs) {
 		_sest = sizeEstimator;
 		_cest = costEstimator;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
index 356c18e381..dc5998c52f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeBinPacking.java
@@ -22,7 +22,7 @@ package org.apache.sysds.runtime.compress.cocode;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 
 /**
@@ -44,7 +44,7 @@ public class CoCodeBinPacking extends AColumnCoCoder {
 	 */
 	// private static double BIN_CAPACITY = 0.000032;
 
-	protected CoCodeBinPacking(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator,
+	protected CoCodeBinPacking(AComEst sizeEstimator, ACostEstimate costEstimator,
 		CompressionSettings cs) {
 		super(sizeEstimator, costEstimator, cs);
 		// mem = new Memorizer(sizeEstimator);
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 ad7100f72c..e8a50d33d1 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
@@ -28,7 +28,7 @@ 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.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.utils.Util;
@@ -38,12 +38,12 @@ public class CoCodeGreedy extends AColumnCoCoder {
 
 	private final Memorizer mem;
 
-	protected CoCodeGreedy(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
+	protected CoCodeGreedy(AComEst sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
 		super(sizeEstimator, costEstimator, cs);
 		mem = new Memorizer(sizeEstimator);
 	}
 
-	protected CoCodeGreedy(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs,
+	protected CoCodeGreedy(AComEst sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs,
 		Memorizer mem) {
 		super(sizeEstimator, costEstimator, cs);
 		this.mem = mem;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeHybrid.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeHybrid.java
index 698758f6ff..554f90d95d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeHybrid.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeHybrid.java
@@ -21,7 +21,7 @@ package org.apache.sysds.runtime.compress.cocode;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
 
@@ -30,7 +30,7 @@ import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
  */
 public class CoCodeHybrid extends AColumnCoCoder {
 
-	protected CoCodeHybrid(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
+	protected CoCodeHybrid(AComEst sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
 		super(sizeEstimator, costEstimator, cs);
 	}
 
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 aec327e5a7..fad5ee1010 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
@@ -31,7 +31,7 @@ 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.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.util.CommonThreadPool;
@@ -48,7 +48,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 
 	private static final int COL_COMBINE_THRESHOLD = 1024;
 
-	protected CoCodePriorityQue(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator,
+	protected CoCodePriorityQue(AComEst sizeEstimator, ACostEstimate costEstimator,
 		CompressionSettings cs) {
 		super(sizeEstimator, costEstimator, cs);
 	}
@@ -60,7 +60,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 	}
 
 	protected static List<CompressedSizeInfoColGroup> join(List<CompressedSizeInfoColGroup> groups,
-		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups, int k) {
+		AComEst sEst, ACostEstimate cEst, int minNumGroups, int k) {
 
 		if(groups.size() > COL_COMBINE_THRESHOLD && k > 1)
 			return combineMultiThreaded(groups, sEst, cEst, minNumGroups, k);
@@ -69,7 +69,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 	}
 
 	private static List<CompressedSizeInfoColGroup> combineMultiThreaded(List<CompressedSizeInfoColGroup> groups,
-		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups, int k) {
+		AComEst sEst, ACostEstimate cEst, int minNumGroups, int k) {
 		try {
 			final ExecutorService pool = CommonThreadPool.get(k);
 			final List<PQTask> tasks = new ArrayList<>();
@@ -94,12 +94,12 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 	}
 
 	private static List<CompressedSizeInfoColGroup> combineSingleThreaded(List<CompressedSizeInfoColGroup> groups,
-		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
+		AComEst sEst, ACostEstimate cEst, int minNumGroups) {
 		return combineBlock(groups, 0, groups.size(), sEst, cEst, minNumGroups);
 	}
 
 	private static List<CompressedSizeInfoColGroup> combineBlock(List<CompressedSizeInfoColGroup> groups, int start,
-		int end, CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
+		int end, AComEst sEst, ACostEstimate cEst, int minNumGroups) {
 		Queue<CompressedSizeInfoColGroup> que = getQue(end - start, cEst);
 
 		for(int i = start; i < end; i++) {
@@ -112,7 +112,7 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 	}
 
 	private static List<CompressedSizeInfoColGroup> combineBlock(Queue<CompressedSizeInfoColGroup> que,
-		CompressedSizeEstimator sEst, ACostEstimate cEst, int minNumGroups) {
+		AComEst sEst, ACostEstimate cEst, int minNumGroups) {
 
 		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
 		CompressedSizeInfoColGroup l = null;
@@ -163,11 +163,11 @@ public class CoCodePriorityQue extends AColumnCoCoder {
 		private final List<CompressedSizeInfoColGroup> _groups;
 		private final int _start;
 		private final int _end;
-		private final CompressedSizeEstimator _sEst;
+		private final AComEst _sEst;
 		private final ACostEstimate _cEst;
 		private final int _minNumGroups;
 
-		protected PQTask(List<CompressedSizeInfoColGroup> groups, int start, int end, CompressedSizeEstimator sEst,
+		protected PQTask(List<CompressedSizeInfoColGroup> groups, int start, int end, AComEst sEst,
 			ACostEstimate cEst, int minNumGroups) {
 			_groups = groups;
 			_start = start;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeStatic.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeStatic.java
index 37e23995e3..49c24d06bb 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeStatic.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeStatic.java
@@ -21,7 +21,7 @@ package org.apache.sysds.runtime.compress.cocode;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 
 /**
@@ -29,7 +29,7 @@ import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
  */
 public class CoCodeStatic extends AColumnCoCoder {
 
-	protected CoCodeStatic(CompressedSizeEstimator sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
+	protected CoCodeStatic(AComEst sizeEstimator, ACostEstimate costEstimator, CompressionSettings cs) {
 		super(sizeEstimator, costEstimator, cs);
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCoderFactory.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCoderFactory.java
index 216817f044..1515a77367 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCoderFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCoderFactory.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cost.ACostEstimate;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
@@ -51,7 +51,7 @@ public interface CoCoderFactory {
 	 * @param cs            The compression settings used in the compression.
 	 * @return The estimated (hopefully) best groups of ColGroups.
 	 */
-	public static CompressedSizeInfo findCoCodesByPartitioning(CompressedSizeEstimator est, CompressedSizeInfo colInfos,
+	public static CompressedSizeInfo findCoCodesByPartitioning(AComEst est, CompressedSizeInfo colInfos,
 		int k, ACostEstimate costEstimator, CompressionSettings cs) {
 
 		// Use column group partitioner to create partitions of columns
@@ -97,7 +97,7 @@ public interface CoCoderFactory {
 		return colInfos;
 	}
 
-	private static AColumnCoCoder createColumnGroupPartitioner(PartitionerType type, CompressedSizeEstimator est,
+	private static AColumnCoCoder createColumnGroupPartitioner(PartitionerType type, AComEst est,
 		ACostEstimate costEstimator, CompressionSettings cs) {
 		switch(type) {
 			case AUTO:
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 854de11716..db77a32bf6 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
@@ -24,15 +24,15 @@ 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.AComEst;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 
 public class Memorizer {
-	private final CompressedSizeEstimator _sEst;
+	private final AComEst _sEst;
 	private final Map<ColIndexes, CompressedSizeInfoColGroup> mem;
 	private int st1 = 0, st2 = 0, st3 = 0, st4 = 0;
 
-	public Memorizer(CompressedSizeEstimator sEst) {
+	public Memorizer(AComEst sEst) {
 		_sEst = sEst;
 		mem = new HashMap<>();
 	}
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 557c0269b3..b9ba6a5df4 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
@@ -24,7 +24,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Arrays;
-import java.util.List;
+import java.util.Collection;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -166,7 +166,7 @@ public abstract class AColGroup implements Serializable {
 	 * @param out data output
 	 * @throws IOException if IOException occurs
 	 */
-	public void write(DataOutput out) throws IOException {
+	protected void write(DataOutput out) throws IOException {
 		out.writeByte(getColGroupType().ordinal());
 		out.writeInt(_colIndexes.length);
 		// write col indices
@@ -180,7 +180,7 @@ public abstract class AColGroup implements Serializable {
 	 * @param in data input
 	 * @throws IOException if IOException occurs
 	 */
-	public void readFields(DataInput in) throws IOException {
+	protected void readFields(DataInput in) throws IOException {
 		// column group type is read in ColGroupIO
 		final int numCols = in.readInt();
 		_colIndexes = new int[numCols];
@@ -269,7 +269,7 @@ public abstract class AColGroup implements Serializable {
 	 * @param nRows  The number of rows in the groups
 	 * @return The given res list, where the sum of the column groups is added
 	 */
-	public static double[] colSum(List<AColGroup> groups, double[] res, int nRows) {
+	public static double[] colSum(Collection<AColGroup> groups, double[] res, int nRows) {
 		for(AColGroup g : groups)
 			g.computeColSums(res, nRows);
 		return res;
@@ -358,7 +358,20 @@ public abstract class AColGroup implements Serializable {
 	 * @param right The MatrixBlock on the right of this matrix multiplication
 	 * @return The new Column Group or null that is the result of the matrix multiplication.
 	 */
-	public abstract AColGroup rightMultByMatrix(MatrixBlock right);
+	public final AColGroup rightMultByMatrix(MatrixBlock right){
+		return rightMultByMatrix(right, null);
+	}
+
+	/**
+	 * Right matrix multiplication with this column group.
+	 * 
+	 * This method can return null, meaning that the output overlapping group would have been empty.
+	 * 
+	 * @param right The MatrixBlock on the right of this matrix multiplication
+	 * @param allCols A pre-materialized list of all col indexes, that can be shared across all column groups if use full, can be set to null.
+	 * @return The new Column Group or null that is the result of the matrix multiplication.
+	 */
+	public abstract AColGroup rightMultByMatrix(MatrixBlock right, int[] allCols);
 
 	/**
 	 * Do a transposed self matrix multiplication on the left side t(x) %*% x. but only with this column group.
@@ -391,10 +404,11 @@ public abstract class AColGroup implements Serializable {
 	 * Left side matrix multiplication with a column group that is transposed.
 	 * 
 	 * @param lhs    The left hand side Column group to multiply with, the left hand side should be considered
-	 *               transposed.
+	 *               transposed. Also it should be guaranteed that this column group is not empty.
 	 * @param result The result matrix to insert the result of the multiplication into
+	 * @param nRows   Number of rows in the lhs colGroup
 	 */
-	public abstract void leftMultByAColGroup(AColGroup lhs, MatrixBlock result);
+	public abstract void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows);
 
 	/**
 	 * Matrix multiply with this other column group, but:
@@ -446,11 +460,13 @@ public abstract class AColGroup implements Serializable {
 	 * Unary Aggregate operator, since aggregate operators require new object output, the output becomes an uncompressed
 	 * matrix.
 	 * 
+	 * The range of rl to ru only applies to row aggregates. (ReduceCol)
+	 * 
 	 * @param op    The operator used
 	 * @param c     The output matrix block
 	 * @param nRows The total number of rows in the Column Group
-	 * @param rl    The Starting Row to do aggregation from
-	 * @param ru    The last Row to do aggregation to (not included)
+	 * @param rl    The starting row to do aggregation from
+	 * @param ru    The last row to do aggregation to (not included)
 	 */
 	public abstract void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int nRows, int rl, int ru);
 
@@ -563,6 +579,13 @@ public abstract class AColGroup implements Serializable {
 
 	public abstract AColGroup unaryOperation(UnaryOperator op);
 
+	/**
+	 * Get if the group is only containing zero
+	 * 
+	 * @return true if empty
+	 */
+	public abstract boolean isEmpty();
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -570,7 +593,6 @@ public abstract class AColGroup implements Serializable {
 		sb.append(this.getClass().getSimpleName());
 		sb.append(String.format("\n%15s", "Columns: "));
 		sb.append(Arrays.toString(_colIndexes));
-
 		return sb.toString();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupCompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupCompressed.java
index 2464bbcfd6..24c95f2de5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupCompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupCompressed.java
@@ -19,12 +19,14 @@
 
 package org.apache.sysds.runtime.compress.colgroup;
 
+import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.DMLScriptException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Builtin.BuiltinCode;
+import org.apache.sysds.runtime.functionobjects.IndexFunction;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
 import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
 import org.apache.sysds.runtime.functionobjects.Multiply;
@@ -61,6 +63,14 @@ public abstract class AColGroupCompressed extends AColGroup {
 
 	protected abstract void computeColSumsSq(double[] c, int nRows);
 
+	/**
+	 * Compute row sums, note that this function works even for row SQ, since the preaggregate is correct.
+	 * 
+	 * @param c      target to aggregate into
+	 * @param rl     row to start from
+	 * @param ru     row to end at (not inclusive)
+	 * @param preAgg the pre-aggregated rows from this column group.
+	 */
 	protected abstract void computeRowSums(double[] c, int rl, int ru, double[] preAgg);
 
 	protected abstract void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg);
@@ -79,8 +89,8 @@ public abstract class AColGroupCompressed extends AColGroup {
 
 	protected abstract double[] preAggBuiltinRows(Builtin builtin);
 
-	public double[] preAggRows(AggregateUnaryOperator op) {
-		final ValueFunction fn = op.aggOp.increOp.fn;
+	public double[] preAggRows(ValueFunction fn) {
+		// final ValueFunction fn = op.aggOp.increOp.fn;
 		if(fn instanceof KahanPlusSq)
 			return preAggSumSqRows();
 		else if(fn instanceof Plus || fn instanceof KahanPlus)
@@ -96,7 +106,7 @@ public abstract class AColGroupCompressed extends AColGroup {
 				throw new DMLScriptException("unsupported builtin type: " + bop);
 		}
 		else
-			throw new DMLScriptException("Unknown UnaryAggregate operator on CompressedMatrixBlock " + op);
+			throw new DMLScriptException("Row Aggregate ValueFunction operator on CompressedMatrixBlock " + fn);
 	}
 
 	@Override
@@ -111,52 +121,73 @@ public abstract class AColGroupCompressed extends AColGroup {
 
 	@Override
 	public final void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int nRows, int rl, int ru) {
-		unaryAggregateOperations(op, c, nRows, rl, ru, null);
+		unaryAggregateOperations(op, c, nRows, rl, ru,
+			(op.indexFn instanceof ReduceCol) ? preAggRows(op.aggOp.increOp.fn) : null);
 	}
 
 	public final void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int nRows, int rl, int ru,
 		double[] preAgg) {
 		final ValueFunction fn = op.aggOp.increOp.fn;
-		if(fn instanceof KahanPlusSq) {
-			if(op.indexFn instanceof ReduceAll)
-				computeSumSq(c, nRows);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowSums(c, rl, ru, preAgg);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColSumsSq(c, nRows);
-		}
-		else if(fn instanceof Plus || fn instanceof KahanPlus) {
-			if(op.indexFn instanceof ReduceAll)
-				computeSum(c, nRows);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowSums(c, rl, ru, preAgg);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColSums(c, nRows);
-		}
-		else if(fn instanceof Multiply) {
+		if(fn instanceof KahanPlusSq)
+			sumSq(op.indexFn, c, nRows, rl, ru, preAgg);
+		else if(fn instanceof Plus || fn instanceof KahanPlus)
+			sum(op.indexFn, c, nRows, rl, ru, preAgg);
+		else if(fn instanceof Multiply)
+			prod(op.indexFn, c, nRows, rl, ru, preAgg);
+		else if(fn instanceof Builtin)
+			builtin(op, c, nRows, rl, ru, preAgg);
+		else
+			throw new DMLRuntimeException("Unknown UnaryAggregate operator on CompressedMatrixBlock");
+	}
+
+	private final void sumSq(IndexFunction idx, double[] c, int nRows, int rl, int ru, double[] preAgg) {
+		if(idx instanceof ReduceAll)
+			computeSumSq(c, nRows);
+		else if(idx instanceof ReduceCol)
+			computeRowSums(c, rl, ru, preAgg);
+		else if(idx instanceof ReduceRow)
+			computeColSumsSq(c, nRows);
+		else
+			throw new DMLRuntimeException("unsupported index type in colgroup: " + idx);
+	}
+
+	private final void sum(IndexFunction idx, double[] c, int nRows, int rl, int ru, double[] preAgg) {
+		if(idx instanceof ReduceAll)
+			computeSum(c, nRows);
+		else if(idx instanceof ReduceCol)
+			computeRowSums(c, rl, ru, preAgg);
+		else if(idx instanceof ReduceRow)
+			computeColSums(c, nRows);
+		else
+			throw new DMLRuntimeException("unsupported index type in colgroup: " + idx);
+	}
+
+	private final void prod(IndexFunction idx, double[] c, int nRows, int rl, int ru, double[] preAgg) {
+		if(idx instanceof ReduceAll)
+			computeProduct(c, nRows);
+		else if(idx instanceof ReduceCol)
+			computeRowProduct(c, rl, ru, preAgg);
+		else if(idx instanceof ReduceRow)
+			computeColProduct(c, nRows);
+		else
+			throw new DMLRuntimeException("unsupported index type in colgroup: " + idx);
+	}
+
+	private final void builtin(AggregateUnaryOperator op, double[] c, int nRows, int rl, int ru, double[] preAgg) {
+		Builtin bop = (Builtin) op.aggOp.increOp.fn;
+		BuiltinCode bopC = bop.getBuiltinCode();
+		if(bopC == BuiltinCode.MAX || bopC == BuiltinCode.MIN) {
 			if(op.indexFn instanceof ReduceAll)
-				computeProduct(c, nRows);
+				c[0] = computeMxx(c[0], bop);
 			else if(op.indexFn instanceof ReduceCol)
-				computeRowProduct(c, rl, ru, preAgg);
+				computeRowMxx(c, bop, rl, ru, preAgg);
 			else if(op.indexFn instanceof ReduceRow)
-				computeColProduct(c, nRows);
-		}
-		else if(fn instanceof Builtin) {
-			Builtin bop = (Builtin) fn;
-			BuiltinCode bopC = bop.getBuiltinCode();
-			if(bopC == BuiltinCode.MAX || bopC == BuiltinCode.MIN) {
-				if(op.indexFn instanceof ReduceAll)
-					c[0] = computeMxx(c[0], bop);
-				else if(op.indexFn instanceof ReduceCol)
-					computeRowMxx(c, bop, rl, ru, preAgg);
-				else if(op.indexFn instanceof ReduceRow)
-					computeColMxx(c, bop);
-			}
+				computeColMxx(c, bop);
 			else
-				throw new DMLScriptException("unsupported builtin type: " + bop);
+				throw new DMLRuntimeException("unsupported index type in colgroup: " + op.indexFn);
 		}
 		else
-			throw new DMLScriptException("Unknown UnaryAggregate operator on CompressedMatrixBlock");
+			throw new DMLRuntimeException("unsupported builtin type: " + bop);
 	}
 
 	@Override
@@ -170,24 +201,23 @@ public abstract class AColGroupCompressed extends AColGroup {
 
 	protected static void tsmm(double[] result, int numColumns, int[] counts, ADictionary dict, int[] colIndexes) {
 		dict = dict.getMBDict(colIndexes.length);
-		if(dict instanceof MatrixBlockDictionary) {
-			MatrixBlockDictionary mbd = (MatrixBlockDictionary) dict;
-			MatrixBlock mb = mbd.getMatrixBlock();
-			if(mb.isEmpty())
-				return;
-			else if(mb.isInSparseFormat())
-				tsmmSparse(result, numColumns, mb.getSparseBlock(), counts, colIndexes);
+		if(dict != null) {
+			if(dict instanceof MatrixBlockDictionary) {
+				MatrixBlockDictionary mbd = (MatrixBlockDictionary) dict;
+				MatrixBlock mb = mbd.getMatrixBlock();
+				// Guaranteed not to be empty
+				if(mb.isInSparseFormat())
+					tsmmSparse(result, numColumns, mb.getSparseBlock(), counts, colIndexes);
+				else
+					tsmmDense(result, numColumns, mb.getDenseBlockValues(), counts, colIndexes);
+			}
 			else
-				tsmmDense(result, numColumns, mb.getDenseBlockValues(), counts, colIndexes);
+				tsmmDense(result, numColumns, dict.getValues(), counts, colIndexes);
 		}
-		else
-			tsmmDense(result, numColumns, dict.getValues(), counts, colIndexes);
 
 	}
 
 	protected static void tsmmDense(double[] result, int numColumns, double[] values, int[] counts, int[] colIndexes) {
-		if(values == null)
-			return;
 		final int nCol = colIndexes.length;
 		final int nRow = counts.length;
 		for(int k = 0; k < nRow; k++) {
@@ -220,4 +250,9 @@ public abstract class AColGroupCompressed extends AColGroup {
 			}
 		}
 	}
+
+	@Override
+	public boolean isEmpty() {
+		return false;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupOffset.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupOffset.java
index 5beea9104a..2e263df164 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupOffset.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroupOffset.java
@@ -22,13 +22,8 @@ package org.apache.sysds.runtime.compress.colgroup;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.Arrays;
 
-import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
-import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
-import org.apache.sysds.runtime.functionobjects.Builtin;
-import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 import org.apache.sysds.utils.MemoryEstimates;
 
 /**
@@ -37,24 +32,32 @@ import org.apache.sysds.utils.MemoryEstimates;
  * NOTES: * OLE: separate storage segment length and bitmaps led to a 30% improvement but not applied because more
  * difficult to support both data layouts at the same time (distributed/local as well as w/ and w/o low-level opt)
  */
-public abstract class AColGroupOffset extends AColGroupValue {
+public abstract class AColGroupOffset extends APreAgg {
+
 	private static final long serialVersionUID = -4105103687174067602L;
-	/** Bitmaps, one per uncompressed value tuple in {@link #_dict}. */
+
+	/** Bitmaps, one per uncompressed value tuple in dict. */
 	protected int[] _ptr;
 	/** Linearized bitmaps (variable lengths) */
 	protected char[] _data;
 
+	final protected int _numRows;
+
+	protected boolean _zeros;
+
 	/**
 	 * Constructor for serialization
 	 * 
 	 * @param numRows Number of rows contained
 	 */
 	protected AColGroupOffset(int numRows) {
-		super(numRows);
+		super();
+		_numRows = numRows;
 	}
 
 	protected AColGroupOffset(int[] colIndices, int numRows, boolean zeros, ADictionary dict, int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
+		super(colIndices, dict, cachedCounts);
+		_numRows = numRows;
 		_zeros = zeros;
 	}
 
@@ -62,17 +65,15 @@ public abstract class AColGroupOffset extends AColGroupValue {
 		return _ptr[k + 1] - _ptr[k];
 	}
 
-	protected void createCompressedBitmaps(int numVals, int totalLen, char[][] lbitmaps) {
+	protected static void createCompressedBitmaps(int[] bitmap, char[] data, char[][] lbitmaps) {
 		// compact bitmaps to linearized representation
-		_ptr = new int[numVals + 1];
-		_data = new char[totalLen];
-		for(int i = 0, off = 0; i < numVals; i++) {
+		for(int i = 0, off = 0; i < bitmap.length - 1; i++) {
 			int len = lbitmaps[i].length;
-			_ptr[i] = off;
-			System.arraycopy(lbitmaps[i], 0, _data, off, len);
+			bitmap[i] = off;
+			System.arraycopy(lbitmaps[i], 0, data, off, len);
 			off += len;
 		}
-		_ptr[numVals] = totalLen;
+		bitmap[bitmap.length - 1] = data.length;
 	}
 
 	@Override
@@ -80,42 +81,10 @@ public abstract class AColGroupOffset extends AColGroupValue {
 		long size = super.estimateInMemorySize();
 		size += MemoryEstimates.intArrayCost(_ptr.length);
 		size += MemoryEstimates.charArrayCost(_data.length);
+		size += 4 + 1 + 3;
 		return size;
 	}
 
-	protected final void sumAllValues(double[] b, double[] c) {
-		final int numVals = getNumValues();
-		final int numCols = getNumCols();
-		final double[] values = _dict.getValues();
-
-		// vectMultiplyAdd over cols instead of dotProduct over vals because
-		// usually more values than columns
-		for(int i = 0, off = 0; i < numCols; i++, off += numVals)
-			LibMatrixMult.vectMultiplyAdd(b[i], values, c, off, 0, numVals);
-	}
-
-	protected final double mxxValues(int bitmapIx, Builtin builtin, double[] values) {
-		final int numCols = getNumCols();
-		final int valOff = bitmapIx * numCols;
-		double val = values[valOff];
-		for(int i = 1; i < numCols; i++)
-			val = builtin.execute(val, values[valOff + i]);
-
-		return val;
-	}
-
-	public char[] getBitmaps() {
-		return _data;
-	}
-
-	public int[] getBitmapOffsets() {
-		return _ptr;
-	}
-
-	public boolean hasZeros() {
-		return _zeros;
-	}
-
 	/**
 	 * Utility function of sparse-unsafe operations.
 	 * 
@@ -141,14 +110,14 @@ public abstract class AColGroupOffset extends AColGroupValue {
 
 		// read bitmaps
 		_ptr = new int[in.readInt()];
-		for(int i = 0; i < _ptr.length; i++) {
+		for(int i = 0; i < _ptr.length; i++)
 			_ptr[i] = in.readInt();
-		}
-		int totalLen = in.readInt();
-		_data = new char[totalLen];
-		for(int i = 0; i < totalLen; i++) {
+
+		_data = new char[in.readInt()];
+		for(int i = 0; i < _data.length; i++)
 			_data[i] = in.readChar();
-		}
+
+		_zeros = in.readBoolean();
 	}
 
 	@Override
@@ -156,56 +125,30 @@ public abstract class AColGroupOffset extends AColGroupValue {
 		super.write(out);
 		// write bitmaps (lens and data, offset later recreated)
 		out.writeInt(_ptr.length);
-		for(int i = 0; i < _ptr.length; i++) {
+		for(int i = 0; i < _ptr.length; i++)
 			out.writeInt(_ptr[i]);
-		}
+
 		out.writeInt(_data.length);
-		for(int i = 0; i < _data.length; i++) {
+		for(int i = 0; i < _data.length; i++)
 			out.writeChar(_data[i]);
-		}
 
+		out.writeBoolean(_zeros);
 	}
 
 	@Override
 	public long getExactSizeOnDisk() {
 		long ret = super.getExactSizeOnDisk();
 		// actual bitmaps
-		ret += 4; // total length // _ptr list
+		ret += 4; // _ptr list length
 		ret += 4 * _ptr.length;
-		ret += 4; // _data list
+		ret += 4; // _data list length
 		ret += 2 * _data.length;
+		ret += 1; // boolean
 
 		return ret;
 	}
 
-	protected abstract boolean[] computeZeroIndicatorVector();
-
-	public abstract void countNonZerosPerRow(int[] rnnz, int rl, int ru);
-
-	@Override
-	public double getCost(ComputationCostEstimator e, int nRows) {
-		throw new NotImplementedException();
-	}
-
-	@Override
-	public String toString() {
-		StringBuilder sb = new StringBuilder();
-		sb.append(super.toString());
-		sb.append(String.format("\n%15s%5d", "Pointers:", this._ptr.length));
-		sb.append(Arrays.toString(this._ptr));
-		return sb.toString();
-	}
-
-	protected static String charsToString(char[] data) {
-		StringBuilder sb = new StringBuilder();
-		sb.append("[");
-		for(int x = 0; x < data.length; x++) {
-			sb.append(((int) data[x]));
-			if(x != data.length - 1)
-				sb.append(", ");
-		}
-		sb.append("]");
-		return sb.toString();
+	public boolean containZerosTuples() {
+		return _zeros;
 	}
-
 }
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 0f6d433843..615f0cc5e6 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
@@ -19,168 +19,47 @@
 
 package org.apache.sysds.runtime.compress.colgroup;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.lang.ref.SoftReference;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
 
-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.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;
 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.CMOperator;
 
-/**
- * Base class for column groups encoded with value dictionary. This include column groups such as DDC OLE and RLE.
- * 
- */
-public abstract class AColGroupValue extends AColGroupCompressed implements Cloneable {
+public abstract class AColGroupValue extends ADictBasedColGroup implements Cloneable {
 	private static final long serialVersionUID = -6835757655517301955L;
 
-	/** The number of rows in the column group */
-	final protected int _numRows;
-
-	/**
-	 * ColGroup Implementation Contains zero tuple. Note this is not if it contains a zero value. If false then the
-	 * stored values are filling the ColGroup making it a dense representation, that can be leveraged in operations.
-	 * 
-	 * TODO remove
-	 */
-	protected boolean _zeros = false;
-
-	/** Distinct value tuples associated with individual bitmaps. */
-	protected ADictionary _dict;
+	private static final boolean soft = true;
 
 	/** The count of each distinct value contained in the dictionary */
 	private SoftReference<int[]> counts = null;
 
-	protected AColGroupValue(int numRows) {
+	protected AColGroupValue() {
 		super();
-		_numRows = numRows;
 	}
 
 	/**
-	 * A Abstract class for column groups that contain ADictionary for values.
+	 * A abstract class for column groups that contain ADictionary for values.
 	 * 
 	 * @param colIndices   The Column indexes
-	 * @param numRows      The number of rows contained in this group
 	 * @param dict         The dictionary to contain the distinct tuples
 	 * @param cachedCounts The cached counts of the distinct tuples (can be null since it should be possible to
 	 *                     reconstruct the counts on demand)
 	 */
-	protected AColGroupValue(int[] colIndices, int numRows, ADictionary dict, int[] cachedCounts) {
-		super(colIndices);
-		_numRows = numRows;
-		_dict = dict;
-		if(dict == null)
-			throw new NullPointerException("null dict is invalid");
+	protected AColGroupValue(int[] colIndices, ADictionary dict, int[] cachedCounts) {
+		super(colIndices, dict);
 		if(cachedCounts != null)
 			counts = new SoftReference<>(cachedCounts);
 	}
 
-	@Override
-	public final void decompressToDenseBlock(DenseBlock db, int rl, int ru, int offR, int offC) {
-		if(_dict instanceof MatrixBlockDictionary) {
-			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
-			final MatrixBlock mb = md.getMatrixBlock();
-			// The dictionary is never empty.
-			if(mb.isInSparseFormat())
-				decompressToDenseBlockSparseDictionary(db, rl, ru, offR, offC, mb.getSparseBlock());
-			else
-				decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, mb.getDenseBlockValues());
-		}
-		else
-			decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, _dict.getValues());
-	}
-
-	@Override
-	public final void decompressToSparseBlock(SparseBlock sb, int rl, int ru, int offR, int offC) {
-		if(_dict instanceof MatrixBlockDictionary) {
-			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
-			final MatrixBlock mb = md.getMatrixBlock();
-			if(mb.isEmpty()) // Early abort if the dictionary is empty.
-				return;
-			else if(mb.isInSparseFormat())
-				decompressToSparseBlockSparseDictionary(sb, rl, ru, offR, offC, mb.getSparseBlock());
-			else
-				decompressToSparseBlockDenseDictionary(sb, rl, ru, offR, offC, mb.getDenseBlockValues());
-		}
-		else
-			decompressToSparseBlockDenseDictionary(sb, rl, ru, offR, offC, _dict.getValues());
-	}
-
-	/**
-	 * Decompress to DenseBlock using a sparse dictionary to lookup into.
-	 * 
-	 * @param db   The dense db block to decompress into
-	 * @param rl   The row to start decompression from
-	 * @param ru   The row to end decompression at
-	 * @param offR The row offset to insert into
-	 * @param offC The column offset to insert into
-	 * @param sb   The sparse dictionary block to take value tuples from
-	 */
-	protected abstract void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
-		SparseBlock sb);
-
-	/**
-	 * Decompress to DenseBlock using a dense dictionary to lookup into.
-	 * 
-	 * @param db     The dense db block to decompress into
-	 * @param rl     The row to start decompression from
-	 * @param ru     The row to end decompression at
-	 * @param offR   The row offset to insert into
-	 * @param offC   The column offset to insert into
-	 * @param values The dense dictionary values, linearized row major.
-	 */
-	protected abstract void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
-		double[] values);
-
-	/**
-	 * Decompress to SparseBlock using a sparse dictionary to lookup into.
-	 * 
-	 * @param ret  The dense ret block to decompress into
-	 * @param rl   The row to start decompression from
-	 * @param ru   The row to end decompression at
-	 * @param offR The row offset to insert into
-	 * @param offC The column offset to insert into
-	 * @param sb   The sparse dictionary block to take value tuples from
-	 */
-	protected abstract void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
-		SparseBlock sb);
-
-	/**
-	 * Decompress to SparseBlock using a dense dictionary to lookup into.
-	 * 
-	 * @param ret    The dense ret block to decompress into
-	 * @param rl     The row to start decompression from
-	 * @param ru     The row to end decompression at
-	 * @param offR   The row offset to insert into
-	 * @param offC   The column offset to insert into
-	 * @param values The dense dictionary values, linearized row major.
-	 */
-	protected abstract void decompressToSparseBlockDenseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
-		double[] values);
-
 	@Override
 	public int getNumValues() {
 		return _dict.getNumberOfValues(_colIndexes.length);
 	}
 
-	public ADictionary getDictionary() {
-		return _dict;
-	}
-
 	/**
 	 * Returns the counts of values inside the dictionary. If already calculated it will return the previous counts. This
 	 * produce an overhead in cases where the count is calculated, but the overhead will be limited to number of distinct
@@ -192,14 +71,18 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 	 * @return The count of each value in the MatrixBlock.
 	 */
 	public final int[] getCounts() {
-		int[] ret = getCachedCounts();
+		if(soft){
 
-		if(ret == null) {
-			ret = getCounts(new int[getNumValues()]);
-			counts = new SoftReference<>(ret);
+			int[] ret = getCachedCounts();
+			if(ret == null) {
+				ret = getCounts(new int[getNumValues()]);
+				counts = new SoftReference<>(ret);
+			}
+			return ret;
+		}
+		else{
+			return getCounts(new int[getNumValues()]);
 		}
-
-		return ret;
 	}
 
 	/**
@@ -209,112 +92,12 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 	 * 
 	 * @return The counts or null.
 	 */
-	public final int[] getCachedCounts() {
-		return counts != null ? counts.get() : null;
-	}
-
-	private int[] rightMMGetColsDense(double[] b, int cl, int cu, int cut) {
-		Set<Integer> aggregateColumnsSet = new HashSet<>();
-		final int retCols = (cu - cl);
-		for(int k = 0; k < _colIndexes.length; k++) {
-			int rowIdxOffset = _colIndexes[k] * cut;
-			for(int h = cl; h < cu; h++) {
-				double v = b[rowIdxOffset + h];
-				if(v != 0.0) {
-					aggregateColumnsSet.add(h);
-				}
-			}
-			if(aggregateColumnsSet.size() == retCols)
-				break;
-		}
-
-		int[] aggregateColumns = aggregateColumnsSet.stream().mapToInt(x -> x).toArray();
-		Arrays.sort(aggregateColumns);
-		return aggregateColumns;
-	}
+	protected final int[] getCachedCounts() {
 
-	private int[] rightMMGetColsSparse(SparseBlock b, int retCols) {
-		Set<Integer> aggregateColumnsSet = new HashSet<>();
-
-		for(int h = 0; h < _colIndexes.length; h++) {
-			int colIdx = _colIndexes[h];
-			if(!b.isEmpty(colIdx)) {
-				int[] sIndexes = b.indexes(colIdx);
-				for(int i = b.pos(colIdx); i < b.size(colIdx) + b.pos(colIdx); i++) {
-					aggregateColumnsSet.add(sIndexes[i]);
-				}
-			}
-			if(aggregateColumnsSet.size() == retCols)
-				break;
-		}
-
-		int[] aggregateColumns = aggregateColumnsSet.stream().mapToInt(x -> x).toArray();
-		Arrays.sort(aggregateColumns);
-		return aggregateColumns;
-	}
-
-	private double[] rightMMPreAggSparse(int numVals, SparseBlock b, int[] aggregateColumns, int cl, int cu, int cut) {
-		final double[] ret = new double[numVals * aggregateColumns.length];
-		for(int h = 0; h < _colIndexes.length; h++) {
-			int colIdx = _colIndexes[h];
-			if(!b.isEmpty(colIdx)) {
-				double[] sValues = b.values(colIdx);
-				int[] sIndexes = b.indexes(colIdx);
-				int retIdx = 0;
-				for(int i = b.pos(colIdx); i < b.size(colIdx) + b.pos(colIdx); i++) {
-					while(aggregateColumns[retIdx] < sIndexes[i])
-						retIdx++;
-					if(sIndexes[i] == aggregateColumns[retIdx])
-						for(int j = 0, offOrg = h;
-							j < numVals * aggregateColumns.length;
-							j += aggregateColumns.length, offOrg += _colIndexes.length) {
-							ret[j + retIdx] += _dict.getValue(offOrg) * sValues[i];
-						}
-				}
-			}
-		}
-		return ret;
-	}
-
-	@Override
-	protected double computeMxx(double c, Builtin builtin) {
-		if(_zeros)
-			c = builtin.execute(c, 0);
-		return _dict.aggregate(c, builtin);
-	}
-
-	@Override
-	protected void computeColMxx(double[] c, Builtin builtin) {
-		if(_zeros)
-			for(int x = 0; x < _colIndexes.length; x++)
-				c[_colIndexes[x]] = builtin.execute(c[_colIndexes[x]], 0);
-
-		_dict.aggregateCols(c, builtin, _colIndexes);
-	}
-
-	@Override
-	public void readFields(DataInput in) throws IOException {
-		super.readFields(in);
-		_zeros = in.readBoolean();
-		_dict = DictionaryFactory.read(in);
-	}
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-		super.write(out);
-		out.writeBoolean(_zeros);
-		_dict.write(out);
-	}
-
-	@Override
-	public long getExactSizeOnDisk() {
-		long ret = super.getExactSizeOnDisk();
-		ret += 1; // zeros boolean
-		ret += _dict.getExactSizeOnDisk();
-		return ret;
+		return counts != null ? counts.get() : null;
 	}
 
-	public abstract int[] getCounts(int[] out);
+	protected abstract int[] getCounts(int[] out);
 
 	@Override
 	protected void computeSum(double[] c, int nRows) {
@@ -346,11 +129,6 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		_dict.colProduct(c, getCounts(), _colIndexes);
 	}
 
-	@Override
-	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
-	}
-
 	@Override
 	protected double[] preAggSumRows() {
 		return _dict.sumAllRowsToDouble(_colIndexes.length);
@@ -363,7 +141,7 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException();
+		return _dict.productAllRowsToDouble(_colIndexes.length);
 	}
 
 	@Override
@@ -387,11 +165,13 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		return clone;
 	}
 
-	private AColGroup copyAndSet(int[] colIndexes, double[] newDictionary) {
-		return copyAndSet(colIndexes, new Dictionary(newDictionary));
+	@Override
+	protected AColGroup copyAndSet(int[] colIndexes, double[] newDictionary) {
+		return copyAndSet(colIndexes, Dictionary.create(newDictionary));
 	}
 
-	private AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary) {
+	@Override
+	protected AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary) {
 		AColGroupValue clone = (AColGroupValue) this.clone();
 		clone._dict = newDictionary;
 		clone.setColIndices(colIndexes);
@@ -431,7 +211,7 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
 		ADictionary retDict = _dict.sliceOutColumnRange(idStart, idEnd, _colIndexes.length);
 		if(retDict == null)
-			return new ColGroupEmpty(_colIndexes);
+			return new ColGroupEmpty(outputCols);
 		final AColGroupValue ret = (AColGroupValue) this.clone();
 		ret._dict = retDict;
 		ret._colIndexes = outputCols;
@@ -445,61 +225,16 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		tsmm(result, numColumns, counts, _dict, _colIndexes);
 	}
 
-	@Override
-	public boolean containsValue(double pattern) {
-		if(pattern == 0 && _zeros)
-			return true;
-		return _dict.containsValue(pattern);
-	}
-
 	@Override
 	public long getNumberNonZeros(int nRows) {
 		int[] counts = getCounts();
 		return _dict.getNumberNonZeros(counts, _colIndexes.length);
 	}
 
-	public synchronized void forceMatrixBlockDictionary() {
-		if(!(_dict instanceof MatrixBlockDictionary))
-			_dict = _dict.getMBDict(_colIndexes.length);
-	}
-
-	@Override
-	public final AColGroup rightMultByMatrix(MatrixBlock right) {
-
-		if(right.isEmpty())
-			return null;
-		final int cl = 0;
-		final int cr = right.getNumColumns();
-		final int numVals = getNumValues();
-		if(right.isInSparseFormat()) {
-			final SparseBlock sb = right.getSparseBlock();
-			final int[] agCols = rightMMGetColsSparse(sb, cr);
-			if(agCols.length == 0)
-				return null;
-			return copyAndSet(agCols, rightMMPreAggSparse(numVals, sb, agCols, cl, cr, cr));
-		}
-		else {
-			final double[] rightV = right.getDenseBlockValues();
-			final int[] agCols = rightMMGetColsDense(rightV, cl, cr, cr);
-			if(agCols.length == 0)
-				return null;
-			ADictionary d = _dict.preaggValuesFromDense(numVals, _colIndexes, agCols, rightV, cr);
-			if(d == null)
-				return null;
-			return copyAndSet(agCols, d);
-		}
-	}
-
 	@Override
 	public long estimateInMemorySize() {
 		long size = super.estimateInMemorySize();
 		size += 8; // Counts reference
-		size += 4; // Int nRows
-		size += 1; // _zeros boolean reference
-		size += 1; // _lossy boolean reference
-		size += 2; // padding
-		size += _dict.getInMemorySize();
-		size += 8; // dict reference
 		return size;
 	}
 
@@ -523,6 +258,10 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 			return copyAndSet(Util.genColsIndices(max), d);
 	}
 
+	protected AColGroup rexpandCols(int max, ADictionary d) {
+		return (d == null) ? ColGroupEmpty.create(max) : copyAndSet(Util.genColsIndices(max), d);
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -531,4 +270,5 @@ public abstract class AColGroupValue extends AColGroupCompressed implements Clon
 		sb.append(_dict.getString(_colIndexes.length));
 		return sb.toString();
 	}
+
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ADictBasedColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ADictBasedColGroup.java
new file mode 100644
index 0000000000..0efdb4aa56
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ADictBasedColGroup.java
@@ -0,0 +1,287 @@
+/*
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+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.matrix.data.MatrixBlock;
+
+public abstract class ADictBasedColGroup extends AColGroupCompressed {
+	/** Distinct value tuples associated with individual bitmaps. */
+	protected ADictionary _dict;
+
+	protected ADictBasedColGroup() {
+		super();
+	}
+
+	/**
+	 * A Abstract class for column groups that contain ADictionary for values.
+	 * 
+	 * @param colIndices The Column indexes
+	 * @param dict       The dictionary to contain the distinct tuples
+	 */
+	protected ADictBasedColGroup(int[] colIndices, ADictionary dict) {
+		super(colIndices);
+		_dict = dict;
+		if(dict == null)
+			throw new NullPointerException("null dict is invalid");
+
+	}
+
+	public ADictionary getDictionary(){
+		return _dict;
+	}
+
+	@Override
+	public final void decompressToDenseBlock(DenseBlock db, int rl, int ru, int offR, int offC) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
+			final MatrixBlock mb = md.getMatrixBlock();
+			// The dictionary is never empty.
+			if(mb.isInSparseFormat())
+				decompressToDenseBlockSparseDictionary(db, rl, ru, offR, offC, mb.getSparseBlock());
+			else
+				decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, mb.getDenseBlockValues());
+		}
+		else
+			decompressToDenseBlockDenseDictionary(db, rl, ru, offR, offC, _dict.getValues());
+	}
+
+	@Override
+	public final void decompressToSparseBlock(SparseBlock sb, int rl, int ru, int offR, int offC) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
+			final MatrixBlock mb = md.getMatrixBlock();
+			// The dictionary is never empty.
+			if(mb.isInSparseFormat())
+				decompressToSparseBlockSparseDictionary(sb, rl, ru, offR, offC, mb.getSparseBlock());
+			else
+				decompressToSparseBlockDenseDictionary(sb, rl, ru, offR, offC, mb.getDenseBlockValues());
+		}
+		else
+			decompressToSparseBlockDenseDictionary(sb, rl, ru, offR, offC, _dict.getValues());
+	}
+
+	/**
+	 * Decompress to DenseBlock using a sparse dictionary to lookup into.
+	 * 
+	 * @param db   The dense db block to decompress into
+	 * @param rl   The row to start decompression from
+	 * @param ru   The row to end decompression at
+	 * @param offR The row offset to insert into
+	 * @param offC The column offset to insert into
+	 * @param sb   The sparse dictionary block to take value tuples from
+	 */
+	protected abstract void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb);
+
+	/**
+	 * Decompress to DenseBlock using a dense dictionary to lookup into.
+	 * 
+	 * @param db     The dense db block to decompress into
+	 * @param rl     The row to start decompression from
+	 * @param ru     The row to end decompression at
+	 * @param offR   The row offset to insert into
+	 * @param offC   The column offset to insert into
+	 * @param values The dense dictionary values, linearized row major.
+	 */
+	protected abstract void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+		double[] values);
+
+	/**
+	 * Decompress to SparseBlock using a sparse dictionary to lookup into.
+	 * 
+	 * @param ret  The dense ret block to decompress into
+	 * @param rl   The row to start decompression from
+	 * @param ru   The row to end decompression at
+	 * @param offR The row offset to insert into
+	 * @param offC The column offset to insert into
+	 * @param sb   The sparse dictionary block to take value tuples from
+	 */
+	protected abstract void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
+		SparseBlock sb);
+
+	/**
+	 * Decompress to SparseBlock using a dense dictionary to lookup into.
+	 * 
+	 * @param ret    The dense ret block to decompress into
+	 * @param rl     The row to start decompression from
+	 * @param ru     The row to end decompression at
+	 * @param offR   The row offset to insert into
+	 * @param offC   The column offset to insert into
+	 * @param values The dense dictionary values, linearized row major.
+	 */
+	protected abstract void decompressToSparseBlockDenseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
+		double[] values);
+
+	@Override
+	public void readFields(DataInput in) throws IOException {
+		super.readFields(in);
+		_dict = DictionaryFactory.read(in);
+	}
+
+	@Override
+	public void write(DataOutput out) throws IOException {
+		super.write(out);
+		_dict.write(out);
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		long ret = super.getExactSizeOnDisk();
+		ret += _dict.getExactSizeOnDisk();
+		return ret;
+	}
+
+	@Override
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		size += _dict.getInMemorySize();
+		size += 8; // dict reference
+		return size;
+	}
+
+	@Override
+	public final AColGroup rightMultByMatrix(MatrixBlock right, int[] allCols) {
+		if(right.isEmpty())
+			return null;
+		final int nCol = right.getNumColumns();
+		// make sure allCols is allocated
+		allCols = allCols == null ? Util.genColsIndices(nCol) : allCols;
+
+		final int[] agCols = (right.isInSparseFormat()) ? // find Cols
+			rightMMGetColsSparse(right.getSparseBlock(), nCol, allCols) : // sparse
+			rightMMGetColsDense(right.getDenseBlockValues(), nCol, allCols, right.getNonZeros()); // dense
+
+		final int nVals = getNumValues();
+		if(agCols == null || nVals == 0)
+			return null;
+		final ADictionary preAgg = (right.isInSparseFormat()) ? // Chose Sparse or Dense
+			rightMMPreAggSparse(nVals, right.getSparseBlock(), agCols, 0, nCol) : // sparse
+			_dict.preaggValuesFromDense(nVals, _colIndexes, agCols, right.getDenseBlockValues(), nCol); // dense
+		return allocateRightMultiplication(right, agCols, preAgg);
+	}
+
+	protected abstract AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg);
+
+	/**
+	 * Find the minimum number of columns that are effected by the right multiplication
+	 * 
+	 * @param b       The dense values in the right matrix
+	 * @param nCols   The max number of columns in the right matrix
+	 * @param allCols The all columns int list
+	 * @param nnz The number of non zero values in b
+	 * @return a list of the column indexes effected in the output column group
+	 */
+	protected int[] rightMMGetColsDense(double[] b, final int nCols, int[] allCols, long nnz) {
+		if(nCols > 200 || nnz > (b.length * 0.7)) // just return the int array
+			return allCols;
+		else { // try to do the best we can
+			Set<Integer> aggregateColumnsSet = new HashSet<>();
+
+			for(int k = 0; k < _colIndexes.length; k++) {
+				int rowIdxOffset = _colIndexes[k] * nCols;
+				for(int h = 0; h < nCols; h++)
+					if(b[rowIdxOffset + h] != 0.0) {
+						aggregateColumnsSet.add(h);
+						continue;
+					}
+
+				if(aggregateColumnsSet.size() == nCols)
+					return allCols;
+			}
+			if(aggregateColumnsSet.size() == 0)
+				return null;
+
+			int[] aggregateColumns = aggregateColumnsSet.stream().mapToInt(x -> x).toArray();
+			Arrays.sort(aggregateColumns);
+			return aggregateColumns;
+		}
+	}
+
+	/**
+	 * Find the minimum number of columns that are effected by the right multiplication
+	 * 
+	 * @param b       The sparse matrix on the right
+	 * @param retCols The number of columns contained in the sparse matrix.
+	 * @return a list of the column indexes effected in the output column group
+	 */
+	protected int[] rightMMGetColsSparse(SparseBlock b, int retCols, int[] allCols) {
+		Set<Integer> aggregateColumnsSet = new HashSet<>();
+
+		for(int h = 0; h < _colIndexes.length; h++) {
+			int colIdx = _colIndexes[h];
+			if(!b.isEmpty(colIdx)) {
+				int[] sIndexes = b.indexes(colIdx);
+				for(int i = b.pos(colIdx); i < b.size(colIdx) + b.pos(colIdx); i++)
+					aggregateColumnsSet.add(sIndexes[i]);
+			}
+			if(aggregateColumnsSet.size() == retCols)
+				return allCols;
+		}
+		if(aggregateColumnsSet.size() == 0)
+			return null;
+
+		int[] aggregateColumns = aggregateColumnsSet.stream().mapToInt(x -> x).toArray();
+		Arrays.sort(aggregateColumns);
+		return aggregateColumns;
+	}
+
+	private ADictionary rightMMPreAggSparse(int numVals, SparseBlock b, int[] aggregateColumns, int cl, int cu) {
+		final double[] ret = new double[numVals * aggregateColumns.length];
+		for(int h = 0; h < _colIndexes.length; h++) {
+			final int colIdx = _colIndexes[h];
+			if(b.isEmpty(colIdx))
+				continue;
+
+			final double[] sValues = b.values(colIdx);
+			final int[] sIndexes = b.indexes(colIdx);
+			int retIdx = 0;
+			for(int i = b.pos(colIdx); i < b.size(colIdx) + b.pos(colIdx); i++) {
+				while(aggregateColumns[retIdx] < sIndexes[i])
+					retIdx++;
+				if(sIndexes[i] == aggregateColumns[retIdx])
+					for(int j = 0, offOrg = h;
+						j < numVals * aggregateColumns.length;
+						j += aggregateColumns.length, offOrg += _colIndexes.length) {
+						ret[j + retIdx] += _dict.getValue(offOrg) * sValues[i];
+					}
+			}
+
+		}
+		return Dictionary.create(ret);
+	}
+
+	protected abstract AColGroup copyAndSet(int[] colIndexes, double[] newDictionary);
+
+	protected abstract AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary);
+
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AMorphingMMColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AMorphingMMColGroup.java
index 0c7a1eb42a..ff609fefce 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AMorphingMMColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AMorphingMMColGroup.java
@@ -21,8 +21,11 @@ package org.apache.sysds.runtime.compress.colgroup;
 
 import org.apache.sysds.runtime.compress.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
+import org.apache.sysds.runtime.compress.lib.CLALibLeftMultBy;
+import org.apache.sysds.runtime.compress.lib.CLALibTSMM;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
+import org.apache.sysds.runtime.data.SparseRow;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 /**
@@ -34,69 +37,186 @@ public abstract class AMorphingMMColGroup extends AColGroupValue {
 
 	/**
 	 * Constructor for serialization
-	 * 
-	 * @param numRows Number of rows contained
 	 */
-	protected AMorphingMMColGroup(int numRows) {
-		super(numRows);
+	protected AMorphingMMColGroup() {
+		super();
 	}
 
 	/**
 	 * A Abstract class for column groups that contain ADictionary for values.
 	 * 
 	 * @param colIndices   The Column indexes
-	 * @param numRows      The number of rows contained in this group
 	 * @param dict         The dictionary to contain the distinct tuples
 	 * @param cachedCounts The cached counts of the distinct tuples (can be null since it should be possible to
 	 *                     reconstruct the counts on demand)
 	 */
-	protected AMorphingMMColGroup(int[] colIndices, int numRows, ADictionary dict, int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
+	protected AMorphingMMColGroup(int[] colIndices, ADictionary dict, int[] cachedCounts) {
+		super(colIndices, dict, cachedCounts);
 	}
 
 	@Override
 	protected final void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
 		SparseBlock sb) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("Should never call decompress on morphing group instead extract common values and combine all commons");
+		double[] cv = new double[db.getDim(1)];
+		AColGroup b = extractCommon(cv);
+		b.decompressToDenseBlock(db, rl, ru, offR, offC);
+		decompressToDenseBlockCommonVector(db, rl, ru, offR, offC, cv);
 	}
 
 	@Override
 	protected final void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("Should never call decompress on morphing group instead extract common values and combine all commons");
+		double[] cv = new double[db.getDim(1)];
+		AColGroup b = extractCommon(cv);
+		b.decompressToDenseBlock(db, rl, ru, offR, offC);
+		decompressToDenseBlockCommonVector(db, rl, ru, offR, offC, cv);
+	}
+
+	private final void decompressToDenseBlockCommonVector(DenseBlock db, int rl, int ru, int offR, int offC,
+		double[] common) {
+		for(int i = rl, offT = rl + offR; i < ru; i++, offT++) {
+			final double[] c = db.values(offT);
+			final int off = db.pos(offT) + offC;
+			for(int j = 0; j < _colIndexes.length; j++)
+				c[off + _colIndexes[j]] += common[j];
+		}
 	}
 
 	@Override
 	protected final void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
 		SparseBlock sb) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("Should never call decompress on morphing group instead extract common values and combine all commons");
+		double[] cv = new double[_colIndexes[_colIndexes.length - 1] + 1];
+		AColGroup b = extractCommon(cv);
+		b.decompressToSparseBlock(ret, rl, ru, offR, offC);
+		decompressToSparseBlockCommonVector(ret, rl, ru, offR, offC, cv);
 	}
 
 	@Override
 	protected final void decompressToSparseBlockDenseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
 		double[] values) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("Should never call decompress on morphing group instead extract common values and combine all commons");
+		double[] cv = new double[_colIndexes[_colIndexes.length - 1] + 1];
+		AColGroup b = extractCommon(cv);
+		b.decompressToSparseBlock(ret, rl, ru, offR, offC);
+		decompressToSparseBlockCommonVector(ret, rl, ru, offR, offC, cv);
+	}
+
+	private final void decompressToSparseBlockCommonVector(SparseBlock sb, int rl, int ru, int offR, int offC,
+		double[] common) {
+		final int nCol = _colIndexes.length;
+		for(int i = rl, offT = rl + offR; i < ru; i++, offT++) {
+			for(int j = 0; j < nCol; j++)
+				if(common[j] != 0)
+					sb.add(offT, _colIndexes[j] + offC, common[j]);
+			final SparseRow sr = sb.get(offT);
+			if(sr != null)
+				sr.compact(1.0E-20);
+		}
 	}
 
 	@Override
-	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
-		throw new DMLCompressionException("This method should never be called");
+	public final void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		LOG.warn("Should never call leftMultByMatrixNoPreAgg on morphing group");
+		double[] cv = new double[result.getNumColumns()];
+		AColGroup b = extractCommon(cv);
+		b.leftMultByMatrixNoPreAgg(matrix, result, rl, ru, cl, cu);
+		final double[] rowSum = (cl != 0 || cu != matrix.getNumColumns()) ? // do partial row sum if range is requested
+			CLALibLeftMultBy.rowSum(matrix, rl, ru, cl, cu) : // partial row sum
+			matrix.rowSum().getDenseBlockValues(); // full row sum
+		ColGroupUtils.outerProduct(rowSum, cv, result.getDenseBlockValues(), rl, ru);
 	}
 
 	@Override
-	public final void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		throw new DMLCompressionException("This method should never be called");
+	public final void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+		LOG.warn("Should never call leftMultByMatrixNoPreAgg on morphing group");
+		double[] cv = new double[result.getNumColumns()];
+		AColGroup b = extractCommon(cv);
+		b.leftMultByAColGroup(lhs, result, nRows);
+		double[] rowSum = new double[result.getNumRows()];
+		lhs.computeColSums(rowSum, nRows);
+		ColGroupUtils.outerProduct(rowSum, cv, result.getDenseBlockValues(), 0, result.getNumRows());
 	}
 
 	@Override
 	public final void tsmmAColGroup(AColGroup other, MatrixBlock result) {
-		throw new DMLCompressionException("This method should never be called");
+		throw new DMLCompressionException("Should not be called tsmm on morphing");
 	}
 
 	@Override
 	protected final void tsmm(double[] result, int numColumns, int nRows) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("tsmm should not be called directly on a morphing column group");
+		final double[] cv = new double[numColumns];
+		AColGroupCompressed b = (AColGroupCompressed) extractCommon(cv);
+		b.tsmm(result, numColumns, nRows);
+		final double[] colSum = new double[numColumns];
+		b.computeColSums(colSum, nRows);
+		CLALibTSMM.addCorrectionLayer(cv, colSum, nRows, result);
+	}
+
+	@Override
+	protected int[] rightMMGetColsDense(double[] b, int nCols, int[] allCols, long nnz) {
+		return allCols;
+	}
+
+	@Override
+	protected int[] rightMMGetColsSparse(SparseBlock b, int nCols, int[] allCols) {
+		return allCols;
 	}
 
+	@Override
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		LOG.warn("right mm should not be called directly on a morphing column group");
+		final double[] common = getCommon();
+		final int rc = right.getNumColumns();
+		final double[] commonMultiplied = new double[rc];
+		final int lc = _colIndexes.length;
+		if(right.isInSparseFormat()) {
+			SparseBlock sb = right.getSparseBlock();
+			for(int r = 0; r < lc; r++) {
+				final int of = _colIndexes[r];
+				if(sb.isEmpty(of))
+					continue;
+				final int apos = sb.pos(of);
+				final int alen = sb.size(of) + apos;
+				final int[] aix = sb.indexes(of);
+				final double[] avals = sb.values(of);
+				final double v = common[r];
+				for(int j = apos; j < alen; j++)
+					commonMultiplied[aix[apos]] += v * avals[j];
+			}
+		}
+		else {
+			final double[] rV = right.getDenseBlockValues();
+			for(int r = 0; r < lc; r++) {
+				final int rOff = rc * _colIndexes[r];
+				final double v = common[r];
+				for(int c = 0; c < rc; c++)
+					commonMultiplied[c] += v * rV[rOff + c];
+			}
+		}
+		return allocateRightMultiplicationCommon(commonMultiplied, colIndexes, preAgg);
+	}
+
+	protected abstract AColGroup allocateRightMultiplicationCommon(double[] common, int[] colIndexes,
+		ADictionary preAgg);
+
+	/** extract common value from group and return non morphing group */
+	/**
+	 * extract common value from group and return non morphing group
+	 * 
+	 * @param constV a vector to contain all values, note length = nCols in total matrix.
+	 * @return A non morphing column group with decompression instructions.
+	 */
 	public abstract AColGroup extractCommon(double[] constV);
+
+	/**
+	 * Get common vector, note this should not materialize anything but simply point to things that are already
+	 * allocated.
+	 * 
+	 * @return the common double vector
+	 */
+	public abstract double[] getCommon();
 }
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 971f6470a1..41f0963e1e 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
@@ -19,7 +19,7 @@
 
 package org.apache.sysds.runtime.compress.colgroup;
 
-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.dictionary.DictLibMatrixMult;
@@ -42,27 +42,25 @@ public abstract class APreAgg extends AColGroupValue {
 	/**
 	 * Constructor for serialization
 	 * 
-	 * @param numRows number of rows
 	 */
-	protected APreAgg(int numRows) {
-		super(numRows);
+	protected APreAgg() {
+		super();
 	}
 
 	/**
 	 * A Abstract class for column groups that contain ADictionary for values.
 	 * 
 	 * @param colIndices   The Column indexes
-	 * @param numRows      The number of rows contained in this group
 	 * @param dict         The dictionary to contain the distinct tuples
 	 * @param cachedCounts The cached counts of the distinct tuples (can be null since it should be possible to
 	 *                     reconstruct the counts on demand)
 	 */
-	protected APreAgg(int[] colIndices, int numRows, ADictionary dict, int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
+	protected APreAgg(int[] colIndices, ADictionary dict, int[] cachedCounts) {
+		super(colIndices, dict, cachedCounts);
 	}
 
 	@Override
-	public void tsmmAColGroup(AColGroup other, MatrixBlock result) {
+	public final void tsmmAColGroup(AColGroup other, MatrixBlock result) {
 		if(other instanceof ColGroupEmpty)
 			return;
 		else if(other instanceof APreAgg)
@@ -75,10 +73,9 @@ public abstract class APreAgg extends AColGroupValue {
 	}
 
 	@Override
-	public final void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		if(lhs instanceof ColGroupEmpty)
-			return;
-		else if(lhs instanceof APreAgg)
+	public final void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+		// Not checking if empty since it should be guaranteed on call.
+		if(lhs instanceof APreAgg)
 			leftMultByColGroupValue((APreAgg) lhs, result);
 		else if(lhs instanceof ColGroupUncompressed)
 			leftMultByUncompressedColGroup((ColGroupUncompressed) lhs, result);
@@ -87,24 +84,6 @@ public abstract class APreAgg extends AColGroupValue {
 				"Not supported left multiplication with A ColGroup of type: " + lhs.getClass().getSimpleName());
 	}
 
-	@Deprecated
-	private final void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
-		if(matrix.isEmpty())
-			return;
-		final int nCol = _colIndexes.length;
-		final int numVals = getNumValues();
-		// Pre aggregate the matrix into same size as dictionary
-		final MatrixBlock preAgg = new MatrixBlock(ru - rl, numVals, false);
-		preAgg.allocateDenseBlock();
-		preAggregate(matrix, preAgg.getDenseBlockValues(), rl, ru);
-		preAgg.recomputeNonZeros();
-		final MatrixBlock tmpRes = new MatrixBlock(preAgg.getNumRows(), nCol, false);
-		forceMatrixBlockDictionary();
-		final MatrixBlock dictM = _dict.getMBDict(nCol).getMatrixBlock();
-		LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
-		addMatrixToResult(tmpRes, result, _colIndexes, rl, ru);
-	}
-
 	/**
 	 * Pre aggregate into a dictionary. It is assumed that "that" have more distinct values than, "this".
 	 * 
@@ -113,7 +92,8 @@ public abstract class APreAgg extends AColGroupValue {
 	 */
 	public final ADictionary preAggregateThatIndexStructure(APreAgg that) {
 		int outputLength = that._colIndexes.length * this.getNumValues();
-		Dictionary ret = new Dictionary(new double[outputLength]);
+		// create empty Dictionary that we slowly fill, hence the dictionary is empty and no check
+		final Dictionary ret = Dictionary.createNoCheck(new double[outputLength]);
 
 		if(that instanceof ColGroupDDC)
 			preAggregateThatDDCStructure((ColGroupDDC) that, ret);
@@ -121,12 +101,12 @@ public abstract class APreAgg extends AColGroupValue {
 			preAggregateThatSDCSingleZerosStructure((ColGroupSDCSingleZeros) that, ret);
 		else if(that instanceof ColGroupSDCZeros)
 			preAggregateThatSDCZerosStructure((ColGroupSDCZeros) that, ret);
-		else {
-			final String cThis = this.getClass().getSimpleName();
-			final String cThat = that.getClass().getSimpleName();
-			throw new NotImplementedException(
-				"Not supported pre aggregate using index structure of :" + cThat + " in " + cThis);
-		}
+		else if(that instanceof ColGroupRLE)
+			preAggregateThatRLEStructure((ColGroupRLE) that, ret);
+		else
+			throw new DMLRuntimeException("Not supported pre aggregate using index structure of :"
+				+ that.getClass().getSimpleName() + " in " + this.getClass().getSimpleName());
+
 		return ret.getMBDict(that._colIndexes.length);
 	}
 
@@ -138,7 +118,7 @@ public abstract class APreAgg extends AColGroupValue {
 	 * @param rl     Row lower on the left side matrix
 	 * @param ru     Row upper on the left side matrix
 	 */
-	public void preAggregate(MatrixBlock m, double[] preAgg, int rl, int ru) {
+	public final void preAggregate(MatrixBlock m, double[] preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
 			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
 		else
@@ -165,6 +145,8 @@ public abstract class APreAgg extends AColGroupValue {
 
 	protected abstract void preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret);
 
+	protected abstract void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret);
+
 	protected abstract boolean sameIndexStructure(AColGroupCompressed that);
 
 	public int getPreAggregateSize() {
@@ -251,8 +233,6 @@ public abstract class APreAgg extends AColGroupValue {
 	}
 
 	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");
 		final MatrixBlock tmp = LibMatrixReorg.transpose(lhs.getData(), InfrastructureAnalyzer.getLocalParallelism());
 		final int numVals = getNumValues();
@@ -262,8 +242,10 @@ public abstract class APreAgg extends AColGroupValue {
 		preAgg.recomputeNonZeros();
 		final MatrixBlock tmpRes = new MatrixBlock(preAgg.getNumRows(), _colIndexes.length, false);
 		final MatrixBlock dictM = _dict.getMBDict(getNumCols()).getMatrixBlock();
-		LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
-		addMatrixToResult(tmpRes, result, lhs._colIndexes);
+		if(dictM != null) {
+			LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
+			addMatrixToResult(tmpRes, result, lhs._colIndexes);
+		}
 	}
 
 	private void addMatrixToResult(MatrixBlock tmp, MatrixBlock result, int[] rowIndexes) {
@@ -299,9 +281,9 @@ public abstract class APreAgg extends AColGroupValue {
 		final int nCols = result.getNumColumns();
 		final MatrixBlock otherMBT = LibMatrixReorg.transpose(((ColGroupUncompressed) other).getData());
 		final int nRows = otherMBT.getNumRows();
-		final MatrixBlock tmp = new MatrixBlock(otherMBT.getNumRows(), nCols, false);
+		final MatrixBlock tmp = new MatrixBlock(nRows, nCols, false);
 		tmp.allocateDenseBlock();
-		leftMultByMatrix(otherMBT, tmp, 0, nRows);
+		leftMultByMatrixNoPreAgg(otherMBT, tmp, 0, nRows, 0, otherMBT.getNumColumns());
 
 		final double[] r = tmp.getDenseBlockValues();
 		final double[] resV = result.getDenseBlockValues();
@@ -311,7 +293,7 @@ public abstract class APreAgg extends AColGroupValue {
 			final int oid = other._colIndexes[i];
 			final int offR = i * nCols;
 			for(int j = 0; j < thisLen; j++)
-				DictLibMatrixMult.addToUpperTriangle(nCols, _colIndexes[j], oid, resV, r[offR + _colIndexes[j]]);
+				DictLibMatrixMult.addToUpperTriangle(nCols, oid, _colIndexes[j], resV, r[offR + _colIndexes[j]]);
 		}
 	}
 
@@ -326,7 +308,6 @@ public abstract class APreAgg extends AColGroupValue {
 	}
 
 	public void mmWithDictionary(MatrixBlock preAgg, MatrixBlock tmpRes, MatrixBlock ret, int k, int rl, int ru) {
-
 		// Shallow copy the preAgg to allow sparse PreAgg multiplication but do not remove the original dense allocation
 		// since the dense allocation is reused.
 		final MatrixBlock preAggCopy = new MatrixBlock();
@@ -334,45 +315,11 @@ public abstract class APreAgg extends AColGroupValue {
 		final MatrixBlock tmpResCopy = new MatrixBlock();
 		tmpResCopy.copy(tmpRes);
 		// Get dictionary matrixBlock
-		final ADictionary d = getDictionary();
-		final MatrixBlock dict = d.getMBDict(_colIndexes.length).getMatrixBlock();
-		try {
+		final MatrixBlock dict = getDictionary().getMBDict(_colIndexes.length).getMatrixBlock();
+		if(dict != null) {
 			// Multiply
 			LibMatrixMult.matrixMult(preAggCopy, dict, tmpResCopy, k);
-			addMatrixToResult(tmpResCopy, ret, _colIndexes, rl, ru);
-		}
-		catch(Exception e) {
-			throw new DMLCompressionException(
-				"Failed matrix multiply with preAggregate: \n" + preAggCopy + "\n" + dict + "\n" + tmpRes, e);
-		}
-
-	}
-
-	private static void addMatrixToResult(MatrixBlock tmp, MatrixBlock result, int[] colIndexes, int rl, int ru) {
-		if(tmp.isEmpty())
-			return;
-		final double[] retV = result.getDenseBlockValues();
-		final int nColRet = result.getNumColumns();
-		if(tmp.isInSparseFormat()) {
-			final SparseBlock sb = tmp.getSparseBlock();
-			for(int row = rl, offT = 0; row < ru; row++, offT++) {
-				final int apos = sb.pos(offT);
-				final int alen = sb.size(offT);
-				final int[] aix = sb.indexes(offT);
-				final double[] avals = sb.values(offT);
-				final int offR = row * nColRet;
-				for(int i = apos; i < apos + alen; i++)
-					retV[offR + colIndexes[aix[i]]] += avals[i];
-			}
-		}
-		else {
-			final double[] tmpV = tmp.getDenseBlockValues();
-			final int nCol = colIndexes.length;
-			for(int row = rl, offT = 0; row < ru; row++, offT += nCol) {
-				final int offR = row * nColRet;
-				for(int col = 0; col < nCol; col++)
-					retV[offR + colIndexes[col]] += tmpV[offT + col];
-			}
+			ColGroupUtils.addMatrixToResult(tmpResCopy, ret, _colIndexes, rl, ru);
 		}
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDC.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDC.java
new file mode 100644
index 0000000000..e598553b3e
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ASDC.java
@@ -0,0 +1,67 @@
+/*
+ * 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.compress.colgroup.dictionary.ADictionary;
+import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
+
+/**
+ * Column group that sparsely encodes the dictionary values. The idea is that all values is encoded with indexes except
+ * the most common one. the most common one can be inferred by not being included in the indexes.
+ * 
+ * This column group is handy in cases where sparse unsafe operations is executed on very sparse columns. Then the zeros
+ * would be materialized in the group without any overhead.
+ */
+public abstract class ASDC extends AMorphingMMColGroup {
+	private static final long serialVersionUID = 769993538831949086L;
+
+	/** Sparse row indexes for the data */
+	protected AOffset _indexes;
+	
+	final protected int _numRows;
+
+	/**
+	 * Constructor for serialization
+	 * 
+	 * @param numRows Number of rows contained
+	 */
+	protected ASDC(int numRows) {
+		super();
+		_numRows = numRows;
+	}
+
+	protected ASDC(int[] colIndices, int numRows, ADictionary dict,  AOffset offsets,
+		int[] cachedCounts) {
+		super(colIndices, dict, cachedCounts);
+
+		_indexes = offsets;
+		_numRows = numRows;
+	}
+
+	public int getNumRows(){
+		return _numRows;
+	}
+
+	public abstract double[] getDefaultTuple();
+
+	public AOffset getOffsets(){
+		return _indexes;
+	}
+}
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 3cdc0904f2..a0a1fd58e1 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
@@ -19,7 +19,6 @@
 
 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;
@@ -33,15 +32,17 @@ public abstract class ASDCZero extends APreAgg {
 
 	/** Sparse row indexes for the data */
 	protected AOffset _indexes;
+	final protected int _numRows;
 
 	protected ASDCZero(int numRows) {
-		super(numRows);
+		super();
+		_numRows = numRows;
 	}
 
 	protected ASDCZero(int[] colIndices, int numRows, ADictionary dict, AOffset offsets, int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
+		super(colIndices, dict, cachedCounts);
 		_indexes = offsets;
-		_zeros = true;
+		_numRows = numRows;
 	}
 
 	@Override
@@ -57,16 +58,14 @@ public abstract class ASDCZero extends APreAgg {
 			leftMultByMatrixNoPreAggRows(matrix, result, rl, ru, cl, cu, it);
 	}
 
-	protected final void leftMultByMatrixNoPreAggSingleRow(MatrixBlock mb, MatrixBlock result, int r, int cl, int cu,
+	private final void leftMultByMatrixNoPreAggSingleRow(MatrixBlock mb, MatrixBlock result, int r, int cl, int cu,
 		AIterator it) {
 		final double[] resV = result.getDenseBlockValues();
 		final int nCols = result.getNumColumns();
 		final int offRet = nCols * r;
 		if(mb.isInSparseFormat()) {
 			final SparseBlock sb = mb.getSparseBlock();
-			if(cl != 0 && cu != _numRows)
-				throw new NotImplementedException();
-			leftMultByMatrixNoPreAggSingleRowSparse(sb, resV, offRet, r, it);
+			leftMultByMatrixNoPreAggSingleRowSparse(sb, resV, offRet, r, cu, it);
 		}
 		else {
 			final DenseBlock db = mb.getDenseBlock();
@@ -76,7 +75,7 @@ public abstract class ASDCZero extends APreAgg {
 		}
 	}
 
-	protected final void leftMultByMatrixNoPreAggSingleRowDense(double[] mV, int off, double[] resV, int offRet, int r,
+	private final void leftMultByMatrixNoPreAggSingleRowDense(double[] mV, int off, double[] resV, int offRet, int r,
 		int cl, int cu, AIterator it) {
 		final int last = _indexes.getOffsetToLast();
 		while(it.isNotOver(cu)) {
@@ -89,8 +88,8 @@ public abstract class ASDCZero extends APreAgg {
 		_indexes.cacheIterator(it, cu);
 	}
 
-	protected synchronized final void leftMultByMatrixNoPreAggSingleRowSparse(SparseBlock sb, double[] resV, int offRet,
-		int r, AIterator it) {
+	private final void leftMultByMatrixNoPreAggSingleRowSparse(final SparseBlock sb, final double[] resV,
+		final int offRet, final int r, final int cu, final AIterator it) {
 		if(sb.isEmpty(r))
 			return;
 		final int last = _indexes.getOffsetToLast();
@@ -98,9 +97,22 @@ public abstract class ASDCZero extends APreAgg {
 		final int alen = sb.size(r) + apos;
 		final int[] aix = sb.indexes(r);
 		final double[] aval = sb.values(r);
-
 		int v = it.value();
-		if(aix[alen - 1] < last) {
+		while(apos < alen && aix[apos] < v)
+			apos++; // go though sparse block until offset start.
+		if(cu < last) {
+			while(v < cu && apos < alen) {
+				if(aix[apos] == v) {
+					multiplyScalar(aval[apos++], resV, offRet, it);
+					v = it.next();
+				}
+				else if(aix[apos] < v)
+					apos++;
+				else
+					v = it.next();
+			}
+		}
+		else if(aix[alen - 1] < last) {
 			while(apos < alen) {
 				if(aix[apos] == v) {
 					multiplyScalar(aval[apos++], resV, offRet, it);
@@ -131,30 +143,26 @@ public abstract class ASDCZero extends APreAgg {
 		}
 	}
 
-	protected final void leftMultByMatrixNoPreAggRows(MatrixBlock mb, MatrixBlock result, int rl, int ru, int cl, int cu,
+	private final void leftMultByMatrixNoPreAggRows(MatrixBlock mb, MatrixBlock result, int rl, int ru, int cl, int cu,
 		AIterator it) {
 		final double[] resV = result.getDenseBlockValues();
 		final int nCols = result.getNumColumns();
-		if(mb.isInSparseFormat()) {
-			final SparseBlock sb = mb.getSparseBlock();
-			leftMultByMatrixNoPreAggRowsSparse(sb, resV, nCols, rl, ru, cl, cu, it);
-		}
+		if(mb.isInSparseFormat())
+			leftMultByMatrixNoPreAggRowsSparse(mb.getSparseBlock(), resV, nCols, rl, ru, cl, cu, it);
 		else
 			leftMultByMatrixNoPreAggRowsDense(mb, resV, nCols, rl, ru, cl, cu, it);
 
 	}
 
-	protected final void leftMultByMatrixNoPreAggRowsSparse(SparseBlock sb, double[] resV, int nCols, int rl, int ru,
+	private final void leftMultByMatrixNoPreAggRowsSparse(SparseBlock sb, double[] resV, int nCols, int rl, int ru,
 		int cl, int cu, AIterator it) {
-		if(cl != 0 && cu != _numRows)
-			throw new NotImplementedException();
 		for(int r = rl; r < ru; r++) {
 			final int offRet = nCols * r;
-			leftMultByMatrixNoPreAggSingleRowSparse(sb, resV, offRet, r, it.clone());
+			leftMultByMatrixNoPreAggSingleRowSparse(sb, resV, offRet, r, cu, it.clone());
 		}
 	}
 
-	protected final void leftMultByMatrixNoPreAggRowsDense(MatrixBlock mb, double[] resV, int nCols, int rl, int ru,
+	private final void leftMultByMatrixNoPreAggRowsDense(MatrixBlock mb, double[] resV, int nCols, int rl, int ru,
 		int cl, int cu, AIterator it) {
 		final DenseBlock db = mb.getDenseBlock();
 		for(int r = rl; r < ru; r++) {
@@ -181,10 +189,11 @@ public abstract class ASDCZero extends APreAgg {
 			final MatrixBlock mb = md.getMatrixBlock();
 			// The dictionary is never empty.
 			if(mb.isInSparseFormat())
-				// TODO make one where the iterator is known in argument
+				// TODO make sparse decompression 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);
+				decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, mb.getDenseBlockValues(),
+					it);
 		}
 		else
 			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, _dict.getValues(), it);
@@ -194,12 +203,10 @@ public abstract class ASDCZero extends APreAgg {
 		decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, _dict.getValues(), it);
 	}
 
-	public abstract void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC,
-		double[] values, AIterator it);
+	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/ColGroupConst.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConst.java
index 5c0392bbe9..57db52f158 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConst.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConst.java
@@ -19,35 +19,27 @@
 
 package org.apache.sysds.runtime.compress.colgroup;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-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.DictionaryFactory;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
+import org.apache.sysds.runtime.compress.lib.CLALibLeftMultBy;
 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;
 import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
-import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 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;
 
-public class ColGroupConst extends AColGroupCompressed {
+public class ColGroupConst extends ADictBasedColGroup {
 
 	private static final long serialVersionUID = -7387793538322386611L;
 
-	protected ADictionary _dict;
-
 	/** Constructor for serialization */
 	protected ColGroupConst() {
 		super();
@@ -60,8 +52,7 @@ public class ColGroupConst extends AColGroupCompressed {
 	 * @param dict       The dictionary containing one tuple for the entire compression.
 	 */
 	private ColGroupConst(int[] colIndices, ADictionary dict) {
-		super(colIndices);
-		this._dict = dict;
+		super(colIndices, dict);
 	}
 
 	/**
@@ -100,6 +91,10 @@ public class ColGroupConst extends AColGroupCompressed {
 	 * @return A Constant column group.
 	 */
 	public static AColGroup create(int[] cols, double value) {
+		if(cols.length == 0)
+			throw new DMLCompressionException("Invalid number of columns");
+		else if(value == 0)
+			return new ColGroupEmpty(cols);
 		final int numCols = cols.length;
 		double[] values = new double[numCols];
 		for(int i = 0; i < numCols; i++)
@@ -117,8 +112,18 @@ public class ColGroupConst extends AColGroupCompressed {
 	public static AColGroup create(int[] cols, double[] values) {
 		if(cols.length != values.length)
 			throw new DMLCompressionException("Invalid size of values compared to columns");
-		ADictionary dict = new Dictionary(values);
-		return ColGroupConst.create(cols, dict);
+		boolean allZero = true;
+		for(double d : values)
+			if(d != 0.0) {
+				allZero = false;
+				break;
+			}
+
+		if(allZero)
+			return new ColGroupEmpty(cols);
+		else
+			return ColGroupConst.create(cols, Dictionary.create(values));
+
 	}
 
 	/**
@@ -129,7 +134,14 @@ public class ColGroupConst extends AColGroupCompressed {
 	 * @return A Constant column group.
 	 */
 	public static AColGroup create(int numCols, ADictionary dict) {
-		if(numCols != dict.getValues().length)
+		if(dict instanceof MatrixBlockDictionary) {
+			MatrixBlock mbd = ((MatrixBlockDictionary) dict).getMatrixBlock();
+			if(mbd.getNumColumns() != numCols && mbd.getNumRows() != 1) {
+				throw new DMLCompressionException(
+					"Invalid construction of const column group with different number of columns in arguments");
+			}
+		}
+		else if(numCols != dict.getValues().length)
 			throw new DMLCompressionException(
 				"Invalid construction of const column group with different number of columns in arguments");
 		final int[] colIndices = Util.genColsIndices(numCols);
@@ -153,6 +165,15 @@ public class ColGroupConst extends AColGroupCompressed {
 		return ColGroupConst.create(colIndices, value);
 	}
 
+	/**
+	 * Get dense values from colgroupConst.
+	 * 
+	 * @return the dictionary vector stored in this column group
+	 */
+	public double[] getValues() {
+		return _dict.getValues();
+	}
+
 	@Override
 	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
 		double v = preAgg[0];
@@ -170,16 +191,52 @@ public class ColGroupConst extends AColGroupCompressed {
 		return ColGroupType.CONST;
 	}
 
-	@Override
-	public void decompressToDenseBlock(DenseBlock db, int rl, int ru, int offR, int offC) {
+	protected void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb) {
+		// guaranteed to be containing some values therefore no check for empty.
+		final int apos = sb.pos(0);
+		final int alen = sb.size(0);
+		final int[] aix = sb.indexes(0);
+		final double[] avals = sb.values(0);
+
+		for(int i = rl, offT = rl + offR; i < ru; i++, offT++) {
+			final double[] c = db.values(offT);
+			final int off = db.pos(offT) + offC;
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]]] += avals[j];
+		}
+	}
+
+	protected void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+		double[] values) {
 		if(db.isContiguous() && _colIndexes.length == db.getDim(1) && offC == 0)
 			decompressToDenseBlockAllColumnsContiguous(db, rl, ru, offR, offC);
 		else
 			decompressToDenseBlockGeneric(db, rl, ru, offR, offC);
 	}
 
-	private void decompressToDenseBlockAllColumnsContiguous(DenseBlock db, int rl, int ru, int offR, int offC) {
+	protected void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
+		SparseBlock sb) {
+		final int apos = sb.pos(0);
+		final int alen = sb.size(0);
+		final int[] aix = sb.indexes(0);
+		final double[] avals = sb.values(0);
+
+		for(int i = rl, offT = rl + offR; i < ru; i++, offT++)
+			for(int j = apos; j < alen; j++)
+				ret.append(offT, _colIndexes[aix[j]] + offC, avals[j]);
+
+	}
 
+	protected void decompressToSparseBlockDenseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
+		double[] values) {
+		final int nCol = _colIndexes.length;
+		for(int i = rl, offT = rl + offR; i < ru; i++, offT++)
+			for(int j = 0; j < nCol; j++)
+				ret.append(offT, _colIndexes[j] + offC, _dict.getValue(j));
+	}
+
+	private void decompressToDenseBlockAllColumnsContiguous(DenseBlock db, int rl, int ru, int offR, int offC) {
 		final double[] c = db.values(0);
 		final int nCol = _colIndexes.length;
 		final double[] values = _dict.getValues();
@@ -199,14 +256,6 @@ public class ColGroupConst extends AColGroupCompressed {
 		}
 	}
 
-	@Override
-	public void decompressToSparseBlock(SparseBlock ret, int rl, int ru, int offR, int offC) {
-		final int nCol = _colIndexes.length;
-		for(int i = rl, offT = rl + offR; i < ru; i++, offT++)
-			for(int j = 0; j < nCol; j++)
-				ret.append(offT, _colIndexes[j] + offC, _dict.getValue(j));
-	}
-
 	@Override
 	public double getIdx(int r, int colIdx) {
 		return _dict.getValue(colIdx);
@@ -238,14 +287,31 @@ public class ColGroupConst extends AColGroupCompressed {
 	 * 
 	 * @param constV The output columns.
 	 */
-	public void addToCommon(double[] constV) {
-		final double[] values = _dict.getValues();
+	public final void addToCommon(double[] constV) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			MatrixBlock mb = ((MatrixBlockDictionary) _dict).getMatrixBlock();
+			if(mb.isInSparseFormat())
+				addToCommonSparse(constV, mb.getSparseBlock());
+			else
+				addToCommonDense(constV, mb.getDenseBlockValues());
+		}
+		else
+			addToCommonDense(constV, _dict.getValues());
+	}
+
+	private final void addToCommonDense(double[] constV, double[] values) {
 		for(int i = 0; i < _colIndexes.length; i++)
 			constV[_colIndexes[i]] += values[i];
 	}
 
-	public double[] getValues() {
-		return _dict.getValues();
+	private final void addToCommonSparse(double[] constV, SparseBlock sb) {
+
+		final int alen = sb.size(0);
+		final int[] aix = sb.indexes(0);
+		final double[] aval = sb.values(0);
+		for(int i = 0; i < alen; i++)
+			constV[_colIndexes[aix[i]]] += aval[i];
+
 	}
 
 	@Override
@@ -290,34 +356,6 @@ public class ColGroupConst extends AColGroupCompressed {
 		return 1;
 	}
 
-	private synchronized MatrixBlock forceValuesToMatrixBlock() {
-		_dict = _dict.getMBDict(_colIndexes.length);
-		MatrixBlock ret = ((MatrixBlockDictionary) _dict).getMatrixBlock();
-		return ret;
-	}
-
-	@Override
-	public AColGroup rightMultByMatrix(MatrixBlock right) {
-		if(right.isEmpty())
-			return null;
-		final int rr = right.getNumRows();
-		final int cr = right.getNumColumns();
-		if(_colIndexes.length == rr) {
-			final MatrixBlock left = forceValuesToMatrixBlock();
-			if(left == null)
-				return null;
-			final MatrixBlock ret = new MatrixBlock(1, cr, false);
-			LibMatrixMult.matrixMult(left, right, ret);
-			if(ret.isEmpty())
-				return null;
-			final ADictionary d = new MatrixBlockDictionary(ret, cr);
-			return create(cr, d);
-		}
-		else {
-			throw new NotImplementedException();
-		}
-	}
-
 	@Override
 	public void tsmm(double[] result, int numColumns, int nRows) {
 		tsmm(result, numColumns, new int[] {nRows}, _dict, _colIndexes);
@@ -325,12 +363,36 @@ public class ColGroupConst extends AColGroupCompressed {
 
 	@Override
 	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
-		throw new DMLCompressionException("This method should never be called");
+		LOG.warn("Do not use leftMultByMatrixNoPreAgg on ColGroupConst");
+		final double[] rowSum = (cl != 0 && cu != matrix.getNumColumns()) ? // do partial row sum if range is requested
+			CLALibLeftMultBy.rowSum(matrix, rl, ru, cl, cu) : // partial row sum
+			matrix.rowSum().getDenseBlockValues(); // full row sum
+
+		leftMultByRowSum(rowSum, result, rl, ru);
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		throw new DMLCompressionException("Should not be called");
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+		LOG.warn("Should never call leftMultByMatrixByAColGroup on ColGroupConst");
+		final double[] rowSum = new double[result.getNumRows()];
+		lhs.computeColSums(rowSum, nRows);
+		leftMultByRowSum(rowSum, result, 0, result.getNumRows());
+	}
+
+	private void leftMultByRowSum(double[] rowSum, MatrixBlock result, int rl, int ru) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			MatrixBlock mb = ((MatrixBlockDictionary) _dict).getMatrixBlock();
+			if(mb.isInSparseFormat())
+				ColGroupUtils.outerProduct(rowSum, mb.getSparseBlock(), _colIndexes, result.getDenseBlockValues(),
+					result.getNumColumns(), rl, ru);
+			else
+				ColGroupUtils.outerProduct(rowSum, _dict.getValues(), _colIndexes, result.getDenseBlockValues(),
+					result.getNumColumns(), rl, ru);
+		}
+		else
+			ColGroupUtils.outerProduct(rowSum, _dict.getValues(), _colIndexes, result.getDenseBlockValues(),
+				result.getNumColumns(), rl, ru);
+
 	}
 
 	@Override
@@ -345,7 +407,7 @@ public class ColGroupConst extends AColGroupCompressed {
 		if(v == 0)
 			return new ColGroupEmpty(colIndexes);
 		else {
-			ADictionary retD = new Dictionary(new double[] {_dict.getValue(idx)});
+			ADictionary retD = Dictionary.create(new double[] {_dict.getValue(idx)});
 			return create(colIndexes, retD);
 		}
 	}
@@ -377,25 +439,6 @@ public class ColGroupConst extends AColGroupCompressed {
 		return create(_colIndexes, replaced);
 	}
 
-	@Override
-	public void readFields(DataInput in) throws IOException {
-		super.readFields(in);
-		_dict = DictionaryFactory.read(in);
-	}
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-		super.write(out);
-		_dict.write(out);
-	}
-
-	@Override
-	public long getExactSizeOnDisk() {
-		long ret = super.getExactSizeOnDisk();
-		ret += _dict.getExactSizeOnDisk();
-		return ret;
-	}
-
 	@Override
 	protected void computeProduct(double[] c, int nRows) {
 		_dict.product(c, new int[] {nRows}, _colIndexes.length);
@@ -403,12 +446,14 @@ public class ColGroupConst extends AColGroupCompressed {
 
 	@Override
 	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
+		final double v = preAgg[0];
+		for(int rix = rl; rix < ru; rix++)
+			c[rix] *= v;
 	}
 
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
-		throw new NotImplementedException();
+		_dict.colProduct(c, new int[] {nRows}, _colIndexes);
 	}
 
 	@Override
@@ -423,7 +468,7 @@ public class ColGroupConst extends AColGroupCompressed {
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException();
+		return _dict.productAllRowsToDouble(_colIndexes.length);
 	}
 
 	@Override
@@ -431,14 +476,6 @@ public class ColGroupConst extends AColGroupCompressed {
 		return _dict.aggregateRows(builtin, _colIndexes.length);
 	}
 
-	@Override
-	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-		size += _dict.getInMemorySize();
-		size += 8; // dict reference
-		return size;
-	}
-
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
 		CM_COV_Object ret = new CM_COV_Object();
@@ -461,6 +498,22 @@ public class ColGroupConst extends AColGroupCompressed {
 		return e.getCost(nRows, 1, nCols, 1, 1.0);
 	}
 
+	protected AColGroup copyAndSet(int[] colIndexes, double[] newDictionary) {
+		return create(colIndexes, Dictionary.create(newDictionary));
+	}
+
+	protected AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary) {
+		return create(colIndexes, newDictionary);
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		if(colIndexes != null && preAgg != null)
+			return create(colIndexes, preAgg);
+		else
+			return null;
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
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 c4e37fa907..f5cf44e53f 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
@@ -30,7 +30,7 @@ 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;
+import org.apache.sysds.runtime.compress.colgroup.offset.AOffsetIterator;
 import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
@@ -50,30 +50,30 @@ public class ColGroupDDC extends APreAgg {
 
 	protected AMapToData _data;
 
-	/**
-	 * Constructor for serialization
-	 * 
-	 * @param numRows number of rows
-	 */
-	protected ColGroupDDC(int numRows) {
-		super(numRows);
+	/** Constructor for serialization */
+	protected ColGroupDDC() {
+		super();
 	}
 
-	private ColGroupDDC(int[] colIndexes, int numRows, ADictionary dict, AMapToData data, int[] cachedCounts) {
-		super(colIndexes, numRows, dict, cachedCounts);
+	private ColGroupDDC(int[] colIndexes, ADictionary dict, AMapToData data, int[] cachedCounts) {
+		super(colIndexes, 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;
 	}
 
-	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AMapToData data,
-		int[] cachedCounts) {
-		if(dict == null)
+	public static AColGroup create(int[] colIndexes, ADictionary dict, AMapToData data, int[] cachedCounts) {
+		if(data.getUnique() == 1)
+			return ColGroupConst.create(colIndexes, dict);
+		else if(dict == null)
 			return new ColGroupEmpty(colIndexes);
 		else
-			return new ColGroupDDC(colIndexes, numRows, dict, data, cachedCounts);
+			return new ColGroupDDC(colIndexes, dict, data, cachedCounts);
+	}
+
+	public AColGroup sparsifyFOR() {
+		return ColGroupDDCFOR.sparsifyFOR(this);
 	}
 
 	public CompressionType getCompType() {
@@ -101,16 +101,20 @@ public class ColGroupDDC extends APreAgg {
 	@Override
 	protected void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values) {
-		if(db.isContiguous() && _colIndexes.length == 1) {
-			if(db.getDim(1) == 1)
+		if(db.isContiguous()) {
+			if(_colIndexes.length == 1 && db.getDim(1) == 1)
 				decompressToDenseBlockDenseDictSingleColOutContiguous(db, rl, ru, offR, offC, values);
-			else
+			else if(_colIndexes.length == 1)
 				decompressToDenseBlockDenseDictSingleColContiguous(db, rl, ru, offR, offC, values);
+			else if(_colIndexes.length == db.getDim(1)) // offC == 0 implied
+				decompressToDenseBlockDenseDictAllColumnsContiguous(db, rl, ru, offR, values);
+			else if(offC == 0 && offR == 0)
+				decompressToDenseBlockDenseDictNoOff(db, rl, ru, values);
+			else if(offC == 0)
+				decompressToDenseBlockDenseDictNoColOffset(db, rl, ru, offR, values);
+			else
+				decompressToDenseBlockDenseDictGeneric(db, rl, ru, offR, offC, values);
 		}
-		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);
 	}
@@ -146,7 +150,6 @@ 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;
@@ -158,9 +161,20 @@ public class ColGroupDDC extends APreAgg {
 		}
 	}
 
+	private void decompressToDenseBlockDenseDictNoOff(DenseBlock db, int rl, int ru, double[] values) {
+		final int nCol = _colIndexes.length;
+		final int nColU = db.getDim(1);
+		final double[] c = db.values(0);
+		for(int i = rl; i < ru; i++) {
+			final int off = i * nColU;
+			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
 		final int nCol = _colIndexes.length;
 		for(int i = rl, offT = rl + offR; i < ru; i++, offT++) {
 			final double[] c = db.values(offT);
@@ -174,7 +188,17 @@ public class ColGroupDDC extends APreAgg {
 	@Override
 	protected void decompressToSparseBlockSparseDictionary(SparseBlock ret, 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 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++)
+				ret.append(offT, offC + _colIndexes[aix[j]], aval[j]);
+		}
 	}
 
 	@Override
@@ -190,7 +214,7 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	public double getIdx(int r, int colIdx) {
-		return _dict.getValue(_data.getIndex(r) * _colIndexes.length + colIdx);
+		return _dict.getValue(_data.getIndex(r), colIdx, _colIndexes.length);
 	}
 
 	@Override
@@ -205,6 +229,12 @@ public class ColGroupDDC extends APreAgg {
 			c[i] = builtin.execute(c[i], preAgg[_data.getIndex(i)]);
 	}
 
+	@Override
+	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		for(int rix = rl; rix < ru; rix++)
+			c[rix] *= preAgg[_data.getIndex(rix)];
+	}
+
 	@Override
 	public int[] getCounts(int[] counts) {
 		return _data.getCounts(counts);
@@ -212,7 +242,6 @@ public class ColGroupDDC extends APreAgg {
 
 	@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
@@ -226,14 +255,17 @@ public class ColGroupDDC extends APreAgg {
 		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);
+		if(matrix.isInSparseFormat()) {
+			if(cl != 0 || cu != _data.size())
+				throw new NotImplementedException();
+			lmSparseMatrixNoPreAggSingleCol(matrix.getSparseBlock(), nColM, retV, nColRet, dictVals, rl, ru);
+		}
 		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) {
+		int rl, int ru) {
 		final int colOut = _colIndexes[0];
 
 		for(int r = rl; r < ru; r++) {
@@ -256,18 +288,22 @@ public class ColGroupDDC extends APreAgg {
 			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)];
+				retV[offR + colOut] += mV[offL + c] * vals[_data.getIndex(c)];
 		}
 	}
 
 	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);
+		if(matrix.isInSparseFormat()) {
+			if(cl != 0 || cu != _data.size())
+				throw new NotImplementedException(
+					"Not implemented left multiplication on sparse without it being entire input");
+			lmSparseMatrixNoPreAggMultiCol(matrix, result, rl, ru);
+		}
 		else
 			lmDenseMatrixNoPreAggMultiCol(matrix, result, rl, ru, cl, cu);
 	}
 
-	private void lmSparseMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+	private void lmSparseMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
 		final double[] retV = result.getDenseBlockValues();
 		final int nColRet = result.getNumColumns();
 		final SparseBlock sb = matrix.getSparseBlock();
@@ -296,7 +332,6 @@ public class ColGroupDDC extends APreAgg {
 			for(int c = cl; c < cu; c++)
 				_dict.multiplyScalar(mV[offL + c], retV, offR, _data.getIndex(c), _colIndexes);
 		}
-
 	}
 
 	@Override
@@ -321,7 +356,7 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	public void preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
-		final AIterator itThat = that._indexes.getIterator();
+		final AOffsetIterator itThat = that._indexes.getOffsetIterator();
 		final int nCol = that._colIndexes.length;
 		final int finalOff = that._indexes.getOffsetToLast();
 		final double[] v = ret.getValues();
@@ -334,6 +369,11 @@ public class ColGroupDDC extends APreAgg {
 		}
 	}
 
+	@Override
+	protected void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret) {
+		_data.preAggregateDDC_RLE(that._ptr, that._data, that._dict, ret, that._colIndexes.length);
+	}
+
 	@Override
 	public boolean sameIndexStructure(AColGroupCompressed that) {
 		return that instanceof ColGroupDDC && ((ColGroupDDC) that)._data == _data;
@@ -353,26 +393,25 @@ 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()) {
+		if((op.fn instanceof Plus || op.fn instanceof Minus)) {
 			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);
+			final double[] reference = ColGroupUtils.createReference(_colIndexes.length, v0);
+			return ColGroupDDCFOR.create(_colIndexes, _dict, _data, getCachedCounts(), reference);
 		}
-		return create(_colIndexes, _numRows, _dict.applyScalarOp(op), _data, getCachedCounts());
+		return create(_colIndexes, _dict.applyScalarOp(op), _data, getCachedCounts());
 	}
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		return create(_colIndexes, _numRows, _dict.applyUnaryOp(op), _data, getCachedCounts());
+		return create(_colIndexes, _dict.applyUnaryOp(op), _data, getCachedCounts());
 	}
 
 	@Override
 	public AColGroup binaryRowOpLeft(BinaryOperator op, double[] v, boolean isRowSafe) {
 		ADictionary ret = _dict.binOpLeft(op, v, _colIndexes);
-		return create(_colIndexes, _numRows, ret, _data, getCachedCounts());
+		return create(_colIndexes, ret, _data, getCachedCounts());
 	}
 
 	@Override
@@ -380,10 +419,10 @@ public class ColGroupDDC extends APreAgg {
 		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);
+			return ColGroupDDCFOR.create(_colIndexes, _dict, _data, getCachedCounts(), reference);
 		}
 		final ADictionary ret = _dict.binOpRight(op, v, _colIndexes);
-		return create(_colIndexes, _numRows, ret, _data, getCachedCounts());
+		return create(_colIndexes, ret, _data, getCachedCounts());
 	}
 
 	@Override
@@ -414,7 +453,30 @@ public class ColGroupDDC extends APreAgg {
 
 	@Override
 	protected int numRowsToMultiply() {
-		return _numRows;
+		return _data.size();
+	}
+
+	@Override
+	protected double computeMxx(double c, Builtin builtin) {
+		return _dict.aggregate(c, builtin);
+	}
+
+	@Override
+	protected void computeColMxx(double[] c, Builtin builtin) {
+		_dict.aggregateCols(c, builtin, _colIndexes);
+	}
+
+	@Override
+	public boolean containsValue(double pattern) {
+		return _dict.containsValue(pattern);
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		if(preAgg != null)
+			return create(colIndexes, preAgg, _data, getCachedCounts());
+		else
+			return null;
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
index 8f004c9246..9def9b075d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDCFOR.java
@@ -28,11 +28,11 @@ 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.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.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;
@@ -57,37 +57,55 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 	/** Reference values in this column group */
 	protected double[] _reference;
 
-	/**
-	 * Constructor for serialization
-	 * 
-	 * @param numRows number of rows
-	 */
-	protected ColGroupDDCFOR(int numRows) {
-		super(numRows);
+	/** Constructor for serialization */
+	protected ColGroupDDCFOR() {
+		super();
 	}
 
-	private ColGroupDDCFOR(int[] colIndexes, int numRows, ADictionary dict, double[] reference, AMapToData data,
-		int[] cachedCounts) {
-		super(colIndexes, numRows, dict, cachedCounts);
+	private ColGroupDDCFOR(int[] colIndexes, ADictionary dict, double[] reference, AMapToData data, int[] cachedCounts) {
+		super(colIndexes, 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;
 		_reference = reference;
 	}
 
-	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AMapToData data,
-		int[] cachedCounts, double[] reference) {
-		final boolean allZero = FORUtil.allZero(reference);
+	public static AColGroup create(int[] colIndexes, ADictionary dict, AMapToData data, int[] cachedCounts,
+		double[] reference) {
+		final boolean allZero = ColGroupUtils.allZero(reference);
 		if(dict == null && allZero)
 			return new ColGroupEmpty(colIndexes);
 		else if(dict == null)
 			return ColGroupConst.create(colIndexes, reference);
+		else if(data.getUnique() == 1)
+			return ColGroupConst.create(colIndexes,
+				dict.binOpRight(new BinaryOperator(Plus.getPlusFnObject()), reference));
 		else if(allZero)
-			return ColGroupDDC.create(colIndexes, numRows, dict, data, cachedCounts);
+			return ColGroupDDC.create(colIndexes, dict, data, cachedCounts);
 		else
-			return new ColGroupDDCFOR(colIndexes, numRows, dict, reference, data, cachedCounts);
+			return new ColGroupDDCFOR(colIndexes, dict, reference, data, cachedCounts);
+	}
+
+	public static AColGroup sparsifyFOR(ColGroupDDC g) {
+		// It is assumed whoever call this does not use an empty Dictionary in g.
+		final int nCol = g.getColIndices().length;
+		final MatrixBlockDictionary mbd = g._dict.getMBDict(nCol);
+		if(mbd != null){
+
+			final MatrixBlock mb = mbd.getMatrixBlock();
+			
+			final double[] ref = ColGroupUtils.extractMostCommonValueInColumns(mb);
+			if(ref != null) {
+				MatrixBlockDictionary mDict = mbd.binOpRight(new BinaryOperator(Minus.getMinusFnObject()), ref);
+				return create(g.getColIndices(), mDict, g._data, g.getCachedCounts(), ref);
+			}
+			else
+				return g;
+		}
+		else {
+			throw new NotImplementedException("The dictionary was empty... highly unlikely");
+		}
 	}
 
 	public CompressionType getCompType() {
@@ -96,7 +114,7 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public double getIdx(int r, int colIdx) {
-		return _dict.getValue(_data.getIndex(r) * _colIndexes.length + colIdx) + _reference[colIdx];
+		return _dict.getValue(_data.getIndex(r), colIdx, _colIndexes.length) + _reference[colIdx];
 	}
 
 	@Override
@@ -116,93 +134,6 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 		return _data.getCounts(counts);
 	}
 
-	@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 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])];
-		}
-	}
-
-	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
 	public ColGroupType getColGroupType() {
 		return ColGroupType.DDCFOR;
@@ -222,22 +153,22 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 		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, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _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, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 		}
 		else {
 			final ADictionary newDict = _dict.applyScalarOpWithReference(op, _reference, newRef);
-			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		final double[] newRef = FORUtil.unaryOperator(op, _reference);
+		final double[] newRef = ColGroupUtils.unaryOperator(op, _reference);
 		final ADictionary newDict = _dict.applyUnaryOpWithReference(op, _reference, newRef);
-		return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
+		return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 	}
 
 	@Override
@@ -247,15 +178,15 @@ public class ColGroupDDCFOR 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, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _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, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, 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, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
@@ -266,15 +197,15 @@ public class ColGroupDDCFOR 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 create(_colIndexes, _numRows, _dict, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, _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 create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 		}
 		else { // have to apply reference while processing
 			final ADictionary newDict = _dict.binOpRightWithReference(op, v, _colIndexes, _reference, newRef);
-			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), newRef);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), newRef);
 		}
 	}
 
@@ -312,19 +243,26 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public AColGroup replace(double pattern, double replace) {
+		final ADictionary newDict = _dict.replaceWithReference(pattern, replace, _reference);
 		boolean patternInReference = false;
 		for(double d : _reference)
 			if(pattern == d) {
 				patternInReference = true;
 				break;
 			}
+		if(patternInReference) {
+			double[] nRef = new double[_reference.length];
+			for(int i = 0; i < _reference.length; i++)
+				if(pattern == _reference[i])
+					nRef[i] = replace;
+				else
+					nRef[i] = _reference[i];
 
-		if(patternInReference)
-			throw new NotImplementedException("Not Implemented replace where a value in reference should be replaced");
-		else {
-			final ADictionary newDict = _dict.replaceWithReference(pattern, replace, _reference);
-			return create(_colIndexes, _numRows, newDict, _data, getCachedCounts(), _reference);
+			return create(_colIndexes, newDict, _data, getCachedCounts(), nRef);
 		}
+		else
+			return create(_colIndexes, newDict, _data, getCachedCounts(), _reference);
+
 	}
 
 	@Override
@@ -342,7 +280,7 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 		// trick, use normal sum
 		super.computeSum(c, nRows);
 		// and add all sum of reference multiplied with nrows.
-		final double refSum = FORUtil.refSum(_reference);
+		final double refSum = ColGroupUtils.refSum(_reference);
 		c[0] += refSum * nRows;
 	}
 
@@ -357,21 +295,12 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected void computeSumSq(double[] c, int nRows) {
-		// square sum the dictionary.
 		c[0] += _dict.sumSqWithReference(getCounts(), _reference);
-		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());
 	}
 
 	@Override
 	protected void computeColSumsSq(double[] c, int nRows) {
-		_dict = _dict.getMBDict(_colIndexes.length);
-		// square sum the dictionary
 		_dict.colSumSqWithReference(c, getCounts(), _colIndexes, _reference);
-		// Square sum of the reference values only for the rows that is not represented in the Offsets.
-		for(int i = 0; i < _colIndexes.length; i++) // correct for the reference sum.
-			c[_colIndexes[i]] += _reference[i] * _reference[i] * (_numRows - _data.size());
 	}
 
 	@Override
@@ -386,7 +315,7 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException();
+		return _dict.productAllRowsToDoubleWithReference(_reference);
 	}
 
 	@Override
@@ -396,18 +325,18 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected void computeProduct(double[] c, int nRows) {
-		final int count = _numRows - _data.size();
-		_dict.productWithReference(c, getCounts(), _reference, count);
+		_dict.productWithReference(c, getCounts(), _reference, 0);
 	}
 
 	@Override
 	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException("Not Implemented PFOR");
+		for(int rix = rl; rix < ru; rix++)
+			c[rix] *= preAgg[_data.getIndex(rix)];
 	}
 
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
-		throw new NotImplementedException("Not Implemented PFOR");
+		_dict.colProductWithReference(c, getCounts(), _colIndexes, _reference);
 	}
 
 	@Override
@@ -432,45 +361,31 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public boolean containsValue(double pattern) {
-		if(pattern == 0 && _zeros)
-			return true;
-		else if(Double.isNaN(pattern) || Double.isInfinite(pattern))
-			return FORUtil.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
+
+		if(Double.isNaN(pattern) || Double.isInfinite(pattern))
+			return ColGroupUtils.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
 		else
 			return _dict.containsValueWithReference(pattern, _reference);
 	}
 
 	@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 {
-			nnz += _dict.getNumberNonZerosWithReference(getCounts(), _reference, nRows);
-			nnz += refCount * nRows;
-		}
-
-		return Math.min((long) _colIndexes.length * nRows, nnz);
+		// to be safe just assume the worst fully dense for DDCFOR
+		return (long) _colIndexes.length * nRows;
 	}
 
 	@Override
 	public AColGroup extractCommon(double[] constV) {
 		for(int i = 0; i < _colIndexes.length; i++)
 			constV[_colIndexes[i]] += _reference[i];
-		return ColGroupDDC.create(_colIndexes, _numRows, _dict, _data, getCounts());
+		return ColGroupDDC.create(_colIndexes, _dict, _data, getCounts());
 	}
 
 	@Override
 	public AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows) {
-		final double def = _reference[0];
+		final int def = (int) _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);
@@ -484,29 +399,38 @@ public class ColGroupDDCFOR extends AMorphingMMColGroup {
 			int[] outCols = Util.genColsIndices(max);
 			if(def <= 0) {
 				if(ignore)
-					return ColGroupDDC.create(outCols, nRows, d, _data, getCachedCounts());
+					return ColGroupDDC.create(outCols, 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());
+				return ColGroupDDC.create(outCols, d, _data, getCachedCounts());
 			else {
 				double[] retDef = new double[max];
 				retDef[((int) def) - 1] = 1;
-				return create(outCols, nRows, d, _data, getCachedCounts(), retDef);
+				return ColGroupDDCFOR.create(outCols, d, _data, getCachedCounts(), retDef);
 			}
 		}
+
 	}
 
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
 		// should be guaranteed to be one column therefore only one reference value.
 		CM_COV_Object ret = _dict.centralMomentWithReference(op.fn, getCounts(), _reference[0], nRows);
-		int count = _numRows - _data.size();
-		op.fn.execute(ret, _reference[0], count);
 		return ret;
 	}
 
+	@Override
+	public double[] getCommon() {
+		return _reference;
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplicationCommon(double[] common, int[] colIndexes, ADictionary preAgg) {
+		return create(colIndexes, preAgg, _data, getCachedCounts(), common);
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDeltaDDC.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDeltaDDC.java
index 3ee360d58f..c26504ecfe 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDeltaDDC.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDeltaDDC.java
@@ -34,30 +34,23 @@ public class ColGroupDeltaDDC extends ColGroupDDC {
 
 	private static final long serialVersionUID = -1045556313148564147L;
 
-	/**
-	 * Constructor for serialization
-	 *
-	 * @param numRows number of rows
-	 */
-	protected ColGroupDeltaDDC(int numRows) {
-		super(numRows);
+	/** Constructor for serialization */
+	protected ColGroupDeltaDDC() {
 	}
 
-	private ColGroupDeltaDDC(int[] colIndexes, int numRows, ADictionary dict, AMapToData data, int[] cachedCounts) {
-		super(numRows);
+	private ColGroupDeltaDDC(int[] colIndexes, ADictionary dict, AMapToData data, int[] cachedCounts) {
+		super();
 		LOG.info("Carefully use of DeltaDDC since implementation is not finished.");
 		_colIndexes = colIndexes;
 		_dict = dict;
-		_zeros = false;
 		_data = data;
 	}
 
-	public static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AMapToData data,
-		int[] cachedCounts) {
+	public static AColGroup create(int[] colIndices, ADictionary dict, AMapToData data, int[] cachedCounts) {
 		if(dict == null)
 			throw new NotImplementedException("Not implemented constant delta group");
 		else
-			return new ColGroupDeltaDDC(colIndices, numRows, dict, data, cachedCounts);
+			return new ColGroupDeltaDDC(colIndices, dict, data, cachedCounts);
 	}
 
 	public CompressionType getCompType() {
@@ -89,6 +82,6 @@ public class ColGroupDeltaDDC extends ColGroupDDC {
 
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
-		return new ColGroupDeltaDDC(_colIndexes, _numRows, _dict.applyScalarOp(op), _data, getCachedCounts());
+		return new ColGroupDeltaDDC(_colIndexes, _dict.applyScalarOp(op), _data, getCachedCounts());
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupEmpty.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupEmpty.java
index d53a9c97af..2a427caea9 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupEmpty.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupEmpty.java
@@ -89,7 +89,7 @@ public class ColGroupEmpty extends AColGroupCompressed {
 			return this;
 		double[] retV = new double[_colIndexes.length];
 		Arrays.fill(retV, v);
-		return ColGroupConst.create(_colIndexes, new Dictionary(retV));
+		return ColGroupConst.create(_colIndexes, Dictionary.create(retV));
 	}
 
 	@Override
@@ -99,7 +99,7 @@ public class ColGroupEmpty extends AColGroupCompressed {
 			return this;
 		double[] retV = new double[_colIndexes.length];
 		Arrays.fill(retV, v);
-		return ColGroupConst.create(_colIndexes, new Dictionary(retV));
+		return ColGroupConst.create(_colIndexes, Dictionary.create(retV));
 	}
 
 	@Override
@@ -109,13 +109,9 @@ public class ColGroupEmpty extends AColGroupCompressed {
 		final ValueFunction fn = op.fn;
 		final double[] retVals = new double[_colIndexes.length];
 		final int lenV = _colIndexes.length;
-		boolean allZero = true;
 		for(int i = 0; i < lenV; i++)
-			allZero = 0 == (retVals[i] = fn.execute(v[_colIndexes[i]], 0)) && allZero;
-
-		if(allZero)
-			return this;
-		return ColGroupConst.create(_colIndexes, new Dictionary(retVals));
+			retVals[i] = fn.execute(v[_colIndexes[i]], 0);
+		return ColGroupConst.create(_colIndexes, Dictionary.create(retVals));
 	}
 
 	@Override
@@ -125,12 +121,9 @@ public class ColGroupEmpty extends AColGroupCompressed {
 		final ValueFunction fn = op.fn;
 		final double[] retVals = new double[_colIndexes.length];
 		final int lenV = _colIndexes.length;
-		boolean allZero = true;
 		for(int i = 0; i < lenV; i++)
-			allZero = 0 == (retVals[i] = fn.execute(0, v[_colIndexes[i]])) && allZero;
-		if(allZero)
-			return this;
-		return ColGroupConst.create(_colIndexes, new Dictionary(retVals));
+			retVals[i] = fn.execute(0, v[_colIndexes[i]]);
+		return ColGroupConst.create(_colIndexes, Dictionary.create(retVals));
 	}
 
 	@Override
@@ -139,19 +132,18 @@ public class ColGroupEmpty extends AColGroupCompressed {
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock c) {
-		// do nothing
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock c, int nRows) {
+		// do nothing, but should never be called
 	}
 
 	@Override
 	public void tsmmAColGroup(AColGroup other, MatrixBlock result) {
-		// do nothing
+		// do nothing, but should never be called
 	}
 
 	@Override
 	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
-		// do nothing
-		// but should never be called
+		// do nothing, but should never be called
 	}
 
 	@Override
@@ -180,7 +172,7 @@ public class ColGroupEmpty extends AColGroupCompressed {
 	}
 
 	@Override
-	public AColGroup rightMultByMatrix(MatrixBlock right) {
+	public AColGroup rightMultByMatrix(MatrixBlock right, int[] allCols) {
 		return null;
 	}
 
@@ -307,4 +299,9 @@ public class ColGroupEmpty extends AColGroupCompressed {
 		final int nCols = getNumCols();
 		return e.getCost(nRows, 1, nCols, 1, 0.00001);
 	}
+
+	@Override
+	public boolean isEmpty() {
+		return true;
+	}
 }
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 c9a8e894c7..7257cb1002 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
@@ -42,7 +42,6 @@ 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.functional.LinearRegression;
 import org.apache.sysds.runtime.compress.colgroup.insertionsort.AInsertionSorter;
 import org.apache.sysds.runtime.compress.colgroup.insertionsort.InsertionSorterFactory;
@@ -60,28 +59,32 @@ import org.apache.sysds.runtime.compress.utils.DblArrayCountHashMap;
 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.DenseBlock;
 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;
 
 /**
  * Factory class for constructing ColGroups.
  */
 public class ColGroupFactory {
-	static final Log LOG = LogFactory.getLog(ColGroupFactory.class.getName());
+	protected static final Log LOG = LogFactory.getLog(ColGroupFactory.class.getName());
 
+	/** Input matrix to compress */
 	private final MatrixBlock in;
+	/** Compression information to compress based on */
 	private final CompressedSizeInfo csi;
+	/** Compression settings specifying for instance if the input is transposed */
 	private final CompressionSettings cs;
+	/** The cost estimator to use to calculate cost of compression */
 	private final ACostEstimate ce;
+	/** Parallelization degree */
 	private final int k;
-
+	/** number of rows in input (taking into account if the input is transposed) */
 	private final int nRow;
+	/** number of columns in input (taking into account if the input is transposed) */
 	private final int nCol;
-
+	/** Thread pool to use in execution of compression */
 	private final ExecutorService pool;
 
 	private ColGroupFactory(MatrixBlock in, CompressedSizeInfo csi, CompressionSettings cs, ACostEstimate ce, int k) {
@@ -97,6 +100,20 @@ public class ColGroupFactory {
 		this.pool = (k > 1) ? CommonThreadPool.get(k) : null;
 	}
 
+	/**
+	 * The actual compression method, that handles the logic of compressing multiple columns together.
+	 * 
+	 * @param in  The input matrix, that could have been transposed. If it is transposed the compSettings should specify
+	 *            this.
+	 * @param csi The compression information extracted from the estimation, this contains which groups of columns to
+	 *            compress together.
+	 * @param cs  The compression settings to specify how to compress.
+	 * @return A resulting array of ColGroups, containing the compressed information from the input matrix block.
+	 */
+	public static List<AColGroup> compressColGroups(MatrixBlock in, CompressedSizeInfo csi, CompressionSettings cs) {
+		return compressColGroups(in, csi, cs, 1);
+	}
+
 	/**
 	 * The actual compression method, that handles the logic of compressing multiple columns together.
 	 * 
@@ -144,8 +161,6 @@ public class ColGroupFactory {
 	}
 
 	private List<AColGroup> compressExecute() {
-		for(CompressedSizeInfoColGroup g : csi.getInfo())
-			g.clearMap();
 		if(in.isEmpty()) {
 			AColGroup empty = ColGroupEmpty.create(cs.transposed ? in.getNumRows() : in.getNumColumns());
 			return Collections.singletonList(empty);
@@ -158,9 +173,8 @@ public class ColGroupFactory {
 
 	private List<AColGroup> compressColGroupsSingleThreaded() {
 		List<AColGroup> ret = new ArrayList<>(csi.getNumberColGroups());
-		List<CompressedSizeInfoColGroup> groups = csi.getInfo();
 
-		for(CompressedSizeInfoColGroup g : groups)
+		for(CompressedSizeInfoColGroup g : csi.getInfo())
 			ret.add(compressColGroup(g));
 
 		return ret;
@@ -209,196 +223,82 @@ public class ColGroupFactory {
 		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));
+			// StringBuilder sb = new StringBuilder();
+			// sb.append("The estimate cost is significantly off : " + est);
+			// sb.append(est.getNumVals());
+			// sb.append(" ");
+			// sb.append(act.getNumValues());
+			// sb.append(" estimate offsets:");
+			// sb.append(est.getNumOffs());
+			String warning = "The estimate cost is significantly off : " + est;
+
+			LOG.debug(String.format("time[ms]: %10.2f %25s est %10.0f -- act %10.0f distinct:%5d cols:%s wanted:%s\n\t\t%s", time, retType,
+				estC, actC, act.getNumValues(), 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));
+			LOG.debug(String.format("time[ms]: %10.2f %25s est %10.0f -- act %10.0f distinct:%5d cols:%s wanted:%s", time, retType,
+				estC, actC, act.getNumValues(), 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("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);
-
-				}
-			}
-			else {
-				if(g instanceof ColGroupSDCZeros)
-					g = ColGroupSDCFOR.create(g.getColIndices(), nRow, mbd, ((ColGroupSDCZeros) clg)._indexes,
-						((ColGroupSDCZeros) clg)._data, clg.getCachedCounts(), ref);
-			}
-
-		}
+		AColGroup g = compress(cg);
+		if(ce != null && ce.shouldSparsify() && nCol >= 4)
+			g = sparsifyFOR(g);
 		return g;
 	}
 
-	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 boolean isSparsifyingColGroup(AColGroup g) {
-		return g instanceof ColGroupDDC || g instanceof ColGroupSDC;
+	private static AColGroup sparsifyFOR(AColGroup g) {
+		if(g instanceof ColGroupDDC)
+			return ((ColGroupDDC) g).sparsifyFOR();
+		else if(g instanceof ColGroupSDC)
+			return ((ColGroupSDC) g).sparsifyFOR();
+		else
+			return g;
 	}
 
-	private AColGroup compressColGroupInitial(CompressedSizeInfoColGroup cg) {
+	private AColGroup compress(CompressedSizeInfoColGroup cg) {
 		final int[] colIndexes = cg.getColumns();
-		final int nrUniqueEstimate = cg.getNumVals();
-		CompressionType ct = cg.getBestCompressionType();
+		final CompressionType ct = cg.getBestCompressionType();
+		final boolean t = cs.transposed;
 
-		if(ct == CompressionType.EMPTY && !cs.transposed)
+		// Fast path compressions
+		if(ct == CompressionType.EMPTY && !t)
 			return new ColGroupEmpty(colIndexes);
 		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 &&
+			return ColGroupUncompressed.create(colIndexes, in, t);
+		else if((ct == CompressionType.SDC || ct == CompressionType.CONST) && in.isInSparseFormat() && t &&
 			((colIndexes.length > 1 && cg.getNumOffs() < 0.3 * nRow) || colIndexes.length == 1))
-			return compressSDCFromSparseTransposedBlock(colIndexes, nrUniqueEstimate, cg.getTupleSparsity());
+			return compressSDCFromSparseTransposedBlock(colIndexes, cg.getNumVals(), cg.getTupleSparsity());
 		else if(ct == CompressionType.DDC)
 			return directCompressDDC(colIndexes, cg);
 		else if(ct == CompressionType.LinearFunctional)
 			return compressLinearFunctional(colIndexes, in, cs);
-		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);
-			return compress(colIndexes, numRows, ubm, ct, cs, cg.getTupleSparsity());
-		}
-	}
-
-	private static AColGroup compress(int[] colIndexes, int rlen, ABitmap ubm, CompressionType compType,
-		CompressionSettings cs, double tupleSparsity) {
 
-		if(ubm == null)
-			// If ubm is null then there was no values to extract
-			// Therefore compress to empty column group
+		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, cg.getNumVals(), cs);
+		if(ubm == null) // no values ... therefore empty
 			return new ColGroupEmpty(colIndexes);
 
 		final IntArrayList[] of = ubm.getOffsetList();
-		if(of.length == 1 && of[0].size() == rlen) // If this always constant
+		if(of.length == 1 && of[0].size() == nRow) // If this always constant
 			return ColGroupConst.create(colIndexes, DictionaryFactory.create(ubm));
 
-		// only consider sparse dictionaries if cocoded more than 4 columns.
-		tupleSparsity = colIndexes.length > 4 ? tupleSparsity : 1.0;
-		switch(compType) {
-			case DDC:
-				return compressDDC(colIndexes, rlen, ubm, cs, tupleSparsity);
+		final double tupleSparsity = colIndexes.length > 4 ? cg.getTupleSparsity() : 1.0;
+
+		switch(ct) {
 			case RLE:
-				return compressRLE(colIndexes, rlen, ubm, cs, tupleSparsity);
+				return ColGroupRLE.compressRLE(colIndexes, ubm, nRow, tupleSparsity);
 			case OLE:
-				return compressOLE(colIndexes, rlen, ubm, cs, tupleSparsity);
+				return ColGroupOLE.compressOLE(colIndexes, ubm, nRow, tupleSparsity);
 			case CONST: // in case somehow one requested const, but it was not const fall back to SDC.
-				LOG.warn("Requested const on non constant column, fallback to SDC");
 			case EMPTY:
+				LOG.warn("Requested " + ct + " on non constant column, fallback to SDC");
 			case SDC:
-				return compressSDC(colIndexes, rlen, ubm, cs, tupleSparsity);
+				return compressSDC(colIndexes, nRow, ubm, cs, tupleSparsity);
+			case DDC: // DDC have direct/fast path compression without use of ABitmap.
 			default:
-				throw new DMLCompressionException("Not implemented compression of " + compType + " in factory.");
+				throw new DMLCompressionException("Not implemented compression of " + ct + " in factory.");
 		}
 	}
 
@@ -423,7 +323,7 @@ public class ColGroupFactory {
 		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);
+		return ColGroupDDC.create(colIndexes, dict, resData, null);
 	}
 
 	private AColGroup directCompressDDCMultiCol(int[] colIndexes, CompressedSizeInfoColGroup cg) {
@@ -455,8 +355,7 @@ public class ColGroupFactory {
 			final int nUnique = map.size() + (extra ? 1 : 0);
 
 			final AMapToData resData = MapToFactory.resize(d, nUnique);
-			return ColGroupDDC.create(colIndexes, nRow, dict, resData, null);
-
+			return ColGroupDDC.create(colIndexes, dict, resData, null);
 		}
 		catch(Exception e) {
 			ReaderColumnSelection reader = ReaderColumnSelection.createReader(in, colIndexes, cs.transposed, 0, nRow);
@@ -524,10 +423,8 @@ public class ColGroupFactory {
 
 	private void readToMapDDCTransposed(int col, DoubleCountHashMap map, AMapToData data) {
 		if(in.isInSparseFormat()) {
-			// good
-			SparseBlock sb = in.getSparseBlock();
-			if(sb.isEmpty(col))
-				return;
+			final SparseBlock sb = in.getSparseBlock();
+			// It should never be empty here.
 
 			final int apos = sb.pos(col);
 			final int alen = sb.size(col) + apos;
@@ -541,16 +438,12 @@ public class ColGroupFactory {
 				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("");
+			final DenseBlock db = in.getDenseBlock();
+			final double[] dv = db.values(col);
+			int off = db.pos(col);
+			for(int r = 0; r < nRow; r++, off++)
+				data.set(r, map.increment(dv[off]));
 		}
 	}
 
@@ -603,7 +496,7 @@ public class ColGroupFactory {
 		else if((ubm.getNumValues() == 2 && numZeros == 0) || (ubm.getNumValues() == 1 && numZeros < largestOffset)) {
 			double[] defaultTuple = new double[colIndexes.length];
 			ADictionary dict = DictionaryFactory.create(ubm, largestIndex, defaultTuple, tupleSparsity, numZeros > 0);
-			return compressSDCSingle(colIndexes, rlen, ubm, dict, defaultTuple);
+			return compressSDCSingle(colIndexes, rlen, ubm,largestIndex, dict, defaultTuple);
 		}
 		else if(numZeros >= largestOffset) {
 			ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
@@ -636,25 +529,31 @@ public class ColGroupFactory {
 		return ColGroupSDC.create(colIndexes, rlen, dict, defaultTuple, indexes, _data, null);
 	}
 
-	private static AColGroup compressSDCSingle(int[] colIndexes, int rlen, ABitmap ubm, ADictionary dict,
+	private static AColGroup compressSDCSingle(int[] colIndexes, int rlen, ABitmap ubm, int largestIndex, ADictionary dict,
 		double[] defaultTuple) {
-		IntArrayList inv = ubm.getOffsetsList(0);
-		int[] indexes = new int[rlen - inv.size()];
-		int p = 0;
-		int v = 0;
-		for(int i = 0; i < inv.size(); i++) {
-			int j = inv.get(i);
-			while(v < j)
-				indexes[p++] = v++;
-			if(v == j)
+		if(ubm.getOffsetList().length > 1){
+			// flipping first bit is same as saying index 1 if zero else index 0 if one or !
+			AOffset off = OffsetFactory.createOffset(ubm.getOffsetsList(largestIndex ^ 1));
+			return ColGroupSDCSingle.create(colIndexes, rlen, dict, defaultTuple, off, null);
+		}
+		else{
+			IntArrayList inv = ubm.getOffsetsList(0);
+			int[] indexes = new int[rlen - inv.size()];
+			int p = 0;
+			int v = 0;
+			for(int i = 0; i < inv.size(); i++) {
+				int j = inv.get(i);
+				while(v < j)
+					indexes[p++] = v++;
 				v++;
+			}
+	
+			while(v < rlen)
+				indexes[p++] = v++;
+			AOffset off = OffsetFactory.createOffset(indexes);
+	
+			return ColGroupSDCSingle.create(colIndexes, rlen, dict, defaultTuple, off, null);
 		}
-
-		while(v < rlen)
-			indexes[p++] = v++;
-		AOffset off = OffsetFactory.createOffset(indexes);
-
-		return ColGroupSDCSingle.create(colIndexes, rlen, dict, defaultTuple, off, null);
 	}
 
 	private static AColGroup compressLinearFunctional(int[] colIndexes, MatrixBlock in, CompressionSettings cs) {
@@ -663,64 +562,40 @@ public class ColGroupFactory {
 		return ColGroupLinearFunctional.create(colIndexes, coefficients, numRows);
 	}
 
-	private static AColGroup compressDDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
-		double tupleSparsity) {
-		boolean zeros = ubm.getNumOffsets() < rlen;
-		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity, zeros);
-		AMapToData data = MapToFactory.create(rlen, zeros, ubm.getOffsetList());
-		return ColGroupDDC.create(colIndexes, rlen, dict, data, null);
-	}
-
-	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
-		double tupleSparsity) {
+	// private static AColGroup compressDDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
+	// double tupleSparsity) {
+	// boolean zeros = ubm.getNumOffsets() < rlen;
+	// ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity, zeros);
+	// AMapToData data = MapToFactory.create(rlen, zeros, ubm.getOffsetList());
+	// return ColGroupDDC.create(colIndexes, rlen, dict, data, null);
+	// }
 
-		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
-		ColGroupOLE ole = new ColGroupOLE(rlen);
+	// private static AColGroup compressOLE(int[] colIndexes, ABitmap ubm, double tupleSparsity) {
 
-		final int numVals = ubm.getNumValues();
-		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;
-		}
+	// ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
+	// ColGroupOLE ole = new ColGroupOLE(nRow);
 
-		// compact bitmaps to linearized representation
-		ole.createCompressedBitmaps(numVals, totalLen, lBitMaps);
-		ole._dict = dict;
-		ole._zeros = ubm.getNumOffsets() < (long) rlen;
-		ole._colIndexes = colIndexes;
-		return ole;
-	}
+	// final int numVals = ubm.getNumValues();
+	// 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;
+	// }
 
-	private static AColGroup compressRLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
-		double tupleSparsity) {
-
-		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
-		ColGroupRLE rle = new ColGroupRLE(rlen);
-		// compress the bitmaps
-		final int numVals = ubm.getNumValues();
-		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;
-		}
-		// compact bitmaps to linearized representation
-		rle.createCompressedBitmaps(numVals, totalLen, lBitMaps);
-		rle._dict = dict;
-		rle._zeros = ubm.getNumOffsets() < (long) rlen;
-		rle._colIndexes = colIndexes;
-		return rle;
-	}
+	// ColGroupOffset.createCompressedBitmaps()
+	// // compact bitmaps to linearized representation
+	// ole.createCompressedBitmaps(numVals, totalLen, lBitMaps);
+	// ole._dict = dict;
+	// ole._colIndexes = colIndexes;
+	// return ole;
+	// }
 
 	private AColGroup compressSDCFromSparseTransposedBlock(int[] cols, int nrUniqueEstimate, double tupleSparsity) {
 		if(cols.length > 1)
 			return compressMultiColSDCFromSparseTransposedBlock(cols, nrUniqueEstimate, tupleSparsity);
 		else
 			return compressSingleColSDCFromSparseTransposedBlock(cols, nrUniqueEstimate);
-
 	}
 
 	private AColGroup compressMultiColSDCFromSparseTransposedBlock(int[] cols, int nrUniqueEstimate,
@@ -815,12 +690,12 @@ public class ColGroupFactory {
 
 			final AOffset offsets = OffsetFactory.createOffset(sb.indexes(sbRow), apos, alen);
 			if(entries.length <= 1)
-				return ColGroupSDCSingleZeros.create(cols, nRow, new Dictionary(dict), offsets, counts);
+				return ColGroupSDCSingleZeros.create(cols, nRow, Dictionary.create(dict), offsets, counts);
 			else {
 				final AMapToData mapToData = MapToFactory.create((alen - apos), entries.length);
 				for(int j = apos; j < alen; j++)
 					mapToData.set(j - apos, map.get(vals[j]));
-				return ColGroupSDCZeros.create(cols, nRow, new Dictionary(dict), offsets, mapToData, counts);
+				return ColGroupSDCZeros.create(cols, nRow, Dictionary.create(dict), offsets, mapToData, counts);
 			}
 		}
 		else if(entries.length == 1) {
@@ -828,7 +703,6 @@ public class ColGroupFactory {
 			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);
@@ -847,7 +721,7 @@ public class ColGroupFactory {
 
 			final AOffset offsets = OffsetFactory.createOffset(notZeroOffsets);
 
-			return ColGroupSDCSingle.create(cols, nRow, zeroDict, defaultTuple, offsets, counts);
+			return ColGroupSDCSingle.create(cols, nRow, null, defaultTuple, offsets, counts);
 		}
 		else {
 			final ABitmap ubm = BitmapEncoder.extractBitmap(cols, in, true, entries.length, true);
@@ -878,7 +752,6 @@ public class ColGroupFactory {
 				return null;
 			}
 			catch(Exception e) {
-				e.printStackTrace();
 				throw e;
 			}
 		}
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 d4b5172705..a15e36d2ee 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
@@ -23,6 +23,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -30,13 +31,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.ColGroupType;
 
-/**
- * This has the IO responsibility of ColGroups, such that it enables to read and write ColGroups to and from a DataInput
- * and DataOutput
- */
-public class ColGroupIO {
+/** IO for ColGroups, it enables read and write ColGroups */
+public interface ColGroupIO {
 
-	protected static final Log LOG = LogFactory.getLog(ColGroupIO.class.getName());
+	static final Log LOG = LogFactory.getLog(ColGroupIO.class.getName());
 
 	/**
 	 * Read groups from a file. Note that the information about how many should be in the file already.
@@ -51,9 +49,7 @@ public class ColGroupIO {
 		// Read in how many colGroups there are
 		final int nColGroups = in.readInt();
 		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);
 
@@ -77,7 +73,7 @@ public class ColGroupIO {
 	 * @param colGroups List of the ColGroups to write to file.
 	 * @throws IOException Throws IO Exception if the out refuses to write.
 	 */
-	public static void writeGroups(DataOutput out, List<AColGroup> colGroups) throws IOException {
+	public static void writeGroups(DataOutput out, Collection<AColGroup> colGroups) throws IOException {
 		// Write out how many ColGroups to save.
 		out.writeInt(colGroups.size());
 		for(AColGroup grp : colGroups)
@@ -92,12 +88,12 @@ public class ColGroupIO {
 	 */
 	public static long getExactSizeOnDisk(List<AColGroup> colGroups) {
 		long ret = 4; // int for number of colGroups.
-		for(AColGroup grp : colGroups) 
+		for(AColGroup grp : colGroups)
 			ret += grp.getExactSizeOnDisk();
 		return ret;
 	}
 
-	private static AColGroup constructColGroup(ColGroupType ctype, int nRows){
+	private static AColGroup constructColGroup(ColGroupType ctype, int nRows) {
 		switch(ctype) {
 			case UNCOMPRESSED:
 				return new ColGroupUncompressed();
@@ -106,9 +102,9 @@ public class ColGroupIO {
 			case RLE:
 				return new ColGroupRLE(nRows);
 			case DDC:
-				return new ColGroupDDC(nRows);
+				return new ColGroupDDC();
 			case DeltaDDC:
-				return new ColGroupDeltaDDC(nRows);
+				return new ColGroupDeltaDDC();
 			case CONST:
 				return new ColGroupConst();
 			case EMPTY:
@@ -124,7 +120,7 @@ public class ColGroupIO {
 			case SDCFOR:
 				return new ColGroupSDCFOR(nRows);
 			case DDCFOR:
-				return new ColGroupDDCFOR(nRows);
+				return new ColGroupDDCFOR();
 			default:
 				throw new DMLRuntimeException("Unsupported ColGroup Type used:  " + ctype);
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupLinearFunctional.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupLinearFunctional.java
index ecb516724a..832207b94b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupLinearFunctional.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupLinearFunctional.java
@@ -371,9 +371,9 @@ public class ColGroupLinearFunctional extends AColGroupCompressed {
 	}
 
 	@Override
-	public AColGroup rightMultByMatrix(MatrixBlock right) {
+	public AColGroup rightMultByMatrix(MatrixBlock right, int[] allCols) {
 		final int nColR = right.getNumColumns();
-		final int[] outputCols = Util.genColsIndices(nColR);
+		final int[] outputCols = allCols != null ? allCols : Util.genColsIndices(nColR);
 
 		// TODO: add specialization for sparse/dense matrix blocks
 		MatrixBlock result = new MatrixBlock(_numRows, nColR, false);
@@ -417,11 +417,12 @@ public class ColGroupLinearFunctional extends AColGroupCompressed {
 		throw new DMLCompressionException("This method should never be called");
 	}
 
+
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		if(lhs instanceof ColGroupEmpty) {
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+		if(lhs instanceof ColGroupEmpty) 
 			return;
-		}
+		
 
 		MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
index 44ec3da73f..19ad6ddaf6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
@@ -23,7 +23,11 @@ import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.bitmap.ABitmap;
 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.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -48,13 +52,36 @@ public class ColGroupOLE extends AColGroupOffset {
 		super(numRows);
 	}
 
-	protected ColGroupOLE(int[] colIndices, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
-		int[] bitmapOffs, int[] counts) {
-		super(colIndices, numRows, zeros, dict, counts);
+	private ColGroupOLE(int[] colIndices, int numRows, boolean zero, ADictionary dict, char[] bitmaps, int[] bitmapOffs,
+		int[] counts) {
+		super(colIndices, numRows, zero, dict, counts);
 		_data = bitmaps;
 		_ptr = bitmapOffs;
 	}
 
+	protected static AColGroup create(int[] colIndices, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
+		int[] bitmapOffs, int[] counts) {
+		return new ColGroupOLE(colIndices, numRows, zeros, dict, bitmaps, bitmapOffs, counts);
+	}
+
+	protected static AColGroup compressOLE(int[] colIndexes, ABitmap ubm, int nRow, double tupleSparsity) {
+
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
+
+		final int numVals = ubm.getNumValues();
+		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;
+		}
+		int[] bitmap = new int[numVals + 1];
+		char[] data = new char[totalLen];
+		createCompressedBitmaps(bitmap, data, lBitMaps);
+
+		return create(colIndexes, nRow, false, dict, data, bitmap, null);
+	}
+
 	@Override
 	public CompressionType getCompType() {
 		return CompressionType.OLE;
@@ -292,40 +319,40 @@ public class ColGroupOLE extends AColGroupOffset {
 		// }
 	}
 
+	@Override
+	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		throw new NotImplementedException();
+	}
+
 	@Override
 	protected final void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
+		throw new NotImplementedException();
 		// NOTE: zeros handled once for all column groups outside
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numVals = getNumValues();
-		final double[] values = _dict.getValues();
-		// double[] c = result.getDenseBlockValues();
+		// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+		// final int numVals = getNumValues();
+		// final double[] values = _dict.getValues();
+		// // double[] c = result.getDenseBlockValues();
 
-		// iterate over all values and their bitmaps
-		for(int k = 0; k < numVals; k++) {
-			// prepare value-to-add for entire value bitmap
-			int boff = _ptr[k];
-			int blen = len(k);
-			double val = mxxValues(k, builtin, values);
+		// // iterate over all values and their bitmaps
+		// for(int k = 0; k < numVals; k++) {
+		// // prepare value-to-add for entire value bitmap
+		// int boff = _ptr[k];
+		// int blen = len(k);
+		// double val = mxxValues(k, builtin, values);
 
-			// iterate over bitmap blocks and add values
-			int slen;
-			int bix = skipScanVal(k, rl);
-			for(int off = ((rl + 1) / blksz) * blksz; bix < blen && off < ru; bix += slen + 1, off += blksz) {
-				slen = _data[boff + bix];
-				for(int i = 1; i <= slen; i++) {
-					int rix = off + _data[boff + bix + i];
-					c[rix] = builtin.execute(c[rix], val);
-				}
-			}
-		}
+		// // iterate over bitmap blocks and add values
+		// int slen;
+		// int bix = skipScanVal(k, rl);
+		// for(int off = ((rl + 1) / blksz) * blksz; bix < blen && off < ru; bix += slen + 1, off += blksz) {
+		// slen = _data[boff + bix];
+		// for(int i = 1; i <= slen; i++) {
+		// int rix = off + _data[boff + bix + i];
+		// c[rix] = builtin.execute(c[rix], val);
+		// }
+		// }
+		// }
 	}
 
-	/**
-	 * Utility function of sparse-unsafe operations.
-	 * 
-	 * @return zero indicator vector
-	 */
-	@Override
 	protected boolean[] computeZeroIndicatorVector() {
 		boolean[] ret = new boolean[_numRows];
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
@@ -354,40 +381,40 @@ public class ColGroupOLE extends AColGroupOffset {
 		return ret;
 	}
 
-	@Override
-	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-		final int numVals = getNumValues();
-		final int numCols = getNumCols();
-
-		// current pos per OLs / output values
-		int[] apos = skipScan(numVals, rl);
-
-		// cache conscious count via horizontal scans
-		for(int bi = rl; bi < ru; bi += blksz2) {
-			int bimax = Math.min(bi + blksz2, ru);
-
-			// iterate over all values and their bitmaps
-			for(int k = 0; k < numVals; k++) {
-				// prepare value-to-add for entire value bitmap
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-
-				// iterate over bitmap blocks and add values
-				for(int off = bi; bix < blen && off < bimax; off += blksz) {
-					int slen = _data[boff + bix];
-					for(int blckIx = 1; blckIx <= slen; blckIx++) {
-						rnnz[off + _data[boff + bix + blckIx] - rl] += numCols;
-					}
-					bix += slen + 1;
-				}
+	// @Override
+	// public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
+	// final int numVals = getNumValues();
+	// final int numCols = getNumCols();
+
+	// // current pos per OLs / output values
+	// int[] apos = skipScan(numVals, rl);
+
+	// // cache conscious count via horizontal scans
+	// for(int bi = rl; bi < ru; bi += blksz2) {
+	// int bimax = Math.min(bi + blksz2, ru);
+
+	// // iterate over all values and their bitmaps
+	// for(int k = 0; k < numVals; k++) {
+	// // prepare value-to-add for entire value bitmap
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int bix = apos[k];
+
+	// // iterate over bitmap blocks and add values
+	// for(int off = bi; bix < blen && off < bimax; off += blksz) {
+	// int slen = _data[boff + bix];
+	// for(int blckIx = 1; blckIx <= slen; blckIx++) {
+	// rnnz[off + _data[boff + bix + blckIx] - rl] += numCols;
+	// }
+	// bix += slen + 1;
+	// }
 
-				apos[k] = bix;
-			}
-		}
-	}
+	// apos[k] = bix;
+	// }
+	// }
+	// }
 
 	@Override
 	public double getIdx(int r, int colIdx) {
@@ -442,47 +469,85 @@ public class ColGroupOLE extends AColGroupOffset {
 		return ret;
 	}
 
-	private int skipScanVal(int k, int rl) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// private int skipScanVal(int k, int rl) {
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 
-		if(rl > 0) { // rl aligned with blksz
-			int boff = _ptr[k];
-			int blen = len(k);
-			int start = 0;
-			int bix = 0;
-			for(int i = start; i < rl && bix < blen; i += blksz) {
-				bix += _data[boff + bix] + 1;
-			}
-			return bix;
-		}
+	// if(rl > 0) { // rl aligned with blksz
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int start = 0;
+	// int bix = 0;
+	// for(int i = start; i < rl && bix < blen; i += blksz) {
+	// bix += _data[boff + bix] + 1;
+	// }
+	// return bix;
+	// }
 
-		return 0;
-	}
+	// return 0;
+	// }
 
 	@Override
 	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
 		throw new NotImplementedException();
 	}
 
+	// @Override
+	// public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+	// throw new NotImplementedException();
+	// }
+
+	@Override
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		throw new NotImplementedException();
+	}
+
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
+	protected double computeMxx(double c, Builtin builtin) {
 		throw new NotImplementedException();
+		// return _dict.aggregate(c, builtin);
+	}
+
+	@Override
+	protected void computeColMxx(double[] c, Builtin builtin) {
+		throw new NotImplementedException();
+		// if(isZero())
+		// for(int x = 0; x < _colIndexes.length; x++)
+		// c[_colIndexes[x]] = builtin.execute(c[_colIndexes[x]], 0);
+
+		// _dict.aggregateCols(c, builtin, _colIndexes);
 	}
 
 	@Override
-	public void tsmmAColGroup(AColGroup other, MatrixBlock result) {
+	public boolean containsValue(double pattern) {
 		throw new NotImplementedException();
+		// if(pattern == 0 && isZero())
+		// return true;
+		// return _dict.containsValue(pattern);
 	}
 
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append(super.toString());
+		sb.append(String.format("\n%15s%5d", "Pointers:", this._ptr.length));
+		sb.append(Arrays.toString(this._ptr));
 		sb.append(String.format("\n%15s%5d", "Data:", this._data.length));
 		sb.append(charsToString(_data));
 		return sb.toString();
 	}
 
+	protected static String charsToString(char[] data) {
+		StringBuilder sb = new StringBuilder();
+		sb.append("[");
+		for(int x = 0; x < data.length; x++) {
+			sb.append(((int) data[x]));
+			if(x != data.length - 1)
+				sb.append(", ");
+		}
+		sb.append("]");
+		return sb.toString();
+	}
+
 	/**
 	 * Encodes the bitmap in blocks of offsets. Within each block, the bits are stored as absolute offsets from the start
 	 * of the block.
@@ -534,4 +599,49 @@ public class ColGroupOLE extends AColGroupOffset {
 
 		return encodedBlocks;
 	}
+
+	@Override
+	public void preAggregateDense(MatrixBlock m, double[] preAgg, int rl, int ru, int cl, int cu) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public void preAggregateSparse(SparseBlock sb, double[] preAgg, int rl, int ru) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected void preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected void preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected void preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected boolean sameIndexStructure(AColGroupCompressed that) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected int numRowsToMultiply() {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	protected void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public double getCost(ComputationCostEstimator e, int nRows) {
+		throw new NotImplementedException();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
index b676989468..9da00bdc26 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
@@ -24,7 +24,13 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.compress.bitmap.ABitmap;
 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.offset.AIterator;
+import org.apache.sysds.runtime.compress.colgroup.offset.AOffsetIterator;
+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;
@@ -46,13 +52,45 @@ public class ColGroupRLE extends AColGroupOffset {
 		super(numRows);
 	}
 
-	protected ColGroupRLE(int[] colIndices, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
-		int[] bitmapOffs, int[] cachedCounts) {
-		super(colIndices, numRows, zeros, dict, cachedCounts);
+	private ColGroupRLE(int[] colIndexes, int numRows, boolean zeros, ADictionary dict, char[] bitmaps, int[] bitmapOffs,
+		int[] cachedCounts) {
+		super(colIndexes, numRows, zeros, dict, cachedCounts);
 		_data = bitmaps;
 		_ptr = bitmapOffs;
 	}
 
+	protected static AColGroup create(int[] colIndexes, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
+		int[] bitmapOffs, int[] cachedCounts) {
+		if(dict == null)
+			return new ColGroupEmpty(colIndexes);
+		else
+			return new ColGroupRLE(colIndexes, numRows, zeros, dict, bitmaps, bitmapOffs, cachedCounts);
+	}
+
+	protected static AColGroup compressRLE(int[] colIndexes, ABitmap ubm, int nRow, double tupleSparsity) {
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
+		// ColGroupRLE rle = new ColGroupRLE(nRow);
+		// compress the bitmaps
+		final int numVals = ubm.getNumValues();
+		char[][] lBitMaps = new char[numVals][];
+		int totalLen = 0;
+		int sumLength = 0;
+		for(int k = 0; k < numVals; k++) {
+			int l = ubm.getNumOffsets(k);
+			sumLength += l;
+			lBitMaps[k] = ColGroupRLE.genRLEBitmap(ubm.getOffsetsList(k).extractValues(), l);
+			totalLen += lBitMaps[k].length;
+		}
+		int[] bitmap = new int[numVals + 1];
+		char[] data = new char[totalLen];
+		// compact bitmaps to linearized representation
+		createCompressedBitmaps(bitmap, data, lBitMaps);
+
+		boolean zeros = sumLength < nRow;
+
+		return create(colIndexes, nRow, zeros, dict, data, bitmap, null);
+	}
+
 	@Override
 	public CompressionType getCompType() {
 		return CompressionType.RLE;
@@ -64,380 +102,483 @@ public class ColGroupRLE extends AColGroupOffset {
 	}
 
 	@Override
-	protected void decompressToDenseBlockDenseDictionary(DenseBlock target, int rl, int ru, int offR, int offC,
+	protected void decompressToDenseBlockDenseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
 		double[] values) {
-		throw new NotImplementedException();
-		// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		// final int numCols = getNumCols();
-		// final int numVals = getNumValues();
-
-		// // position and start offset arrays
-		// int[] astart = new int[numVals];
-		// int[] apos = skipScan(numVals, rl, astart);
-
-		// double[] c = target.getDenseBlockValues();
-		// // cache conscious append via horizontal scans
-		// for(int bi = rl; bi < ru; bi += blksz) {
-		// int bimax = Math.min(bi + blksz, ru);
-		// for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-		// int boff = _ptr[k];
-		// int blen = len(k);
-		// int bix = apos[k];
-		// int start = astart[k];
-		// for(; bix < blen & start < bimax; bix += 2) {
-		// start += _data[boff + bix];
-		// int len = _data[boff + bix + 1];
-		// for(int i = Math.max(rl, start) - (rl - offT); i < Math.min(start + len, ru) - (rl - offT); i++) {
-
-		// int rc = i * target.getNumColumns();
-		// for(int j = 0; j < numCols; j++)
-		// c[rc + _colIndexes[j]] += values[off + j];
-
-		// }
-		// start += len;
-		// }
-		// apos[k] = bix;
-		// astart[k] = start;
-		// }
-		// }
+		final int numVals = getNumValues();
+		final int nCol = _colIndexes.length;
+		for(int k = 0; k < numVals; k++) {
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+			final int rowIndex = k * nCol;
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc, offT = rsc + offR; rix < ru; rix++, offT++) {
+						final double[] c = db.values(offT);
+						final int off = db.pos(offT) + offC;
+						for(int j = 0; j < nCol; j++)
+							c[off + _colIndexes[j]] += values[rowIndex + j];
+					}
+					break;
+				}
+				else {
+					for(int rix = rsc, offT = rsc + offR; rix < re; rix++, offT++) {
+						final double[] c = db.values(offT);
+						final int off = db.pos(offT) + offC;
+						for(int j = 0; j < nCol; j++)
+							c[off + _colIndexes[j]] += values[rowIndex + j];
+					}
+				}
+			}
+		}
 	}
 
 	@Override
-	protected void decompressToDenseBlockSparseDictionary(DenseBlock target, int rl, int ru, int offR, int offC,
-		SparseBlock values) {
-		throw new NotImplementedException();
+	protected void decompressToDenseBlockSparseDictionary(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb) {
+		final int numVals = getNumValues();
+		for(int k = 0; k < numVals; k++) {
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+			final int sbApos = sb.pos(k);
+			final int sbAlen = sb.size(k) + sbApos;
+			final int[] sbAix = sb.indexes(k);
+			final double[] sbAval = sb.values(k);
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc, offT = rsc + offR; rix < ru; rix++, offT++) {
+						final double[] c = db.values(offT);
+						final int off = db.pos(offT) + offC;
+						for(int j = sbApos; j < sbAlen; j++)
+							c[off + _colIndexes[sbAix[j]]] += sbAval[j];
+					}
+					break;
+				}
+				else {
+					for(int rix = rsc, offT = rsc + offR; rix < re; rix++, offT++) {
+						final double[] c = db.values(offT);
+						final int off = db.pos(offT) + offC;
+
+						for(int j = sbApos; j < sbAlen; j++)
+							c[off + _colIndexes[sbAix[j]]] += sbAval[j];
+					}
+				}
+			}
+		}
 	}
 
 	@Override
 	protected void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
 		SparseBlock sb) {
-		throw new NotImplementedException();
+		final int numVals = getNumValues();
+		for(int k = 0; k < numVals; k++) {
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+			final int sbApos = sb.pos(k);
+			final int sbAlen = sb.size(k) + sbApos;
+			final int[] sbAix = sb.indexes(k);
+			final double[] sbAval = sb.values(k);
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc, offT = rsc + offR; rix < ru; rix++, offT++) {
+						for(int j = sbApos; j < sbAlen; j++)
+							ret.append(offT, _colIndexes[sbAix[j]] + offC, sbAval[j]);
+					}
+					break;
+				}
+				else {
+					for(int rix = rsc, offT = rsc + offR; rix < re; rix++, offT++) {
+						for(int j = sbApos; j < sbAlen; j++)
+							ret.append(offT, _colIndexes[sbAix[j]] + offC, sbAval[j]);
+					}
+				}
+			}
+		}
 	}
 
 	@Override
 	protected void decompressToSparseBlockDenseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
 		double[] values) {
-		throw new NotImplementedException();
-	}
-
-	@Override
-	public int[] getCounts(int[] counts) {
 		final int numVals = getNumValues();
-		int sum = 0;
+		final int nCol = _colIndexes.length;
 		for(int k = 0; k < numVals; k++) {
-			int boff = _ptr[k];
-			int blen = len(k);
-			int count = 0;
-			for(int bix = 0; bix < blen; bix += 2) {
-				count += _data[boff + bix + 1];
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+			final int rowIndex = k * nCol;
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc, offT = rsc + offR; rix < ru; rix++, offT++)
+						for(int j = 0; j < nCol; j++)
+							ret.append(offT, _colIndexes[j] + offC, values[rowIndex + j]);
+
+					break;
+				}
+				else {
+					for(int rix = rsc, offT = rsc + offR; rix < re; rix++, offT++)
+						for(int j = 0; j < nCol; j++)
+							ret.append(offT, _colIndexes[j] + offC, values[rowIndex + j]);
+
+				}
 			}
-			sum += count;
-			counts[k] = count;
-		}
-		if(_zeros) {
-			counts[counts.length - 1] = _numRows - sum;
 		}
+	}
+
+	@Override
+	public int[] getCounts(int[] counts) {
+		for(int k = 0; k < getNumValues(); k++)
+			for(int bix = _ptr[k]; bix < _ptr[k + 1]; bix += 2)
+				counts[k] += _data[bix + 1]; // add length of run
+
 		return counts;
 	}
 
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
-		double val0 = op.executeScalar(0);
+		final double val0 = op.executeScalar(0);
 		// fast path: sparse-safe operations
 		// Note that bitmaps don't change and are shallow-copied
-		if(op.sparseSafe || val0 == 0 || !_zeros) {
-			return new ColGroupRLE(_colIndexes, _numRows, _zeros, _dict.applyScalarOp(op), _data, _ptr, getCachedCounts());
-		}
-
-		// slow path: sparse-unsafe operations (potentially create new bitmap)
-		// note: for efficiency, we currently don't drop values that become 0
-		boolean[] lind = computeZeroIndicatorVector();
-		int[] loff = computeOffsets(lind);
-		if(loff.length == 0) { // empty offset list: go back to fast path
-			return new ColGroupRLE(_colIndexes, _numRows, false, _dict.applyScalarOp(op), _data, _ptr, getCachedCounts());
-		}
-
-		throw new NotImplementedException(
-			"Not implemented because dictionaries no longer should support extending by a tuple"
-				+ " Ideally implement a modification such that RLE becomes SDC group when materializing Zero tuples");
-
-		// ADictionary rvalues = _dict.applyScalarOp(op, val0, getNumCols());
-		// char[] lbitmap = genRLEBitmap(loff, loff.length);
+		if(op.sparseSafe || val0 == 0 || !_zeros)
+			return create(_colIndexes, _numRows, _zeros, _dict.applyScalarOp(op), _data, _ptr, getCachedCounts());
 
-		// char[] rbitmaps = Arrays.copyOf(_data, _data.length + lbitmap.length);
-		// System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
-		// int[] rbitmapOffs = Arrays.copyOf(_ptr, _ptr.length + 1);
-		// rbitmapOffs[rbitmapOffs.length - 1] = rbitmaps.length;
-		// return new ColGroupRLE(_colIndexes, _numRows, false, rvalues, rbitmaps, rbitmapOffs, getCachedCounts());
+		// TODO: add support for FORRLE if applicable case.
+		// slow path: sparse-unsafe operations
+		return appendRun(_dict.applyScalarOpAndAppend(op, val0, getNumCols()));
 	}
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		throw new NotImplementedException();
+		final double val0 = op.fn.execute(0);
+		// fast path: sparse-safe operations
+		// Note that bitmaps don't change and are shallow-copied
+		if(op.sparseSafe || val0 == 0 || !_zeros)
+			return create(_colIndexes, _numRows, _zeros, _dict.applyUnaryOp(op), _data, _ptr, getCachedCounts());
+
+		// TODO: add support for FORRLE if applicable case.
+		// slow path: sparse-unsafe operations
+		return appendRun(_dict.applyUnaryOpAndAppend(op, val0, getNumCols()));
 	}
 
 	@Override
 	public AColGroup binaryRowOpLeft(BinaryOperator op, double[] v, boolean isRowSafe) {
-		throw new NotImplementedException();
+		boolean sparseSafe = isRowSafe || !_zeros;
+
+		// fast path: sparse-safe operations
+		// Note that bitmaps don't change and are shallow-copied
+		if(sparseSafe)
+			return create(_colIndexes, _numRows, _zeros, _dict.binOpLeft(op, v, _colIndexes), _data, _ptr,
+				getCachedCounts());
+
+		return appendRun(_dict.binOpLeftAndAppend(op, v, _colIndexes));
 	}
 
 	@Override
 	public AColGroup binaryRowOpRight(BinaryOperator op, double[] v, boolean isRowSafe) {
-		throw new NotImplementedException();
-	}
-
-	// @Override
-	// public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-	// sparseSafe = sparseSafe || !_zeros;
-
-	// // fast path: sparse-safe operations
-	// // Note that bitmaps don't change and are shallow-copied
-	// if(sparseSafe) {
-	// return new ColGroupRLE(_colIndexes, _numRows, _zeros, applyBinaryRowOp(op, v, sparseSafe, left), _data, _ptr,
-	// getCachedCounts());
-	// }
-
-	// // slow path: sparse-unsafe operations (potentially create new bitmap)
-	// // note: for efficiency, we currently don't drop values that become 0
-	// boolean[] lind = computeZeroIndicatorVector();
-	// int[] loff = computeOffsets(lind);
-	// if(loff.length == 0) { // empty offset list: go back to fast path
-	// return new ColGroupRLE(_colIndexes, _numRows, false, applyBinaryRowOp(op, v, true, left), _data, _ptr,
-	// getCachedCounts());
-	// }
-
-	// ADictionary rvalues = applyBinaryRowOp(op, v, sparseSafe, left);
-	// char[] lbitmap = genRLEBitmap(loff, loff.length);
-	// char[] rbitmaps = Arrays.copyOf(_data, _data.length + lbitmap.length);
-	// System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
-	// int[] rbitmapOffs = Arrays.copyOf(_ptr, _ptr.length + 1);
-	// rbitmapOffs[rbitmapOffs.length - 1] = rbitmaps.length;
-
-	// // Also note that for efficiency of following operations (and less memory usage because they share index
-	// // structures),
-	// // the materialized is also applied to this.
-	// // so that following operations don't suffer from missing zeros.
-	// _data = rbitmaps;
-	// _ptr = rbitmapOffs;
-	// _zeros = false;
-	// _dict = _dict.cloneAndExtend(_colIndexes.length);
-
-	// return new ColGroupRLE(_colIndexes, _numRows, false, rvalues, rbitmaps, rbitmapOffs, getCachedCounts());
-	// }
-
-	@Override
-	protected void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
-		// final int numVals = getNumValues();
-
-		// if(numVals > 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-		// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-
-		// // step 1: prepare position and value arrays
-
-		// // current pos / values per RLE list
-		// int[] astart = new int[numVals];
-		// int[] apos = skipScan(numVals, rl, astart);
-		// double[] aval = _dict.sumAllRowsToDouble(square, _colIndexes.length);
-
-		// // step 2: cache conscious matrix-vector via horizontal scans
-		// for(int bi = rl; bi < ru; bi += blksz) {
-		// int bimax = Math.min(bi + blksz, ru);
-
-		// // horizontal segment scan, incl pos maintenance
-		// for(int k = 0; k < numVals; k++) {
-		// int boff = _ptr[k];
-		// int blen = len(k);
-		// double val = aval[k];
-		// int bix = apos[k];
-		// int start = astart[k];
-
-		// // compute partial results, not aligned
-		// while(bix < blen) {
-		// int lstart = _data[boff + bix];
-		// int llen = _data[boff + bix + 1];
-		// int from = Math.max(bi, start + lstart);
-		// int to = Math.min(start + lstart + llen, bimax);
-		// for(int rix = from; rix < to; rix++)
-		// c[rix] += val;
-
-		// if(start + lstart + llen >= bimax)
-		// break;
-		// start += lstart + llen;
-		// bix += 2;
-		// }
-
-		// apos[k] = bix;
-		// astart[k] = start;
-		// }
-		// }
-		// }
-		// else {
-		// for(int k = 0; k < numVals; k++) {
-		// int boff = _ptr[k];
-		// int blen = len(k);
-		// double val = _dict.sumRow(k, square, _colIndexes.length);
-
-		// if(val != 0.0) {
-		// Pair<Integer, Integer> tmp = skipScanVal(k, rl);
-		// int bix = tmp.getKey();
-		// int curRunStartOff = tmp.getValue();
-		// int curRunEnd = tmp.getValue();
-		// for(; bix < blen && curRunEnd < ru; bix += 2) {
-		// curRunStartOff = curRunEnd + _data[boff + bix];
-		// curRunEnd = curRunStartOff + _data[boff + bix + 1];
-		// for(int rix = curRunStartOff; rix < curRunEnd && rix < ru; rix++)
-		// c[rix] += val;
-
-		// }
-		// }
-		// }
-		// }
-	}
-
-	// @Override
-	// protected void computeRowSumsSq(double[] c, int rl, int ru, double[] preAgg) {
-	// throw new NotImplementedException();
-	// // final int numVals = getNumValues();
-
-	// // if(numVals > 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// // final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-
-	// // // step 1: prepare position and value arrays
-
-	// // // current pos / values per RLE list
-	// // int[] astart = new int[numVals];
-	// // int[] apos = skipScan(numVals, rl, astart);
-	// // double[] aval = _dict.sumAllRowsToDouble(square, _colIndexes.length);
-
-	// // // step 2: cache conscious matrix-vector via horizontal scans
-	// // for(int bi = rl; bi < ru; bi += blksz) {
-	// // int bimax = Math.min(bi + blksz, ru);
-
-	// // // horizontal segment scan, incl pos maintenance
-	// // for(int k = 0; k < numVals; k++) {
-	// // int boff = _ptr[k];
-	// // int blen = len(k);
-	// // double val = aval[k];
-	// // int bix = apos[k];
-	// // int start = astart[k];
-
-	// // // compute partial results, not aligned
-	// // while(bix < blen) {
-	// // int lstart = _data[boff + bix];
-	// // int llen = _data[boff + bix + 1];
-	// // int from = Math.max(bi, start + lstart);
-	// // int to = Math.min(start + lstart + llen, bimax);
-	// // for(int rix = from; rix < to; rix++)
-	// // c[rix] += val;
-
-	// // if(start + lstart + llen >= bimax)
-	// // break;
-	// // start += lstart + llen;
-	// // bix += 2;
-	// // }
-
-	// // apos[k] = bix;
-	// // astart[k] = start;
-	// // }
-	// // }
-	// // }
-	// // else {
-	// // for(int k = 0; k < numVals; k++) {
-	// // int boff = _ptr[k];
-	// // int blen = len(k);
-	// // double val = _dict.sumRow(k, square, _colIndexes.length);
-
-	// // if(val != 0.0) {
-	// // Pair<Integer, Integer> tmp = skipScanVal(k, rl);
-	// // int bix = tmp.getKey();
-	// // int curRunStartOff = tmp.getValue();
-	// // int curRunEnd = tmp.getValue();
-	// // for(; bix < blen && curRunEnd < ru; bix += 2) {
-	// // curRunStartOff = curRunEnd + _data[boff + bix];
-	// // curRunEnd = curRunStartOff + _data[boff + bix + 1];
-	// // for(int rix = curRunStartOff; rix < curRunEnd && rix < ru; rix++)
-	// // c[rix] += val;
-
-	// // }
-	// // }
-	// // }
-	// // }
-	// }
+		boolean sparseSafe = isRowSafe || !_zeros;
+
+		// fast path: sparse-safe operations
+		// Note that bitmaps don't change and are shallow-copied
+		if(sparseSafe)
+			return create(_colIndexes, _numRows, _zeros, _dict.binOpRight(op, v, _colIndexes), _data, _ptr,
+				getCachedCounts());
+
+		return appendRun(_dict.binOpRightAndAppend(op, v, _colIndexes));
+	}
+
+	private AColGroup appendRun(ADictionary dict) {
+		// find the locations missing runs
+		final boolean[] lind = computeZeroIndicatorVector();
+		// compute them as offsets... waste full
+		// TODO create rle from boolean list.
+		final int[] loff = computeOffsets(lind);
+		// new map for the materialized zero runs
+		final char[] lbitmap = genRLEBitmap(loff, loff.length);
+		// copy old maps and add space for new map waste full
+		final char[] rbitmaps = Arrays.copyOf(_data, _data.length + lbitmap.length);
+		// copy new map into last location
+		System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
+		// map new pointers first copy old
+		final int[] rbitmapOffs = Arrays.copyOf(_ptr, _ptr.length + 1);
+		// then add new pointer
+		rbitmapOffs[rbitmapOffs.length - 1] = rbitmaps.length;
+		return create(_colIndexes, _numRows, false, dict, rbitmaps, rbitmapOffs, getCachedCounts());
+	}
 
 	@Override
-	protected final void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
-		// NOTE: zeros handled once for all column groups outside
-		// final int numVals = getNumValues();
-		// // double[] c = result.getDenseBlockValues();
-		// final double[] values = _dict.getValues();
-
-		// for(int k = 0; k < numVals; k++) {
-		// int boff = _ptr[k];
-		// int blen = len(k);
-		// double val = mxxValues(k, builtin, values);
-
-		// Pair<Integer, Integer> tmp = skipScanVal(k, rl);
-		// int bix = tmp.getKey();
-		// int curRunStartOff = tmp.getValue();
-		// int curRunEnd = tmp.getValue();
-		// for(; bix < blen && curRunEnd < ru; bix += 2) {
-		// curRunStartOff = curRunEnd + _data[boff + bix];
-		// curRunEnd = curRunStartOff + _data[boff + bix + 1];
-		// for(int rix = curRunStartOff; rix < curRunEnd && rix < ru; rix++)
-		// c[rix] = builtin.execute(c[rix], val);
-		// }
-		// }
+	protected final void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
+		// same for both sparse and dense allocation.
+		final int numVals = getNumValues();
+
+		for(int k = 0; k < numVals; k++) {
+			// TODO add cache blocking
+			// https://github.com/apache/systemds/blob/ab5959991e33cec2a1f76ed3356a6e8b2f7a08a3/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java#L229
+			final double val = preAgg[k];
+
+			if(val != 0.0) { // cheap to check and avoid following code.
+				final int blen = _ptr[k + 1]; // 2 short to handle last differently
+				final skipPair tmp = skipScanVal(k, rl);
+				// rs is runStart and re is runEnd
+				int apos = tmp.apos;
+				int rs = 0;
+				int re = tmp.astart;
+				for(; apos < blen; apos += 2) {
+					// for each run find new start and end
+					rs = re + (int) _data[apos];
+					re = rs + (int) _data[apos + 1];
+					// TODO make specialized version that ignore rl if rl == 0.
+					// move start to new variable but minimum rl
+					final int rsc = Math.max(rs, rl); // runStartCorrected
+					// TODO make specialized version that ignore ru if ru == nRows.
+					if(re >= ru) {
+						for(int rix = rsc; rix < ru; rix++)
+							c[rix] += val;
+						break;
+					}
+					else {
+						for(int rix = rsc; rix < re; rix++)
+							c[rix] += val;
+					}
+				}
+			}
+		}
 	}
 
 	@Override
-	public boolean[] computeZeroIndicatorVector() {
-		boolean[] ret = new boolean[_numRows];
+	protected void computeProduct(double[] c, int nRows) {
+		if(_zeros)
+			c[0] = 0;
+		else
+			_dict.product(c, getCounts(), _colIndexes.length);
+	}
+
+	@Override
+	protected void computeColProduct(double[] c, int nRows) {
+		if(_zeros)
+			for(int i = 0; i < _colIndexes.length; i++)
+				c[_colIndexes[i]] = 0;
+		else
+			_dict.colProduct(c, getCounts(), _colIndexes);
+	}
+
+	@Override
+	protected final void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		if(_zeros)
+			computeRowProductSparseRLE(c, rl, ru, preAgg);
+		else
+			computeRowProductDenseRLE(c, rl, ru, preAgg);
+	}
+
+	private final void computeRowProductSparseRLE(double[] c, int rl, int ru, double[] preAgg) {
 		final int numVals = getNumValues();
+		// waste full but works
+		final boolean[] zeroRows = new boolean[ru - rl];
+		for(int k = 0; k < numVals; k++) {
+			// TODO add cache blocking
+			// https://github.com/apache/systemds/blob/ab5959991e33cec2a1f76ed3356a6e8b2f7a08a3/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java#L229
+			final double val = preAgg[k];
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc; rix < ru; rix++) {
+						c[rix] *= val;
+						zeroRows[rix - rl] = true;
+					}
+					break;
+				}
+				else {
+					for(int rix = rsc; rix < re; rix++) {
+						c[rix] *= val;
+						zeroRows[rix - rl] = true;
+					}
+				}
+			}
+		}
+		// process zeros
+		for(int i = 0; i < zeroRows.length; i++)
+			if(!zeroRows[i])
+				c[i + rl] = 0;
 
-		// initialize everything with zero
-		Arrays.fill(ret, true);
+	}
+
+	private final void computeRowProductDenseRLE(double[] c, int rl, int ru, double[] preAgg) {
+		final int numVals = getNumValues();
 
 		for(int k = 0; k < numVals; k++) {
-			int boff = _ptr[k];
-			int blen = len(k);
+			// TODO add cache blocking
+			// https://github.com/apache/systemds/blob/ab5959991e33cec2a1f76ed3356a6e8b2f7a08a3/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java#L229
+			final double val = preAgg[k];
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc; rix < ru; rix++)
+						c[rix] *= val;
+					break;
+				}
+				else {
+					for(int rix = rsc; rix < re; rix++)
+						c[rix] *= val;
+				}
+			}
 
-			int curRunStartOff = 0;
-			int curRunEnd = 0;
-			for(int bix = 0; bix < blen; bix += 2) {
-				curRunStartOff = curRunEnd + _data[boff + bix];
-				curRunEnd = curRunStartOff + _data[boff + bix + 1];
-				Arrays.fill(ret, curRunStartOff, curRunEnd, false);
+		}
+	}
+
+	@Override
+	protected final void computeRowMxx(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
+		if(_zeros)
+			computeRowMxxSparseRLE(c, builtin, rl, ru, preAgg);
+		else
+			computeRowMxxDenseRLE(c, builtin, rl, ru, preAgg);
+	}
+
+	private final void computeRowMxxSparseRLE(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
+		final int numVals = getNumValues();
+		// waste full but works
+		final boolean[] zeroRows = new boolean[ru - rl];
+		for(int k = 0; k < numVals; k++) {
+			// TODO add cache blocking
+			// https://github.com/apache/systemds/blob/ab5959991e33cec2a1f76ed3356a6e8b2f7a08a3/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java#L229
+			final double val = preAgg[k];
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc; rix < ru; rix++) {
+						c[rix] = builtin.execute(c[rix], val);
+						zeroRows[rix - rl] = true;
+					}
+					break;
+				}
+				else {
+					for(int rix = rsc; rix < re; rix++) {
+						c[rix] = builtin.execute(c[rix], val);
+						zeroRows[rix - rl] = true;
+					}
+				}
 			}
 		}
+		// process zeros
+		for(int i = 0; i < zeroRows.length; i++)
+			if(!zeroRows[i]) {
+				final int id = i + rl;
+				c[id] = builtin.execute(c[id], 0);
+				;
+			}
+	}
 
-		return ret;
+	private final void computeRowMxxDenseRLE(double[] c, Builtin builtin, int rl, int ru, double[] preAgg) {
+		final int numVals = getNumValues();
+		for(int k = 0; k < numVals; k++) {
+			// TODO add cache blocking
+			// https://github.com/apache/systemds/blob/ab5959991e33cec2a1f76ed3356a6e8b2f7a08a3/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java#L229
+			final double val = preAgg[k];
+			final int blen = _ptr[k + 1]; // 2 short to handle last differently
+			final skipPair tmp = skipScanVal(k, rl);
+
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = tmp.astart, re = tmp.astart; apos < blen; apos += 2) {
+				// for each run find new start and end
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// TODO make specialized version that ignore rl if rl == 0.
+				// move start to new variable but minimum rl
+				final int rsc = Math.max(rs, rl); // runStartCorrected
+				// TODO make specialized version that ignore ru if ru == nRows.
+				if(re >= ru) {
+					for(int rix = rsc; rix < ru; rix++)
+						c[rix] = builtin.execute(c[rix], val);
+					break;
+				}
+				else {
+					for(int rix = rsc; rix < re; rix++)
+						c[rix] = builtin.execute(c[rix], val);
+				}
+			}
+		}
 	}
 
-	@Override
-	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public boolean[] computeZeroIndicatorVector() {
+		boolean[] ret = new boolean[_numRows];
 		final int numVals = getNumValues();
-		final int numCols = getNumCols();
 
-		// current pos / values per RLE list
-		int[] astart = new int[numVals];
-		int[] apos = skipScan(numVals, rl, astart);
+		// initialize everything with zero
+		Arrays.fill(ret, true);
 
 		for(int k = 0; k < numVals; k++) {
 			int boff = _ptr[k];
 			int blen = len(k);
-			int bix = apos[k];
 
 			int curRunStartOff = 0;
 			int curRunEnd = 0;
-			for(; bix < blen && curRunStartOff < ru; bix += 2) {
+			for(int bix = 0; bix < blen; bix += 2) {
 				curRunStartOff = curRunEnd + _data[boff + bix];
 				curRunEnd = curRunStartOff + _data[boff + bix + 1];
-				for(int i = Math.max(curRunStartOff, rl); i < Math.min(curRunEnd, ru); i++)
-					rnnz[i - rl] += numCols;
+				Arrays.fill(ret, curRunStartOff, curRunEnd, false);
 			}
 		}
+
+		return ret;
 	}
 
 	@Override
@@ -462,96 +603,381 @@ public class ColGroupRLE extends AColGroupOffset {
 		return 0;
 	}
 
-	/////////////////////////////////
-	// internal helper functions
-
 	/**
-	 * Scans to given row_lower position by scanning run length fields. Returns array of positions for all values and
-	 * modifies given array of start positions for all values too.
+	 * Skip through the k's values run until a run containing or greater than rl
 	 * 
-	 * @param numVals number of values
-	 * @param rl      lower row position
-	 * @param astart  start positions
-	 * @return array of positions for all values
+	 * @param k  the k's value to skip inside
+	 * @param rl The row to either contain or be greater than
+	 * @return A skipPair of position in data, and starting offset for that run.
 	 */
-	private int[] skipScan(int numVals, int rl, int[] astart) {
-		int[] apos = new int[numVals];
-
-		if(rl > 0) { // rl aligned with blksz
-			for(int k = 0; k < numVals; k++) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = 0;
-				int start = 0;
-				while(bix < blen) {
-					int lstart = _data[boff + bix]; // start
-					int llen = _data[boff + bix + 1]; // len
-					if(start + lstart + llen >= rl)
+	private skipPair skipScanVal(int k, int rl) {
+		final int blen = _ptr[k + 1];
+		int apos = _ptr[k];
+		int start = 0;
+		do {
+			// Next run start index start
+			final int nStart = start + _data[apos] + _data[apos + 1];
+			// If it starts after rl then skip found.
+			if(nStart >= rl)
+				break;
+			// increment
+			start = nStart;
+			apos += 2;
+		}
+		while(apos < blen);
+
+		return new skipPair(apos, start);
+	}
+
+	private class skipPair {
+		protected final int apos;
+		protected final int astart;
+
+		protected skipPair(int apos, int astart) {
+			this.apos = apos;
+			this.astart = astart;
+		}
+	}
+
+	@Override
+	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
+		if(matrix.isInSparseFormat()) {
+			if(cl != 0 || cu != _numRows)
+				throw new NotImplementedException(
+					"Not implemented left multiplication on sparse without it being entire input");
+			lmSparseMatrixNoPreAggMultiCol(matrix, result, rl, ru);
+		}
+		else
+			lmDenseMatrixNoPreAggMultiCol(matrix, result, rl, ru, cl, cu);
+	}
+
+	private void lmSparseMatrixNoPreAggMultiCol(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
+		final double[] retV = result.getDenseBlockValues();
+		final int nColRet = result.getNumColumns();
+		final SparseBlock sb = matrix.getSparseBlock();
+		final int nv = getNumValues();
+		for(int r = rl; r < ru; r++) {
+			if(sb.isEmpty(r))
+				continue;
+			final int sbApos = sb.pos(r);
+			final int sbAlen = sb.size(r) + sbApos;
+			final int[] sbAix = sb.indexes(r);
+			final double[] sbAval = sb.values(r);
+			final int offR = r * nColRet;
+
+			for(int k = 0; k < nv; k++) { // for each run in RLE
+				int i = sbApos;
+				final int blen = _ptr[k + 1];
+				for(int apos = _ptr[k], rs = 0, re = 0; apos < blen && i < sbAlen; apos += 2) {
+					rs = re + _data[apos];
+					re = rs + _data[apos + 1];
+					while(i < sbAlen && sbAix[i] < rs)
+						i++;
+					for(; i < sbAlen && sbAix[i] < re; i++)
+						_dict.multiplyScalar(sbAval[i], retV, offR, k, _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();
+		final int nv = getNumValues();
+		// find each index in RLE, and aggregate into those.
+		for(int r = rl; r < ru; r++) { // TODO move rl and ru to innermost loop.
+			final int offL = r * nColM;
+			final int offR = r * nColRet;
+			for(int k = 0; k < nv; k++) { // for each run in RLE
+				final int blen = _ptr[k + 1];
+				final skipPair tmp = skipScanVal(k, cl);
+				// rs is runStart and re is runEnd
+
+				for(int apos = tmp.apos, rs = 0, re = tmp.astart; apos < blen; apos += 2) {
+					rs = re + _data[apos];
+					re = rs + _data[apos + 1];
+					final int rsc = Math.max(rs, cl); // runStartCorrected
+					// TODO make specialized version that ignore cu if cu == nRows.
+					if(re >= cu) {
+						for(int rix = rsc; rix < cu; rix++)
+							_dict.multiplyScalar(mV[offL + rix], retV, offR, k, _colIndexes);
 						break;
-					start += lstart + llen;
-					bix += 2;
+					}
+					else {
+						for(int rix = rsc; rix < re; rix++)
+							_dict.multiplyScalar(mV[offL + rix], retV, offR, k, _colIndexes);
+					}
 				}
-				apos[k] = bix;
-				astart[k] = start;
 			}
 		}
+	}
 
-		return apos;
-	}
-
-	// private Pair<Integer, Integer> skipScanVal(int k, int rl) {
-	// int apos = 0;
-	// int astart = 0;
-
-	// if(rl > 0) { // rl aligned with blksz
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// int bix = 0;
-	// int start = 0;
-	// while(bix < blen) {
-	// int lstart = _data[boff + bix]; // start
-	// int llen = _data[boff + bix + 1]; // len
-	// if(start + lstart + llen >= rl)
-	// break;
-	// start += lstart + llen;
-	// bix += 2;
-	// }
-	// apos = bix;
-	// astart = start;
-	// }
-	// return new Pair<>(apos, astart);
-	// }
+	@Override
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		if(preAgg == null)
+			return null;
+		return create(colIndexes, _numRows, _zeros, preAgg, _data, _ptr, getCachedCounts());
+	}
 
 	@Override
-	public void leftMultByMatrixNoPreAgg(MatrixBlock matrix, MatrixBlock result, int rl, int ru, int cl, int cu) {
-		throw new NotImplementedException();
+	protected double computeMxx(double c, Builtin builtin) {
+		if(_zeros)
+			c = builtin.execute(c, 0);
+		return _dict.aggregate(c, builtin);
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		throw new NotImplementedException();
+	protected void computeColMxx(double[] c, Builtin builtin) {
+		if(_zeros)
+			for(int x = 0; x < _colIndexes.length; x++)
+				c[_colIndexes[x]] = builtin.execute(c[_colIndexes[x]], 0);
+		_dict.aggregateCols(c, builtin, _colIndexes);
 	}
 
 	@Override
-	public void tsmmAColGroup(AColGroup other, MatrixBlock result) {
-		throw new NotImplementedException();
+	public boolean containsValue(double pattern) {
+		if(pattern == 0 && _zeros)
+			return true;
+		return _dict.containsValue(pattern);
 	}
 
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append(super.toString());
-		sb.append(String.format("\n%15s%5d", "Data:", this._data.length));
-		sb.append("{");
-		sb.append(((int) _data[0]) + "-" + ((int) _data[1]));
+		sb.append(String.format("\n%14s len(%d) Zeros:%b", "Data:", this._data.length, _zeros));
+		sb.append("\n{{");
+		sb.append(pair(_data, 0));
+		int p = 1;
 		for(int i = 2; i < _data.length; i += 2) {
-			sb.append(", " + ((int) _data[i]) + "-" + ((int) _data[i + 1]));
+			if(_ptr[p] == i) {
+				if(_ptr[p] + 2 == _ptr[p + 1])
+					sb.append("}, {" + pair(_data, i));
+				else
+
+					sb.append("},\n {" + pair(_data, i));
+				p++;
+			}
+			else
+				sb.append(", " + pair(_data, i));
+
 		}
-		sb.append("}");
+		sb.append("}}");
 
 		return sb.toString();
 	}
 
+	private String pair(char[] d, int off) {
+		if((int) _data[off + 1] == 1)
+			return ((int) _data[off]) + "";
+		else
+			return ((int) _data[off]) + "-" + ((int) _data[off + 1]);
+	}
+
+	@Override
+	public void preAggregateDense(MatrixBlock m, double[] preAgg, final int rl, final int ru, final int cl,
+		final int cu) {
+		final DenseBlock db = m.getDenseBlock();
+		if(!db.isContiguous())
+			throw new NotImplementedException("Not implemented support for preAggregate non contiguous dense matrix");
+		final double[] mV = m.getDenseBlockValues();
+		final int nCol = m.getNumColumns();
+		final int nv = getNumValues();
+
+		for(int k = 0; k < nv; k++) { // for each run in RLE
+			final int blen = _ptr[k + 1];
+			final skipPair tmp = skipScanVal(k, cl);
+			// rs is runStart and re is runEnd
+			for(int apos = tmp.apos, rs = 0, re = tmp.astart; apos < blen; apos += 2) {
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				final int rsc = Math.max(rs, cl); // runStartCorrected
+				// TODO make specialized version that ignore cu if cu == nRows.
+
+				if(re >= cu) {
+					for(int r = rl; r < ru; r++) {
+						final int off = (r - rl) * nv + k;
+						final int offI = nCol * r;
+						for(int rix = rsc + offI; rix < cu + offI; rix++) {
+							preAgg[off] += mV[rix];
+						}
+					}
+					break;
+				}
+				else {
+					for(int r = rl; r < ru; r++) {
+						final int off = (r - rl) * nv + k;
+						final int offI = nCol * r;
+						for(int rix = rsc + offI; rix < re + offI; rix++)
+							preAgg[off] += mV[rix];
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public void preAggregateSparse(SparseBlock sb, double[] preAgg, int rl, int ru) {
+		final int nv = getNumValues();
+
+		for(int r = rl; r < ru; r++) { // for each row
+			if(sb.isEmpty(r))
+				continue;
+			final int sbApos = sb.pos(r);
+			final int sbAlen = sb.size(r) + sbApos;
+			final int[] sbAix = sb.indexes(r);
+			final double[] sbAval = sb.values(r);
+			for(int k = 0; k < nv; k++) { // for each unique value in RLE
+				final int blen = _ptr[k + 1];
+				final int offR = (r - rl) * nv + k;
+				int i = sbApos;
+				for(int apos = _ptr[k], rs = 0, re = 0; apos < blen; apos += 2) { // for each run
+					rs = re + _data[apos];
+					re = rs + _data[apos + 1];
+
+					while(i < sbAlen && sbAix[i] < rs) // skip into sparse until run
+						i++;
+					for(; i < sbAlen && sbAix[i] < re; i++) // process in run
+						preAgg[offR] += sbAval[i];
+				}
+			}
+		}
+	}
+
+	@Override
+	protected void preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
+		that._data.preAggregateRLE_DDC(_ptr, _data, that._dict, ret, that._colIndexes.length);
+	}
+
+	@Override
+	protected void preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
+		final int finalOff = that._indexes.getOffsetToLast();
+		final double[] v = ret.getValues();
+		final int nv = getNumValues();
+		final int nCol = that._colIndexes.length;
+		for(int k = 0; k < nv; k++) {
+			final AIterator itThat = that._indexes.getIterator();
+			final int blen = _ptr[k + 1];
+			for(int apos = _ptr[k], rs = 0, re = 0; apos < blen; apos += 2) {
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// if index is later than run continue
+				if(itThat.value() >= re || rs == re || rs > finalOff)
+					continue;
+				// while lower than run iterate through
+				while(itThat.value() < rs && itThat.value() != finalOff)
+					itThat.next();
+				// process inside run
+				for(int rix = itThat.value(); rix < re; rix = itThat.value()) { // nice skip inside runs
+					that._dict.addToEntry(v, that._data.getIndex(itThat.getDataIndex()), k, nCol);
+					if(itThat.value() == finalOff) // break if final.
+						break;
+					itThat.next();
+				}
+			}
+		}
+	}
+
+	@Override
+	protected void preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
+		final int finalOff = that._indexes.getOffsetToLast();
+		final double[] v = ret.getValues();
+		final int nv = getNumValues();
+		final int nCol = that._colIndexes.length;
+		for(int k = 0; k < nv; k++) {
+			final AOffsetIterator itThat = that._indexes.getOffsetIterator();
+			final int blen = _ptr[k + 1];
+			for(int apos = _ptr[k], rs = 0, re = 0; apos < blen; apos += 2) {
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				// if index is later than run continue
+				if(itThat.value() >= re || rs == re || rs > finalOff)
+					continue;
+				// while lower than run iterate through
+				while(itThat.value() < rs && itThat.value() != finalOff)
+					itThat.next();
+				// process inside run
+				for(int rix = Math.max(rs, itThat.value()); rix < re; rix = itThat.value()) { // nice skip inside runs
+					that._dict.addToEntry(v, 0, k, nCol);
+					if(itThat.value() == finalOff) // break if final.
+						break;
+					itThat.next();
+				}
+			}
+		}
+	}
+
+	@Override
+	protected boolean sameIndexStructure(AColGroupCompressed that) {
+		if(that.getCompType() == this.getCompType()) {
+			final ColGroupRLE rle = (ColGroupRLE) that;
+			return rle._ptr == this._ptr && rle._data == this._data;
+		}
+		else
+			return false;
+	}
+
+	@Override
+	protected int numRowsToMultiply() {
+		return _data.length / 2;
+	}
+
+	@Override
+	protected void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret) {
+		final double[] v = ret.getValues();
+		final int nv = getNumValues();
+		final int tnv = that.getNumValues();
+		final int nCol = that._colIndexes.length;
+		final int[] skip = new int[tnv];
+		final int[] skipV = new int[tnv];
+		for(int k = 0; k < nv; k++) {
+			for(int tk = 0; tk < tnv; tk++) {
+				skip[tk] = that._ptr[tk];
+				skipV[tk] = 0;
+			}
+			final int blen = _ptr[k + 1];
+			for(int apos = _ptr[k], rs = 0, re = 0; apos < blen; apos += 2) {
+				rs = re + _data[apos];
+				re = rs + _data[apos + 1];
+				if(rs == re)// empty run
+					continue;
+				for(int tk = 0; tk < tnv; tk++) {
+					final int tblen = that._ptr[tk + 1];
+					int tapos = skip[tk];
+					int trs = 0, tre = skipV[tk];
+					for(; tapos < tblen; tapos += 2) {
+						trs = tre + that._data[tapos];
+						tre = trs + that._data[tapos + 1];
+						if(trs == tre || // if run is zero length do not check just remember skip
+							tre <= rs) { // if before run take next run
+							skip[tk] = tapos;
+							skipV[tk] = trs - that._data[tapos];
+							continue;
+						}
+						else if(trs >= re) // if we are past run break.
+							break;
+						else if((trs >= rs && trs < re) || // inside low
+							(tre <= re && tre > rs) || // inside high
+							(trs <= rs && tre > re)) { // encapsulate
+							final int crs = Math.max(rs, trs); // common largest run start
+							final int cre = Math.min(re, tre); // common smallest run end
+							that._dict.addToEntry(v, tk, k, nCol, cre - crs);
+						}
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public double getCost(ComputationCostEstimator e, int nRows) {
+		final int nVals = getNumValues();
+		final int nCols = getNumCols();
+		return e.getCost(_numRows, _data.length, nCols, nVals, _dict.getSparsity());
+	}
+
 	/**
 	 * Encodes the bitmap as a series of run lengths and offsets.
 	 * 
@@ -563,6 +989,10 @@ public class ColGroupRLE extends AColGroupOffset {
 	 */
 	public static char[] genRLEBitmap(int[] offsets, int len) {
 
+		final char CM = Character.MAX_VALUE;
+		final int CMi = CM;
+		final char c0 = (char) 0;
+
 		// Use an ArrayList for correctness at the expense of temp space
 		List<Character> buf = new ArrayList<>();
 
@@ -586,16 +1016,16 @@ public class ColGroupRLE extends AColGroupOffset {
 		int firstOff = offsets[0];
 
 		// The first run may start more than a short's worth of bits in
-		while(firstOff > Character.MAX_VALUE) {
-			buf.add(Character.MAX_VALUE);
-			buf.add((char) 0);
-			firstOff -= Character.MAX_VALUE;
-			lastRunEnd += Character.MAX_VALUE;
+		while(firstOff > CM) {
+			buf.add(CM);
+			buf.add(c0);
+			firstOff -= CM;
+			lastRunEnd += CM;
 		}
 
 		// Create the first run with an initial size of 1
 		curRunOff = firstOff;
-		curRunLen = 1;
+		curRunLen = 1; // 1 because there is at least 1 value in the next offset.
 
 		// Process the remaining offsets
 		for(int i = 1; i < len; i++) {
@@ -603,9 +1033,9 @@ public class ColGroupRLE extends AColGroupOffset {
 			int absOffset = offsets[i];
 
 			// 1 + (last position in run)
-			int curRunEnd = lastRunEnd + curRunOff + curRunLen;
+			final int curRunEnd = lastRunEnd + curRunOff + curRunLen;
 
-			if(absOffset > curRunEnd || curRunLen >= Character.MAX_VALUE) {
+			if(absOffset > curRunEnd || curRunLen >= CMi) {
 				// End of a run, either because we hit a run of 0's or because the
 				// number of 1's won't fit in 16 bits. Add run to bitmap and start a new one.
 				buf.add((char) curRunOff);
@@ -614,13 +1044,13 @@ public class ColGroupRLE extends AColGroupOffset {
 				lastRunEnd = curRunEnd;
 				curRunOff = absOffset - lastRunEnd;
 
-				while(curRunOff > Character.MAX_VALUE) {
+				while(curRunOff > CMi) {
 					// SPECIAL CASE: Offset to next run doesn't fit into 16 bits.
 					// Add zero-length runs until the offset is small enough.
-					buf.add(Character.MAX_VALUE);
-					buf.add((char) 0);
-					lastRunEnd += Character.MAX_VALUE;
-					curRunOff -= Character.MAX_VALUE;
+					buf.add(CM);
+					buf.add(c0);
+					lastRunEnd += CMi;
+					curRunOff -= CMi;
 				}
 
 				curRunLen = 1;
@@ -631,13 +1061,6 @@ public class ColGroupRLE extends AColGroupOffset {
 			}
 		}
 
-		// Edge case, if the last run overlaps the character length bound.
-		if(curRunOff + curRunLen > Character.MAX_VALUE) {
-			buf.add(Character.MAX_VALUE);
-			buf.add((char) 0);
-			curRunOff -= Character.MAX_VALUE;
-		}
-
 		// Add the final Run.
 		buf.add((char) curRunOff);
 		buf.add((char) curRunLen);
@@ -649,5 +1072,4 @@ public class ColGroupRLE extends AColGroupOffset {
 
 		return ret;
 	}
-
 }
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 e0ebe986d5..eec893df05 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
@@ -24,13 +24,11 @@ import java.io.DataOutput;
 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.dictionary.Dictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
-import org.apache.sysds.runtime.compress.colgroup.mapping.MapToBit;
 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;
@@ -51,11 +49,9 @@ import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
  * This column group is handy in cases where sparse unsafe operations is executed on very sparse columns. Then the zeros
  * would be materialized in the group without any overhead.
  */
-public class ColGroupSDC extends AMorphingMMColGroup {
+public class ColGroupSDC extends ASDC {
 	private static final long serialVersionUID = 769993538831949086L;
 
-	/** Sparse row indexes for the data */
-	protected AOffset _indexes;
 	/** Pointers to row indexes in the dictionary. */
 	protected AMapToData _data;
 	/** The default value stored in this column group */
@@ -70,9 +66,9 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		super(numRows);
 	}
 
-	private ColGroupSDC(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple, AOffset offsets,
+	protected ColGroupSDC(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple, AOffset offsets,
 		AMapToData data, int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
+		super(colIndices, numRows, dict, offsets, cachedCounts);
 		if(data.getUnique() != dict.getNumberOfValues(colIndices.length)) {
 			if(data.getUnique() != data.getMax())
 				throw new DMLCompressionException(
@@ -80,29 +76,31 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 			throw new DMLCompressionException("Invalid construction of SDC group: number uniques: " + data.getUnique()
 				+ " vs." + dict.getNumberOfValues(colIndices.length));
 		}
+		if(defaultTuple.length != colIndices.length)
+			throw new DMLCompressionException("Invalid construction of SDC group");
 
-		_indexes = offsets;
 		_data = data;
-		_zeros = false;
 		_defaultTuple = defaultTuple;
 
-		if(data instanceof MapToBit && ((MapToBit) data).isEmpty())
-			throw new DMLCompressionException("Error in SDC construction should have been SDCSingle");
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple,
+	public static AColGroup create(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple,
 		AOffset offsets, AMapToData data, int[] cachedCounts) {
-		final boolean allZero = FORUtil.allZero(defaultTuple);
+		final boolean allZero = ColGroupUtils.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(dict == null || dict.getNumberOfValues(colIndices.length) == 1)
+			return ColGroupSDCSingle.create(colIndices, numRows, dict, 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);
 	}
 
+	public AColGroup sparsifyFOR() {
+		return ColGroupSDCFOR.sparsifyFOR(this);
+	}
+
 	@Override
 	public CompressionType getCompType() {
 		return CompressionType.SDC;
@@ -113,23 +111,23 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		return ColGroupType.SDC;
 	}
 
+	@Override
+	public  double[] getDefaultTuple(){
+		return _defaultTuple;
+	}
+
+	public AMapToData getMapping(){
+		return _data;
+	}
+
 	@Override
 	public double getIdx(int r, int colIdx) {
 		final AIterator it = _indexes.getIterator(r);
 		if(it == null || it.value() != r)
 			return _defaultTuple[colIdx];
+		else
+			return _dict.getValue(_data.getIndex(it.getDataIndex()), colIdx, _colIndexes.length);
 
-		else {
-			final int rowOff = _data.getIndex(it.getDataIndex());
-			final int nCol = _colIndexes.length;
-			return _dict.getValue(rowOff * nCol + colIdx);
-		}
-	}
-
-	@Override
-	public ADictionary getDictionary() {
-		throw new NotImplementedException(
-			"Not implemented getting the dictionary out, and i think we should consider removing the option");
 	}
 
 	@Override
@@ -144,7 +142,7 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException("Should implement preAgg with extra cell");
+		return _dict.productAllRowsToDoubleWithDefault(_defaultTuple);
 	}
 
 	@Override
@@ -177,42 +175,43 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		int r = rl;
 		final AIterator it = indexes.getIterator(rl);
 		final double def = preAgg[preAgg.length - 1];
-		if(it != null && it.value() > ru)
-			indexes.cacheIterator(it, ru);
-		else if(it != null && ru >= indexes.getOffsetToLast()) {
-			final int maxId = data.size() - 1;
-			while(true) {
-				if(it.value() == r) {
-					c[r] += preAgg[data.getIndex(it.getDataIndex())];
-					if(it.getDataIndex() < maxId)
-						it.next();
-					else {
-						r++;
-						break;
+		if(it != null) {
+
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxId = data.size() - 1;
+
+				while(true) {
+					if(it.value() == r) {
+						c[r] += preAgg[data.getIndex(it.getDataIndex())];
+						if(it.getDataIndex() < maxId)
+							it.next();
+						else {
+							r++;
+							break;
+						}
 					}
+					else
+						c[r] += def;
+					r++;
 				}
-				else
-					c[r] += def;
-				r++;
 			}
-		}
-		else if(it != null) {
-			while(r < ru) {
-				if(it.value() == r) {
-					c[r] += preAgg[data.getIndex(it.getDataIndex())];
-					it.next();
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r++] += preAgg[data.getIndex(it.getDataIndex())];
+						it.next();
+					}
+					else
+						c[r++] += def;
 				}
-				else
-					c[r] += def;
-				r++;
+				indexes.cacheIterator(it, ru);
 			}
-			indexes.cacheIterator(it, ru);
 		}
 
-		while(r < ru) {
-			c[r] += def;
-			r++;
-		}
+		while(r < ru)
+			c[r++] += def;
 	}
 
 	@Override
@@ -224,42 +223,45 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		AMapToData data, AOffset indexes, int nRows, double def) {
 		int r = rl;
 		final AIterator it = indexes.getIterator(rl);
-		if(it != null && it.value() > ru)
-			indexes.cacheIterator(it, ru);
-		else if(it != null && ru >= indexes.getOffsetToLast()) {
-			final int maxId = data.size() - 1;
-			while(true) {
-				if(it.value() == r) {
-					c[r] = builtin.execute(c[r], preAgg[data.getIndex(it.getDataIndex())]);
-					if(it.getDataIndex() < maxId)
-						it.next();
-					else {
-						r++;
-						break;
+		if(it != null) {
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxId = data.size() - 1;
+				while(true) {
+					if(it.value() == r) {
+						c[r] = builtin.execute(c[r], preAgg[data.getIndex(it.getDataIndex())]);
+						if(it.getDataIndex() < maxId)
+							it.next();
+						else {
+							r++;
+							break;
+						}
 					}
+					else
+						c[r] = builtin.execute(c[r], def);
+					r++;
 				}
-				else
-					c[r] = builtin.execute(c[r], def);
-				r++;
 			}
-		}
-		else if(it != null) {
-			while(r < ru) {
-				if(it.value() == r) {
-					c[r] = builtin.execute(c[r], preAgg[data.getIndex(it.getDataIndex())]);
-					it.next();
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r] = builtin.execute(c[r], preAgg[data.getIndex(it.getDataIndex())]);
+						it.next();
+					}
+					else
+						c[r] = builtin.execute(c[r], def);
+					r++;
 				}
-				else
-					c[r] = builtin.execute(c[r], def);
-				r++;
+				indexes.cacheIterator(it, ru);
 			}
-			indexes.cacheIterator(it, ru);
 		}
 
 		while(r < ru) {
 			c[r] = builtin.execute(c[r], def);
 			r++;
 		}
+
 	}
 
 	@Override
@@ -303,13 +305,62 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
 		super.computeColProduct(c, nRows);
-		for(int x = 0; x < _colIndexes.length; x++)
-			c[_colIndexes[x]] *= _defaultTuple[x];
+		final int count = _numRows - _data.size();
+		for(int x = 0; x < _colIndexes.length; x++) {
+			double v = c[_colIndexes[x]];
+			c[_colIndexes[x]] = v != 0 ? v * Math.pow(_defaultTuple[x], count) : 0;
+		}
 	}
 
 	@Override
 	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
+		computeRowProduct(c, rl, ru, preAgg, _data, _indexes, _numRows);
+	}
+
+	protected static final void computeRowProduct(double[] c, int rl, int ru, double[] preAgg, AMapToData data,
+		AOffset indexes, int nRows) {
+		int r = rl;
+		final AIterator it = indexes.getIterator(rl);
+		final double def = preAgg[preAgg.length - 1];
+		if(it != null) {
+
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxId = data.size() - 1;
+				while(true) {
+					if(it.value() == r) {
+						c[r] *= preAgg[data.getIndex(it.getDataIndex())];
+						if(it.getDataIndex() < maxId)
+							it.next();
+						else {
+							r++;
+							break;
+						}
+					}
+					else
+						c[r] *= def;
+					r++;
+				}
+			}
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r] *= preAgg[data.getIndex(it.getDataIndex())];
+						it.next();
+					}
+					else
+						c[r] *= def;
+					r++;
+				}
+				indexes.cacheIterator(it, ru);
+			}
+		}
+
+		while(r < ru) {
+			c[r] *= def;
+			r++;
+		}
 	}
 
 	@Override
@@ -425,44 +476,42 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
-		CM_COV_Object ret = super.centralMoment(op, nRows);
-		int count = _numRows - _data.size();
-		op.fn.execute(ret, _defaultTuple[0], count);
-		return ret;
+		return _dict.centralMomentWithDefault(op.fn, getCounts(), _defaultTuple[0], nRows);
 	}
 
 	@Override
 	public AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows) {
 		ADictionary d = _dict.rexpandCols(max, ignore, cast, _colIndexes.length);
-		return rexpandCols(max, ignore, cast, nRows, d, _indexes, _data, getCachedCounts(), _defaultTuple[0]);
+		return rexpandCols(max, ignore, cast, nRows, d, _indexes, _data, getCachedCounts(), (int) _defaultTuple[0]);
 	}
 
 	protected static AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows, ADictionary d,
-		AOffset indexes, AMapToData data, int[] counts, double def) {
-		// final double def = _defaultTuple[0];
+		AOffset indexes, AMapToData data, int[] counts, int def) {
+
 		if(d == null) {
 			if(def <= 0 || def > max)
 				return ColGroupEmpty.create(max);
 			else {
 				double[] retDef = new double[max];
 				retDef[((int) def) - 1] = 1;
-				return ColGroupSDCSingle.create(Util.genColsIndices(max), nRows, new Dictionary(new double[max]), retDef,
+				return ColGroupSDCSingle.create(Util.genColsIndices(max), nRows, Dictionary.create(new double[max]), retDef,
 					indexes, null);
 			}
 		}
 		else {
+			final int[] outCols = Util.genColsIndices(max);
 			if(def <= 0) {
 				if(ignore)
-					return ColGroupSDCZeros.create(Util.genColsIndices(max), nRows, d, indexes, data, counts);
+					return ColGroupSDCZeros.create(outCols, nRows, d, indexes, data, counts);
 				else
 					throw new DMLRuntimeException("Invalid content of zero in rexpand");
 			}
 			else if(def > max)
-				return ColGroupSDCZeros.create(Util.genColsIndices(max), nRows, d, indexes, data, counts);
+				return ColGroupSDCZeros.create(outCols, nRows, d, indexes, data, counts);
 			else {
 				double[] retDef = new double[max];
 				retDef[((int) def) - 1] = 1;
-				return ColGroupSDC.create(Util.genColsIndices(max), nRows, d, retDef, indexes, data, counts);
+				return ColGroupSDC.create(outCols, nRows, d, retDef, indexes, data, counts);
 			}
 		}
 	}
@@ -494,11 +543,8 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 
 	@Override
 	public boolean containsValue(double pattern) {
-		if(pattern == 0 && _zeros)
+		if(_dict.containsValue(pattern))
 			return true;
-		boolean ret = _dict.containsValue(pattern);
-		if(ret == true)
-			return ret;
 		else {
 			for(double v : _defaultTuple)
 				if(v == pattern)
@@ -507,6 +553,16 @@ public class ColGroupSDC extends AMorphingMMColGroup {
 		}
 	}
 
+	@Override
+	public double[] getCommon() {
+		return _defaultTuple;
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplicationCommon(double[] common, int[] colIndexes, ADictionary preAgg) {
+		return create(colIndexes, _numRows, preAgg, common, _indexes, _data, getCachedCounts());
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
index 91b9ff22ae..ab5005f3c6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCFOR.java
@@ -24,7 +24,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-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.mapping.AMapToData;
@@ -54,13 +53,10 @@ import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
  * with no modifications.
  * 
  */
-public class ColGroupSDCFOR extends AMorphingMMColGroup {
+public class ColGroupSDCFOR extends ASDC {
 
 	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. */
 	protected AMapToData _data;
 
@@ -78,18 +74,17 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	private ColGroupSDCFOR(int[] colIndices, int numRows, ADictionary dict, AOffset indexes, AMapToData data,
 		int[] cachedCounts, double[] reference) {
-		super(colIndices, numRows, dict, cachedCounts);
+		super(colIndices, numRows, dict, indexes, cachedCounts);
 		if(data.getUnique() != dict.getNumberOfValues(colIndices.length))
 			throw new DMLCompressionException("Invalid construction of SDCZero group");
 		_data = data;
 		_indexes = indexes;
-		_zeros = false;
 		_reference = reference;
 	}
 
-	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
+	public static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
 		int[] cachedCounts, double[] reference) {
-		final boolean allZero = FORUtil.allZero(reference);
+		final boolean allZero = ColGroupUtils.allZero(reference);
 		if(allZero && dict == null)
 			return new ColGroupEmpty(colIndexes);
 		else if(dict == null)
@@ -100,6 +95,13 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 			return new ColGroupSDCFOR(colIndexes, numRows, dict, offsets, data, cachedCounts, reference);
 	}
 
+	public static AColGroup sparsifyFOR(ColGroupSDC g) {
+		// subtract default.
+		final double[] constV = ((ColGroupSDC) g)._defaultTuple;
+		final AColGroupValue clg = (AColGroupValue) g.subtractDefaultTuple();
+		return create(g.getColIndices(), g._numRows, clg._dict, g._indexes, g._data, g.getCachedCounts(), constV);
+	}
+
 	@Override
 	public CompressionType getCompType() {
 		return CompressionType.SDCFOR;
@@ -110,6 +112,11 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 		return ColGroupType.SDCFOR;
 	}
 
+	@Override
+	public  double[] getDefaultTuple(){
+		return _reference;
+	}
+
 	@Override
 	public int[] getCounts(int[] counts) {
 		return _data.getCounts(counts);
@@ -128,11 +135,9 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 	@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];
+		return _dict.getValue(_data.getIndex(it.getDataIndex()), colIdx, _colIndexes.length) + _reference[colIdx];
 	}
 
 	@Override
@@ -154,7 +159,7 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public AColGroup unaryOperation(UnaryOperator op) {
-		final double[] newRef = FORUtil.unaryOperator(op, _reference);
+		final double[] newRef = ColGroupUtils.unaryOperator(op, _reference);
 		final ADictionary newDict = _dict.applyUnaryOpWithReference(op, _reference, newRef);
 		return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), newRef);
 	}
@@ -236,21 +241,26 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public AColGroup replace(double pattern, double replace) {
+
+		final ADictionary newDict = _dict.replaceWithReference(pattern, replace, _reference);
 		boolean patternInReference = false;
 		for(double d : _reference)
 			if(pattern == d) {
 				patternInReference = true;
 				break;
 			}
-
 		if(patternInReference) {
-			throw new NotImplementedException("Not Implemented replace where a value in reference should be replaced");
-			// _dict.replace(pattern, replace, _reference, _newReplace);
+			double[] nRef = new double[_reference.length];
+			for(int i = 0; i < _reference.length; i++)
+				if(pattern == _reference[i])
+					nRef[i] = replace;
+				else
+					nRef[i] = _reference[i];
+
+			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), nRef);
 		}
-		else {
-			final ADictionary newDict = _dict.replaceWithReference(pattern, replace, _reference);
+		else
 			return create(_colIndexes, _numRows, newDict, _indexes, _data, getCachedCounts(), _reference);
-		}
 
 	}
 
@@ -269,7 +279,7 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 		// trick, use normal sum
 		super.computeSum(c, nRows);
 		// and add all sum of reference multiplied with nrows.
-		final double refSum = FORUtil.refSum(_reference);
+		final double refSum = ColGroupUtils.refSum(_reference);
 		c[0] += refSum * nRows;
 	}
 
@@ -286,14 +296,13 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 	protected void computeSumSq(double[] c, int nRows) {
 		// square sum the dictionary.
 		c[0] += _dict.sumSqWithReference(getCounts(), _reference);
-		final double refSum = FORUtil.refSumSq(_reference);
+		final double refSum = ColGroupUtils.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());
 	}
 
 	@Override
 	protected void computeColSumsSq(double[] c, int nRows) {
-		_dict = _dict.getMBDict(_colIndexes.length);
 		// square sum the dictionary
 		_dict.colSumSqWithReference(c, getCounts(), _colIndexes, _reference);
 		// Square sum of the reference values only for the rows that is not represented in the Offsets.
@@ -313,7 +322,7 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException();
+		return _dict.productAllRowsToDoubleWithReference(_reference);
 	}
 
 	@Override
@@ -329,12 +338,18 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException("Not Implemented PFOR");
+		ColGroupSDC.computeRowProduct(c, rl, ru, preAgg, _data, _indexes, _numRows);
 	}
 
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
-		throw new NotImplementedException("Not Implemented PFOR");
+		_dict.colProductWithReference(c, getCounts(), _colIndexes, _reference);
+		final int count = _numRows - _data.size();
+
+		for(int x = 0; x < _colIndexes.length; x++) {
+			double v = c[_colIndexes[x]];
+			c[_colIndexes[x]] = v != 0 ? v * Math.pow(_reference[x], count) : 0;
+		}
 	}
 
 	@Override
@@ -359,12 +374,16 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@Override
 	public boolean containsValue(double pattern) {
-		if(pattern == 0 && _zeros)
-			return true;
-		else if(Double.isNaN(pattern) || Double.isInfinite(pattern))
-			return FORUtil.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
-		else
+		if(Double.isNaN(pattern) || Double.isInfinite(pattern))
+			return ColGroupUtils.containsInfOrNan(pattern, _reference) || _dict.containsValue(pattern);
+		else {
+			// if the value is in reference then return true.
+			for(double v : _reference)
+				if(v == pattern)
+					return true;
+
 			return _dict.containsValueWithReference(pattern, _reference);
+		}
 	}
 
 	@Override
@@ -386,17 +405,15 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 
 	@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]);
+		ADictionary d = _dict.rexpandColsWithReference(max, ignore, cast, (int) _reference[0]);
+		return ColGroupSDC.rexpandCols(max, ignore, cast, nRows, d, _indexes, _data, getCachedCounts(),
+			(int) _reference[0]);
 	}
 
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
 		// should be guaranteed to be one column therefore only one reference value.
-		CM_COV_Object ret = _dict.centralMomentWithReference(op.fn, getCounts(), _reference[0], nRows);
-		int count = _numRows - _data.size();
-		op.fn.execute(ret, _reference[0], count);
-		return ret;
+		return _dict.centralMomentWithReference(op.fn, getCounts(), _reference[0], nRows);
 	}
 
 	@Override
@@ -407,6 +424,16 @@ public class ColGroupSDCFOR extends AMorphingMMColGroup {
 		return e.getCost(nRows, nRowsScanned, nCols, nVals, _dict.getSparsity());
 	}
 
+	@Override
+	public double[] getCommon() {
+		return _reference;
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplicationCommon(double[] common, int[] colIndexes, ADictionary preAgg) {
+		return create(colIndexes, _numRows, preAgg, _indexes, _data, getCachedCounts(), common);
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
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 21bf950ddc..e83401a957 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
@@ -30,6 +30,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.offset.AIterator;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
+import org.apache.sysds.runtime.compress.colgroup.offset.AOffsetIterator;
 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;
@@ -48,10 +49,9 @@ import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
  * This column group is handy in cases where sparse unsafe operations is executed on very sparse columns. Then the zeros
  * would be materialized in the group without any overhead.
  */
-public class ColGroupSDCSingle extends AMorphingMMColGroup {
+public class ColGroupSDCSingle extends ASDC {
 	private static final long serialVersionUID = 3883228464052204200L;
-	/** Sparse row indexes for the data */
-	protected AOffset _indexes;
+
 	/** The default value stored in this column group */
 	protected double[] _defaultTuple;
 
@@ -66,28 +66,56 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 
 	private ColGroupSDCSingle(int[] colIndices, int numRows, ADictionary dict, double[] defaultTuple, AOffset offsets,
 		int[] cachedCounts) {
-		super(colIndices, numRows, dict, cachedCounts);
-		_indexes = offsets;
-		_zeros = false;
+		super(colIndices, numRows, dict == null ? Dictionary.createNoCheck(new double[colIndices.length]) : dict, offsets,
+			cachedCounts);
 		_defaultTuple = defaultTuple;
-
-		if(_indexes == null)
-			throw new NullPointerException("null indexes is invalid for SDCSingle");
 	}
 
-	protected static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, double[] defaultTuple,
+	public static AColGroup create(int[] colIndexes, int numRows, ADictionary dict, double[] defaultTuple,
 		AOffset offsets, int[] cachedCounts) {
-		final boolean allZero = FORUtil.allZero(defaultTuple);
+		final boolean allZero = ColGroupUtils.allZero(defaultTuple);
 		if(dict == null && allZero)
 			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);
+			if(offsets.getSize() * 2 > numRows + 2) {
+				AOffset rev = reverse(numRows, offsets);
+				return ColGroupSDCSingleZeros.create(colIndexes, numRows, Dictionary.create(defaultTuple), rev,
+					cachedCounts);
+			}
+			else
+				return new ColGroupSDCSingle(colIndexes, numRows, null, defaultTuple, offsets, cachedCounts);
 		}
 		else if(allZero)
 			return ColGroupSDCSingleZeros.create(colIndexes, numRows, dict, offsets, cachedCounts);
-		else
-			return new ColGroupSDCSingle(colIndexes, numRows, dict, defaultTuple, offsets, cachedCounts);
+		else {
+			if(offsets.getSize() * 2.0 > numRows + 2.0) {
+				AOffset rev = reverse(numRows, offsets);
+				return new ColGroupSDCSingle(colIndexes, numRows, null, dict.getValues(), rev, null);
+			}
+			else
+				return new ColGroupSDCSingle(colIndexes, numRows, dict, defaultTuple, offsets, cachedCounts);
+		}
+	}
+
+	private static AOffset reverse(int numRows, AOffset offsets) {
+		int[] newOff = new int[numRows - offsets.getSize()];
+		final AOffsetIterator it = offsets.getOffsetIterator();
+		final int last = offsets.getOffsetToLast();
+		int i = 0;
+		int j = 0;
+
+		while(i < last) {
+			if(i == it.value()) {
+				i++;
+				it.next();
+			}
+			else
+				newOff[j++] = i++;
+		}
+		i++; // last
+		while(i < numRows)
+			newOff[j++] = i++;
+		return OffsetFactory.createOffset(newOff);
 	}
 
 	@Override
@@ -100,9 +128,14 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		return ColGroupType.SDCSingle;
 	}
 
+	@Override
+	public double[] getDefaultTuple() {
+		return _defaultTuple;
+	}
+
 	@Override
 	public double getIdx(int r, int colIdx) {
-		final AIterator it = _indexes.getIterator(r);
+		final AOffsetIterator it = _indexes.getOffsetIterator(r);
 		if(it == null || it.value() != r)
 			return _defaultTuple[colIdx];
 		else
@@ -127,7 +160,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 
 	@Override
 	protected double[] preAggProductRows() {
-		throw new NotImplementedException("Should implement preAgg with extra cell");
+		return _dict.productAllRowsToDoubleWithDefault(_defaultTuple);
 	}
 
 	@Override
@@ -160,40 +193,43 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		final AIterator it = indexes.getIterator(rl);
 		final double def = preAgg[1];
 		final double norm = preAgg[0];
-		if(it != null && it.value() > ru)
-			indexes.cacheIterator(it, ru);
-		else if(it != null && ru >= indexes.getOffsetToLast()) {
-			final int maxOff = indexes.getOffsetToLast();
-			while(true) {
-				if(it.value() == r) {
-					c[r] += norm;
-					if(it.value() < maxOff)
-						it.next();
-					else {
-						r++;
-						break;
+		if(it != null) {
+
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxOff = indexes.getOffsetToLast();
+				while(true) {
+					if(it.value() == r) {
+						c[r] += norm;
+						if(it.value() < maxOff)
+							it.next();
+						else {
+							r++;
+							break;
+						}
 					}
+					else
+						c[r] += def;
+					r++;
 				}
-				else
-					c[r] += def;
-				r++;
 			}
-		}
-		else if(it != null) {
-			while(r < ru) {
-				if(it.value() == r)
-					c[r] += norm;
-				else
-					c[r] += def;
-				r++;
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r++] += norm;
+						it.next();
+					}
+					else
+						c[r++] += def;
+				}
+				indexes.cacheIterator(it, ru);
 			}
-			indexes.cacheIterator(it, ru);
 		}
 
-		while(r < ru) {
-			c[r] += def;
-			r++;
-		}
+		while(r < ru)
+			c[r++] += def;
+
 	}
 
 	@Override
@@ -205,36 +241,39 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		double def, double norm) {
 		int r = rl;
 		final AIterator it = indexes.getIterator(rl);
-		if(it != null && it.value() > ru)
-			indexes.cacheIterator(it, ru);
-		else if(it != null && ru >= indexes.getOffsetToLast()) {
-			final int maxOff = indexes.getOffsetToLast();
-			while(true) {
-				if(it.value() == r) {
-					c[r] = builtin.execute(c[r], norm);
-					if(it.value() < maxOff)
-						it.next();
-					else {
-						r++;
-						break;
+		if(it != null) {
+
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxOff = indexes.getOffsetToLast();
+				while(true) {
+					if(it.value() == r) {
+						c[r] = builtin.execute(c[r], norm);
+						if(it.value() < maxOff)
+							it.next();
+						else {
+							r++;
+							break;
+						}
 					}
+					else
+						c[r] = builtin.execute(c[r], def);
+					r++;
 				}
-				else
-					c[r] = builtin.execute(c[r], def);
-				r++;
 			}
-		}
-		else if(it != null) {
-			while(r < ru) {
-				if(it.value() == r) {
-					c[r] = builtin.execute(c[r], norm);
-					it.next();
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r] = builtin.execute(c[r], norm);
+						it.next();
+					}
+					else
+						c[r] = builtin.execute(c[r], def);
+					r++;
 				}
-				else
-					c[r] = builtin.execute(c[r], def);
-				r++;
+				indexes.cacheIterator(it, ru);
 			}
-			indexes.cacheIterator(it, ru);
 		}
 
 		while(r < ru) {
@@ -243,10 +282,56 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		}
 	}
 
+	@Override
+	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		computeRowProduct(c, rl, ru, _indexes, _numRows, preAgg[1], preAgg[0]);
+	}
+
+	protected static final void computeRowProduct(double[] c, int rl, int ru, AOffset indexes, int nRows, double def,
+		double norm) {
+		int r = rl;
+		final AIterator it = indexes.getIterator(rl);
+		if(it != null) {
+			if(it.value() > ru)
+				indexes.cacheIterator(it, ru);
+			else if(ru > indexes.getOffsetToLast()) {
+				final int maxOff = indexes.getOffsetToLast();
+				while(true) {
+					if(it.value() == r) {
+						c[r] *= norm;
+						if(it.value() < maxOff)
+							it.next();
+						else {
+							r++;
+							break;
+						}
+					}
+					else
+						c[r] *= def;
+					r++;
+				}
+			}
+			else {
+				while(r < ru) {
+					if(it.value() == r) {
+						c[r++] *= norm;
+						it.next();
+					}
+					else
+						c[r++] *= def;
+				}
+				indexes.cacheIterator(it, ru);
+			}
+		}
+
+		while(r < ru)
+			c[r++] *= def;
+	}
+
 	@Override
 	protected void computeSum(double[] c, int nRows) {
 		super.computeSum(c, nRows);
-		int count = _numRows - getCounts()[0];
+		int count = _numRows - _indexes.getSize();
 		for(int x = 0; x < _defaultTuple.length; x++)
 			c[0] += _defaultTuple[x] * count;
 	}
@@ -254,7 +339,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 	@Override
 	public void computeColSums(double[] c, int nRows) {
 		super.computeColSums(c, nRows);
-		int count = _numRows - getCounts()[0];
+		int count = _numRows - _indexes.getSize();
 		for(int x = 0; x < _colIndexes.length; x++)
 			c[_colIndexes[x]] += _defaultTuple[x] * count;
 	}
@@ -262,7 +347,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 	@Override
 	protected void computeSumSq(double[] c, int nRows) {
 		super.computeSumSq(c, nRows);
-		int count = _numRows - getCounts()[0];
+		int count = _numRows - _indexes.getSize();
 		for(int x = 0; x < _colIndexes.length; x++)
 			c[0] += _defaultTuple[x] * _defaultTuple[x] * count;
 	}
@@ -270,28 +355,25 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 	@Override
 	protected void computeColSumsSq(double[] c, int nRows) {
 		super.computeColSumsSq(c, nRows);
-		int count = _numRows - getCounts()[0];
+		int count = _numRows - _indexes.getSize();
 		for(int x = 0; x < _colIndexes.length; x++)
 			c[_colIndexes[x]] += _defaultTuple[x] * _defaultTuple[x] * count;
 	}
 
 	@Override
 	protected void computeProduct(double[] c, int nRows) {
-		final int count = _numRows - getCounts()[0];
+		final int count = _numRows - _indexes.getSize();
 		_dict.productWithDefault(c, getCounts(), _defaultTuple, count);
 	}
 
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
 		super.computeColProduct(c, nRows);
-		int count = _numRows - getCounts()[0];
-		for(int x = 0; x < _colIndexes.length; x++)
-			c[_colIndexes[x]] *= Math.pow(_defaultTuple[x], count);
-	}
-
-	@Override
-	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		throw new NotImplementedException();
+		int count = _numRows - _indexes.getSize();
+		for(int x = 0; x < _colIndexes.length; x++) {
+			double v = c[_colIndexes[x]];
+			c[_colIndexes[x]] = v != 0 ? v * Math.pow(_defaultTuple[x], count) : 0;
+		}
 	}
 
 	@Override
@@ -341,7 +423,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		for(int i = 0; i < _defaultTuple.length; i++)
 			newDefaultTuple[i] = op.fn.execute(_defaultTuple[i], v[_colIndexes[i]]);
 		final ADictionary newDict = _dict.binOpRight(op, v, _colIndexes);
-		return new ColGroupSDCSingle(_colIndexes, _numRows, newDict, newDefaultTuple, _indexes, getCachedCounts());
+		return create(_colIndexes, _numRows, newDict, newDefaultTuple, _indexes, getCachedCounts());
 	}
 
 	@Override
@@ -350,7 +432,6 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		_indexes.write(out);
 		for(double d : _defaultTuple)
 			out.writeDouble(d);
-
 	}
 
 	@Override
@@ -392,7 +473,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 	@Override
 	public long getNumberNonZeros(int nRows) {
 		long nnz = super.getNumberNonZeros(nRows);
-		final int count = _numRows - getCounts()[0];
+		final int count = _numRows - _indexes.getSize();
 		for(int x = 0; x < _colIndexes.length; x++)
 			nnz += _defaultTuple[x] != 0 ? count : 0;
 		return nnz;
@@ -400,24 +481,20 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 
 	@Override
 	public CM_COV_Object centralMoment(CMOperator op, int nRows) {
-		CM_COV_Object ret = super.centralMoment(op, nRows);
-		int count = _numRows - getCounts()[0];
-		op.fn.execute(ret, _defaultTuple[0], count);
-		return ret;
+		return _dict.centralMomentWithDefault(op.fn, getCounts(), _defaultTuple[0], nRows);
 	}
 
 	@Override
 	public AColGroup rexpandCols(int max, boolean ignore, boolean cast, int nRows) {
 		ADictionary d = _dict.rexpandCols(max, ignore, cast, _colIndexes.length);
-		final double def = _defaultTuple[0];
+		final int def = (int) _defaultTuple[0];
 		if(d == null) {
 			if(def <= 0 || def > max)
 				return ColGroupEmpty.create(max);
 			else {
 				double[] retDef = new double[max];
 				retDef[((int) _defaultTuple[0]) - 1] = 1;
-				return new ColGroupSDCSingle(Util.genColsIndices(max), nRows, new Dictionary(new double[max]), retDef,
-					_indexes, null);
+				return ColGroupSDCSingle.create(Util.genColsIndices(max), nRows, null, retDef, _indexes, null);
 			}
 		}
 		else {
@@ -432,7 +509,7 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 			else {
 				double[] retDef = new double[max];
 				retDef[((int) _defaultTuple[0]) - 1] = 1;
-				return new ColGroupSDCSingle(Util.genColsIndices(max), nRows, d, retDef, _indexes, getCachedCounts());
+				return ColGroupSDCSingle.create(Util.genColsIndices(max), nRows, d, retDef, _indexes, getCachedCounts());
 			}
 		}
 	}
@@ -441,34 +518,41 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 	public double getCost(ComputationCostEstimator e, int nRows) {
 		final int nVals = getNumValues();
 		final int nCols = getNumCols();
-		final int nRowsScanned = getCounts()[0];
+		final int nRowsScanned = _indexes.getSize();
 		return e.getCost(nRows, nRowsScanned, nCols, nVals, _dict.getSparsity());
 	}
 
 	@Override
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
-		ColGroupSDCSingle ret = (ColGroupSDCSingle) super.sliceMultiColumns(idStart, idEnd, outputCols);
-		ret._defaultTuple = new double[idEnd - idStart];
+		final AColGroup ret = super.sliceMultiColumns(idStart, idEnd, outputCols);
+		final double[] defTuple = new double[idEnd - idStart];
 		for(int i = idStart, j = 0; i < idEnd; i++, j++)
-			ret._defaultTuple[j] = _defaultTuple[i];
-		return ret;
+			defTuple[j] = _defaultTuple[i];
+		if(ret instanceof ColGroupEmpty)
+			return create(ret._colIndexes, _numRows, null, defTuple, _indexes, null);
+		else {
+			ColGroupSDCSingle retSDC = (ColGroupSDCSingle) ret;
+			return create(retSDC._colIndexes, _numRows, retSDC._dict, defTuple, _indexes, null);
+		}
 	}
 
 	@Override
 	protected AColGroup sliceSingleColumn(int idx) {
-		ColGroupSDCSingle ret = (ColGroupSDCSingle) super.sliceSingleColumn(idx);
-		ret._defaultTuple = new double[1];
-		ret._defaultTuple[0] = _defaultTuple[idx];
-		return ret;
+		AColGroup ret = super.sliceSingleColumn(idx);
+		double[] defTuple = new double[1];
+		defTuple[0] = _defaultTuple[idx];
+		if(ret instanceof ColGroupEmpty)
+			return create(ret._colIndexes, _numRows, null, defTuple, _indexes, null);
+		else {
+			ColGroupSDCSingle retSDC = (ColGroupSDCSingle) ret;
+			return create(retSDC._colIndexes, _numRows, retSDC._dict, defTuple, _indexes, null);
+		}
 	}
 
 	@Override
 	public boolean containsValue(double pattern) {
-		if(pattern == 0 && _zeros)
+		if(_dict.containsValue(pattern))
 			return true;
-		boolean ret = _dict.containsValue(pattern);
-		if(ret == true)
-			return ret;
 		else {
 			for(double v : _defaultTuple)
 				if(v == pattern)
@@ -477,6 +561,16 @@ public class ColGroupSDCSingle extends AMorphingMMColGroup {
 		}
 	}
 
+	@Override
+	public double[] getCommon() {
+		return _defaultTuple;
+	}
+
+	@Override
+	protected AColGroup allocateRightMultiplicationCommon(double[] common, int[] colIndexes, ADictionary preAgg) {
+		return create(colIndexes, _numRows, preAgg, common, _indexes, getCachedCounts());
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
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 72259d25f1..139b433c0c 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
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 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.offset.AIterator;
@@ -63,10 +64,12 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 	private ColGroupSDCSingleZeros(int[] colIndices, int numRows, ADictionary dict, AOffset offsets,
 		int[] cachedCounts) {
 		super(colIndices, numRows, dict, offsets, cachedCounts);
-		_zeros = true;
+		if(offsets.getSize() * 2 > numRows + 2)
+			throw new DMLCompressionException("Wrong direction of SDCSingleZero compression should be other way " + numRows
+				+ " vs " + _indexes + "\n" + this);
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset offsets,
+	public static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset offsets,
 		int[] cachedCounts) {
 		if(dict == null)
 			return new ColGroupEmpty(colIndices);
@@ -92,13 +95,15 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else
+		else{
 			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, values, it);
+			_indexes.cacheIterator(it, ru);
+		}
 	}
 
 	@Override
-	public void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC, double[] values,
-		AIterator it) {
+	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)
 			return;
@@ -106,10 +111,9 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 			decompressToDenseBlockDenseDictionaryPost(db, rl, ru, offR, offC, values, it);
 		else {
 			if(_colIndexes.length == 1 && db.getDim(1) == 1)
-				decompressToDenseBlockDenseDictionaryPreSingleColOutContiguous(db, rl, ru, offR, offC, values, it);
+				decompressToDenseBlockDenseDictionaryPreSingleColOutContiguous(db, rl, ru, offR, offC, values[0], it);
 			else
 				decompressToDenseBlockDenseDictionaryPre(db, rl, ru, offR, offC, values, it);
-			_indexes.cacheIterator(it, ru);
 		}
 	}
 
@@ -134,11 +138,8 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 	}
 
 	private void decompressToDenseBlockDenseDictionaryPreSingleColOutContiguous(DenseBlock db, int rl, int ru, int offR,
-		int offC, double[] values, AIterator it) {
-		// final int nCol = _colIndexes.length;
+		int offC, double v, AIterator it) {
 		final double[] c = db.values(0);
-		// final int off = db.pos(row);
-		final double v = values[0];
 		int r = it.value();
 		while(r < ru) {
 			c[offR + r] += v;
@@ -169,36 +170,74 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else if(ru > _indexes.getOffsetToLast()) {
-			throw new NotImplementedException();
+
+		final int last = _indexes.getOffsetToLast();
+		if(ru > last)
+			decompressToDenseBlockSparseDictionaryPost(db, rl, ru, offR, offC, sb, it, last);
+		else
+			decompressToDenseBlockSparseDictionaryPre(db, rl, ru, offR, offC, sb, it);
+
+	}
+
+	private final void decompressToDenseBlockSparseDictionaryPost(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb, AIterator it, int last) {
+		final int apos = sb.pos(0);
+		final int alen = sb.size(0) + apos;
+		final double[] avals = sb.values(0);
+		final int[] aix = sb.indexes(0);
+		while(true) {
+			final int idx = offR + it.value();
+			final double[] c = db.values(idx);
+
+			final int off = db.pos(idx) + offC;
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]]] += avals[j];
+			if(it.value() == last)
+				return;
+			it.next();
 		}
-		else {
-			final int apos = sb.pos(0);
-			final int alen = sb.size(0) + apos;
-			final int[] aix = sb.indexes(0);
-			final double[] avals = sb.values(0);
-			while(it.isNotOver(ru)) {
-				final int row = offR + it.value();
-				final double[] c = db.values(row);
-				final int off = db.pos(row);
-				for(int j = apos; j < alen; j++)
-					c[off + _colIndexes[aix[j]] + offC] += avals[j];
-				it.next();
-			}
-			_indexes.cacheIterator(it, ru);
+	}
+
+	private final void decompressToDenseBlockSparseDictionaryPre(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb, AIterator it) {
+		final int apos = sb.pos(0);
+		final int alen = sb.size(0) + apos;
+		final int[] aix = sb.indexes(0);
+		final double[] avals = sb.values(0);
+		while(it.isNotOver(ru)) {
+			final int row = offR + it.value();
+			final double[] c = db.values(row);
+			final int off = db.pos(row);
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]] + offC] += avals[j];
+			it.next();
 		}
+		_indexes.cacheIterator(it, ru);
 	}
 
 	@Override
 	protected void decompressToSparseBlockSparseDictionary(SparseBlock ret, int rl, int ru, int offR, int offC,
 		SparseBlock sb) {
 		final AIterator it = _indexes.getIterator(rl);
+		final int last = _indexes.getOffsetToLast();
 		if(it == null)
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else if(ru > _indexes.getOffsetToLast()) {
-			throw new NotImplementedException();
+		else if(ru > last) {
+			final int apos = sb.pos(0);
+			final int alen = sb.size(0) + apos;
+			final int[] aix = sb.indexes(0);
+			final double[] avals = sb.values(0);
+			while(it.value() < last) {
+				final int row = offR + it.value();
+				for(int j = apos; j < alen; j++)
+					ret.append(row, _colIndexes[aix[j]] + offC, avals[j]);
+				it.next();
+			}
+			final int row = offR + it.value();
+			for(int j = apos; j < alen; j++)
+				ret.append(row, _colIndexes[aix[j]] + offC, avals[j]);
 		}
 		else {
 			final int apos = sb.pos(0);
@@ -292,6 +331,11 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 		ColGroupSDCSingle.computeRowMxx(c, builtin, rl, ru, _indexes, _numRows, 0, preAgg[0]);
 	}
 
+	@Override
+	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		ColGroupSDCSingle.computeRowProduct(c, rl, ru, _indexes, _numRows, 0, preAgg[0]);
+	}
+
 	@Override
 	public int[] getCounts(int[] counts) {
 		counts[0] = _indexes.getSize();
@@ -305,6 +349,8 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 
 	@Override
 	public void preAggregateDense(MatrixBlock m, double[] preAgg, int rl, int ru, int cl, int cu) {
+		if(!m.getDenseBlock().isContiguous())
+			throw new NotImplementedException("Not implemented support for preAggregate non contiguous dense matrix");
 		final AIterator it = _indexes.getIterator(cl);
 		final double[] vals = m.getDenseBlockValues();
 		final int nCol = m.getNumColumns();
@@ -394,6 +440,8 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 
 		while(i < last) { // while we are not done iterating
 			for(int r = rl; r < ru; r++) {
+				if(sb.isEmpty(r))
+					continue;
 				final int off = r - rl;
 				int apos = aOffs[off]; // current offset
 				final int alen = sb.size(r) + sb.pos(r);
@@ -410,6 +458,8 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 
 		// process final element
 		for(int r = rl; r < ru; r++) {
+			if(sb.isEmpty(r))
+				continue;
 			final int off = r - rl;
 			int apos = aOffs[off];
 			final int alen = sb.size(r) + sb.pos(r);
@@ -434,10 +484,10 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		final double val0 = op.executeScalar(0);
-		final boolean isSparseSafeOp = op.sparseSafe || val0 == 0;
+		final boolean isSparseSafeOp = val0 == 0;
 		final ADictionary nDict = _dict.applyScalarOp(op);
 		if(isSparseSafeOp)
-			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, nDict, _indexes, getCachedCounts());
+			return create(_colIndexes, _numRows, nDict, _indexes, getCachedCounts());
 		else {
 			final double[] defaultTuple = new double[_colIndexes.length];
 			Arrays.fill(defaultTuple, val0);
@@ -458,6 +508,25 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 		}
 	}
 
+	@Override
+	protected double computeMxx(double c, Builtin builtin) {
+		c = builtin.execute(c, 0);
+		return _dict.aggregate(c, builtin);
+	}
+
+	@Override
+	protected void computeColMxx(double[] c, Builtin builtin) {
+		for(int x = 0; x < _colIndexes.length; x++)
+			c[_colIndexes[x]] = builtin.execute(c[_colIndexes[x]], 0);
+
+		_dict.aggregateCols(c, builtin, _colIndexes);
+	}
+
+	@Override
+	public boolean containsValue(double pattern) {
+		return (pattern == 0) || _dict.containsValue(pattern);
+	}
+
 	@Override
 	public AColGroup binaryRowOpLeft(BinaryOperator op, double[] v, boolean isRowSafe) {
 		if(isRowSafe) {
@@ -477,7 +546,7 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 	public AColGroup binaryRowOpRight(BinaryOperator op, double[] v, boolean isRowSafe) {
 		if(isRowSafe) {
 			ADictionary ret = _dict.binOpRight(op, v, _colIndexes);
-			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, ret, _indexes, getCachedCounts());
+			return ColGroupSDCSingleZeros.create(_colIndexes, _numRows, ret, _indexes, getCachedCounts());
 		}
 		else {
 			ADictionary newDict = _dict.binOpRight(op, v, _colIndexes);
@@ -655,6 +724,35 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 
 	}
 
+	@Override
+	protected void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret) {
+		final int finalOff = _indexes.getOffsetToLast();
+		final double[] v = ret.getValues();
+		final int nv = that.getNumValues();
+		final int nCol = that._colIndexes.length;
+		for(int k = 0; k < nv; k++) {
+			final AOffsetIterator itThis = _indexes.getOffsetIterator();
+			final int blen = that._ptr[k + 1];
+			for(int apos = that._ptr[k], rs = 0, re = 0; apos < blen; apos += 2) {
+				rs = re + that._data[apos];
+				re = rs + that._data[apos + 1];
+				// if index is later than run continue
+				if(itThis.value() >= re || rs == re || rs > finalOff)
+					continue;
+				// while lower than run iterate through
+				while(itThis.value() < rs && itThis.value() != finalOff)
+					itThis.next();
+				// process inside run
+				for(int rix = itThis.value(); rix < re; rix = itThis.value()) { // nice skip inside runs
+					that._dict.addToEntry(v, k, 0, nCol);
+					if(itThis.value() == finalOff) // break if final.
+						break;
+					itThis.next();
+				}
+			}
+		}
+	}
+
 	@Override
 	public int getPreAggregateSize() {
 		return 1;
@@ -677,16 +775,10 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 		c[0] = 0;
 	}
 
-	@Override
-	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		for(int i = 0; i < c.length; i++)
-			c[i] = 0;
-	}
-
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
-		for(int i = 0; i < c.length; i++)
-			c[i] = 0;
+		for(int i = 0; i < _colIndexes.length; i++)
+			c[_colIndexes[i]] = 0;
 	}
 
 	@Override
@@ -698,13 +790,16 @@ public class ColGroupSDCSingleZeros extends ASDCZero {
 	}
 
 	@Override
-	protected int getIndexesSize() {
+	protected int numRowsToMultiply() {
 		return getCounts()[0];
 	}
 
 	@Override
-	protected int numRowsToMultiply() {
-		return getCounts()[0];
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		if(colIndexes != null && preAgg != null)
+			return create(colIndexes, _numRows, preAgg, _indexes, getCachedCounts());
+		else
+			return null;
 	}
 
 	@Override
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 52ee9efc52..6e3b399ccb 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
@@ -75,10 +75,9 @@ public class ColGroupSDCZeros extends ASDCZero {
 			throw new DMLCompressionException("Invalid construction of SDCZero group: number uniques: " + data.getUnique()
 				+ " vs." + dict.getNumberOfValues(colIndices.length));
 		_data = data;
-		_zeros = true;
 	}
 
-	protected static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
+	public static AColGroup create(int[] colIndices, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
 		int[] cachedCounts) {
 		if(dict == null)
 			return new ColGroupEmpty(colIndices);
@@ -105,15 +104,15 @@ public class ColGroupSDCZeros extends ASDCZero {
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else{
+		else {
 			decompressToDenseBlockDenseDictionaryWithProvidedIterator(db, rl, ru, offR, offC, values, it);
 			_indexes.cacheIterator(it, ru);
 		}
 	}
 
 	@Override
-	public final void decompressToDenseBlockDenseDictionaryWithProvidedIterator(DenseBlock db, int rl, int ru, int offR, int offC,
-		double[] values, AIterator it) {
+	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)
 			return;
@@ -239,55 +238,65 @@ public class ColGroupSDCZeros extends ASDCZero {
 			return;
 		else if(it.value() >= ru)
 			_indexes.cacheIterator(it, ru);
-		else if(ru > _indexes.getOffsetToLast()) {
-			final int lastOff = _indexes.getOffsetToLast();
-			while(true) {
-				final int idx = offR + it.value();
-				final double[] c = db.values(idx);
-				final int dx = it.getDataIndex();
-				final int dictIndex = _data.getIndex(dx);
-				if(sb.isEmpty(dictIndex)) {
-					if(it.value() == lastOff)
-						return;
-					it.next();
-					continue;
-				}
 
-				final int off = db.pos(idx) + offC;
-				final int apos = sb.pos(dictIndex);
-				final int alen = sb.size(dictIndex) + apos;
-				final double[] avals = sb.values(dictIndex);
-				final int[] aix = sb.indexes(dictIndex);
-				for(int j = apos; j < alen; j++)
-					c[off + _colIndexes[aix[j]]] += avals[j];
-				if(it.value() == lastOff)
+		final int last = _indexes.getOffsetToLast();
+		if(ru > last)
+			decompressToDenseBlockSparseDictionaryPost(db, rl, ru, offR, offC, sb, it, last);
+		else
+			decompressToDenseBlockSparseDictionaryPre(db, rl, ru, offR, offC, sb, it);
+
+	}
+
+	private final void decompressToDenseBlockSparseDictionaryPost(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb, AIterator it, int last) {
+		while(true) {
+			final int idx = offR + it.value();
+			final double[] c = db.values(idx);
+			final int dx = it.getDataIndex();
+			final int dictIndex = _data.getIndex(dx);
+			if(sb.isEmpty(dictIndex)) {
+				if(it.value() == last)
 					return;
 				it.next();
+				continue;
 			}
-		}
-		else {
-			while(it.isNotOver(ru)) {
-				final int idx = offR + it.value();
-				final int dx = it.getDataIndex();
-				final int dictIndex = _data.getIndex(dx);
-				if(sb.isEmpty(dictIndex)) {
-					it.next();
-					continue;
-				}
 
-				final double[] c = db.values(idx);
-				final int off = db.pos(idx) + offC;
-				final int apos = sb.pos(dictIndex);
-				final int alen = sb.size(dictIndex) + apos;
-				final double[] avals = sb.values(dictIndex);
-				final int[] aix = sb.indexes(dictIndex);
-				for(int j = apos; j < alen; j++)
-					c[off + _colIndexes[aix[j]]] += avals[j];
+			final int off = db.pos(idx) + offC;
+			final int apos = sb.pos(dictIndex);
+			final int alen = sb.size(dictIndex) + apos;
+			final double[] avals = sb.values(dictIndex);
+			final int[] aix = sb.indexes(dictIndex);
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]]] += avals[j];
+			if(it.value() == last)
+				return;
+			it.next();
+		}
+	}
 
+	private final void decompressToDenseBlockSparseDictionaryPre(DenseBlock db, int rl, int ru, int offR, int offC,
+		SparseBlock sb, AIterator it) {
+		while(it.isNotOver(ru)) {
+			final int idx = offR + it.value();
+			final int dx = it.getDataIndex();
+			final int dictIndex = _data.getIndex(dx);
+			if(sb.isEmpty(dictIndex)) {
 				it.next();
+				continue;
 			}
-			_indexes.cacheIterator(it, ru);
+
+			final double[] c = db.values(idx);
+			final int off = db.pos(idx) + offC;
+			final int apos = sb.pos(dictIndex);
+			final int alen = sb.size(dictIndex) + apos;
+			final double[] avals = sb.values(dictIndex);
+			final int[] aix = sb.indexes(dictIndex);
+			for(int j = apos; j < alen; j++)
+				c[off + _colIndexes[aix[j]]] += avals[j];
+
+			it.next();
 		}
+		_indexes.cacheIterator(it, ru);
 	}
 
 	@Override
@@ -392,11 +401,21 @@ public class ColGroupSDCZeros extends ASDCZero {
 		return _dict.getValue(_data.getIndex(it.getDataIndex()) * nCol + colIdx);
 	}
 
+	@Override
+	protected double[] preAggProductRows() {
+		return _dict.productAllRowsToDoubleWithDefault(new double[_colIndexes.length]);
+	}
+
 	@Override
 	protected void computeRowSums(double[] c, int rl, int ru, double[] preAgg) {
 		computeRowSums(c, rl, ru, preAgg, _data, _indexes, _numRows);
 	}
 
+	@Override
+	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
+		ColGroupSDC.computeRowProduct(c, rl, ru, preAgg, _data, _indexes, _numRows);
+	}
+
 	protected static final void computeRowSums(double[] c, int rl, int ru, double[] preAgg, AMapToData data,
 		AOffset indexes, int nRows) {
 		final AIterator it = indexes.getIterator(rl);
@@ -404,7 +423,7 @@ public class ColGroupSDCZeros extends ASDCZero {
 			return;
 		else if(it.value() > ru)
 			indexes.cacheIterator(it, ru);
-		else if(ru >= indexes.getOffsetToLast()) {
+		else if(ru > indexes.getOffsetToLast()) {
 			final int maxId = data.size() - 1;
 			c[it.value()] += preAgg[data.getIndex(it.getDataIndex())];
 			while(it.getDataIndex() < maxId) {
@@ -462,12 +481,12 @@ public class ColGroupSDCZeros extends ASDCZero {
 		if(isSparseSafeOp)
 			return create(_colIndexes, _numRows, _dict.applyScalarOp(op), _indexes, _data, getCachedCounts());
 		else if(op.fn instanceof Plus || (op.fn instanceof Minus && op instanceof RightScalarOperator)) {
-			final double[] reference = FORUtil.createReference(_colIndexes.length, val0);
+			final double[] reference = ColGroupUtils.createReference(_colIndexes.length, val0);
 			return ColGroupSDCFOR.create(_colIndexes, _numRows, _dict, _indexes, _data, getCachedCounts(), reference);
 		}
 		else {
 			final ADictionary newDict = _dict.applyScalarOp(op);
-			final double[] defaultTuple = FORUtil.createReference(_colIndexes.length, val0);
+			final double[] defaultTuple = ColGroupUtils.createReference(_colIndexes.length, val0);
 			return ColGroupSDC.create(_colIndexes, _numRows, newDict, defaultTuple, _indexes, _data, getCachedCounts());
 		}
 	}
@@ -600,6 +619,35 @@ public class ColGroupSDCZeros extends ASDCZero {
 		}
 	}
 
+	@Override
+	protected void preAggregateThatRLEStructure(ColGroupRLE that, Dictionary ret) {
+		final int finalOff = _indexes.getOffsetToLast();
+		final double[] v = ret.getValues();
+		final int nv = that.getNumValues();
+		final int nCol = that._colIndexes.length;
+		for(int k = 0; k < nv; k++) {
+			final AIterator itThis = _indexes.getIterator();
+			final int blen = that._ptr[k + 1];
+			for(int apos = that._ptr[k], rs = 0, re = 0; apos < blen; apos += 2) {
+				rs = re + that._data[apos];
+				re = rs + that._data[apos + 1];
+				// if index is later than run continue
+				if(itThis.value() >= re || rs == re || rs > finalOff)
+					continue;
+				// while lower than run iterate through
+				while(itThis.value() < rs && itThis.value() != finalOff)
+					itThis.next();
+				// process inside run
+				for(int rix = itThis.value(); rix < re; rix = itThis.value()) { // nice skip inside runs
+					that._dict.addToEntry(v, k, _data.getIndex(itThis.getDataIndex()), nCol);
+					if(itThis.value() == finalOff) // break if final.
+						break;
+					itThis.next();
+				}
+			}
+		}
+	}
+
 	@Override
 	public AColGroup replace(double pattern, double replace) {
 		ADictionary replaced = _dict.replace(pattern, replace, _colIndexes.length);
@@ -618,16 +666,10 @@ public class ColGroupSDCZeros extends ASDCZero {
 		c[0] = 0;
 	}
 
-	@Override
-	protected void computeRowProduct(double[] c, int rl, int ru, double[] preAgg) {
-		for(int i = 0; i < c.length; i++)
-			c[i] = 0;
-	}
-
 	@Override
 	protected void computeColProduct(double[] c, int nRows) {
-		for(int i = 0; i < c.length; i++)
-			c[i] = 0;
+		for(int i = 0; i < _colIndexes.length; i++)
+			c[_colIndexes[i]] = 0;
 	}
 
 	@Override
@@ -639,13 +681,35 @@ public class ColGroupSDCZeros extends ASDCZero {
 	}
 
 	@Override
-	protected int getIndexesSize() {
+	protected int numRowsToMultiply() {
 		return _data.size();
 	}
 
 	@Override
-	protected int numRowsToMultiply() {
-		return _data.size();
+	protected AColGroup allocateRightMultiplication(MatrixBlock right, int[] colIndexes, ADictionary preAgg) {
+		if(colIndexes != null && preAgg != null)
+			return create(colIndexes, _numRows, preAgg, _indexes, _data, getCachedCounts());
+		else
+			return null;
+	}
+
+	@Override
+	protected double computeMxx(double c, Builtin builtin) {
+		c = builtin.execute(c, 0);
+		return _dict.aggregate(c, builtin);
+	}
+
+	@Override
+	protected void computeColMxx(double[] c, Builtin builtin) {
+		for(int x = 0; x < _colIndexes.length; x++)
+			c[_colIndexes[x]] = builtin.execute(c[_colIndexes[x]], 0);
+
+		_dict.aggregateCols(c, builtin, _colIndexes);
+	}
+
+	@Override
+	public boolean containsValue(double pattern) {
+		return (pattern == 0) || _dict.containsValue(pattern);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSizes.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSizes.java
index 5f273e9242..9ba5c5df71 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSizes.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSizes.java
@@ -28,8 +28,8 @@ import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
-public final class ColGroupSizes {
-	protected static final Log LOG = LogFactory.getLog(ColGroupSizes.class.getName());
+public interface ColGroupSizes {
+	static final Log LOG = LogFactory.getLog(ColGroupSizes.class.getName());
 
 	public static long estimateInMemorySizeGroup(int nrColumns) {
 		long size = 16; // Object header
@@ -40,12 +40,8 @@ public final class ColGroupSizes {
 	public static long estimateInMemorySizeGroupValue(int nrColumns, int nrValues, double tupleSparsity, boolean lossy) {
 		long size = estimateInMemorySizeGroup(nrColumns);
 		size += 8; // Counts reference
-		size += 4; // Int nRows
-		size += 1; // _zeros boolean reference
-		size += 1; // _lossy boolean reference
-		size += 2; // padding
-		size += DictionaryFactory.getInMemorySize(nrValues, nrColumns, tupleSparsity, lossy);
 		size += 8; // Reference to Dict.
+		size += DictionaryFactory.getInMemorySize(nrValues, nrColumns, tupleSparsity, lossy);
 		return size;
 	}
 
@@ -59,6 +55,9 @@ public final class ColGroupSizes {
 	public static long estimateInMemorySizeOffset(int nrColumns, int nrValues, int pointers, int offsetLength,
 		double tupleSparsity, boolean lossy) {
 		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, tupleSparsity, lossy);
+		size += 4; // Int nRows
+		size += 1; // _zeros boolean reference
+		size += 3; // padding
 		size += MemoryEstimates.intArrayCost(pointers);
 		size += MemoryEstimates.charArrayCost(offsetLength);
 		return size;
@@ -74,9 +73,16 @@ public final class ColGroupSizes {
 
 	public static long estimateInMemorySizeRLE(int nrColumns, int nrValues, int nrRuns, int nrRows, double tupleSparsity,
 		boolean lossy) {
-		int offsetLength = (nrRuns) * 2;
-		long size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues) + 1, offsetLength, tupleSparsity, lossy);
-		return size;
+		// Correct low number of runs if very large input.
+		// This correction handles the case where the skip runs are added in a safe manner
+		if(nrRows > Character.MAX_VALUE ) {
+
+			final double extra = (double)nrRows / Character.MAX_VALUE;
+			// we assume that half unique values contain extra runs if we have few runs to begin with.
+			// This is not 100% guaranteeing larger estimate than real but most likely 
+			nrRuns += (extra / 2) * nrValues;
+		}
+		return estimateInMemorySizeOffset(nrColumns, nrValues, nrValues + 1, nrRuns * 2, tupleSparsity, lossy);
 	}
 
 	public static long estimateInMemorySizeSDC(int nrColumns, int nrValues, int nrRows, int largestOff,
@@ -112,7 +118,8 @@ public final class ColGroupSizes {
 		long size = 0;
 		// Since the Object is a col group the overhead from the Memory Size group is added
 		size += estimateInMemorySizeGroup(nrColumns);
-		size += MemoryEstimates.doubleArrayCost(2L * nrColumns); // coefficients; per column, we store 2 doubles (slope & intercept)
+		size += MemoryEstimates.doubleArrayCost(2L * nrColumns); // coefficients; per column, we store 2 doubles (slope &
+																					// intercept)
 		size += 4; // _numRows
 		return size;
 	}
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 0d78b95dc1..097ee43183 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
@@ -25,16 +25,23 @@ 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.dictionary.DictLibMatrixMult;
+import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.cost.ComputationCostEstimator;
 import org.apache.sysds.runtime.compress.utils.Util;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 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.Builtin.BuiltinCode;
+import org.apache.sysds.runtime.functionobjects.CM;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.functionobjects.ReduceRow;
+import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
@@ -69,7 +76,7 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	protected static AColGroup create(MatrixBlock mb, int[] colIndexes) {
-		if(mb.isEmpty())
+		if(mb == null || mb.isEmpty())
 			return new ColGroupEmpty(colIndexes);
 		else
 			return new ColGroupUncompressed(mb, colIndexes);
@@ -130,27 +137,8 @@ public class ColGroupUncompressed extends AColGroup {
 
 	}
 
-	/**
-	 * Constructor for internal use. Used when a method needs to build an instance of this class from scratch.
-	 * 
-	 * @param colIndices column mapping for this column group
-	 * @param data       matrix block
-	 */
-	protected ColGroupUncompressed(int[] colIndices, MatrixBlock data) {
-		super(colIndices);
-		_data = data;
-		_data.recomputeNonZeros();
-	}
-
-	/**
-	 * Constructor for allocating a single uncompressed column group.
-	 * 
-	 * @param data matrix block
-	 */
-	public ColGroupUncompressed(MatrixBlock data) {
-		super(Util.genColsIndices(data.getNumColumns()));
-		_data = data;
-		_data.recomputeNonZeros();
+	public static AColGroup create(MatrixBlock data) {
+		return create(Util.genColsIndices(data.getNumColumns()), data, false);
 	}
 
 	@Override
@@ -179,9 +167,8 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public void decompressToDenseBlock(DenseBlock db, int rl, int ru, int offR, int offC) {
-		if(_data.isEmpty())
-			return;
-		else if(_data.isInSparseFormat())
+		// _data is never empty
+		if(_data.isInSparseFormat())
 			decompressToDenseBlockSparseData(db, rl, ru, offR, offC);
 		else
 			decompressToDenseBlockDenseData(db, rl, ru, offR, offC);
@@ -219,9 +206,8 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public void decompressToSparseBlock(SparseBlock ret, int rl, int ru, int offR, int offC) {
-		if(_data.isEmpty())
-			return;
-		else if(_data.isInSparseFormat())
+		// data is never empty
+		if(_data.isInSparseFormat())
 			decompressToSparseBlockSparseData(ret, rl, ru, offR, offC);
 		else
 			decompressToSparseBlockDenseData(ret, rl, ru, offR, offC);
@@ -263,12 +249,20 @@ public class ColGroupUncompressed extends AColGroup {
 		final double[] retV = result.getDenseBlockValues();
 		if(matrix.isInSparseFormat())
 			lmmNPSparse(matrix.getSparseBlock(), nCol, retV, nColRet, rl, ru, cl, cu);
-		else
-			lmmNPDense(matrix.getDenseBlockValues(), nCol, retV, nColRet, rl, ru, cl, cu);
+		else {
+			final DenseBlock db = matrix.getDenseBlock();
+			if(db.isContiguous())
+				lmmNPDense(db.values(0), nCol, retV, nColRet, rl, ru, cl, cu);
+			else
+				throw new NotImplementedException(
+					"Not implemented support for leftMultByMatrixNoPreAgg non contiguous dense matrix");
+		}
 
 	}
 
 	protected void lmmNPSparse(SparseBlock sb, int nCol, double[] retV, int nColRet, int rl, int ru, int cl, int cu) {
+		if(cl != 0 || cu != _data.getNumRows())
+			throw new NotImplementedException();
 		if(_data.isInSparseFormat()) {
 			final SparseBlock dsb = _data.getSparseBlock();
 			for(int r = rl; r < ru; r++) {
@@ -316,6 +310,7 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	protected void lmmNPDense(double[] mV, int nCol, double[] retV, int nColRet, int rl, int ru, int cl, int cu) {
+
 		if(_data.isInSparseFormat()) {
 			final SparseBlock sb = _data.getSparseBlock();
 			for(int r = rl; r < ru; r++) {
@@ -338,53 +333,19 @@ public class ColGroupUncompressed extends AColGroup {
 		else {
 			final double[] dV = _data.getDenseBlockValues();
 			final int nColD = _colIndexes.length;
-			for(int r = rl; r < ru; r++) {
+			for(int r = rl; r < ru; r++) { // I
 				final int off = r * nCol;
 				final int offR = r * nColRet;
-				for(int c = cl; c < cu; c++) {
+				for(int c = cl; c < cu; c++) { // K
 					final int offD = c * nColD;
 					final double v = mV[off + c];
-					for(int i = 0; i < nColD; i++)
+					for(int i = 0; i < nColD; i++) // J
 						retV[offR + _colIndexes[i]] += v * dV[offD + i];
 				}
 			}
 		}
 	}
 
-//	 @Override
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
-
-		final MatrixBlock tmpRet = new MatrixBlock(ru - rl, _data.getNumColumns(), false);
-		tmpRet.allocateDenseBlock();
-		final MatrixBlock leftSlice = matrix.slice(rl, ru - 1, false);
-		LibMatrixMult.matrixMult(leftSlice, _data, tmpRet);
-		int offT = result.getNumColumns() * rl;
-		final double[] resV = result.getDenseBlockValues();
-		if(tmpRet.isEmpty())
-			return;
-		else if(tmpRet.isInSparseFormat()) {
-			final SparseBlock sb = tmpRet.getSparseBlock();
-			for(int rowIdx = 0; rowIdx < ru - rl; rowIdx++, offT += result.getNumColumns()) {
-				if(sb.isEmpty(rowIdx))
-					continue;
-
-				final int apos = sb.pos(rowIdx);
-				final int alen = sb.size(rowIdx) + apos;
-				final int[] aix = sb.indexes(rowIdx);
-				final double[] avals = sb.values(rowIdx);
-				for(int col = apos; col < alen; col++)
-					resV[offT + _colIndexes[aix[col]]] += avals[col];
-
-			}
-		}
-		else {
-			final double[] tmpRetV = tmpRet.getDenseBlockValues();
-			for(int j = rl, offTemp = 0; j < ru; j++, offTemp += _colIndexes.length, offT += result.getNumColumns())
-				for(int i = 0; i < _colIndexes.length; i++)
-					resV[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
-		}
-	}
-
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		MatrixBlock retContent = _data.scalarOperations(op, new MatrixBlock());
@@ -399,27 +360,36 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public AColGroup binaryRowOpLeft(BinaryOperator op, double[] v, boolean isRowSafe) {
-		throw new NotImplementedException("Binary row op left is not supported for Uncompressed Matrix, "
+		LOG.warn("Binary row op left is not supported for Uncompressed Matrix, "
 			+ "Implement support for VMr in MatrixBlock Binary Cell operations");
+		MatrixBlockDictionary d = new MatrixBlockDictionary(_data);
+		ADictionary dm = d.binOpLeft(op, v, _colIndexes);
+		if(dm == null)
+			return create(null, _colIndexes);
+		else
+			return create(((MatrixBlockDictionary) dm).getMatrixBlock(), _colIndexes);
 	}
 
 	@Override
 	public AColGroup binaryRowOpRight(BinaryOperator op, double[] v, boolean isRowSafe) {
 		MatrixBlock rowVector = Util.extractValues(v, _colIndexes);
-		return create(_data.binaryOperations(op, rowVector, null), getColIndices());
+		return create(_data.binaryOperations(op, rowVector, null), _colIndexes);
 	}
 
 	@Override
 	public void unaryAggregateOperations(AggregateUnaryOperator op, double[] result, int nRows, int rl, int ru) {
-		if(op.aggOp.increOp.fn instanceof Multiply && op.indexFn instanceof ReduceAll && result[0] == 0) {
+		final ValueFunction fn = op.aggOp.increOp.fn;
+		if(fn instanceof Multiply && op.indexFn instanceof ReduceAll && result[0] == 0)
 			return; // product
-		}
+		else if((fn instanceof Builtin && ((Builtin) fn).getBuiltinCode() == BuiltinCode.MAXINDEX) // index
+			|| (fn instanceof CM))
+			throw new DMLRuntimeException("Not supported type of Unary Aggregate on colGroup");
 
-		// LOG.warn("Inefficient Unary Aggregate because of Uncompressed ColumnGroup");
-		// Since usually Uncompressed column groups are used in case of extreme sparsity, it is fine
+		// inefficient 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);
-		MatrixBlock tmp = tmpData.aggregateUnaryOperations(op, new MatrixBlock(), _data.getNumRows(),
+
+		final MatrixBlock tmpData = (rl == 0 && ru == nRows) ? _data : _data.slice(rl, ru - 1, false);
+		MatrixBlock tmp = tmpData.aggregateUnaryOperations(op, new MatrixBlock(), tmpData.getNumRows(),
 			new MatrixIndexes(1, 1), true);
 
 		if(tmp.isEmpty()) {
@@ -434,12 +404,22 @@ public class ColGroupUncompressed extends AColGroup {
 					for(int row = rl; row < ru; row++)
 						result[row] = b.execute(result[row], 0);
 			}
+			else if(op.aggOp.increOp.fn instanceof Multiply) {
+				if(op.indexFn instanceof ReduceRow)
+					for(int i = 0; i < _colIndexes.length; i++)
+						result[_colIndexes[i]] = 0;
+				else if(op.indexFn instanceof ReduceAll)
+					result[0] = 0;
+				else
+					Arrays.fill(result, rl, ru, 0);
+			}
+			// sum etc.
 			return;
 		}
 
 		tmp.sparseToDense();
 		// The output is always dense in unary aggregates.
-		double[] tmpV = tmp.getDenseBlockValues();
+		final double[] tmpV = tmp.getDenseBlockValues();
 
 		if(op.aggOp.increOp.fn instanceof Builtin) {
 			Builtin b = (Builtin) op.aggOp.increOp.fn;
@@ -449,17 +429,17 @@ public class ColGroupUncompressed extends AColGroup {
 			else if(op.indexFn instanceof ReduceAll)
 				result[0] = b.execute(result[0], tmpV[0]);
 			else
-				for(int i = 0, row = rl; i < tmpV.length; i++, row++)
+				for(int i = 0, row = rl; row < ru; i++, row++)
 					result[row] = b.execute(result[row], tmpV[i]);
 		}
 		else if(op.aggOp.increOp.fn instanceof Multiply) {
 			if(op.indexFn instanceof ReduceRow)
 				for(int i = 0; i < tmpV.length; i++)
-					result[_colIndexes[i]] *= tmpV[i];
+					result[_colIndexes[i]] = tmpV[i];
 			else if(op.indexFn instanceof ReduceAll)
 				result[0] *= tmpV[0];
 			else
-				for(int i = 0, row = rl; i < tmpV.length; i++, row++)
+				for(int i = 0, row = rl; row < ru; i++, row++)
 					result[row] *= tmpV[i];
 		}
 		else {
@@ -469,7 +449,7 @@ public class ColGroupUncompressed extends AColGroup {
 			else if(op.indexFn instanceof ReduceAll)
 				result[0] += tmpV[0];
 			else
-				for(int i = 0, row = rl; i < tmpV.length; i++, row++)
+				for(int i = 0, row = rl; row < ru; i++, row++)
 					result[row] += tmpV[i];
 		}
 	}
@@ -504,8 +484,7 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public final void tsmm(MatrixBlock ret, int nRows) {
-		if(_data.isEmpty())
-			return; // early abort
+
 		final int tCol = _colIndexes.length;
 		final MatrixBlock tmp = new MatrixBlock(tCol, tCol, true);
 
@@ -539,123 +518,144 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
-		if(lhs instanceof ColGroupEmpty || getData().isEmpty())
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result, int nRows) {
+		if(lhs instanceof ColGroupUncompressed)
+			leftMultByAColGroupUncompressed((ColGroupUncompressed) lhs, result);
+		else if(lhs instanceof APreAgg)
+			leftMultByAPreAggColGroup((APreAgg) lhs, result);
+		else
+			throw new DMLCompressionException("Not supported leftMult colgroup type: " + lhs.getClass().getSimpleName());
+	}
+
+	private void leftMultByAPreAggColGroup(APreAgg paCG, MatrixBlock result) {
+		LOG.warn("\nInefficient transpose of uncompressed to fit to"
+			+ " t(AColGroup) %*% UncompressedColGroup mult by colGroup uncompressed column"
+			+ "\nCurrently solved by t(t(Uncompressed) %*% AColGroup)");
+		final int k = InfrastructureAnalyzer.getLocalParallelism();
+		final MatrixBlock ucCGT = LibMatrixReorg.transpose(getData(), k);
+		final int nCols = paCG.getNumCols();
+		final MatrixBlock preAgg = new MatrixBlock(1, paCG.getNumValues(), false);
+		final MatrixBlock tmpRes = new MatrixBlock(1, nCols, false);
+		final MatrixBlock dictM = paCG._dict.getMBDict(nCols).getMatrixBlock();
+		if(dictM == null)
 			return;
-		else if(lhs instanceof ColGroupUncompressed) {
-			ColGroupUncompressed lhsUC = (ColGroupUncompressed) lhs;
-			MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
+		preAgg.allocateDenseBlock();
+		tmpRes.allocateDenseBlock();
+		final int nRowsTransposed = ucCGT.getNumRows();
+		final double[] retV = result.getDenseBlockValues();
+		final double[] tmpV = tmpRes.getDenseBlockValues();
+		final int retCols = result.getNumColumns();
 
-			if(lhsUC._data == this._data) {
+		// Process a row at a time in the transposed block.
+		for(int i = 0; i < nRowsTransposed; i++) {
+			if(ucCGT.isInSparseFormat() && ucCGT.getSparseBlock().isEmpty(i))
+				continue;
+			paCG.preAggregate(ucCGT, preAgg.getDenseBlockValues(), i, i + 1);
+			preAgg.recomputeNonZeros();
+			if(preAgg.isEmpty())
+				continue;
+			// Fixed ret to enforce that we do not allocate again.
+			LibMatrixMult.matrixMult(preAgg, dictM, tmpRes, true);
 
-				LibMatrixMult.matrixMultTransposeSelf(this._data, tmpRet, true,
-					InfrastructureAnalyzer.getLocalParallelism());
+			final int rowOut = _colIndexes[i];
+			for(int j = 0; j < nCols; j++) {
+				final int colOut = paCG._colIndexes[j] * retCols;
+				retV[rowOut + colOut] += tmpV[j];
 			}
-			else {
-				LOG.warn("Inefficient Left Matrix Multiplication with transpose of left hand side : t(l) %*% r");
-				MatrixBlock lhData = lhsUC._data;
-				MatrixBlock transposed = LibMatrixReorg.transpose(lhData, InfrastructureAnalyzer.getLocalParallelism());
-				transposed.setNonZeros(lhData.getNonZeros());
-				// do transposed left hand side, matrix multiplication.
-				LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
+			if(i < nRowsTransposed - 1) {
+				preAgg.reset(1, paCG.getPreAggregateSize());
+				tmpRes.reset(1, nCols);
 			}
-
-			ColGroupUtils.copyValuesColGroupMatrixBlocks(lhs, this, tmpRet, result);
 		}
-		else if(lhs instanceof APreAgg) {
-			// throw new NotImplementedException();
-			LOG.warn("\nInefficient transpose of uncompressed to fit to"
-				+ " t(AColGroup) %*% UncompressedColGroup mult by colGroup uncompressed column"
-				+ "\nCurrently solved by t(t(Uncompressed) %*% AColGroup)");
-
-			final MatrixBlock ucCGT = LibMatrixReorg.transpose(getData(), InfrastructureAnalyzer.getLocalParallelism());
-
-			final APreAgg paCG = (APreAgg) lhs;
-			final MatrixBlock preAgg = new MatrixBlock(1, lhs.getNumValues(), false);
-			final MatrixBlock tmpRes = new MatrixBlock(1, this.getNumCols(), false);
-			final MatrixBlock dictM = paCG._dict.getMBDict(paCG.getNumCols()).getMatrixBlock();
-			preAgg.allocateDenseBlock();
-			tmpRes.allocateDenseBlock();
-			final int nRows = ucCGT.getNumRows();
-			final int nCols = lhs.getNumCols();
-			final double[] retV = result.getDenseBlockValues();
-			final double[] tmpV = tmpRes.getDenseBlockValues();
-			final int retCols = result.getNumColumns();
-			for(int i = 0; i < nRows; i++) {
-				if(ucCGT.isInSparseFormat() && ucCGT.getSparseBlock().isEmpty(i))
+	}
+
+	private void leftMultByAColGroupUncompressed(ColGroupUncompressed lhs, MatrixBlock result) {
+		LOG.warn("Inefficient Left Matrix Multiplication with transpose of left hand side : t(l) %*% r");
+		final MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
+		final int k = InfrastructureAnalyzer.getLocalParallelism();
+
+		// multiply to temp
+		MatrixBlock lhData = lhs._data;
+		MatrixBlock transposed = LibMatrixReorg.transpose(lhData, k);
+		transposed.setNonZeros(lhData.getNonZeros());
+		// do transposed left hand side, matrix multiplication.
+		LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
+
+		// add temp to output
+		final double[] resV = result.getDenseBlockValues();
+		final int nColOut = result.getNumColumns();
+		// Guaranteed not empty both sides, therefore safe to not check for empty
+		if(tmpRet.isInSparseFormat()) {
+			SparseBlock sb = tmpRet.getSparseBlock();
+			for(int row = 0; row < lhs._colIndexes.length; row++) {
+				if(sb.isEmpty(row))
 					continue;
-				paCG.preAggregate(ucCGT, preAgg.getDenseBlockValues(), i, i + 1);
-				preAgg.recomputeNonZeros();
-				LibMatrixMult.matrixMult(preAgg, dictM, tmpRes, true);
-
-				final int rowOut = _colIndexes[i];
-				for(int j = 0; j < nCols; j++) {
-					final int colOut = lhs._colIndexes[j] * retCols;
-					retV[rowOut + colOut] += tmpV[j];
-				}
-				if(i < nRows - 1) {
-					preAgg.reset(1, lhs.getNumValues());
-					tmpRes.reset(1, this.getNumCols());
-				}
+				final int apos = sb.pos(row);
+				final int alen = sb.size(row) + apos;
+				final int[] aix = sb.indexes(row);
+				final double[] avals = sb.values(row);
+				final int offRes = lhs._colIndexes[row] * nColOut;
+				for(int col = apos; col < alen; col++)
+					resV[offRes + _colIndexes[aix[col]]] += avals[col];
 			}
 		}
 		else {
-			throw new NotImplementedException();
+			final double[] tmpRetV = tmpRet.getDenseBlockValues();
+			for(int row = 0; row < lhs._colIndexes.length; row++) {
+				final int offRes = lhs._colIndexes[row] * nColOut;
+				final int offTmp = _colIndexes.length * row;
+				for(int col = 0; col < _colIndexes.length; col++)
+					resV[offRes + _colIndexes[col]] += tmpRetV[offTmp + col];
+			}
 		}
 	}
 
 	@Override
 	public void tsmmAColGroup(AColGroup lhs, MatrixBlock result) {
-		if(this._data.isEmpty())
-			return; // early abort
-		if(lhs instanceof ColGroupUncompressed) {
-			ColGroupUncompressed lhsUC = (ColGroupUncompressed) lhs;
-			if(lhsUC._data.isEmpty())
-				return; // early abort
-
-			MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
-			if(lhsUC._data == this._data) {
-				LibMatrixMult.matrixMultTransposeSelf(this._data, tmpRet, true,
-					InfrastructureAnalyzer.getLocalParallelism());
-			}
-			else {
-				LOG.warn("Inefficient Left Matrix Multiplication with transpose of left hand side : t(l) %*% r");
-				MatrixBlock lhData = lhsUC._data;
-				MatrixBlock transposed = LibMatrixReorg.transpose(lhData, InfrastructureAnalyzer.getLocalParallelism());
-				transposed.setNonZeros(lhData.getNonZeros());
-				// do transposed left hand side, matrix multiplication.
-				LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
-			}
-			final double[] resV = result.getDenseBlockValues();
-			final int nCols = result.getNumColumns();
-			if(tmpRet.isEmpty())
-				return;
-			else if(tmpRet.isInSparseFormat()) {
-				SparseBlock sb = tmpRet.getSparseBlock();
-				for(int row = 0; row < lhs._colIndexes.length; row++) {
-					if(sb.isEmpty(row))
-						continue;
-					final int apos = sb.pos(row);
-					final int alen = sb.size(row) + apos;
-					final int[] aix = sb.indexes(row);
-					final double[] avals = sb.values(row);
-					for(int col = apos; col < alen; col++)
-						DictLibMatrixMult.addToUpperTriangle(nCols, lhs._colIndexes[row], _colIndexes[aix[col]], resV,
-							avals[col]);
-				}
+		// this is never empty therefore process:
+		if(lhs instanceof ColGroupUncompressed)
+			tsmmUncompressedColGroup((ColGroupUncompressed) lhs, result);
+		else
+			lhs.tsmmAColGroup(this, result);
+	}
+
+	private void tsmmUncompressedColGroup(ColGroupUncompressed lhs, MatrixBlock result) {
+		final MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
+		final int k = InfrastructureAnalyzer.getLocalParallelism();
+
+		if(lhs._data == this._data)
+			LibMatrixMult.matrixMultTransposeSelf(this._data, tmpRet, true, k);
+		else {
+			LOG.warn("Inefficient Left Matrix Multiplication with transpose of left hand side : t(l) %*% r");
+			LibMatrixMult.matrixMult(LibMatrixReorg.transpose(lhs._data, k), this._data, tmpRet);
+		}
+
+		final double[] resV = result.getDenseBlockValues();
+		final int nCols = result.getNumColumns();
+		// guaranteed non empty
+		if(tmpRet.isInSparseFormat()) {
+			SparseBlock sb = tmpRet.getSparseBlock();
+			for(int row = 0; row < lhs._colIndexes.length; row++) {
+				if(sb.isEmpty(row))
+					continue;
+				final int apos = sb.pos(row);
+				final int alen = sb.size(row) + apos;
+				final int[] aix = sb.indexes(row);
+				final double[] avals = sb.values(row);
+				for(int col = apos; col < alen; col++)
+					DictLibMatrixMult.addToUpperTriangle(nCols, lhs._colIndexes[row], _colIndexes[aix[col]], resV,
+						avals[col]);
 			}
-			else {
-				double[] tmpRetV = tmpRet.getDenseBlockValues();
-				for(int row = 0; row < lhs._colIndexes.length; row++) {
-					final int offTmp = lhs._colIndexes.length * row;
-					for(int col = 0; col < _colIndexes.length; col++)
-						DictLibMatrixMult.addToUpperTriangle(nCols, lhs._colIndexes[row], _colIndexes[col], resV,
-							tmpRetV[offTmp + col]);
-				}
+		}
+		else {
+			double[] tmpRetV = tmpRet.getDenseBlockValues();
+			for(int row = 0; row < lhs._colIndexes.length; row++) {
+				final int offTmp = lhs._colIndexes.length * row;
+				final int oid = lhs._colIndexes[row];
+				for(int col = 0; col < _colIndexes.length; col++)
+					DictLibMatrixMult.addToUpperTriangle(nCols, oid, _colIndexes[col], resV, tmpRetV[offTmp + col]);
 			}
 		}
-		else
-			lhs.tsmmAColGroup(this, result);
 	}
 
 	@Override
@@ -670,12 +670,12 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	@Override
-	public AColGroup rightMultByMatrix(MatrixBlock right) {
+	public AColGroup rightMultByMatrix(MatrixBlock right, int[] allCols) {
 		final int nColR = right.getNumColumns();
-		final int[] outputCols = Util.genColsIndices(nColR);
+		final int[] outputCols = allCols != null ? allCols : Util.genColsIndices(nColR);
 
-		if(_data.isEmpty() || right.isEmpty())
-			return new ColGroupEmpty(outputCols);
+		if(right.isEmpty())
+			return null;
 
 		MatrixBlock subBlockRight;
 
@@ -725,9 +725,7 @@ public class ColGroupUncompressed extends AColGroup {
 	@Override
 	public void computeColSums(double[] c, int nRows) {
 		final MatrixBlock colSum = _data.colSum();
-		if(colSum.isEmpty())
-			return;
-		else if(colSum.isInSparseFormat()) {
+		if(colSum.isInSparseFormat()) {
 			SparseBlock sb = colSum.getSparseBlock();
 			double[] rv = sb.values(0);
 			int[] idx = sb.indexes(0);
@@ -759,6 +757,11 @@ public class ColGroupUncompressed extends AColGroup {
 		return e.getCost(nRows, nRows, nCols, nVals, _data.getSparsity());
 	}
 
+	@Override
+	public boolean isEmpty() {
+		return _data.isEmpty();
+	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
@@ -768,15 +771,9 @@ public class ColGroupUncompressed extends AColGroup {
 		sb.append(" numRows : " + _data.getNumRows());
 		sb.append(" nonZeros: " + _data.getNonZeros());
 		sb.append(" Sparse : " + _data.isInSparseFormat());
-		if(_data.isEmpty()) {
-			sb.append(" empty");
-			return sb.toString();
-		}
 
 		sb.append("\n");
-		if(!_data.isInSparseFormat() && _data.getNumRows() < 1000)
-			sb.append(Arrays.toString(_data.getDenseBlockValues()));
-		else if(_data.getNumRows() < 100)
+		if(_data.getNumRows() < 1000)
 			sb.append(_data.toString());
 		else
 			sb.append(" don't print uncompressed matrix because it is to big.");
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUtils.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUtils.java
index 55b7be3243..425c82c2c5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUtils.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUtils.java
@@ -19,12 +19,15 @@
 
 package org.apache.sysds.runtime.compress.colgroup;
 
+import org.apache.sysds.runtime.compress.utils.DoubleCountHashMap;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
+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.matrix.operators.UnaryOperator;
 
-public class ColGroupUtils {
+public interface ColGroupUtils {
 
 	/**
 	 * Calculate the result of performing the binary operation on an empty row to the left
@@ -36,7 +39,7 @@ public class ColGroupUtils {
 	 * @param colIndexes The column indexes to extract
 	 * @return The result as a double array.
 	 */
-	protected final static double[] binaryDefRowLeft(BinaryOperator op, double[] v, int[] colIndexes) {
+	public static double[] binaryDefRowLeft(BinaryOperator op, double[] v, int[] colIndexes) {
 		final ValueFunction fn = op.fn;
 		final int len = colIndexes.length;
 		final double[] ret = new double[len];
@@ -55,7 +58,7 @@ public class ColGroupUtils {
 	 * @param colIndexes The column indexes to extract
 	 * @return The result as a double array.
 	 */
-	protected final static double[] binaryDefRowRight(BinaryOperator op, double[] v, int[] colIndexes) {
+	public static double[] binaryDefRowRight(BinaryOperator op, double[] v, int[] colIndexes) {
 		final ValueFunction fn = op.fn;
 		final int len = colIndexes.length;
 		final double[] ret = new double[len];
@@ -67,12 +70,13 @@ public class ColGroupUtils {
 	/**
 	 * Copy values from tmpResult into correct positions of result (according to colIndexes in lhs and rhs)
 	 *
-	 * @param lhs        Left ColumnGroup
-	 * @param rhs        Right ColumnGroup
-	 * @param tmpResult  The matrix block to move values from
-	 * @param result     The result matrix block to move values to
+	 * @param lhs       Left ColumnGroup
+	 * @param rhs       Right ColumnGroup
+	 * @param tmpResult The matrix block to move values from
+	 * @param result    The result matrix block to move values to
 	 */
-	protected final static void copyValuesColGroupMatrixBlocks(AColGroup lhs, AColGroup rhs, MatrixBlock tmpResult, MatrixBlock result) {
+	public static void copyValuesColGroupMatrixBlocks(AColGroup lhs, AColGroup rhs, MatrixBlock tmpResult,
+		MatrixBlock result) {
 		final double[] resV = result.getDenseBlockValues();
 		if(tmpResult.isEmpty())
 			return;
@@ -101,4 +105,203 @@ public class ColGroupUtils {
 			}
 		}
 	}
+
+	/*
+	 * Returns null if all zero
+	 * 
+	 * @param mb Matrix Block to find most common value in all columns.
+	 * 
+	 * @return Double array with most common values.
+	 */
+	public static double[] extractMostCommonValueInColumns(MatrixBlock mb) {
+		final int nCol = mb.getNumColumns();
+		final int nVal = mb.getNumRows();
+		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);
+		else
+			return null;
+
+		contains = false;
+		for(int i = 0; i < nCol; i++)
+			if(ref[i] != 0) {
+				contains = true;
+				break;
+			}
+		if(contains == false)
+			return null;
+		else
+			return ref;
+	}
+
+	public static double refSum(double[] reference) {
+		double ret = 0;
+		for(double d : reference)
+			ret += d;
+		return ret;
+	}
+
+	public static double refSumSq(double[] reference) {
+		double ret = 0;
+		for(double d : reference)
+			ret += d * d;
+		return ret;
+	}
+
+	public static boolean allZero(double[] in) {
+		for(double v : in)
+			if(v != 0)
+				return false;
+		return true;
+	}
+
+	public 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;
+		}
+	}
+
+	public static double[] createReference(int nCol, double val) {
+		double[] reference = new double[nCol];
+		for(int i = 0; i < nCol; i++)
+			reference[i] = val;
+		return reference;
+	}
+
+	public 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;
+	}
+
+	public static void outerProduct(final double[] leftRowSum, final double[] rightColumnSum, final double[] result,
+		int rl, int ru) {
+		for(int row = rl; row < ru; row++) {
+			final int offOut = rightColumnSum.length * row;
+			final double vLeft = leftRowSum[row];
+			for(int col = 0; col < rightColumnSum.length; col++) {
+				result[offOut + col] += vLeft * rightColumnSum[col];
+			}
+		}
+	}
+
+	public static void outerProduct(final double[] leftRowSum, final double[] rightColumnSum, final int[] colIdxRight,
+		final double[] result, final int nColR, final int rl, final int ru) {
+		for(int row = rl; row < ru; row++) {
+			final int offOut = nColR * row;
+			final double vLeft = leftRowSum[row];
+			for(int col = 0; col < rightColumnSum.length; col++) {
+				result[offOut + colIdxRight[col]] += vLeft * rightColumnSum[col];
+			}
+		}
+	}
+
+	public static void outerProduct(final double[] leftRowSum, final SparseBlock rightColSum, final int[] colIdxRight,
+		final double[] result, final int nColR, final int rl, final int ru) {
+		final int alen = rightColSum.size(0);
+		final int[] aix = rightColSum.indexes(0);
+		final double[] aval = rightColSum.values(0);
+		for(int row = rl; row < ru; row++) {
+			final int offOut = nColR * row;
+			final double vLeft = leftRowSum[row];
+			for(int j = 0; j < alen; j++)
+				result[offOut + colIdxRight[aix[j]]] += vLeft * aval[j];
+		}
+	}
+
+	private static 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();
+	}
+
+
+	public static void addMatrixToResult(MatrixBlock tmp, MatrixBlock result, int[] colIndexes, int rl, int ru) {
+		if(tmp.isEmpty())
+			return;
+		final double[] retV = result.getDenseBlockValues();
+		final int nColRet = result.getNumColumns();
+		if(tmp.isInSparseFormat()) {
+			final SparseBlock sb = tmp.getSparseBlock();
+			for(int row = rl, offT = 0; row < ru; row++, offT++) {
+				final int apos = sb.pos(offT);
+				final int alen = sb.size(offT);
+				final int[] aix = sb.indexes(offT);
+				final double[] avals = sb.values(offT);
+				final int offR = row * nColRet;
+				for(int i = apos; i < apos + alen; i++)
+					retV[offR + colIndexes[aix[i]]] += avals[i];
+			}
+		}
+		else {
+			final double[] tmpV = tmp.getDenseBlockValues();
+			final int nCol = colIndexes.length;
+			for(int row = rl, offT = 0; row < ru; row++, offT += nCol) {
+				final int offR = row * nColRet;
+				for(int col = 0; col < nCol; col++)
+					retV[offR + colIndexes[col]] += tmpV[offT + col];
+			}
+		}
+	}
+
 }
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
deleted file mode 100644
index 1124c6b882..0000000000
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/FORUtil.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.runtime.compress.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 b3320fb277..9e330f0f79 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
@@ -23,7 +23,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 
-import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.data.SparseBlock;
@@ -59,6 +58,16 @@ public abstract class ADictionary implements Serializable {
 	 */
 	public abstract double getValue(int i);
 
+	/**
+	 * Get Specific value contain in dictionary at index.
+	 * 
+	 * @param r    Row target
+	 * @param col  Col target
+	 * @param nCol nCol in dictionary
+	 * @return value
+	 */
+	public abstract double getValue(int r, int col, int nCol);
+
 	/**
 	 * Returns the memory usage of the dictionary.
 	 * 
@@ -131,29 +140,50 @@ public abstract class ADictionary implements Serializable {
 	 * @param c          The target double array, this contains the full number of columns, therefore the colIndexes for
 	 *                   this specific dictionary is needed.
 	 * @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 reference  The reference offset values to add to each cell.
 	 * @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,
 		boolean def);
 
 	/**
-	 * Allocate a new dictionary and applies the scalar operation on each cell of the to then return the new dictionary.
+	 * Allocate a new dictionary and applies the scalar operation on each cell of to then return the new dictionary.
 	 * 
 	 * @param op The operator.
 	 * @return The new dictionary to return.
 	 */
 	public abstract ADictionary applyScalarOp(ScalarOperator op);
 
+	/**
+	 * Allocate a new dictionary with one extra row and applies the scalar operation on each cell of to then return the
+	 * new dictionary.
+	 * 
+	 * @param op   The operator
+	 * @param v0   The new value to put into each cell in the new row
+	 * @param nCol The number of columns in the dictionary
+	 * @return The new dictionary to return.
+	 */
+	public abstract ADictionary applyScalarOpAndAppend(ScalarOperator op, double v0, int nCol);
+
 	/**
 	 * Allocate a new dictionary and apply the unary operator on each cell.
 	 * 
-	 * @param op the operator.
+	 * @param op The operator.
 	 * @return The new dictionary to return.
 	 */
 	public abstract ADictionary applyUnaryOp(UnaryOperator op);
 
+	/**
+	 * Allocate a new dictionary with one extra row and apply the unary operator on each cell.
+	 * 
+	 * @param op   The operator.
+	 * @param v0   The new value to put into each cell in the new row
+	 * @param nCol The number of columns in the dictionary
+	 * @return The new dictionary to return.
+	 */
+	public abstract ADictionary applyUnaryOpAndAppend(UnaryOperator op, double v0, int nCol);
+
 	/**
 	 * Allocate a new dictionary and apply the scalar operation on each cell to then return a new dictionary.
 	 * 
@@ -179,23 +209,24 @@ public abstract class ADictionary implements Serializable {
 	public abstract ADictionary applyUnaryOpWithReference(UnaryOperator op, double[] reference, double[] newReference);
 
 	/**
-	 * Applies the scalar operation on the dictionary. Note that this operation modifies the underlying data, and
-	 * normally require a copy of the original Dictionary to preserve old objects.
+	 * Apply binary row operation on the left side
 	 * 
-	 * @param op The operator to apply to the dictionary values.
-	 * @return this dictionary with modified values.
+	 * @param op         The operation to this dictionary
+	 * @param v          The values to use on the left hand side.
+	 * @param colIndexes The column indexes to consider inside v.
+	 * @return A new dictionary containing the updated values.
 	 */
-	public abstract ADictionary inplaceScalarOp(ScalarOperator op);
+	public abstract ADictionary binOpLeft(BinaryOperator op, double[] v, int[] colIndexes);
 
 	/**
-	 * Apply binary row operation on the left side in place
+	 * Apply binary row operation on the left side with one extra row evaluating with zeros.
 	 * 
 	 * @param op         The operation to this dictionary
 	 * @param v          The values to use on the left hand side.
 	 * @param colIndexes The column indexes to consider inside v.
 	 * @return A new dictionary containing the updated values.
 	 */
-	public abstract ADictionary binOpLeft(BinaryOperator op, double[] v, int[] colIndexes);
+	public abstract ADictionary binOpLeftAndAppend(BinaryOperator op, double[] v, int[] colIndexes);
 
 	/**
 	 * Apply the binary operator such that each value is offset by the reference before application. Then put the result
@@ -225,7 +256,17 @@ public abstract class ADictionary implements Serializable {
 	public abstract ADictionary binOpRight(BinaryOperator op, double[] v, int[] colIndexes);
 
 	/**
-	 * Apply binary row operation on the right side.
+	 * Apply binary row operation on the right side with one extra row evaluating with zeros.
+	 * 
+	 * @param op         The operation to this dictionary
+	 * @param v          The values to use on the right hand side.
+	 * @param colIndexes The column indexes to consider inside v.
+	 * @return A new dictionary containing the updated values.
+	 */
+	public abstract ADictionary binOpRightAndAppend(BinaryOperator op, double[] v, int[] colIndexes);
+
+	/**
+	 * Apply binary row operation on the right side as with no columns to extract from v.
 	 * 
 	 * @param op The operation to this dictionary
 	 * @param v  The values to apply on the dictionary (same number of cols as the dictionary)
@@ -339,6 +380,31 @@ public abstract class ADictionary implements Serializable {
 	 */
 	public abstract double[] sumAllRowsToDoubleSqWithReference(double[] reference);
 
+	/**
+	 * Method to product all rows to a column vector.
+	 * 
+	 * @param nrColumns The number of columns in the ColGroup to know how to get the values from the dictionary.
+	 * @return A row product
+	 */
+	public abstract double[] productAllRowsToDouble(int nrColumns);
+
+	/**
+	 * Method to product all rows to a column vector with a default value added in the end.
+	 * 
+	 * @param defaultTuple The default row that aggregate to last cell
+	 * @return A row product
+	 */
+	public abstract double[] productAllRowsToDoubleWithDefault(double[] defaultTuple);
+
+	/**
+	 * Method to product all rows to a column vector with a reference values added to all cells, and a reference product
+	 * in the end
+	 * 
+	 * @param reference The reference row
+	 * @return A row product
+	 */
+	public abstract double[] productAllRowsToDoubleWithReference(double[] reference);
+
 	/**
 	 * Get the column sum of the values contained in the dictionary
 	 * 
@@ -570,7 +636,7 @@ public abstract class ADictionary implements Serializable {
 	 * @param ret       The result dense double array (containing one value)
 	 * @param counts    The counts of each entry in the dictionary
 	 * @param reference The reference value.
-	 * @param refCount  The number of occurences of the ref value.
+	 * @param refCount  The number of occurrences of the ref value.
 	 */
 	public abstract void productWithReference(double[] ret, int[] counts, double[] reference, int refCount);
 
@@ -581,9 +647,17 @@ public abstract class ADictionary implements Serializable {
 	 * @param counts     The weighted count of individual tuples
 	 * @param colIndexes The column indexes.
 	 */
-	public void colProduct(double[] res, int[] counts, int[] colIndexes) {
-		throw new NotImplementedException();
-	}
+	public abstract void colProduct(double[] res, int[] counts, int[] colIndexes);
+
+	/**
+	 * Calculate the column product of the dictionary weighted by counts.
+	 * 
+	 * @param res        The result vector to put the result into
+	 * @param counts     The weighted count of individual tuples
+	 * @param colIndexes The column indexes.
+	 * @param reference  The reference value.
+	 */
+	public abstract void colProductWithReference(double[] res, int[] counts, int[] colIndexes, double[] reference);
 
 	/**
 	 * Central moment function to calculate the central moment of this column group. MUST be on a single column
@@ -610,6 +684,34 @@ public abstract class ADictionary implements Serializable {
 	 */
 	public abstract CM_COV_Object centralMoment(CM_COV_Object ret, ValueFunction fn, int[] counts, int nRows);
 
+	/**
+	 * Central moment function to calculate the central moment of this column group with a default offset on all missing
+	 * tuples. MUST be on a single column dictionary.
+	 * 
+	 * @param fn     The value function to apply
+	 * @param counts The weight of individual tuples
+	 * @param def    The default values to offset the tuples with
+	 * @param nRows  The number of rows in total of the column group
+	 * @return The central moment Object
+	 */
+	public CM_COV_Object centralMomentWithDefault(ValueFunction fn, int[] counts, double def, int nRows) {
+		return centralMomentWithDefault(new CM_COV_Object(), fn, counts, def, nRows);
+	}
+
+	/**
+	 * Central moment function to calculate the central moment of this column group with a default offset on all missing
+	 * tuples. MUST be on a single column dictionary.
+	 * 
+	 * @param ret    The Central Moment object to be modified and returned
+	 * @param fn     The value function to apply
+	 * @param counts The weight of individual tuples
+	 * @param def    The default values to offset the tuples with
+	 * @param nRows  The number of rows in total of the column group
+	 * @return The central moment Object
+	 */
+	public abstract CM_COV_Object centralMomentWithDefault(CM_COV_Object ret, ValueFunction fn, int[] counts, double def,
+		int nRows);
+
 	/**
 	 * Central moment function to calculate the central moment of this column group with a reference offset on each
 	 * tuple. MUST be on a single column dictionary.
@@ -658,7 +760,7 @@ public abstract class ADictionary implements Serializable {
 	 * @param reference A reference value to add to all tuples before expanding
 	 * @return A new dictionary
 	 */
-	public abstract ADictionary rexpandColsWithReference(int max, boolean ignore, boolean cast, double reference);
+	public abstract ADictionary rexpandColsWithReference(int max, boolean ignore, boolean cast, int reference);
 
 	/**
 	 * Get the sparsity of the dictionary.
@@ -788,4 +890,17 @@ public abstract class ADictionary implements Serializable {
 	protected abstract void TSMMToUpperTriangleSparseScaling(SparseBlock left, int[] rowsLeft, int[] colsRight,
 		int[] scale, MatrixBlock result);
 
+	protected String doubleToString(double v) {
+		if(v == (long) v)
+			return Long.toString(((long) v));
+		else
+			return Double.toString(v);
+	}
+
+	protected static void correctNan(double[] res, int[] colIndexes) {
+		// since there is no nan values every in a dictionary, we exploit that
+		// nan oly occur if we multiplied infinity with 0.
+		for(int j = 0; j < colIndexes.length; j++)
+			res[colIndexes[j]] = Double.isNaN(res[colIndexes[j]]) ? 0 : res[colIndexes[j]];
+	}
 }
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 a44fd1cb19..e2a5945ddf 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
@@ -50,12 +50,27 @@ public class Dictionary extends ADictionary {
 
 	protected final double[] _values;
 
-	public Dictionary(double[] values) {
+	protected Dictionary(double[] values) {
 		if(values == null || values.length == 0)
 			throw new DMLCompressionException("Invalid construction of dictionary with null array");
 		_values = values;
 	}
 
+	public static Dictionary create(double[] values) {
+		boolean nonZero = false;
+		for(double d : values) {
+			if(d != 0) {
+				nonZero = true;
+				break;
+			}
+		}
+		return nonZero ? new Dictionary(values) : null;
+	}
+
+	public static Dictionary createNoCheck(double[] values){
+		return new Dictionary(values);
+	}
+
 	@Override
 	public double[] getValues() {
 		return _values;
@@ -66,6 +81,11 @@ public class Dictionary extends ADictionary {
 		return _values[i];
 	}
 
+	@Override
+	public final double getValue(int r, int c, int nCol) {
+		return _values[r * nCol + c];
+	}
+
 	@Override
 	public long getInMemorySize() {
 		// object + values array + double
@@ -156,7 +176,17 @@ public class Dictionary extends ADictionary {
 		final double[] retV = new double[_values.length];
 		for(int i = 0; i < _values.length; i++)
 			retV[i] = op.executeScalar(_values[i]);
-		return new Dictionary(retV);
+		return create(retV);
+	}
+
+	@Override
+	public ADictionary applyScalarOpAndAppend(ScalarOperator op, double v0, int nCol) {
+		final double[] retV = new double[_values.length + nCol];
+		for(int i = 0; i < _values.length; i++)
+			retV[i] = op.executeScalar(_values[i]);
+		for(int i = _values.length; i < retV.length; i++)
+			retV[i] = v0;
+		return create(retV);
 	}
 
 	@Override
@@ -164,7 +194,17 @@ public class Dictionary extends ADictionary {
 		final double[] retV = new double[_values.length];
 		for(int i = 0; i < _values.length; i++)
 			retV[i] = op.fn.execute(_values[i]);
-		return new Dictionary(retV);
+		return create(retV);
+	}
+
+	@Override
+	public ADictionary applyUnaryOpAndAppend(UnaryOperator op, double v0, int nCol) {
+		final double[] retV = new double[_values.length + nCol];
+		for(int i = 0; i < _values.length; i++)
+			retV[i] = op.fn.execute(_values[i]);
+		for(int i = _values.length; i < retV.length; i++)
+			retV[i] = v0;
+		return create(retV);
 	}
 
 	@Override
@@ -179,7 +219,7 @@ public class Dictionary extends ADictionary {
 				off++;
 			}
 		}
-		return new Dictionary(retV);
+		return create(retV);
 	}
 
 	@Override
@@ -194,15 +234,7 @@ public class Dictionary extends ADictionary {
 				off++;
 			}
 		}
-		return new Dictionary(retV);
-	}
-
-	@Override
-	public Dictionary inplaceScalarOp(ScalarOperator op) {
-		int len = size();
-		for(int i = 0; i < len; i++)
-			_values[i] = op.executeScalar(_values[i]);
-		return this;
+		return create(retV);
 	}
 
 	@Override
@@ -213,20 +245,32 @@ public class Dictionary extends ADictionary {
 		final int lenV = colIndexes.length;
 		for(int i = 0; i < len; i++)
 			retVals[i] = fn.execute(_values[i], v[colIndexes[i % lenV]]);
-		return new Dictionary(retVals);
+		return create(retVals);
 	}
 
 	@Override
-	public Dictionary binOpRight(BinaryOperator op, double[] v){
+	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);
+		return create(retVals);
 	}
 
+	@Override
+	public ADictionary binOpRightAndAppend(BinaryOperator op, double[] v, int[] colIndexes) {
+		final ValueFunction fn = op.fn;
+		final double[] retVals = new double[_values.length + colIndexes.length];
+		final int lenV = colIndexes.length;
+		for(int i = 0; i < _values.length; i++)
+			retVals[i] = fn.execute(_values[i], v[colIndexes[i % lenV]]);
+		for(int i = _values.length; i < _values.length; i++)
+			retVals[i] = fn.execute(0, v[colIndexes[i % lenV]]);
+
+		return create(retVals);
+	}
 
 	@Override
 	public Dictionary binOpRightWithReference(BinaryOperator op, double[] v, int[] colIndexes, double[] reference,
@@ -242,18 +286,30 @@ public class Dictionary extends ADictionary {
 				off++;
 			}
 		}
-		return new Dictionary(retV);
+		return create(retV);
 	}
 
 	@Override
 	public final Dictionary binOpLeft(BinaryOperator op, double[] v, int[] colIndexes) {
 		final ValueFunction fn = op.fn;
 		final double[] retVals = new double[_values.length];
-		final int len = size();
 		final int lenV = colIndexes.length;
-		for(int i = 0; i < len; i++)
+		for(int i = 0; i < _values.length; i++)
+			retVals[i] = fn.execute(v[colIndexes[i % lenV]], _values[i]);
+		return create(retVals);
+	}
+
+	@Override
+	public ADictionary binOpLeftAndAppend(BinaryOperator op, double[] v, int[] colIndexes) {
+		final ValueFunction fn = op.fn;
+		final double[] retVals = new double[_values.length + colIndexes.length];
+		final int lenV = colIndexes.length;
+		for(int i = 0; i < _values.length; i++)
 			retVals[i] = fn.execute(v[colIndexes[i % lenV]], _values[i]);
-		return new Dictionary(retVals);
+		for(int i = _values.length; i < _values.length; i++)
+			retVals[i] = fn.execute(v[colIndexes[i % lenV]], 0);
+
+		return create(retVals);
 	}
 
 	@Override
@@ -270,12 +326,12 @@ public class Dictionary extends ADictionary {
 				off++;
 			}
 		}
-		return new Dictionary(retV);
+		return create(retV);
 	}
 
 	@Override
 	public Dictionary clone() {
-		return new Dictionary(_values.clone());
+		return createNoCheck(_values.clone());
 	}
 
 	public static Dictionary read(DataInput in) throws IOException {
@@ -284,7 +340,7 @@ public class Dictionary extends ADictionary {
 		double[] values = new double[numVals];
 		for(int i = 0; i < numVals; i++)
 			values[i] = in.readDouble();
-		return new Dictionary(values);
+		return Dictionary.createNoCheck(values);
 	}
 
 	@Override
@@ -370,6 +426,41 @@ public class Dictionary extends ADictionary {
 		return ret;
 	}
 
+	@Override
+	public double[] productAllRowsToDouble(int nCol) {
+		final int numVals = getNumberOfValues(nCol);
+		final double[] ret = new double[numVals];
+		for(int k = 0; k < numVals; k++)
+			ret[k] = prodRow(k, nCol);
+		return ret;
+	}
+
+	@Override
+	public double[] productAllRowsToDoubleWithDefault(double[] defaultTuple) {
+		final int nCol = defaultTuple.length;
+		final int numVals = getNumberOfValues(nCol);
+		final double[] ret = new double[numVals + 1];
+		for(int k = 0; k < numVals; k++)
+			ret[k] = prodRow(k, nCol);
+		ret[ret.length - 1] = defaultTuple[0];
+		for(int i = 1; i < nCol; i++)
+			ret[ret.length - 1] *= defaultTuple[i];
+		return ret;
+	}
+
+	@Override
+	public double[] productAllRowsToDoubleWithReference(double[] reference) {
+		final int nCol = reference.length;
+		final int numVals = getNumberOfValues(nCol);
+		final double[] ret = new double[numVals + 1];
+		for(int k = 0; k < numVals; k++)
+			ret[k] = prodRowWithReference(k, nCol, reference);
+		ret[ret.length - 1] = reference[0];
+		for(int i = 1; i < nCol; i++)
+			ret[ret.length - 1] *= reference[i];
+		return ret;
+	}
+
 	@Override
 	public double[] sumAllRowsToDoubleSqWithReference(double[] reference) {
 		final int nCol = reference.length;
@@ -406,6 +497,23 @@ public class Dictionary extends ADictionary {
 		return res;
 	}
 
+	private double prodRow(int k, int nrColumns) {
+		final int valOff = k * nrColumns;
+		double res = _values[valOff];
+		for(int i = 1; i < nrColumns; i++)
+			res *= _values[valOff + i];
+		return res;
+	}
+
+	private double prodRowWithReference(int k, int nrColumns, double[] reference) {
+		final int valOff = k * nrColumns;
+		double res = _values[valOff] + reference[0];
+		for(int i = 1; i < nrColumns; i++)
+			res *= _values[valOff + i] + reference[i];
+		return res;
+
+	}
+
 	private double sumRowSqWithReference(int k, int nrColumns, double[] reference) {
 		final int valOff = k * nrColumns;
 		double res = 0.0;
@@ -441,6 +549,31 @@ public class Dictionary extends ADictionary {
 		}
 	}
 
+	@Override
+	public void colProduct(double[] res, int[] counts, int[] colIndexes) {
+		final int nCol = colIndexes.length;
+		for(int k = 0; k < counts.length; k++) {
+			final int cntk = counts[k];
+			final int off = k * nCol;
+			for(int j = 0; j < nCol; j++)
+				res[colIndexes[j]] *= Math.pow(_values[off + j], cntk);
+		}
+		correctNan(res, colIndexes);
+	}
+
+	@Override
+	public void colProductWithReference(double[] res, int[] counts, int[] colIndexes, double[] reference) {
+		final int nCol = colIndexes.length;
+		for(int k = 0; k < counts.length; k++) {
+			final int cntk = counts[k];
+			final int off = k * nCol;
+			for(int j = 0; j < nCol; j++)
+				res[colIndexes[j]] *= Math.pow(_values[off + j] + reference[j], cntk);
+		}
+
+		correctNan(res, colIndexes);
+	}
+
 	@Override
 	public void colSumSqWithReference(double[] c, int[] counts, int[] colIndexes, double[] reference) {
 		final int nCol = colIndexes.length;
@@ -514,10 +647,10 @@ public class Dictionary extends ADictionary {
 		else {
 			sb.append("[\n\t");
 			for(int i = 0; i < _values.length - 1; i++) {
-				sb.append(_values[i]);
+				sb.append(doubleToString(_values[i]));
 				sb.append((i) % (colIndexes) == colIndexes - 1 ? "\n\t" : ", ");
 			}
-			sb.append(_values[_values.length - 1]);
+			sb.append(doubleToString(_values[_values.length - 1]));
 			sb.append("]");
 		}
 		return sb.toString();
@@ -535,7 +668,7 @@ public class Dictionary extends ADictionary {
 			}
 			orgOffset += previousNumberOfColumns - idxEnd + idxStart;
 		}
-		return new Dictionary(newDictValues);
+		return create(newDictValues);
 	}
 
 	@Override
@@ -601,7 +734,7 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public void addToEntry(double[] v, int fr, int to, int nCol) {
+	public final void addToEntry(double[] v, int fr, int to, int nCol) {
 		final int sf = fr * nCol; // start from
 		final int st = to * nCol; // start to
 		addToOffsets(v, sf, st, nCol);
@@ -613,13 +746,13 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public void addToEntry(double[] v, int fr, int to, int nCol, int rep) {
+	public final void addToEntry(double[] v, int fr, int to, int nCol, int rep) {
 		final int sf = fr * nCol; // start from
 		final int st = to * nCol; // start to
 		addToOffsets(v, sf, st, nCol, rep);
 	}
 
-	private void addToOffsets(double[] v, int sf, int st, int nCol, int rep) {
+	private final void addToOffsets(double[] v, int sf, int st, int nCol, int rep) {
 		for(int i = sf, j = st; i < sf + nCol; i++, j++)
 			v[j] += _values[i] * rep;
 	}
@@ -649,12 +782,12 @@ public class Dictionary extends ADictionary {
 			for(int j = 0; j < tuple.length; i++, j++)
 				newValues[i] = _values[i] - tuple[j];
 
-		return new Dictionary(newValues);
+		return create(newValues);
 	}
 
 	@Override
 	public MatrixBlockDictionary getMBDict(int nCol) {
-		return MatrixBlockDictionary.createDictionary(_values, nCol);
+		return MatrixBlockDictionary.createDictionary(_values, nCol, true);
 	}
 
 	@Override
@@ -689,7 +822,7 @@ public class Dictionary extends ADictionary {
 				off++;
 			}
 		}
-		return new Dictionary(scaledValues);
+		return create(scaledValues);
 	}
 
 	@Override
@@ -704,7 +837,7 @@ public class Dictionary extends ADictionary {
 						ret[off + i] += v * b[idb + aggregateColumns[i]];
 			}
 		}
-		return new Dictionary(ret);
+		return create(ret);
 	}
 
 	@Override
@@ -714,7 +847,7 @@ public class Dictionary extends ADictionary {
 			final double v = _values[i];
 			retV[i] = v == pattern ? replace : v;
 		}
-		return new Dictionary(retV);
+		return create(retV);
 	}
 
 	@Override
@@ -730,7 +863,7 @@ public class Dictionary extends ADictionary {
 
 			}
 		}
-		return new Dictionary(retV);
+		return create(retV);
 	}
 
 	@Override
@@ -815,27 +948,54 @@ public class Dictionary extends ADictionary {
 		// should be guaranteed to only contain one value per tuple in dictionary.
 		for(int i = 0; i < _values.length; i++)
 			fn.execute(ret, _values[i], counts[i]);
+
+		if(ret.getWeight() < nRows)
+			fn.execute(ret, 0, nRows - ret.getWeight());
 		return ret;
 	}
 
 	@Override
-	public CM_COV_Object centralMomentWithReference(CM_COV_Object ret, ValueFunction fn, int[] counts, double reference,
+	public CM_COV_Object centralMomentWithDefault(CM_COV_Object ret, ValueFunction fn, int[] counts, double def,
 		int nRows) {
 		// should be guaranteed to only contain one value per tuple in dictionary.
 		for(int i = 0; i < _values.length; i++)
-			fn.execute(ret, _values[i] + reference, counts[i]);
+			fn.execute(ret, _values[i], counts[i]);
+
+		if(ret.getWeight() < nRows)
+			fn.execute(ret, def, nRows - ret.getWeight());
 		return ret;
 	}
 
 	@Override
-	public ADictionary rexpandCols(int max, boolean ignore, boolean cast, int nCol) {
-		return getMBDict(nCol).rexpandCols(max, ignore, cast, nCol);
+	public CM_COV_Object centralMomentWithReference(CM_COV_Object ret, ValueFunction fn, int[] counts, double reference,
+		int nRows) {
+		// should be guaranteed to only contain one value per tuple in dictionary.
+		for(int i = 0; i < _values.length; i++)
+			fn.execute(ret, _values[i] + reference, counts[i]);
+
+		if(ret.getWeight() < nRows)
+			fn.execute(ret, reference, nRows - ret.getWeight());
+		return ret;
 	}
 
 	@Override
-	public ADictionary rexpandColsWithReference(int max, boolean ignore, boolean cast, double reference) {
-		return getMBDict(1).applyScalarOp(new LeftScalarOperator(Plus.getPlusFnObject(), reference)).rexpandCols(max,
-			ignore, cast, 1);
+	public ADictionary rexpandCols(int max, boolean ignore, boolean cast, int nCol) {
+		MatrixBlockDictionary a = getMBDict(nCol);
+		if(a == null)
+			return null;
+		return a.rexpandCols(max, ignore, cast, nCol);
+	}
+
+	@Override
+	public ADictionary rexpandColsWithReference(int max, boolean ignore, boolean cast, int reference) {
+		MatrixBlockDictionary a = getMBDict(1);
+		if(a == null)
+			a = new MatrixBlockDictionary(new MatrixBlock(_values.length, 1, (double) reference));
+		else
+			a = (MatrixBlockDictionary) a.applyScalarOp(new LeftScalarOperator(Plus.getPlusFnObject(), reference));
+		if(a == null)
+			return null;
+		return a.rexpandCols(max, ignore, cast, 1);
 	}
 
 	@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 be1b095679..c9bc6524bc 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
@@ -34,6 +34,7 @@ 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.data.SparseRowVector;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 public interface DictionaryFactory {
@@ -84,7 +85,7 @@ public interface DictionaryFactory {
 				}
 				retB.recomputeNonZeros();
 				retB.examSparsity(true);
-				return new MatrixBlockDictionary(retB, nCols);
+				return new MatrixBlockDictionary(retB);
 			}
 			else {
 
@@ -93,7 +94,7 @@ public interface DictionaryFactory {
 					final DArrCounts dac = vals.get(i);
 					System.arraycopy(dac.key.getData(), 0, resValues, dac.id * nCols, nCols);
 				}
-				return new Dictionary(resValues);
+				return Dictionary.create(resValues);
 			}
 		}
 		catch(Exception e) {
@@ -113,7 +114,7 @@ public interface DictionaryFactory {
 	public static ADictionary create(ABitmap ubm, double sparsity) {
 		final int nCol = ubm.getNumColumns();
 		if(ubm instanceof Bitmap)
-			return new Dictionary(((Bitmap) ubm).getValues());
+			return Dictionary.create(((Bitmap) ubm).getValues());
 		else if(sparsity < 0.4 && nCol > 4 && ubm instanceof MultiColBitmap) {
 			final MultiColBitmap mcbm = (MultiColBitmap) ubm;
 
@@ -129,7 +130,7 @@ public interface DictionaryFactory {
 			}
 			m.recomputeNonZeros();
 			m.examSparsity(true);
-			return new MatrixBlockDictionary(m, nCol);
+			return new MatrixBlockDictionary(m);
 		}
 		else if(ubm instanceof MultiColBitmap) {
 			MultiColBitmap mcbm = (MultiColBitmap) ubm;
@@ -138,7 +139,7 @@ public interface DictionaryFactory {
 			for(int i = 0; i < nVals; i++)
 				System.arraycopy(mcbm.getValues(i), 0, resValues, i * nCol, nCol);
 
-			return new Dictionary(resValues);
+			return Dictionary.create(resValues);
 		}
 		throw new NotImplementedException("Not implemented creation of bitmap type : " + ubm.getClass().getSimpleName());
 	}
@@ -148,25 +149,24 @@ public interface DictionaryFactory {
 		final int nCol = ubm.getNumColumns();
 		final int nVal = ubm.getNumValues() - (addZero ? 0 : 1);
 		if(nCol > 4 && sparsity < 0.4) {
-			final MultiColBitmap mcbm = (MultiColBitmap) ubm;
+			final MultiColBitmap mcbm = (MultiColBitmap) ubm; // always multi column
+
 			final MatrixBlock m = new MatrixBlock(nVal, nCol, true);
 			m.allocateSparseRowsBlock();
 			final SparseBlock sb = m.getSparseBlock();
 
-			for(int i = 0; i < defaultIndex; i++) {
-				final double[] tuple = mcbm.getValues(i);
-				for(int col = 0; col < nCol; col++)
-					sb.append(i, col, tuple[col]);
-			}
+			for(int i = 0; i < defaultIndex; i++)
+				sb.set(i, new SparseRowVector(mcbm.getValues(i)), false);
+
+			// copy default
 			System.arraycopy(mcbm.getValues(defaultIndex), 0, defaultTuple, 0, nCol);
-			for(int i = defaultIndex; i < ubm.getNumValues() - 1; i++) {
-				final double[] tuple = mcbm.getValues(i);
-				for(int col = 0; col < nCol; col++)
-					sb.append(i, col, tuple[col]);
-			}
+
+			for(int i = defaultIndex; i < ubm.getNumValues() - 1; i++)
+				sb.set(i, new SparseRowVector(mcbm.getValues(i + 1)), false);
+
 			m.recomputeNonZeros();
 			m.examSparsity(true);
-			return new MatrixBlockDictionary(m, nCol);
+			return new MatrixBlockDictionary(m);
 		}
 		else {
 			double[] dict = new double[nCol * nVal];
@@ -177,7 +177,7 @@ public interface DictionaryFactory {
 				System.arraycopy(bmv, defaultIndex + 1, dict, defaultIndex, bmv.length - defaultIndex - 1);
 			}
 			else if(ubm instanceof MultiColBitmap) {
-				MultiColBitmap mcbm = (MultiColBitmap) ubm;
+				final MultiColBitmap mcbm = (MultiColBitmap) ubm;
 				for(int i = 0; i < defaultIndex; i++)
 					System.arraycopy(mcbm.getValues(i), 0, dict, i * nCol, nCol);
 				System.arraycopy(mcbm.getValues(defaultIndex), 0, defaultTuple, 0, nCol);
@@ -187,7 +187,7 @@ public interface DictionaryFactory {
 			else
 				throw new NotImplementedException("not supported ABitmap of type:" + ubm.getClass().getSimpleName());
 
-			return new Dictionary(dict);
+			return Dictionary.create(dict);
 		}
 	}
 
@@ -200,7 +200,7 @@ public interface DictionaryFactory {
 			final double[] resValues = new double[nRows];
 			final double[] from = ((Bitmap) ubm).getValues();
 			System.arraycopy(from, 0, resValues, 0, from.length);
-			return new Dictionary(resValues);
+			return Dictionary.create(resValues);
 		}
 
 		final MultiColBitmap mcbm = (MultiColBitmap) ubm;
@@ -216,18 +216,18 @@ public interface DictionaryFactory {
 			}
 			m.recomputeNonZeros();
 			m.examSparsity(true);
-			return new MatrixBlockDictionary(m, nCols);
+			return new MatrixBlockDictionary(m);
 		}
 
 		final double[] resValues = new double[nRows * nCols];
 		for(int i = 0; i < nVals; i++)
 			System.arraycopy(mcbm.getValues(i), 0, resValues, i * nCols, nCols);
 
-		return new Dictionary(resValues);
+		return Dictionary.create(resValues);
 	}
 
 	public static ADictionary create(DoubleCountHashMap map) {
 		final double[] resValues = map.getDictionary();
-		return new Dictionary(resValues);
+		return Dictionary.create(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 b3baa7db56..1435e29793 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
@@ -41,6 +41,7 @@ import org.apache.sysds.runtime.functionobjects.Minus;
 import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
+import org.apache.sysds.runtime.matrix.data.LibMatrixAgg;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
@@ -59,30 +60,35 @@ public class MatrixBlockDictionary extends ADictionary {
 	 * 
 	 * @param data The matrix block data.
 	 */
-	private MatrixBlockDictionary(MatrixBlock data) {
-		_data = data;
+	public MatrixBlockDictionary(MatrixBlock data) {
+		this(data, true);
 	}
 
-	public MatrixBlockDictionary(MatrixBlock data, int nCol) {
-		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.isInSparseFormat() && data.getSparseBlock() instanceof SparseBlockMCSR) {
-			SparseBlock csr = SparseBlockFactory.copySparseBlock(SparseBlock.Type.CSR, data.getSparseBlock(), false);
-			data.setSparseBlock(csr);
+	/**
+	 * Unsafe private constructor that does not check the data validity. USE WITH CAUTION.
+	 * 
+	 * @param data  The matrix block data.
+	 * @param check Check the nonZeros in the dict
+	 */
+	public MatrixBlockDictionary(MatrixBlock data, boolean check) {
+		if(check) {
+			data.examSparsity(true);
+			if(data.isEmpty())
+				throw new DMLCompressionException("Invalid construction of empty dictionary");
+			else if(data.isInSparseFormat() && data.getSparseBlock() instanceof SparseBlockMCSR) {
+				SparseBlock csr = SparseBlockFactory.copySparseBlock(SparseBlock.Type.CSR, data.getSparseBlock(), false);
+				data.setSparseBlock(csr);
+			}
 		}
-
 		_data = data;
 	}
 
-	public static MatrixBlockDictionary createDictionary(double[] values, int nCol) {
-		MatrixBlock nd = Util.matrixBlockFromDenseArray(values, nCol);
-		nd.examSparsity(true);
-		if(nd.isEmpty())
+	public static MatrixBlockDictionary createDictionary(double[] values, int nCol, boolean check) {
+		final MatrixBlock mb = Util.matrixBlockFromDenseArray(values, nCol, check);
+		if(mb.isEmpty())
 			return null;
-		return new MatrixBlockDictionary(nd);
+		else
+			return new MatrixBlockDictionary(mb, check);
 	}
 
 	public MatrixBlock getMatrixBlock() {
@@ -109,6 +115,11 @@ public class MatrixBlockDictionary extends ADictionary {
 		return _data.quickGetValue(row, col);
 	}
 
+	@Override
+	public final double getValue(int r, int c, int nCol) {
+		return _data.quickGetValue(r, c);
+	}
+
 	@Override
 	public long getInMemorySize() {
 		// object reference to a matrix block + matrix block size.
@@ -140,7 +151,7 @@ public class MatrixBlockDictionary extends ADictionary {
... 8089 lines suppressed ...