You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by ba...@apache.org on 2021/06/01 10:43:07 UTC

[systemds] branch master updated (4cb9321 -> d430902)

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

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


    from 4cb9321  [SYSTEMDS-2991,2994] Initial workload analysis for compression planning
     new bc2602f  [SYSTEMDS-2999] CLA Decompression Unification
     new e7abe87  [SYSTEMDS-2997] CLA MatrixBlock Dictionary
     new f7b20d7  [SYSTEMDS-3001] CLA MM Left using existing kernels
     new e8da72f  [SYSTEMDS-2992] CLA init workload cost functions
     new 0f8e302  [SYSTEMDS-2997] CLA MatrixBlock Dictionary Update
     new 9bfd7ff  [SYSTEMDS-2998] CLA Offset and Mapping Tests
     new d430902  [SYSTEMDS-2746] CLA ReplaceOperation

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


Summary of changes:
 src/main/java/org/apache/sysds/conf/DMLConfig.java |    2 +-
 .../runtime/compress/CompressedMatrixBlock.java    |  210 ++--
 .../compress/CompressedMatrixBlockFactory.java     |   40 +-
 .../runtime/compress/CompressionSettings.java      |   36 +-
 .../compress/CompressionSettingsBuilder.java       |   72 +-
 .../runtime/compress/CompressionStatistics.java    |   32 +-
 .../runtime/compress/cocode/AColumnCoCoder.java    |   12 +-
 .../runtime/compress/cocode/CoCodeBinPacking.java  |    4 +-
 .../sysds/runtime/compress/cocode/CoCodeCost.java  |   33 +-
 .../compress/cocode/CoCodeCostMatrixMult.java      |  138 +++
 .../runtime/compress/cocode/CoCodeCostTSMM.java    |  188 ++++
 .../runtime/compress/cocode/CoCodeStatic.java      |    4 +-
 .../runtime/compress/cocode/PlanningCoCoder.java   |   34 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |  371 ++-----
 .../compress/colgroup/ColGroupCompressed.java      |   96 +-
 .../runtime/compress/colgroup/ColGroupConst.java   |  214 +---
 .../runtime/compress/colgroup/ColGroupDDC.java     |  423 ++------
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   65 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |  197 ++--
 .../runtime/compress/colgroup/ColGroupOLE.java     |  712 +-----------
 .../runtime/compress/colgroup/ColGroupOffset.java  |   24 +-
 .../runtime/compress/colgroup/ColGroupRLE.java     |  615 +----------
 .../runtime/compress/colgroup/ColGroupSDC.java     |  511 +++------
 .../compress/colgroup/ColGroupSDCSingle.java       |  470 +++-----
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  287 ++---
 .../compress/colgroup/ColGroupSDCZeros.java        |  311 ++----
 .../runtime/compress/colgroup/ColGroupSizes.java   |   68 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  367 +++----
 .../runtime/compress/colgroup/ColGroupValue.java   | 1140 +++++++++++---------
 .../compress/colgroup/dictionary/ADictionary.java  |  245 ++++-
 .../compress/colgroup/dictionary/Dictionary.java   |  188 +++-
 .../colgroup/dictionary/DictionaryFactory.java     |  230 +++-
 .../colgroup/dictionary/MatrixBlockDictionary.java |  652 +++++++++++
 .../compress/colgroup/dictionary/QDictionary.java  |  125 ++-
 .../colgroup/insertionsort/AInsertionSorter.java   |   68 ++
 .../insertionsort/InsertionSorterFactory.java      |   47 +
 .../{tree => insertionsort}/MaterializeSort.java   |   69 +-
 .../{tree => insertionsort}/MergeSort.java         |  104 +-
 .../compress/colgroup/mapping/MapToBit.java        |   24 +-
 .../compress/colgroup/mapping/MapToByte.java       |    4 +-
 .../compress/colgroup/mapping/MapToChar.java       |    4 +-
 .../compress/colgroup/mapping/MapToFactory.java    |   45 +-
 .../compress/colgroup/mapping/MapToInt.java        |    4 +-
 .../runtime/compress/colgroup/offset/AOffset.java  |   15 +-
 .../compress/colgroup/offset/OffsetByte.java       |    6 +-
 .../compress/colgroup/offset/OffsetChar.java       |    7 +-
 .../compress/colgroup/offset/OffsetFactory.java    |   30 +-
 .../compress/colgroup/pre/IPreAggregate.java       |   79 --
 .../compress/colgroup/pre/MapPreAggregate.java     |   62 --
 .../compress/colgroup/pre/PreAggregateFactory.java |   41 -
 .../compress/colgroup/tree/AInsertionSorter.java   |   95 --
 .../colgroup/tree/InsertionSorterFactory.java      |   33 -
 .../runtime/compress/colgroup/tree/Naive.java      |  136 ---
 .../compress/estim/CompressedSizeEstimator.java    |  123 ++-
 .../estim/CompressedSizeEstimatorExact.java        |   10 +-
 .../estim/CompressedSizeEstimatorFactory.java      |   52 +-
 .../estim/CompressedSizeEstimatorSample.java       |  122 +--
 .../compress/estim/CompressedSizeInfoColGroup.java |   45 +-
 .../runtime/compress/estim/EstimationFactors.java  |   17 +-
 .../sysds/runtime/compress/lib/BitmapEncoder.java  |   59 +-
 .../runtime/compress/lib/BitmapLossyEncoder.java   |   84 +-
 .../runtime/compress/lib/CLALibBinaryCellOp.java   |   51 +-
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |    3 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |  330 ++----
 .../runtime/compress/lib/CLALibRelationalOp.java   |   13 +-
 .../runtime/compress/lib/CLALibRightMultBy.java    |   82 +-
 .../sysds/runtime/compress/lib/CLALibScalar.java   |    4 +-
 .../sysds/runtime/compress/lib/CLALibSquash.java   |   22 +-
 .../compress/readers/ReaderColumnSelection.java    |    3 +-
 .../readers/ReaderColumnSelectionBitSet.java       |   16 +-
 .../ReaderColumnSelectionDenseMultiBlock.java      |    8 +-
 ...erColumnSelectionDenseMultiBlockTransposed.java |   12 +-
 .../ReaderColumnSelectionDenseSingleBlock.java     |   11 +-
 ...rColumnSelectionDenseSingleBlockTransposed.java |   18 +-
 .../readers/ReaderColumnSelectionSparse.java       |    4 +-
 .../ReaderColumnSelectionSparseTransposed.java     |    5 +-
 .../readers/ReaderCompressedSelection.java         |    6 +-
 .../sysds/runtime/compress/utils/ABitmap.java      |   21 +-
 .../sysds/runtime/compress/utils/Bitmap.java       |   29 +-
 .../sysds/runtime/compress/utils/DblArray.java     |   15 +-
 .../utils/{Bitmap.java => MultiColBitmap.java}     |   43 +-
 .../sysds/runtime/matrix/data/LibMatrixMult.java   |    2 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |   39 +-
 .../sysds/runtime/matrix/data/MatrixValue.java     |    4 +-
 .../runtime/matrix/data/RandomMatrixGenerator.java |    2 +-
 .../org/apache/sysds/utils/MemoryEstimates.java    |    2 +
 src/test/java/org/apache/sysds/test/TestUtils.java |   33 +-
 .../compress/AbstractCompressedUnaryTests.java     |   16 +-
 .../component/compress/CompressedMatrixTest.java   |  292 ++++-
 .../component/compress/CompressedTestBase.java     |  367 +++++--
 .../component/compress/CompressedVectorTest.java   |   11 +-
 .../compress/CompressibleInputGenerator.java       |   15 +
 .../compress/ParCompressedMatrixTest.java          |   30 +-
 .../sysds/test/component/compress/TestBase.java    |   85 +-
 .../test/component/compress/TestConstants.java     |   31 +-
 .../compress/colgroup/JolEstimateOLETest.java      |    6 +-
 .../compress/colgroup/JolEstimateTest.java         |  109 +-
 .../compress/estim/SampleEstimatorTest.java        |  119 ++
 .../insertionsort/TestInsertionSorters.java        |  128 +++
 .../compress/insertionsorter/MergeSortTest.java    |  276 -----
 .../component/compress/mapping/MappingTests.java   |  182 ++++
 .../compress/offset/OffsetNegativeTests.java       |   90 ++
 .../compress/offset/OffsetSingleTests.java}        |   38 +-
 .../component/compress/offset/OffsetTests.java     |  132 ++-
 104 files changed, 5955 insertions(+), 6251 deletions(-)
 create mode 100644 src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
 create mode 100644 src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostTSMM.java
 create mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java
 create mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/insertionsort/AInsertionSorter.java
 create mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/insertionsort/InsertionSorterFactory.java
 rename src/main/java/org/apache/sysds/runtime/compress/colgroup/{tree => insertionsort}/MaterializeSort.java (50%)
 rename src/main/java/org/apache/sysds/runtime/compress/colgroup/{tree => insertionsort}/MergeSort.java (56%)
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/pre/IPreAggregate.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/pre/MapPreAggregate.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/pre/PreAggregateFactory.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/tree/AInsertionSorter.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/tree/InsertionSorterFactory.java
 delete mode 100644 src/main/java/org/apache/sysds/runtime/compress/colgroup/tree/Naive.java
 copy src/main/java/org/apache/sysds/runtime/compress/utils/{Bitmap.java => MultiColBitmap.java} (70%)
 create mode 100644 src/test/java/org/apache/sysds/test/component/compress/estim/SampleEstimatorTest.java
 create mode 100644 src/test/java/org/apache/sysds/test/component/compress/insertionsort/TestInsertionSorters.java
 delete mode 100644 src/test/java/org/apache/sysds/test/component/compress/insertionsorter/MergeSortTest.java
 create mode 100644 src/test/java/org/apache/sysds/test/component/compress/mapping/MappingTests.java
 create mode 100644 src/test/java/org/apache/sysds/test/component/compress/offset/OffsetNegativeTests.java
 rename src/{main/java/org/apache/sysds/runtime/compress/colgroup/pre/ArrPreAggregate.java => test/java/org/apache/sysds/test/component/compress/offset/OffsetSingleTests.java} (59%)

[systemds] 03/07: [SYSTEMDS-3001] CLA MM Left using existing kernels

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

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

commit f7b20d78cff111106ecc4248352efe39db7cefb0
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Tue May 11 21:15:12 2021 +0200

    [SYSTEMDS-3001] CLA MM Left using existing kernels
    
    Compressed left multiplication have two phases, first pre aggregation
    then a matrix multiplication. This commit make the matrix multiplication
    use the default systemds kernels. This allows for exploitation of the
    various dedicated MM kernels already in SystemDS.
---
 src/main/java/org/apache/sysds/conf/DMLConfig.java |   2 +-
 .../runtime/compress/CompressionSettings.java      |   5 +-
 .../compress/CompressionSettingsBuilder.java       |   4 +
 .../sysds/runtime/compress/cocode/CoCodeCost.java  |  27 +--
 .../compress/cocode/CoCodeCostMatrixMult.java      |  41 +++--
 .../sysds/runtime/compress/colgroup/AColGroup.java |  34 ++--
 .../compress/colgroup/ColGroupCompressed.java      |  38 ----
 .../runtime/compress/colgroup/ColGroupConst.java   |  43 +++--
 .../runtime/compress/colgroup/ColGroupDDC.java     |  25 +++
 .../runtime/compress/colgroup/ColGroupEmpty.java   |  10 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |   5 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     |   5 +
 .../runtime/compress/colgroup/ColGroupRLE.java     |   5 +
 .../runtime/compress/colgroup/ColGroupSDC.java     |   5 +
 .../compress/colgroup/ColGroupSDCSingle.java       |   5 +
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |   5 +
 .../compress/colgroup/ColGroupSDCZeros.java        |   9 +-
 .../runtime/compress/colgroup/ColGroupSizes.java   |   2 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  37 ++--
 .../runtime/compress/colgroup/ColGroupValue.java   | 193 ++++++++++-----------
 .../compress/estim/CompressedSizeEstimator.java    |  21 ++-
 .../estim/CompressedSizeEstimatorExact.java        |   7 +-
 .../estim/CompressedSizeEstimatorFactory.java      |  27 ++-
 .../estim/CompressedSizeEstimatorSample.java       |  93 ++++------
 .../compress/estim/CompressedSizeInfoColGroup.java |  14 +-
 .../runtime/compress/estim/EstimationFactors.java  |  17 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |  83 ++++-----
 .../sysds/runtime/compress/utils/ABitmap.java      |  21 ++-
 .../sysds/runtime/compress/utils/Bitmap.java       |   2 +-
 .../compress/colgroup/JolEstimateOLETest.java      |   6 +-
 .../compress/colgroup/JolEstimateTest.java         |  22 ++-
 31 files changed, 413 insertions(+), 400 deletions(-)

diff --git a/src/main/java/org/apache/sysds/conf/DMLConfig.java b/src/main/java/org/apache/sysds/conf/DMLConfig.java
index bf1767b..e978197 100644
--- a/src/main/java/org/apache/sysds/conf/DMLConfig.java
+++ b/src/main/java/org/apache/sysds/conf/DMLConfig.java
@@ -128,7 +128,7 @@ public class DMLConfig
 		_defaultVals.put(COMPRESSED_LOSSY,       "false" );
 		_defaultVals.put(COMPRESSED_VALID_COMPRESSIONS, "SDC,DDC");
 		_defaultVals.put(COMPRESSED_OVERLAPPING, "true" );
-		_defaultVals.put(COMPRESSED_SAMPLING_RATIO, "0.01");
+		_defaultVals.put(COMPRESSED_SAMPLING_RATIO, "NaN");
 		_defaultVals.put(COMPRESSED_COCODE,      "COST");
 		_defaultVals.put(COMPRESSED_TRANSPOSE,   "auto");
 		_defaultVals.put(CODEGEN,                "false" );
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 7be74ef..ddbc60b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -126,8 +126,9 @@ public class CompressionSettings {
 		sb.append("\n Lossy: " + lossy);
 		sb.append("\n sortValuesByLength: " + sortValuesByLength);
 		sb.append("\n column Partitioner: " + columnPartitioner);
-		sb.append("\n max Static ColGroup CoCode " + maxColGroupCoCode);
-		sb.append("\n max cocodePercentage " + coCodePercentage);
+		sb.append("\n Max Static ColGroup CoCode: " + maxColGroupCoCode);
+		sb.append("\n Max cocodePercentage: " + coCodePercentage);
+		sb.append("\n Sample Percentage: " + samplingRatio);
 		// If needed for debugging add more fields to the printing.
 		return sb.toString();
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index 72aeeeb..216267e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -44,6 +44,8 @@ public class CompressionSettingsBuilder {
 	private int maxStaticColGroupCoCode = 10000;
 	private double coCodePercentage = 0.01;
 
+	private final static double defaultSampleRate = 0.01;
+
 	public CompressionSettingsBuilder() {
 
 		DMLConfig conf = ConfigurationManager.getDMLConfig();
@@ -54,6 +56,8 @@ public class CompressionSettingsBuilder {
 			validCompressions.add(CompressionType.valueOf(comp));
 		}
 		samplingRatio = conf.getDoubleValue(DMLConfig.COMPRESSED_SAMPLING_RATIO);
+		if(Double.isNaN(samplingRatio))
+			samplingRatio = defaultSampleRate;
 		columnPartitioner = PartitionerType.valueOf(conf.getTextValue(DMLConfig.COMPRESSED_COCODE));
 
 		transposeInput = conf.getTextValue(DMLConfig.COMPRESSED_TRANSPOSE);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
index d2aec2c..66ad209 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
@@ -26,7 +26,6 @@ import java.util.PriorityQueue;
 import java.util.Queue;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
-import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
@@ -67,12 +66,8 @@ public class CoCodeCost extends AColumnCoCoder {
 		Queue<CompressedSizeInfoColGroup> que = new PriorityQueue<>(currentGroups.size(), comp);
 		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
 
-		for(CompressedSizeInfoColGroup g : currentGroups) {
-			if(g.getBestCompressionType() == CompressionType.CONST)
-				ret.add(g);
-			else
-				que.add(g);
-		}
+		for(CompressedSizeInfoColGroup g : currentGroups)
+			que.add(g);
 
 		boolean finished = false;
 		while(!finished) {
@@ -83,31 +78,27 @@ public class CoCodeCost extends AColumnCoCoder {
 					int worstCaseJoinedSize = l.getNumVals() * r.getNumVals();
 					if(worstCaseJoinedSize < toSmallForAnalysis)
 						que.add(joinWithoutAnalysis(l, r));
-					else if(worstCaseJoinedSize < largestDistinct){
-
+					else if(worstCaseJoinedSize < largestDistinct) {
 						CompressedSizeInfoColGroup g = joinWithAnalysis(l, r);
 						if(g.getNumVals() < largestDistinct)
 							que.add(joinWithAnalysis(l, r));
-						else{
-							finished = true;
-							que.add(l);
+						else {
+							ret.add(l);
 							que.add(r);
 						}
 					}
 					else {
-						finished = true;
-						que.add(l);
+						ret.add(l);
 						que.add(r);
 					}
 				}
-				else {
-					que.add(l);
-					finished = true;
-				}
+				else
+					ret.add(l);
 			}
 			else
 				finished = true;
 		}
+
 		for(CompressedSizeInfoColGroup g : que)
 			ret.add(g);
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
index 09a9990..910c94a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
@@ -26,6 +26,7 @@ import java.util.PriorityQueue;
 import java.util.Queue;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
@@ -40,20 +41,8 @@ import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
  */
 public class CoCodeCostMatrixMult extends AColumnCoCoder {
 
-	/**
-	 * This value specifies the maximum distinct count allowed int a coCoded group. Note that this value is the number
-	 * of distinct tuples not the total number of values. That value can be calculated by multiplying the number of
-	 * tuples with columns in the coCoded group.
-	 */
-	private final int largestDistinct;
-
-	private final int toSmallForAnalysis;
-
 	protected CoCodeCostMatrixMult(CompressedSizeEstimator e, CompressionSettings cs, int numRows) {
 		super(e, cs, numRows);
-		largestDistinct = Math.max(256, (int) (_est.getNumRows() * _est.getNumColumns() * cs.coCodePercentage * 0.2));
-		toSmallForAnalysis = Math.min(Math.max(256, largestDistinct / 4), 1028);
-		LOG.debug("CocodeCost largest Distinct: " + largestDistinct + " toSmallForAnalysis: " + toSmallForAnalysis);
 	}
 
 	@Override
@@ -104,11 +93,31 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 
 		protected CostOfJoin(CompressedSizeInfoColGroup elm) {
 			this.elm = elm;
-			final int nRows = _est.getNumRows();
-			final double commonFraction = elm.getMostCommonFraction();
-			final double rowsToProcess = commonFraction > 0.2 ? nRows * (1 - Math.min(commonFraction, 0.95)) : nRows;
-			this.cost = rowsToProcess + elm.getNumVals() * elm.getColumns().length;
 
+			final double constantOverheadForColGroup = 5;
+			final double nCols = elm.getColumns().length;
+			final double nRows = _est.getNumRows();
+			if(elm.getBestCompressionType() == CompressionType.UNCOMPRESSED)
+				this.cost = nRows * nCols * 2 + constantOverheadForColGroup;
+			else {
+				final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+
+				// LOG.error(constantOverheadForColGroup);
+				final double commonFraction = elm.getMostCommonFraction();
+				final double rowsCost = commonFraction > 0.2 ? nRows * (1 - commonFraction) : nRows;
+				// this.cost = rowsToProcess + elm.getNumVals() * nCols + constantOverheadForColGroup;
+				// this.cost = rowsToProcess + elm.getNumVals() * nCols * (1 - commonFraction) +
+				// constantOverheadForColGroup;
+				// final double sparsity_tuple_effect = elm.getTupleSparsity() > 0.4 ? 1 -
+				// Math.min(elm.getTupleSparsity(), 0.9) : 1;
+				final int numberTuples = elm.getNumVals();
+				final double tuplesCost = (numberTuples < blksz) ? numberTuples : numberTuples * 2;
+
+				// this.cost = elementsCost;
+				// this.cost = rowsCost + tuplesCost * sparsity_tuple_effect + constantOverheadForColGroup;
+
+				this.cost = rowsCost + tuplesCost + constantOverheadForColGroup;
+			}
 		}
 
 		@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
index 8de274f..ddc6195 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
@@ -524,37 +524,33 @@ public abstract class AColGroup implements Serializable {
 	/**
 	 * Left multiply with this column group
 	 * 
-	 * @param matrix  The matrix to multiply with on the left
-	 * @param result  The result to output the values into, always dense for the purpose of the column groups
-	 *                parallelizing
-	 * @param numCols The number of columns contained in the CompressedMatrixBlock that this column group is inside.
+	 * @param matrix The matrix to multiply with on the left
+	 * @param result The result to output the values into, always dense for the purpose of the column groups
+	 *               parallelizing
 	 */
-	public void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols) {
-		leftMultByMatrix(matrix, result, numCols, 0, matrix.getNumRows());
+	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result) {
+		leftMultByMatrix(matrix, result, 0, matrix.getNumRows());
 	}
 
 	/**
 	 * Left multiply with this column group.
 	 * 
-	 * @param matrix  The matrix to multiply with on the left
-	 * @param result  The result to output the values into, always dense for the purpose of the column groups
-	 *                parallelizing
-	 * @param numCols The number of columns contained in the CompressedMatrixBlock that this column group is inside.
-	 * @param rl      The row to begin the multiplication from
-	 * @param ru      The row to end the multiplication at.
+	 * @param matrix The matrix to multiply with on the left
+	 * @param result The result to output the values into, always dense for the purpose of the column groups
+	 *               parallelizing
+	 * @param rl     The row to begin the multiplication from on the lhs matrix
+	 * @param ru     The row to end the multiplication at on the lhs matrix
 	 */
-	public abstract void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru);
+	public abstract void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru);
 
 	/**
 	 * 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.
-	 * @param result  The result matrix to insert the result of the multiplication into
-	 * @param numRows The number of rows in the left hand side matrix
-	 * @param numCols The number of columns in the right hand side matrix
+	 * @param lhs    The left hand side Column group to multiply with, the left hand side should be considered
+	 *               transposed.
+	 * @param result The result matrix to insert the result of the multiplication into
 	 */
-	public abstract void leftMultByAColGroup(AColGroup lhs, double[] result, int numRows, int numCols);
+	public abstract void leftMultByAColGroup(AColGroup lhs, MatrixBlock result);
 
 	/**
 	 * Perform the specified scalar operation directly on the compressed column group, without decompressing individual
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
index c8f9a41..c5d29ba 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
@@ -20,7 +20,6 @@
 package org.apache.sysds.runtime.compress.colgroup;
 
 import org.apache.sysds.runtime.DMLScriptException;
-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.KahanPlus;
@@ -30,7 +29,6 @@ import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.functionobjects.ReduceCol;
 import org.apache.sysds.runtime.functionobjects.ReduceRow;
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 
 /**
@@ -89,42 +87,6 @@ public abstract class ColGroupCompressed extends AColGroup {
 
 	protected abstract boolean sameIndexStructure(ColGroupCompressed that);
 
-	/**
-	 * Multiply with a matrix on the left.
-	 * 
-	 * @param matrix  matrix to left multiply
-	 * @param result  matrix block result
-	 * @param numRows The number of rows in the matrix input
-	 * @param numCols The number of columns in the colGroups parent matrix.
-	 * @param rl      The row to start the matrix multiplication from
-	 * @param ru      The row to stop the matrix multiplication at.
-	 */
-	public abstract void leftMultByMatrix(double[] matrix, double[] result, int numRows, int numCols, int rl, int ru);
-
-	/**
-	 * Multiply with a sparse matrix on the left hand side, and add the values to the output result
-	 * 
-	 * @param sb      The sparse block to multiply with
-	 * @param result  The linearized output matrix
-	 * @param numRows The number of rows in the left hand side input matrix (the sparse one)
-	 * @param numCols The number of columns in the compression.
-	 * @param rl      The row to start the matrix multiplication from
-	 * @param ru      The row to stop the matrix multiplication at.
-	 */
-	public abstract void leftMultBySparseMatrix(SparseBlock sb, double[] result, int numRows, int numCols, int rl,
-		int ru);
-
-	@Override
-	public final void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru) {
-		if(matrix.isEmpty())
-			return;
-		else if(matrix.isInSparseFormat())
-			leftMultBySparseMatrix(matrix.getSparseBlock(), result, matrix.getNumRows(), numCols, rl, ru);
-		else {
-			leftMultByMatrix(matrix.getDenseBlockValues(), result, matrix.getNumRows(), numCols, rl, ru);
-		}
-	}
-
 	@Override
 	public final double getMin() {
 		return computeMxx(Double.POSITIVE_INFINITY, Builtin.getBuiltinFnObject(BuiltinCode.MIN));
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 eb38ee0..8f6299c 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
@@ -210,23 +210,31 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public void leftMultByMatrix(double[] a, double[] c, double[] values, int numRows, int numCols, int rl, int ru) {
-		for(int i = rl; i < ru; i++) {
-			double preAggVals = preAggregateSingle(a, i);
-			int offC = i * numCols;
-			for(int j = 0; j < _colIndexes.length; j++) {
-				c[offC + _colIndexes[j]] += preAggVals * values[j];
+	public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, double[] values, int rl, int ru) {
+		final double[] cV = c.getDenseBlockValues();
+		if(a.isEmpty())
+			return;
+		else if(a.isInSparseFormat()) {
+			SparseBlock sb = a.getSparseBlock();
+			for(int i = rl; i < ru; i++) {
+
+				if(!sb.isEmpty(i)) {
+					double v = preAggregateSparseSingle(sb, i);
+					int offC = i * c.getNumColumns();
+					for(int j = 0; j < _colIndexes.length; j++)
+						cV[offC + _colIndexes[j]] += v * values[j];
+
+				}
 			}
 		}
-	}
+		else {
+			double[] aV = a.getDenseBlockValues();
+			for(int i = rl; i < ru; i++) {
+				double preAggVals = preAggregateSingle(aV, i);
+				int offC = i * c.getNumColumns();
+				for(int j = 0; j < _colIndexes.length; j++)
+					cV[offC + _colIndexes[j]] += preAggVals * values[j];
 
-	@Override
-	public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int row) {
-		if(!sb.isEmpty(row)) {
-			double v = preAggregateSparseSingle(sb, row);
-			int offC = row * numCols;
-			for(int j = 0; j < _colIndexes.length; j++) {
-				c[offC + _colIndexes[j]] += v * values[j];
 			}
 		}
 	}
@@ -315,7 +323,7 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new DMLCompressionException("Does not make sense to call this");
 	}
 
@@ -328,4 +336,9 @@ public class ColGroupConst extends ColGroupValue {
 	protected boolean sameIndexStructure(ColGroupCompressed that) {
 		return that instanceof ColGroupEmpty || that instanceof ColGroupConst;
 	}
+
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		throw new NotImplementedException();
+	}
 }
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 aa8a55f..df45f56 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
@@ -32,6 +32,8 @@ 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.pre.IPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
+import org.apache.sysds.runtime.data.DenseBlock;
+import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -231,6 +233,29 @@ public class ColGroupDDC extends ColGroupValue {
 
 	}
 
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+
+		final int retCols = getNumValues();
+		final int retRows = ru - rl;
+		final double[] vals = allocDVector(retRows * retCols, true);
+		final DenseBlock retB = new DenseBlockFP64(new int[] {retRows, retCols}, vals);
+		final MatrixBlock ret = new MatrixBlock(retRows, retCols, retB);
+
+		final double[] mV = m.getDenseBlockValues();
+
+		ret.setNonZeros(retRows * retCols);
+		for(int k = rl; k < ru; k++) {
+			final int offT = ret.getNumColumns() * k;
+			final int offM = m.getNumColumns() * k;
+			for(int i = 0; i < _numRows; i++) {
+				int index = _data.getIndex(i);
+				vals[offT + index] += mV[offM + i];
+			}
+		}
+		return ret;
+	}
+
 	/**
 	 * Generic get value for byte-length-agnostic access to first column.
 	 * 
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 2046d18..c873b1a 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
@@ -20,7 +20,6 @@
 package org.apache.sysds.runtime.compress.colgroup;
 
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
-import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
@@ -125,12 +124,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	}
 
 	@Override
-	public void leftMultByMatrix(double[] a, double[] c, int numRows, int numCols, int rl, int ru) {
-		// do nothing.
-	}
-
-	@Override
-	public void leftMultBySparseMatrix(SparseBlock sb, double[] c, int numRows, int numCols, int rl, int ru) {
+	public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, int rl, int ru) {
 		// do nothing.
 	}
 
@@ -214,7 +208,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, double[] result, int numRows, int numCols) {
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock c) {
 		// do nothing
 	}
 
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 f16204e..62acf71 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
@@ -191,10 +191,11 @@ public class ColGroupFactory {
 
 	private static AColGroup compressColGroupForced(MatrixBlock in, int[] colIndexes,
 		CompressionSettings compSettings) {
+			
+		ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, compSettings.transposed);
 
-		CompressedSizeEstimator estimator = new CompressedSizeEstimatorExact(in, compSettings, compSettings.transposed);
+		CompressedSizeEstimator estimator = new CompressedSizeEstimatorExact(in, compSettings);
 
-		ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, compSettings.transposed);
 		CompressedSizeInfoColGroup sizeInfo = new CompressedSizeInfoColGroup(
 			estimator.estimateCompressedColGroupSize(ubm, colIndexes), compSettings.validCompressions);
 
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 8101f40..2df2878 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
@@ -1212,4 +1212,9 @@ public class ColGroupOLE extends ColGroupOffset {
 	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		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 1e79abc..1649591 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
@@ -1163,4 +1163,9 @@ public class ColGroupRLE extends ColGroupOffset {
 	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		throw new NotImplementedException();
+	}
 }
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 268f294..e449259 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
@@ -744,4 +744,9 @@ public class ColGroupSDC extends ColGroupValue {
 		return ret;
 	}
 
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		throw new NotImplementedException();
+	}
+
 }
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 14afdf4..58649c3 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
@@ -669,4 +669,9 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		}
 
 	}
+
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		throw new NotImplementedException();
+	}
 }
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 34d148d..94be81c 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
@@ -453,4 +453,9 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		throw new NotImplementedException();
+	}
 }
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 410ec90..23b06a9 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
@@ -534,9 +534,14 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		}
 		return ret;
 	}
-	
+
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary re, boolean preModified) {
+		throw new NotImplementedException();
+	}
+
 	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary re, boolean preModified){
+	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
 		throw new NotImplementedException();
 	}
 }
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 f9bed0e..3c8c50a 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
@@ -124,7 +124,7 @@ public class ColGroupSizes {
 		// Since the Object is a col group the overhead from the Memory Size group is added
 		size += estimateInMemorySizeGroup(nrColumns);
 		size += 8; // reference to MatrixBlock.
-		size += MatrixBlock.estimateSizeInMemory(nrRows, nrColumns, sparsity);
+		size += MatrixBlock.estimateSizeInMemory(nrRows, nrColumns, (nrColumns > 1) ?  sparsity : 1);
 		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 a848031..3cec6fb 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
@@ -315,33 +315,34 @@ public class ColGroupUncompressed extends AColGroup {
 		LibMatrixMult.matrixMult(_data, subMatrix, result);
 	}
 
-	public void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru) {
+	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 = numCols * rl;
+		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 += numCols) {
+			for(int rowIdx = 0; rowIdx < ru - rl; rowIdx++, offT += result.getNumColumns()) {
 				if(!sb.isEmpty(rowIdx)) {
 					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++)
-						result[offT + _colIndexes[aix[col]]] += avals[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 += numCols)
+			for(int j = rl, offTemp = 0; j < ru; j++, offTemp += _colIndexes.length, offT += result.getNumColumns())
 				for(int i = 0; i < _colIndexes.length; i++)
-					result[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
+					resV[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
 		}
 	}
 
@@ -565,7 +566,7 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, double[] result, final int numRows, final int numCols) {
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
 		if(lhs instanceof ColGroupEmpty)
 			return;
 		if(lhs instanceof ColGroupUncompressed) {
@@ -587,18 +588,19 @@ public class ColGroupUncompressed extends AColGroup {
 				LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
 			}
 
+			final double[] resV = result.getDenseBlockValues();
 			if(tmpRet.isEmpty())
 				return;
 			else if(tmpRet.isInSparseFormat()) {
 				SparseBlock sb = tmpRet.getSparseBlock();
-				for(int rowIdx = 0, offT = 0; rowIdx < tmpRet.getNumRows(); rowIdx++, offT += numCols) {
+				for(int rowIdx = 0, offT = 0; rowIdx < tmpRet.getNumRows(); rowIdx++, offT += result.getNumColumns()) {
 					if(!sb.isEmpty(rowIdx)) {
 						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++)
-							result[offT + _colIndexes[aix[col]]] += avals[col];
+							resV[offT + _colIndexes[aix[col]]] += avals[col];
 					}
 				}
 			}
@@ -606,9 +608,9 @@ public class ColGroupUncompressed extends AColGroup {
 				double[] tmpRetV = tmpRet.getDenseBlockValues();
 				for(int j = 0, offTemp = 0, offT = 0;
 					j < tmpRet.getNumRows();
-					j++, offTemp += _colIndexes.length, offT += numCols) {
+					j++, offTemp += _colIndexes.length, offT += result.getNumColumns()) {
 					for(int i = 0; i < _colIndexes.length; i++)
-						result[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
+						resV[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
 				}
 			}
 
@@ -618,16 +620,19 @@ public class ColGroupUncompressed extends AColGroup {
 			LOG.warn("Inefficient transpose of uncompressed to fit to"
 				+ " t(AColGroup) %*% UncompressedColGroup mult by colGroup uncompressed column"
 				+ " Currently solved by t(t(Uncompressed) %*% AColGroup");
-			double[] tmpTransposedResult = new double[result.length];
+			MatrixBlock tmpTransposedResult = new MatrixBlock(result.getNumColumns(), result.getNumRows(), false);
+			tmpTransposedResult.allocateDenseBlock();
 
 			MatrixBlock ucCG = getData();
 			MatrixBlock tmp = new MatrixBlock(ucCG.getNumColumns(), ucCG.getNumRows(), ucCG.isInSparseFormat());
 			LibMatrixReorg.transpose(ucCG, tmp, InfrastructureAnalyzer.getLocalParallelism());
-			lhs.leftMultByMatrix(tmp, tmpTransposedResult, numRows);
+			lhs.leftMultByMatrix(tmp, tmpTransposedResult);
 
-			for(int row = 0; row < numRows; row++) {
-				for(int col = 0; col < numCols; col++) {
-					result[row * numCols + col] += tmpTransposedResult[col * numRows + row];
+			final double[] resV = result.getDenseBlockValues();
+			final double[] tmpV = tmpTransposedResult.getDenseBlockValues();
+			for(int row = 0; row < result.getNumRows(); row++) {
+				for(int col = 0; col < result.getNumColumns(); col++) {
+					resV[row * result.getNumColumns() + col] += tmpV[col * result.getNumRows() + row];
 				}
 			}
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index dc52ca1..fc7da24 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -36,9 +36,12 @@ import org.apache.sysds.runtime.compress.colgroup.dictionary.DictionaryFactory;
 import org.apache.sysds.runtime.compress.colgroup.pre.ArrPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysds.runtime.data.DenseBlock;
+import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
+import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
@@ -607,8 +610,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param c          The output matrix
 	 * @param numVals    The number of values contained in the dictionary.
 	 */
-	protected void postScaling(double[] dictValues, double[] vals, double[] c, int numVals) {
-		postScaling(dictValues, vals, c, numVals, 0, 0);
+	protected void postScaling(double[] dictValues, double[] vals, MatrixBlock c, int numVals) {
+		postScaling(dictValues, vals, c, numVals, 0);
 	}
 
 	/**
@@ -621,41 +624,17 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param row        The row index in the output c to assign the result to.
 	 * @param totalCols  The total number of columns in c.
 	 */
-	protected void postScaling(double[] dictValues, double[] vals, double[] c, int numVals, int row, int totalCols) {
+	protected void postScaling(double[] dictValues, double[] vals, MatrixBlock c, int numVals, int row) {
 		final int ncol = getNumCols();
 		int valOff = 0;
+		final double[] cv = c.getDenseBlockValues();
+		final int totalCols = c.getNumColumns();
 
 		for(int k = 0; k < numVals; k++) {
 			double aval = vals[k];
 			for(int j = 0; j < ncol; j++) {
 				int colIx = _colIndexes[j] + row * totalCols;
-				c[colIx] += aval * dictValues[valOff++];
-			}
-		}
-	}
-
-	/**
-	 * Post scale for left Multiplication
-	 * 
-	 * @param dictValues The dictionary values materialized as double array.
-	 * @param vals       The values aggregated from the left side row vector.
-	 * @param c          The output matrix
-	 * @param numVals    The number of values contained in the dictionary.
-	 * @param row        The row index in the output c to assign the result to.
-	 * @param totalCols  The total number of columns in c.
-	 * @param offT       The offset into C to assign the values from usefull in case the c output is a smaller temporary
-	 *                   array.
-	 */
-	protected void postScaling(double[] dictValues, double[] vals, double[] c, int numVals, int row, int totalCols,
-		int offT) {
-		final int nCol = getNumCols();
-		int valOff = 0;
-
-		for(int k = 0; k < numVals; k++) {
-			double aval = vals[k];
-			for(int j = 0; j < nCol; j++) {
-				int colIx = _colIndexes[j] + row * totalCols;
-				c[offT + colIx] += aval * dictValues[valOff++];
+				cv[colIx] += aval * dictValues[valOff++];
 			}
 		}
 	}
@@ -670,6 +649,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return preAggregate(a, 0);
 	}
 
+	public abstract MatrixBlock preAggregate(MatrixBlock m, int rl, int ru);
+
 	/**
 	 * Pre aggregates for left multiplication
 	 * 
@@ -686,7 +667,14 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @return The pre-aggregated values.
 	 */
 	public double[] preAggregate(SparseBlock sb) {
-		return preAggregateSparse(sb, 0);
+		return preAggregateSparseWithCheck(sb, 0);
+	}
+
+	private double[] preAggregateSparseWithCheck(SparseBlock sb, int row) {
+		if(sb != null && !sb.isEmpty(row))
+			return preAggregateSparse(sb, row);
+		else
+			return null;
 	}
 
 	/**
@@ -797,18 +785,18 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, double[] result, final int numRows, final int numCols) {
+	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
 		if(lhs instanceof ColGroupEmpty)
 			return;
 		else if(lhs instanceof ColGroupValue)
-			leftMultByColGroupValue((ColGroupValue) lhs, result, numRows, numCols);
+			leftMultByColGroupValue((ColGroupValue) lhs, result);
 		else if(lhs instanceof ColGroupUncompressed) {
 			LOG.warn("Inefficient transpose of uncompressed to fit to "
 				+ "template need t(UnCompressedColGroup) %*% AColGroup support");
 			MatrixBlock ucCG = ((ColGroupUncompressed) lhs).getData();
 			MatrixBlock tmp = new MatrixBlock(ucCG.getNumColumns(), ucCG.getNumRows(), ucCG.isInSparseFormat());
 			LibMatrixReorg.transpose(ucCG, tmp, InfrastructureAnalyzer.getLocalParallelism());
-			leftMultByMatrix(tmp, result, numCols);
+			leftMultByMatrix(tmp, result);
 
 		}
 		else
@@ -816,35 +804,38 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				"Not supported left multiplication with A ColGroup of type: " + lhs.getClass().getSimpleName());
 	}
 
-	private void leftMultByColGroupValue(ColGroupValue lhs, double[] result, final int numRows, final int numCols) {
+	private void leftMultByColGroupValue(ColGroupValue lhs, MatrixBlock result) {
 		final int nvL = lhs.getNumValues();
 		final int nvR = this.getNumValues();
 		final double[] lhValues = lhs.getValues();
 		final double[] rhValues = this.getValues();
 		final int lCol = lhs._colIndexes.length;
 		final int rCol = this._colIndexes.length;
+		final double[] resV = result.getDenseBlockValues();
+		final int numCols = result.getNumColumns();
 
 		final double threshold = 0.2;
 
 		if(sameIndexStructure(lhs)) {
 			int[] agI = getCounts();
 			for(int i = 0; i < agI.length; i++) {
-				for(int l = 0; l < lCol; l++) {
-					final int leftOff = lhs._colIndexes[l] * numCols;
-					final double lhV = lhValues[i * lCol + l] * agI[i];
-					if(lhV != 0)
-						for(int r = 0; r < rCol; r++) {
-							final double rhV = rhValues[i * rCol + r];
-							final double va = lhV * rhV;
-							result[leftOff + this._colIndexes[r]] += va;
-						}
-				}
+				if(i < nvL)
+					for(int l = 0; l < lCol; l++) {
+						final int leftOff = lhs._colIndexes[l] * numCols;
+						final double lhV = lhValues[i * lCol + l] * agI[i];
+						if(lhV != 0)
+							for(int r = 0; r < rCol; r++) {
+								final double rhV = rhValues[i * rCol + r];
+								final double va = lhV * rhV;
+								resV[leftOff + this._colIndexes[r]] += va;
+							}
+					}
 			}
 		}
 		else if(lhs instanceof ColGroupConst || this instanceof ColGroupConst) {
-			// double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
-			// double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
-			// vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, result, numCols);
+			double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
+			double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
+			vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, resV, numCols);
 		}
 		else {
 			int[] countsRight = getCounts();
@@ -858,36 +849,36 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 			if(skipRight > threshold && percentageRight > percentageLeft && !(this instanceof ColGroupDDC)) {
 				double[] mct = this._dict.getMostCommonTuple(this.getCounts(), rCol);
-				// double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
-				// if(mct != null)
-				// 	vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, result, numCols);
+				double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
+				if(mct != null)
+					vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, resV, numCols);
 
 				ColGroupValue thisM = (mct != null) ? (ColGroupValue) this
 					.copyAndSet(this._dict.subtractTuple(mct)) : this;
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(thisM, true);
 				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
-					this._colIndexes, result, numCols);
+					this._colIndexes, resV, numCols);
 			}
 			else if(skipLeft > threshold && !(lhs instanceof ColGroupDDC)) {
 				double[] mct = lhs._dict.getMostCommonTuple(lhs.getCounts(), lCol);
-				// double[] thisColSum = this._dict.colSum(getCounts(), rCol);
-				// if(mct != null)
-				// 	vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, result, numCols);
+				double[] thisColSum = this._dict.colSum(getCounts(), rCol);
+				if(mct != null)
+					vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, resV, numCols);
 
 				ColGroupValue lhsM = (mct != null) ? (ColGroupValue) lhs.copyAndSet(lhs._dict.subtractTuple(mct)) : lhs;
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhsM, true);
 				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
-					this._colIndexes, result, numCols);
+					this._colIndexes, resV, numCols);
 			}
 			else if(nvR * rCol < nvL * lCol) {
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(this, false);
 				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
-					this._colIndexes, result, numCols);
+					this._colIndexes, resV, numCols);
 			}
 			else {
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhs, false);
 				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
-					this._colIndexes, result, numCols);
+					this._colIndexes, resV, numCols);
 			}
 		}
 	}
@@ -1014,56 +1005,62 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return !_zeros;
 	}
 
-	@Override
-	public void leftMultBySparseMatrix(SparseBlock sb, double[] result, int numRows, int numCols, int rl, int ru) {
-		double[] values = getValues();
-		for(int i = rl; i < ru; i++) {
-			leftMultBySparseMatrix(sb, result, values, numRows, numCols, i);
-		}
-	}
-
 	/**
 	 * Multiply with a matrix on the left.
 	 * 
-	 * @param matrix  matrix to left multiply
-	 * @param result  matrix block result
-	 * @param values  The materialized values contained in the ColGroupValue
-	 * @param numRows The number of rows in the matrix input
-	 * @param numCols The number of columns in the colGroups parent matrix.
-	 * @param rl      The row to start the matrix multiplication from
-	 * @param ru      The row to stop the matrix multiplication at.
+	 * @param matrix matrix to left multiply
+	 * @param result matrix block result
+	 * @param values The materialized values contained in the ColGroupValue
+	 * @param rl     The row to start the matrix multiplication from
+	 * @param ru     The row to stop the matrix multiplication at.
 	 */
-	public void leftMultByMatrix(double[] matrix, double[] result, double[] values, int numRows, int numCols, int rl,
-		int ru) {
-		int numVals = getNumValues();
+	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, double[] values, int rl, int ru) {
+		final int numVals = getNumValues();
+
+		DenseBlock dictV = new DenseBlockFP64(new int[] {numVals, _colIndexes.length}, values);
+		MatrixBlock dictM = new MatrixBlock(numVals, _colIndexes.length, dictV);
+		dictM.examSparsity();
+		MatrixBlock tmpRes = new MatrixBlock(1, _colIndexes.length, false);
 		for(int i = rl; i < ru; i++) {
-			double[] vals = preAggregate(matrix, i);
-			postScaling(values, vals, result, numVals, i, numCols);
+			double[] vals = matrix.isInSparseFormat() ? preAggregateSparseWithCheck(matrix.getSparseBlock(),
+				i) : preAggregate(matrix.getDenseBlockValues(), i);
+			if(vals != null) {
+				DenseBlock preAggV = new DenseBlockFP64(new int[] {1, numVals}, vals);
+				MatrixBlock preAgg = new MatrixBlock(1, numVals, preAggV);
+				preAgg.setNonZeros(numVals);
+				// LOG.error("PreAgg Sparsity " + preAgg.getSparsity() + " nnz " + preAgg.getNonZeros());
+				LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
+				addToResult(tmpRes, result, i);
+				tmpRes.reset();
+			}
 		}
 	}
 
-	@Override
-	public void leftMultByMatrix(double[] matrix, double[] result, int numRows, int numCols, int rl, int ru) {
-		leftMultByMatrix(matrix, result, getValues(), numRows, numCols, rl, ru);
+	private void addToResult(MatrixBlock tmp, MatrixBlock result, int row) {
+		if(tmp.isEmpty())
+			return;
+		else if(tmp.isInSparseFormat()) {
+			throw new NotImplementedException();
+		}
+		else {
+			final double[] tmpV = tmp.getDenseBlockValues();
+			final double[] retV = result.getDenseBlockValues();
+			final int nColRet = result.getNumColumns();
+			// final int nColTmp = tmp.getNumColumns();
+			final int offR = row * nColRet;
+			// for(int row = rl, offT = 0, offR = rl * nColRet; row < ru; row++, offT += nColTmp, offR += nColRet) {
+			for(int col = 0; col < _colIndexes.length; col++) {
+				final int colOffset = _colIndexes[col];
+				retV[offR + colOffset] += tmpV[col];
+			}
+			// }
+		}
+
 	}
 
-	/**
-	 * Multiply with a sparse matrix on the left hand side, and add the values to the output result
-	 * 
-	 * @param sb      The sparse block to multiply with
-	 * @param result  The linearized output matrix
-	 * @param values  The values contained in the dictionary, this parameter is here to allow materialization once.
-	 * @param numRows The number of rows in the left hand side input matrix (the sparse one)
-	 * @param numCols The number of columns in the compression.
-	 * @param row     The row index of the sparse row to multiply with.
-	 */
-	public void leftMultBySparseMatrix(SparseBlock sb, double[] result, double[] values, int numRows, int numCols,
-		int row) {
-		if(!sb.isEmpty(row)) {
-			final int numVals = getNumValues();
-			double[] vals = preAggregateSparse(sb, row);
-			postScaling(values, vals, result, numVals, row, numCols);
-		}
+	@Override
+	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
+		leftMultByMatrix(matrix, result, getValues(), rl, ru);
 	}
 
 	public AColGroup rightMultByMatrix(MatrixBlock right) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
index 4902cb8..d4b2a63 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
@@ -48,7 +48,7 @@ public abstract class CompressedSizeEstimator {
 	/** The number of columns in the matrix block, extracted to a field because the matrix could be transposed */
 	final protected int _numCols;
 	/** The compression settings to use, for estimating the size, and compress the ColGroups. */
-	final protected CompressionSettings _compSettings;
+	final protected CompressionSettings _cs;
 
 	/**
 	 * Boolean specifying if the _data is in transposed format. This is used to select the correct readers for the
@@ -61,23 +61,22 @@ public abstract class CompressedSizeEstimator {
 	 * 
 	 * protected because the factory should be used to construct the CompressedSizeEstimator
 	 * 
-	 * @param data         The matrix block to extract information from
-	 * @param compSettings The Compression settings used.
+	 * @param data The matrix block to extract information from
+	 * @param cs   The Compression settings used.
 	 */
-	protected CompressedSizeEstimator(MatrixBlock data, CompressionSettings compSettings, boolean transposed) {
+	protected CompressedSizeEstimator(MatrixBlock data, CompressionSettings cs) {
 		_data = data;
-		_transposed = transposed;
+		_transposed = cs.transposed;
 		_numRows = _transposed ? _data.getNumColumns() : _data.getNumRows();
 		_numCols = _transposed ? _data.getNumRows() : _data.getNumColumns();
-		_compSettings = compSettings;
+		_cs = cs;
 	}
 
-
-	public int getNumRows(){
+	public int getNumRows() {
 		return _numRows;
 	}
 
-	public int getNumColumns(){
+	public int getNumColumns() {
 		return _numCols;
 	}
 
@@ -124,8 +123,8 @@ public abstract class CompressedSizeEstimator {
 	 * @return The size factors estimated from the Bit Map.
 	 */
 	public EstimationFactors estimateCompressedColGroupSize(ABitmap ubm, int[] colIndexes) {
-		return EstimationFactors.computeSizeEstimationFactors(ubm,
-			_compSettings.validCompressions.contains(CompressionType.RLE), _numRows, colIndexes);
+		return EstimationFactors.computeSizeEstimationFactors(ubm, _cs.validCompressions.contains(CompressionType.RLE),
+			_numRows, colIndexes);
 	}
 
 	private CompressedSizeInfoColGroup[] CompressedSizeInfoColGroup(int clen) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
index a118722..40677e9 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -29,16 +29,17 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
  */
 public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
 
-	public CompressedSizeEstimatorExact(MatrixBlock data, CompressionSettings compSettings, boolean transposed) {
-		super(data, compSettings, transposed);
+	public CompressedSizeEstimatorExact(MatrixBlock data, CompressionSettings compSettings) {
+		super(data, compSettings);
 	}
 
 	@Override
 	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
 		ABitmap entireBitMap = BitmapEncoder.extractBitmap(colIndexes, _data, _transposed);
 		return new CompressedSizeInfoColGroup(estimateCompressedColGroupSize(entireBitMap, colIndexes),
-			_compSettings.validCompressions);
+			_cs.validCompressions);
 	}
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
index a886eae..cf5437e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
@@ -29,26 +29,19 @@ public class CompressedSizeEstimatorFactory {
 
 	public static final int minimumSampleSize = 2000;
 
-	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, CompressionSettings compSettings) {
-
-		MatrixBlock shallowCopy = new MatrixBlock().copyShallow(data);
-		long elements = compSettings.transposed ? data.getNumColumns() : data.getNumRows();
-		elements = data.getNonZeros() / (compSettings.transposed ? data.getNumRows() : data.getNumColumns());
-		CompressedSizeEstimator est;
+	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, CompressionSettings cs) {
 
+		final long nRows = cs.transposed ? data.getNumColumns() : data.getNumRows();
+		
 		// Calculate the sample size.
 		// If the sample size is very small, set it to the minimum size
-		int sampleSize = Math.max((int) Math.ceil(elements * compSettings.samplingRatio), minimumSampleSize);
-		if(compSettings.samplingRatio >= 1.0 || elements < minimumSampleSize || sampleSize > elements) {
-			est = new CompressedSizeEstimatorExact(shallowCopy, compSettings, compSettings.transposed);
-		}
-		else {
-			int[] sampleRows = CompressedSizeEstimatorSample.getSortedUniformSample(
-				compSettings.transposed ? data.getNumColumns() : data.getNumRows(),
-				sampleSize,
-				compSettings.seed);
-			est = new CompressedSizeEstimatorSample(shallowCopy, compSettings, sampleRows, compSettings.transposed);
-		}
+		final int sampleSize = Math.max((int) Math.ceil(nRows * cs.samplingRatio), minimumSampleSize);
+
+		CompressedSizeEstimator est;
+		if(cs.samplingRatio >= 1.0 || nRows < minimumSampleSize || sampleSize > nRows)
+			est = new CompressedSizeEstimatorExact(data, cs);
+		else
+			est = new CompressedSizeEstimatorSample(data, cs, sampleSize);
 
 		LOG.debug("Estimating using: " + est);
 		return est;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
index af94473..3098394 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -38,61 +38,48 @@ import org.apache.sysds.runtime.util.UtilFunctions;
 
 public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 
-	private static final int FORCE_TRANSPOSE_ON_SAMPLE_THRESHOLD = 8000;
-
 	private final int[] _sampleRows;
-
 	private final MatrixBlock _sample;
 	private HashMap<Integer, Double> _solveCache = null;
 
 	/**
 	 * CompressedSizeEstimatorSample, samples from the input data and estimates the size of the compressed matrix.
 	 * 
-	 * @param data         The input data toSample from
-	 * @param compSettings The Settings used for the sampling, and compression, contains information such as seed.
-	 * @param sampleRows   The rows sampled
-	 * @param transposed   Boolean specifying if the input is already transposed.
+	 * @param data       The input data toSample from
+	 * @param cs         The Settings used for the sampling, and compression, contains information such as seed.
+	 * @param sampleSize The size to sample from the data.
 	 */
-	public CompressedSizeEstimatorSample(MatrixBlock data, CompressionSettings compSettings, int[] sampleRows,
-		boolean transposed) {
-		super(data, compSettings, transposed);
-		_sampleRows = sampleRows;
+	public CompressedSizeEstimatorSample(MatrixBlock data, CompressionSettings cs, int sampleSize) {
+		super(data, cs);
+		_sampleRows = CompressedSizeEstimatorSample.getSortedUniformSample(_numRows, sampleSize, _cs.seed);
 		_solveCache = new HashMap<>();
-		_sample = sampleData(data, compSettings, sampleRows, transposed);
+		_sample = sampleData();
 	}
 
-	protected MatrixBlock sampleData(MatrixBlock data, CompressionSettings compSettings, int[] sampleRows,
-		boolean transposed) {
+	protected MatrixBlock sampleData() {
 		MatrixBlock sampledMatrixBlock;
-		if(data.isInSparseFormat() && !transposed) {
-			sampledMatrixBlock = new MatrixBlock(sampleRows.length, data.getNumColumns(), true);
-			SparseRow[] rows = new SparseRow[sampleRows.length];
-			SparseBlock in = data.getSparseBlock();
-			for(int i = 0; i < sampleRows.length; i++) {
-				rows[i] = in.get(sampleRows[i]);
-			}
+		if(_data.isInSparseFormat() && !_cs.transposed) {
+			sampledMatrixBlock = new MatrixBlock(_sampleRows.length, _data.getNumColumns(), true);
+			SparseRow[] rows = new SparseRow[_sampleRows.length];
+			SparseBlock in = _data.getSparseBlock();
+			for(int i = 0; i < _sampleRows.length; i++)
+				rows[i] = in.get(_sampleRows[i]);
+
 			sampledMatrixBlock.setSparseBlock(new SparseBlockMCSR(rows, false));
 			sampledMatrixBlock.recomputeNonZeros();
 			_transposed = true;
 			sampledMatrixBlock = LibMatrixReorg.transposeInPlace(sampledMatrixBlock, 16);
 		}
 		else {
-			MatrixBlock select = (transposed) ? new MatrixBlock(data.getNumColumns(), 1,
-				true) : new MatrixBlock(data.getNumRows(), 1, true);
-			for(int i = 0; i < sampleRows.length; i++)
-				select.appendValue(sampleRows[i], 0, 1);
+			MatrixBlock select = (_cs.transposed) ? new MatrixBlock(_data.getNumColumns(), 1,
+				false) : new MatrixBlock(_data.getNumRows(), 1, false);
+			for(int i = 0; i < _sampleRows.length; i++)
+				select.appendValue(_sampleRows[i], 0, 1);
 
-			sampledMatrixBlock = data.removeEmptyOperations(new MatrixBlock(), !transposed, true, select);
+			sampledMatrixBlock = _data.removeEmptyOperations(new MatrixBlock(), !_cs.transposed, true, select);
 
 		}
 
-		if(!transposed && sampledMatrixBlock.isInSparseFormat() &&
-			sampleRows.length > FORCE_TRANSPOSE_ON_SAMPLE_THRESHOLD) {
-			_transposed = true;
-			sampledMatrixBlock = LibMatrixReorg.transpose(sampledMatrixBlock,
-				new MatrixBlock(sampleRows.length, data.getNumRows(), true), 1);
-		}
-
 		if(sampledMatrixBlock.isEmpty())
 			throw new DMLCompressionException("Empty sample block");
 
@@ -104,7 +91,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
 		final int sampleSize = _sampleRows.length;
 		// final int numCols = colIndexes.length;
-
+		final double scalingFactor = ((double) _numRows / sampleSize);
 		// extract statistics from sample
 		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, _sample, _transposed);
 		final EstimationFactors fact = EstimationFactors.computeSizeEstimationFactors(ubm, false, _numRows, colIndexes);
@@ -114,50 +101,44 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		if(numZerosInSample == sampleSize || ubm.getOffsetList() == null) {
 			// Since we sample, and this column seems to be empty, we set the return to 1 value detected.
 			// aka 1 value, and 1 offset.
-			// This makes it more robust in the co coding of Columns
-			return new CompressedSizeInfoColGroup(
-				new EstimationFactors(colIndexes, 1, 1, _numRows - 1, 2, 1, _numRows, lossy, true, 0, 0),
-				_compSettings.validCompressions);
+			// This makes it more robust in the coCoding of Columns
+			return new CompressedSizeInfoColGroup(new EstimationFactors(colIndexes, 1, 1, _numRows - 1, 2, 1, _numRows,
+				lossy, true, (double) 1 / _numRows, (double) 1 / ubm.getNumColumns()), _cs.validCompressions);
 		}
 
-		// estimate number of distinct values (incl fixes for anomalies w/ large sample fraction)
+		// Estimate number of distinct values (incl fixes for anomalies w/ large sample fraction)
 		int totalCardinality = getNumDistinctValues(ubm, _numRows, sampleSize, _solveCache);
-		totalCardinality = Math.max(totalCardinality, fact.numVals);
-		totalCardinality = lossy ? Math.min(totalCardinality, 256) : totalCardinality;
-		totalCardinality = Math.min(totalCardinality, _numRows);
-
-		// Number of unseen values
-		// int unseenVals = totalCardinality - fact.numVals;
+		// Number of unique is trivially bounded by the sampled number of uniques and the number of rows.
+		totalCardinality = Math.min(Math.max(totalCardinality, fact.numVals), _numRows);
 
 		// estimate number of non-zeros (conservatively round up)
 		final double C = Math.max(1 - (double) fact.numSingle / sampleSize, (double) sampleSize / _numRows);
-		final int numNonZeros = Math.max(
-			(int) Math.floor(_numRows - (double) (_numRows / sampleSize) * C * numZerosInSample), totalCardinality);
+		final int numNonZeros = Math.max((int) Math.floor(_numRows - scalingFactor * C * numZerosInSample),
+			totalCardinality);
 
 		// estimate number of segments and number of runs incl correction for
 		// empty segments and empty runs (via expected mean of offset value)
 		// int numUnseenSeg = (int) (unseenVals * Math.ceil((double) _numRows / BitmapEncoder.BITMAP_BLOCK_SZ / 2));
-		final int totalNumRuns = _compSettings.validCompressions.contains(CompressionType.RLE) &&
+		final int totalNumRuns = _cs.validCompressions.contains(CompressionType.RLE) &&
 			ubm.getNumValues() > 0 ? getNumRuns(ubm, sampleSize, _numRows, _sampleRows) : 0;
 
 		// Largest instance count ... initiate as the number of zeros.
 		int largestInstanceCount = numZerosInSample;
-		for(IntArrayList a : ubm.getOffsetList()) {
+		for(IntArrayList a : ubm.getOffsetList())
 			if(a.size() > largestInstanceCount)
 				largestInstanceCount = a.size();
-		}
+		
 
 		final boolean zeroIsMostFrequent = largestInstanceCount == numZerosInSample;
 
 		// Scale largest Instance count to correctly reflect the number of instances.
-		largestInstanceCount = (int) (((double) _numRows / sampleSize) * largestInstanceCount);
-
+		largestInstanceCount = (int) (scalingFactor * largestInstanceCount);
 		EstimationFactors totalFacts = new EstimationFactors(colIndexes, totalCardinality, numNonZeros,
-			largestInstanceCount, totalNumRuns, fact.numSingle, _numRows, lossy, zeroIsMostFrequent, fact.tupleSparsity,
-			fact.overAllSparsity);
+			largestInstanceCount, totalNumRuns, fact.numSingle, _numRows, lossy, zeroIsMostFrequent,
+			fact.overAllSparsity, fact.tupleSparsity);
 
 		// construct new size info summary
-		return new CompressedSizeInfoColGroup(totalFacts, _compSettings.validCompressions);
+		return new CompressedSizeInfoColGroup(totalFacts, _cs.validCompressions);
 	}
 
 	private static int getNumDistinctValues(ABitmap ubm, int numRows, int sampleSize,
@@ -330,7 +311,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	 * @param smplSize sample size
 	 * @return sorted array of integers
 	 */
-	protected static int[] getSortedUniformSample(int range, int smplSize, long seed) {
+	private static int[] getSortedUniformSample(int range, int smplSize, long seed) {
 		return UtilFunctions.getSortedSampleIndexes(range, smplSize, seed);
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
index d25f280..50aeb7e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -80,7 +80,8 @@ public class CompressedSizeInfoColGroup {
 
 		_bestCompressionType = bestEntry.getKey();
 		_minSize = bestEntry.getValue();
-		// LOG.error(this);
+		if(LOG.isTraceEnabled())
+			LOG.trace(this);
 	}
 
 	/**
@@ -98,7 +99,6 @@ public class CompressedSizeInfoColGroup {
 		Set<CompressionType> validCompressionTypes) {
 		EstimationFactors fact = new EstimationFactors(columns, oneSide._facts);
 		CompressedSizeInfoColGroup ret = new CompressedSizeInfoColGroup(fact, validCompressionTypes);
-		// LOG.error(ret);
 		return ret;
 	}
 
@@ -144,11 +144,11 @@ public class CompressedSizeInfoColGroup {
 		return _cardinalityRatio;
 	}
 
-	public double getMostCommonFraction(){
+	public double getMostCommonFraction() {
 		return (double) _facts.largestOff / _facts.numRows;
 	}
 
-	public double getTupleSparsity(){
+	public double getTupleSparsity() {
 		return _facts.tupleSparsity;
 	}
 
@@ -215,13 +215,13 @@ public class CompressedSizeInfoColGroup {
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append("Best Type: " + _bestCompressionType);
-		sb.append(" facts: " + _facts);
 		sb.append(" Cardinality: ");
 		sb.append(_cardinalityRatio);
+		sb.append(" mostCommonFraction: ");
+		sb.append(getMostCommonFraction());
 		sb.append(" Sizes: ");
 		sb.append(_sizes);
-
-		sb.append("\n");
+		sb.append(" facts: " + _facts);
 		return sb.toString();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java b/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
index 350d9c4..b42e211 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/EstimationFactors.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.ABitmap.BitmapType;
@@ -102,6 +103,18 @@ public class EstimationFactors {
 		this.containNoZeroValues = numOffs == numRows;
 		this.overAllSparsity = overAllSparsity;
 		this.tupleSparsity = tupleSparsity;
+
+		if(!containNoZeroValues && overAllSparsity >= 1)
+			throw new DMLCompressionException(
+				"Invalid Sparsity, if there is zeroOffsets, then the sparsity should be below 1");
+		if(overAllSparsity > 1 || overAllSparsity < 0)
+			throw new DMLCompressionException("Invalid sparsity");
+		if(tupleSparsity > 1 || tupleSparsity < 0)
+			throw new DMLCompressionException("Invalid sparsity");
+		if(largestOff > numRows)
+			throw new DMLCompressionException(
+				"Invalid number of instance of most common element should be lower than number of rows. " + largestOff
+					+ " > numRows: " + numRows);
 	}
 
 	protected static EstimationFactors computeSizeEstimationFactors(ABitmap ubm, boolean inclRLE, int numRows,
@@ -155,8 +168,7 @@ public class EstimationFactors {
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
-		sb.append("\nrows:" + numRows);
-		sb.append(" cols:" + Arrays.toString(cols));
+		sb.append(" rows:" + numRows);
 		sb.append(" num Offsets:" + numOffs);
 		sb.append(" LargestOffset:" + largestOff);
 		sb.append(" num Singles:" + numSingle);
@@ -164,6 +176,7 @@ public class EstimationFactors {
 		sb.append(" num Unique Vals:" + numVals);
 		sb.append(" overallSparsity:" + overAllSparsity);
 		sb.append(" tupleSparsity:" + tupleSparsity);
+		sb.append(" cols:" + Arrays.toString(cols));
 		return sb.toString();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index 2779a68..9c73f7c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -109,8 +109,7 @@ public class CLALibLeftMultBy {
 
 		if(k <= 1) {
 			for(int i = 0; i < groups.size(); i++)
-				leftMultByCompressedTransposedMatrix(groups.get(i), groups, result.getDenseBlockValues(),
-					result.getNumRows(), result.getNumColumns(), i, groups.size());
+				leftMultByCompressedTransposedMatrix(groups.get(i), groups, result, i, groups.size());
 		}
 		else {
 			try {
@@ -213,8 +212,7 @@ public class CLALibLeftMultBy {
 		public Object call() {
 			try {
 				IPreAggregate.setupThreadLocalMemory(1024);
-				leftMultByCompressedTransposedMatrix(_left, _groups, _ret.getDenseBlockValues(), _ret.getNumRows(),
-					_ret.getNumColumns(), _start, _end);
+				leftMultByCompressedTransposedMatrix(_left, _groups, _ret, _start, _end);
 
 			}
 			catch(Exception e) {
@@ -227,19 +225,15 @@ public class CLALibLeftMultBy {
 
 	private static void leftMultByCompressedTransposedMatrix(List<AColGroup> thisCG, List<AColGroup> thatCG,
 		MatrixBlock ret) {
-		double[] c = ret.getDenseBlockValues();
-		for(AColGroup lhs : thatCG) {
-			leftMultByCompressedTransposedMatrix(lhs, thisCG, c, ret.getNumRows(), ret.getNumColumns(), 0,
-				thisCG.size());
-		}
+		for(AColGroup lhs : thatCG)
+			leftMultByCompressedTransposedMatrix(lhs, thisCG, ret, 0, thisCG.size());
 	}
 
-	private static void leftMultByCompressedTransposedMatrix(AColGroup lhs, List<AColGroup> thisCG, double[] c,
-		int rows, int cols, int colGroupStart, int colGroupEnd) {
+	private static void leftMultByCompressedTransposedMatrix(AColGroup lhs, List<AColGroup> thisCG, MatrixBlock ret,
+		int colGroupStart, int colGroupEnd) {
 
-		for(; colGroupStart < colGroupEnd; colGroupStart++) {
-			thisCG.get(colGroupStart).leftMultByAColGroup(lhs, c, rows, cols);
-		}
+		for(; colGroupStart < colGroupEnd; colGroupStart++)
+			thisCG.get(colGroupStart).leftMultByAColGroup(lhs, ret);
 
 	}
 
@@ -252,29 +246,34 @@ public class CLALibLeftMultBy {
 		}
 
 		ret.allocateDenseBlock();
-		double[] retV = ret.getDenseBlockValues();
 
 		if(k == 1)
 			for(int j = 0; j < colGroups.size(); j++)
-				colGroups.get(j).leftMultByMatrix(that, retV, numColumns);
+				colGroups.get(j).leftMultByMatrix(that, ret);
 		else {
 			try {
 				ExecutorService pool = CommonThreadPool.get(k);
 				// compute remaining compressed column groups in parallel
 				ArrayList<Callable<Object>> tasks = new ArrayList<>();
-				int rowBlockSize = 1;
+				int rowBlockSize = 8;
 				if(overlapping) {
 					for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
-						tasks.add(new LeftMatrixMatrixMultTask(colGroups, that, retV, numColumns, blo,
+						tasks.add(new LeftMatrixMatrixMultTask(colGroups, that, ret, blo,
 							Math.min(blo + rowBlockSize, that.getNumRows()), v));
 					}
 				}
 				else {
-					for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
-						for(AColGroup g : colGroups) {
-							tasks.add(new LeftMatrixColGroupMultTask(g, that, retV, numColumns, blo,
-								Math.min(blo + rowBlockSize, that.getNumRows()), v));
-						}
+					for(AColGroup g : colGroups) {
+						// if(g instanceof ColGroupDDC) {
+						// 	tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, 0, that.getNumRows(), v));
+						// }
+						// else {
+
+							for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
+								tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, blo,
+									Math.min(blo + rowBlockSize, that.getNumRows()), v));
+							}
+						// }
 					}
 				}
 
@@ -364,9 +363,9 @@ public class CLALibLeftMultBy {
 		for(int j = rl; j < ru; j++) {
 			AColGroup.decompressColumnToBlock(lhs, j, thatGroups);
 			if(!lhs.isEmptyBlock(false)) {
-				for(AColGroup grp : thisGroups) {
-					grp.leftMultByMatrix(lhs, tmpret.getDenseBlockValues(), result.getNumColumns());
-				}
+				for(AColGroup grp : thisGroups)
+					grp.leftMultByMatrix(lhs, tmpret);
+
 				double[] tmpRetValues = tmpret.getDenseBlockValues();
 				double[] resultValues = result.getDenseBlockValues();
 				int offset = tmpret.getNumColumns() * j;
@@ -383,18 +382,16 @@ public class CLALibLeftMultBy {
 	private static class LeftMatrixMatrixMultTask implements Callable<Object> {
 		private final List<AColGroup> _group;
 		private final MatrixBlock _that;
-		private final double[] _ret;
-		private final int _numCols;
+		private final MatrixBlock _ret;
 		private final int _rl;
 		private final int _ru;
 		private final Pair<Integer, int[]> _v;
 
-		protected LeftMatrixMatrixMultTask(List<AColGroup> group, MatrixBlock that, double[] ret, int numCols, int rl,
-			int ru, Pair<Integer, int[]> v) {
+		protected LeftMatrixMatrixMultTask(List<AColGroup> group, MatrixBlock that, MatrixBlock ret, int rl, int ru,
+			Pair<Integer, int[]> v) {
 			_group = group;
 			_that = that;
 			_ret = ret;
-			_numCols = numCols;
 			_rl = rl;
 			_ru = ru;
 			_v = v;
@@ -402,16 +399,10 @@ public class CLALibLeftMultBy {
 
 		@Override
 		public Object call() {
-			// setup memory pool for reuse
-
-			double[][] materialized = new double[_group.size()][];
-			for(int i = 0; i < _group.size(); i++) {
-				materialized[i] = _group.get(i).getValues();
-			}
 			try {
-				ColGroupValue.setupThreadLocalMemory(_v.getLeft() + 1);
+				ColGroupValue.setupThreadLocalMemory(_v.getLeft() * (_ru - _rl));
 				for(int j = 0; j < _group.size(); j++)
-					_group.get(j).leftMultByMatrix(_that, _ret, _numCols, _rl, _ru);
+					_group.get(j).leftMultByMatrix(_that, _ret, _rl, _ru);
 			}
 			catch(Exception e) {
 				throw new DMLRuntimeException(e);
@@ -423,18 +414,16 @@ public class CLALibLeftMultBy {
 	private static class LeftMatrixColGroupMultTask implements Callable<Object> {
 		private final AColGroup _group;
 		private final MatrixBlock _that;
-		private final double[] _ret;
-		private final int _numCols;
+		private final MatrixBlock _ret;
 		private final int _rl;
 		private final int _ru;
 		private final Pair<Integer, int[]> _v;
 
-		protected LeftMatrixColGroupMultTask(AColGroup group, MatrixBlock that, double[] ret, int numCols, int rl,
-			int ru, Pair<Integer, int[]> v) {
+		protected LeftMatrixColGroupMultTask(AColGroup group, MatrixBlock that, MatrixBlock ret, int rl, int ru,
+			Pair<Integer, int[]> v) {
 			_group = group;
 			_that = that;
 			_ret = ret;
-			_numCols = numCols;
 			_rl = rl;
 			_ru = ru;
 			_v = v;
@@ -444,8 +433,8 @@ public class CLALibLeftMultBy {
 		public Object call() {
 
 			try {
-				ColGroupValue.setupThreadLocalMemory(_v.getLeft() + 1);
-				_group.leftMultByMatrix(_that, _ret, _numCols, _rl, _ru);
+				ColGroupValue.setupThreadLocalMemory(_v.getLeft() * (_ru - _rl));
+				_group.leftMultByMatrix(_that, _ret, _rl, _ru);
 			}
 			catch(Exception e) {
 				throw new DMLRuntimeException(e);
@@ -510,7 +499,7 @@ public class CLALibLeftMultBy {
 		public Object call() {
 			ColGroupValue.setupThreadLocalMemory(_v.getLeft() + 1);
 			for(int i = _gl; i < _gu; i++) {
-				_grps.get(i).leftMultByMatrix(_rowVector, _result.getDenseBlockValues(), _result.getNumColumns());
+				_grps.get(i).leftMultByMatrix(_rowVector, _result);
 			}
 			return null;
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/ABitmap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/ABitmap.java
index 49e5712..69573a4 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/ABitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/ABitmap.java
@@ -37,24 +37,23 @@ public abstract class ABitmap {
 	/** Bitmaps (as lists of offsets) for each of the values. */
 	protected IntArrayList[] _offsetsLists;
 
-	/** int specifying the number of zero value groups contained in the rows. */
+	/** int specifying the number of zero value tuples contained in the rows. */
 	protected final int _numZeros;
 
 	public ABitmap(int numCols, IntArrayList[] offsetsLists, int rows) {
 		_numCols = numCols;
 		int offsetsTotal = 0;
-		if(offsetsLists != null){
-			for(IntArrayList a: offsetsLists){
+		if(offsetsLists != null) {
+			for(IntArrayList a : offsetsLists)
 				offsetsTotal += a.size();
-			}
+
 			_numZeros = rows - offsetsTotal;
-			if(_numZeros < 0){
+			if(_numZeros < 0)
 				throw new DMLCompressionException("Error in constructing bitmap");
-			}
 		}
-		else{
+		else
 			_numZeros = rows;
-		}
+
 		_offsetsLists = offsetsLists;
 	}
 
@@ -85,6 +84,12 @@ public abstract class ABitmap {
 		return ret;
 	}
 
+	/**
+	 * Get the number of offsets for a specific unique offset.
+	 * 
+	 * @param ix The offset index.
+	 * @return The number of offsets for this unique value.
+	 */
 	public int getNumOffsets(int ix) {
 		return _offsetsLists[ix].size();
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
index 693cf25..09354f6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
@@ -60,7 +60,7 @@ public final class Bitmap extends ABitmap {
 
 	public int getNumNonZerosInOffset(int idx){
 		if(_numCols == 1)
-			return  _offsetsLists[idx].size();
+			return  _values[0] != 0 ? 1 : 0;
 		int nz = 0;
 		for(int i = idx * _numCols; i < (idx+1) * _numCols; i++)
 			nz += _values[i] == 0 ? 0 : 1;
diff --git a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateOLETest.java b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateOLETest.java
index 5602ade..a76b2f1 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateOLETest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateOLETest.java
@@ -76,7 +76,7 @@ public class JolEstimateOLETest extends JolEstimateTest {
 		// Random rounded numbers dense
 		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 1523, 0, 99, 1.0, 7)));
 		tests.add(new Object[] {mb});
-		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 255, 1.0, 7)));
+		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 100, 1.0, 7)));
 		tests.add(new Object[] {mb});
 
 		// Sparse rounded numbers
@@ -88,7 +88,7 @@ public class JolEstimateOLETest extends JolEstimateTest {
 		mb = DataConverter
 			.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 2321, 0, 99, 0.1, 512)));
 		tests.add(new Object[] {mb});
-		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 255, 0.1, 7)));
+		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 100, 0.1, 7)));
 		tests.add(new Object[] {mb});
 
 		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 1523, 0, 99, 0.5, 7)));
@@ -99,7 +99,7 @@ public class JolEstimateOLETest extends JolEstimateTest {
 		mb = DataConverter
 			.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 2321, 0, 99, 0.5, 512)));
 		tests.add(new Object[] {mb});
-		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 255, 0.5, 7)));
+		mb = DataConverter.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(1, 4000, 0, 100, 0.5, 7)));
 		tests.add(new Object[] {mb});
 
 		// Paper
diff --git a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
index 90c3bf6..c32ccbe 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
@@ -32,6 +32,7 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupFactory;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorFactory;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.lib.BitmapEncoder;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -101,32 +102,32 @@ public abstract class JolEstimateTest {
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_90() {
-		compressedSizeInfoEstimatorSample(0.9, 0.99);
+		compressedSizeInfoEstimatorSample(0.9, 0.9);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_50() {
-		compressedSizeInfoEstimatorSample(0.5, 0.95);
+		compressedSizeInfoEstimatorSample(0.5, 0.90);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_20() {
-		compressedSizeInfoEstimatorSample(0.2, 0.90);
+		compressedSizeInfoEstimatorSample(0.2, 0.8);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_10() {
-		compressedSizeInfoEstimatorSample(0.1, 0.9);
+		compressedSizeInfoEstimatorSample(0.1, 0.75);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_5() {
-		compressedSizeInfoEstimatorSample(0.05, 0.9);
+		compressedSizeInfoEstimatorSample(0.05, 0.7);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_1() {
-		compressedSizeInfoEstimatorSample(0.01, 0.9);
+		compressedSizeInfoEstimatorSample(0.01, 0.6);
 	}
 
 	public void compressedSizeInfoEstimatorSample(double ratio, double tolerance) {
@@ -138,13 +139,16 @@ public abstract class JolEstimateTest {
 				.setValidCompressions(EnumSet.of(getCT())).setSeed(seed).create();
 			cs.transposed = true;
 
-			final long estimateCSI = CompressedSizeEstimatorFactory.getSizeEstimator(mbt, cs)
-				.estimateCompressedColGroupSize().getCompressionSize(cg.getCompType());
+			final CompressedSizeInfoColGroup cgsi = CompressedSizeEstimatorFactory.getSizeEstimator(mbt, cs)
+				.estimateCompressedColGroupSize();
+			final long estimateCSI = cgsi.getCompressionSize(cg.getCompType());
 			final double minTolerance = actualSize * tolerance;
 			final double maxTolerance = actualSize / tolerance;
 			final String rangeString = minTolerance + " < " + estimateCSI + " < " + maxTolerance;
 			boolean res = minTolerance < estimateCSI && estimateCSI < maxTolerance;
-			assertTrue("CSI Sampled estimate is not in tolerance range " + rangeString + "\n" + cg.toString(), res);
+			assertTrue(
+				"CSI Sampled estimate is not in tolerance range " + rangeString + "\n" + cgsi + "\n" + cg.toString(),
+				res);
 
 		}
 		catch(Exception e) {

[systemds] 02/07: [SYSTEMDS-2997] CLA MatrixBlock Dictionary

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

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

commit e7abe8765b06f7a9a59c349d6a2f15f67f15685f
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Mon May 10 17:47:25 2021 +0200

    [SYSTEMDS-2997] CLA MatrixBlock Dictionary
    
    - Add Basic MatrixBlock Dictionary
    - CocodeMatrixCost function for estimating the matrix multiplication cost.
---
 .../runtime/compress/CompressionSettings.java      |  23 ++-
 .../compress/CompressionSettingsBuilder.java       |  19 +-
 .../runtime/compress/cocode/AColumnCoCoder.java    |  10 +-
 .../sysds/runtime/compress/cocode/CoCodeCost.java  |   8 +-
 .../{CoCodeCost.java => CoCodeCostMatrixMult.java} |  97 ++++-----
 .../runtime/compress/cocode/PlanningCoCoder.java   |   8 +-
 .../runtime/compress/colgroup/ColGroupSDC.java     |  28 ++-
 .../compress/colgroup/ColGroupSDCSingle.java       |  28 ++-
 .../runtime/compress/colgroup/ColGroupValue.java   |  67 ++-----
 .../compress/colgroup/dictionary/ADictionary.java  |   4 +-
 .../compress/colgroup/dictionary/Dictionary.java   |   3 -
 .../compress/colgroup/dictionary/QDictionary.java  |  50 +++--
 .../colgroup/dictionary/SparseDictionary.java      | 218 +++++++++++++++++++++
 .../compress/estim/CompressedSizeEstimator.java    |  19 +-
 .../estim/CompressedSizeEstimatorSample.java       |   8 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |   8 +
 16 files changed, 440 insertions(+), 158 deletions(-)

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 f1ff14e..7be74ef 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -21,6 +21,8 @@ package org.apache.sysds.runtime.compress;
 
 import java.util.EnumSet;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.PartitionerType;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 
@@ -29,6 +31,7 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
  * CompressionSettingsBuilder for default non static parameters.
  */
 public class CompressionSettings {
+	private static final Log LOG = LogFactory.getLog(CompressionSettings.class.getName());
 
 	/**
 	 * Size of the blocks used in a blocked bitmap representation. Note it is exactly Character.MAX_VALUE. This is not
@@ -79,8 +82,14 @@ public class CompressionSettings {
 	/** The selected method for column partitioning used in CoCoding compressed columns */
 	public final PartitionerType columnPartitioner;
 
-	/** The maximum number of columns CoCoded if the Static CoCoding strategy is selected */
-	public final int maxStaticColGroupCoCode;
+	/** The maximum number of columns CoCoded allowed */
+	public final int maxColGroupCoCode;
+
+	/**
+	 * A Cocode parameter that differ in behavior based on compression method, in general it is a value that reflects
+	 * aggressively likely coCoding is used.
+	 */
+	public final double coCodePercentage;
 
 	/**
 	 * Valid Compressions List, containing the ColGroup CompressionTypes that are allowed to be used for the compression
@@ -91,7 +100,7 @@ public class CompressionSettings {
 	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput,
 		boolean skipList, int seed, boolean investigateEstimate, boolean lossy,
 		EnumSet<CompressionType> validCompressions, boolean sortValuesByLength, PartitionerType columnPartitioner,
-		int maxStaticColGroupCoCode) {
+		int maxColGroupCoCode, double coCodePercentage) {
 		this.samplingRatio = samplingRatio;
 		this.allowSharedDictionary = allowSharedDictionary;
 		this.transposeInput = transposeInput;
@@ -102,7 +111,9 @@ public class CompressionSettings {
 		this.lossy = lossy;
 		this.sortValuesByLength = sortValuesByLength;
 		this.columnPartitioner = columnPartitioner;
-		this.maxStaticColGroupCoCode = maxStaticColGroupCoCode;
+		this.maxColGroupCoCode = maxColGroupCoCode;
+		this.coCodePercentage = coCodePercentage;
+		LOG.debug(this);
 	}
 
 	@Override
@@ -113,6 +124,10 @@ public class CompressionSettings {
 		sb.append("\n DDC1 share dict: " + allowSharedDictionary);
 		sb.append("\n Partitioner: " + columnPartitioner);
 		sb.append("\n Lossy: " + lossy);
+		sb.append("\n sortValuesByLength: " + sortValuesByLength);
+		sb.append("\n column Partitioner: " + columnPartitioner);
+		sb.append("\n max Static ColGroup CoCode " + maxColGroupCoCode);
+		sb.append("\n max cocodePercentage " + coCodePercentage);
 		// If needed for debugging add more fields to the printing.
 		return sb.toString();
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index a53bcfa..72aeeeb 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -41,7 +41,8 @@ public class CompressionSettingsBuilder {
 	private EnumSet<CompressionType> validCompressions;
 	private boolean sortValuesByLength = true;
 	private PartitionerType columnPartitioner;
-	private int maxStaticColGroupCoCode = 10;
+	private int maxStaticColGroupCoCode = 10000;
+	private double coCodePercentage = 0.01;
 
 	public CompressionSettingsBuilder() {
 
@@ -236,6 +237,20 @@ public class CompressionSettingsBuilder {
 	}
 
 	/**
+	 * Set the coCode percentage, the effect is different based on the coCoding strategy, but the general effect is that
+	 * higher values results in more coCoding while lower values result in less.
+	 * 
+	 * Note that with high coCoding the compression ratio would possibly be lower.
+	 * 
+	 * @param coCodePercentage The percentage to set.
+	 * @return The CompressionSettingsBuilder
+	 */
+	public CompressionSettingsBuilder setCoCodePercentage(double coCodePercentage) {
+		this.coCodePercentage = coCodePercentage;
+		return this;
+	}
+
+	/**
 	 * Create the CompressionSettings object to use in the compression.
 	 * 
 	 * @return The CompressionSettings
@@ -243,6 +258,6 @@ public class CompressionSettingsBuilder {
 	public CompressionSettings create() {
 		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, skipList, seed,
 			investigateEstimate, lossy, validCompressions, sortValuesByLength, columnPartitioner,
-			maxStaticColGroupCoCode);
+			maxStaticColGroupCoCode, coCodePercentage);
 	}
 }
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 4dc1f8e..1ede652 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
@@ -30,14 +30,14 @@ public abstract class AColumnCoCoder {
 
 	protected static final Log LOG = LogFactory.getLog(AColumnCoCoder.class.getName());
 
-	protected CompressedSizeEstimator _est;
-	protected CompressionSettings _cs;
-	protected int _numRows;
+	final protected CompressedSizeEstimator _est;
+	final protected CompressionSettings _cs;
+	// final protected int _numRows;
 
 	protected AColumnCoCoder(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
 		_est = sizeEstimator;
 		_cs = cs;
-		_numRows = numRows;
+		// _numRows = numRows;
 	}
 
 	/**
@@ -64,7 +64,7 @@ public abstract class AColumnCoCoder {
 		CompressedSizeInfoColGroup rhs) {
 		int[] joined = Util.join(lhs.getColumns(), rhs.getColumns());
 		int numVals = lhs.getNumVals() + rhs.getNumVals();
-		return new CompressedSizeInfoColGroup(joined, numVals, _numRows);
+		return new CompressedSizeInfoColGroup(joined, numVals, _est.getNumRows());
 	}
 
 	protected CompressedSizeInfoColGroup analyze(CompressedSizeInfoColGroup g) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
index 1cce5b1..d2aec2c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
@@ -48,15 +48,11 @@ public class CoCodeCost extends AColumnCoCoder {
 	 */
 	private final int largestDistinct;
 
-	private final int toSmallForAnalysis;
-
-	private final double percentMaxCardinality = 0.08;
+	private final static int toSmallForAnalysis = 64;
 
 	protected CoCodeCost(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
 		super(sizeEstimator, cs, numRows);
-		largestDistinct = Math.max(256, (int) (_numRows * percentMaxCardinality));
-		toSmallForAnalysis = largestDistinct / 4;
-		LOG.error("CocodeCost largest Distinct: "+ largestDistinct + " toSmallForAnalysis: " + toSmallForAnalysis);
+		largestDistinct = Math.min(4096, Math.max(256, (int) (sizeEstimator.getNumRows() * cs.coCodePercentage)));
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
similarity index 56%
copy from src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
copy to src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
index 1cce5b1..09a9990 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
@@ -20,13 +20,12 @@
 package org.apache.sysds.runtime.compress.cocode;
 
 import java.util.ArrayList;
-import java.util.Comparator;
+import java.util.Arrays;
 import java.util.List;
 import java.util.PriorityQueue;
 import java.util.Queue;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
-import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
@@ -39,7 +38,7 @@ import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
  * This method allows us to compress many more columns than the BinPacking
  * 
  */
-public class CoCodeCost extends AColumnCoCoder {
+public class CoCodeCostMatrixMult extends AColumnCoCoder {
 
 	/**
 	 * This value specifies the maximum distinct count allowed int a coCoded group. Note that this value is the number
@@ -50,13 +49,11 @@ public class CoCodeCost extends AColumnCoCoder {
 
 	private final int toSmallForAnalysis;
 
-	private final double percentMaxCardinality = 0.08;
-
-	protected CoCodeCost(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
-		super(sizeEstimator, cs, numRows);
-		largestDistinct = Math.max(256, (int) (_numRows * percentMaxCardinality));
-		toSmallForAnalysis = largestDistinct / 4;
-		LOG.error("CocodeCost largest Distinct: "+ largestDistinct + " toSmallForAnalysis: " + toSmallForAnalysis);
+	protected CoCodeCostMatrixMult(CompressedSizeEstimator e, CompressionSettings cs, int numRows) {
+		super(e, cs, numRows);
+		largestDistinct = Math.max(256, (int) (_est.getNumRows() * _est.getNumColumns() * cs.coCodePercentage * 0.2));
+		toSmallForAnalysis = Math.min(Math.max(256, largestDistinct / 4), 1028);
+		LOG.debug("CocodeCost largest Distinct: " + largestDistinct + " toSmallForAnalysis: " + toSmallForAnalysis);
 	}
 
 	@Override
@@ -67,54 +64,66 @@ public class CoCodeCost extends AColumnCoCoder {
 
 	private List<CompressedSizeInfoColGroup> join(List<CompressedSizeInfoColGroup> currentGroups) {
 
-		Comparator<CompressedSizeInfoColGroup> comp = Comparator.comparing(CompressedSizeInfoColGroup::getNumVals);
-		Queue<CompressedSizeInfoColGroup> que = new PriorityQueue<>(currentGroups.size(), comp);
-		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
+		Queue<CostOfJoin> que = new PriorityQueue<>(currentGroups.size());
 
-		for(CompressedSizeInfoColGroup g : currentGroups) {
-			if(g.getBestCompressionType() == CompressionType.CONST)
-				ret.add(g);
-			else
-				que.add(g);
-		}
+		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
+		for(CompressedSizeInfoColGroup g : currentGroups)
+			que.add(new CostOfJoin(g));
 
-		boolean finished = false;
-		while(!finished) {
+		while(true) {
 			if(que.peek() != null) {
-				CompressedSizeInfoColGroup l = que.poll();
+				final CostOfJoin l = que.poll();
 				if(que.peek() != null) {
-					CompressedSizeInfoColGroup r = que.poll();
-					int worstCaseJoinedSize = l.getNumVals() * r.getNumVals();
-					if(worstCaseJoinedSize < toSmallForAnalysis)
-						que.add(joinWithoutAnalysis(l, r));
-					else if(worstCaseJoinedSize < largestDistinct){
-
-						CompressedSizeInfoColGroup g = joinWithAnalysis(l, r);
-						if(g.getNumVals() < largestDistinct)
-							que.add(joinWithAnalysis(l, r));
-						else{
-							finished = true;
-							que.add(l);
-							que.add(r);
-						}
-					}
+					final CostOfJoin r = que.poll();
+					final double costIndividual = (l.cost + r.cost);
+					final CostOfJoin g = new CostOfJoin(joinWithAnalysis(l.elm, r.elm));
+					if(g.cost < costIndividual)
+						que.add(g);
 					else {
-						finished = true;
-						que.add(l);
+						ret.add(l.elm);
 						que.add(r);
 					}
 				}
 				else {
-					que.add(l);
-					finished = true;
+					ret.add(l.elm);
+					break;
 				}
 			}
 			else
-				finished = true;
+				break;
 		}
-		for(CompressedSizeInfoColGroup g : que)
-			ret.add(g);
+		for(CostOfJoin g : que)
+			ret.add(g.elm);
 
 		return ret;
 	}
+
+	private class CostOfJoin implements Comparable<CostOfJoin> {
+		protected final CompressedSizeInfoColGroup elm;
+		protected final double cost;
+
+		protected CostOfJoin(CompressedSizeInfoColGroup elm) {
+			this.elm = elm;
+			final int nRows = _est.getNumRows();
+			final double commonFraction = elm.getMostCommonFraction();
+			final double rowsToProcess = commonFraction > 0.2 ? nRows * (1 - Math.min(commonFraction, 0.95)) : nRows;
+			this.cost = rowsToProcess + elm.getNumVals() * elm.getColumns().length;
+
+		}
+
+		@Override
+		public int compareTo(CostOfJoin o) {
+			return cost == o.cost ? 0 : cost > o.cost ? 1 : -1;
+		}
+
+		@Override
+		public String toString() {
+			StringBuilder sb = new StringBuilder();
+			sb.append("\n");
+			sb.append(cost);
+			sb.append(" - ");
+			sb.append(Arrays.toString(elm.getColumns()));
+			return sb.toString();
+		}
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
index 46bf988..94572c9 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
@@ -41,7 +41,7 @@ public class PlanningCoCoder {
 	 * The Valid coCoding techniques
 	 */
 	public enum PartitionerType {
-		BIN_PACKING, STATIC, COST;
+		BIN_PACKING, STATIC, COST, COST_MATRIX_MULT;
 	}
 
 	/**
@@ -66,10 +66,10 @@ public class PlanningCoCoder {
 			constantGroups = new ArrayList<>();
 			List<CompressedSizeInfoColGroup> newGroups = new ArrayList<>();
 			mem = new Memorizer();
-			for(CompressedSizeInfoColGroup g : colInfos.getInfo()){
+			for(CompressedSizeInfoColGroup g : colInfos.getInfo()) {
 				if(g.getBestCompressionType() == CompressionType.CONST)
 					constantGroups.add(g);
-				else{
+				else {
 					mem.put(g);
 					newGroups.add(g);
 				}
@@ -98,6 +98,8 @@ public class PlanningCoCoder {
 				return new CoCodeStatic(est, cs, numRows);
 			case COST:
 				return new CoCodeCost(est, cs, numRows);
+			case COST_MATRIX_MULT:
+				return new CoCodeCostMatrixMult(est, cs, numRows);
 			default:
 				throw new RuntimeException("Unsupported column group partitioner: " + type.toString());
 		}
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 1acbfdc..268f294 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
@@ -643,7 +643,7 @@ public class ColGroupSDC extends ColGroupValue {
 				else
 					itThis.next();
 			}
-	
+
 			while(itThat.hasNext()) {
 				final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
 				that._dict.addToEntry(ret, fr, offsetToDefaultThis, nCol);
@@ -717,7 +717,31 @@ public class ColGroupSDC extends ColGroupValue {
 
 	@Override
 	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
-		throw new NotImplementedException();
+		final AIterator itThat = that._indexes.getIterator();
+		final AIterator itThis = _indexes.getIterator();
+		final int nCol = that._colIndexes.length;
+		final int defThis = this.getNumValues() * nCol - nCol;
+
+		if(preModified) {
+			while(itThat.hasNext() && itThis.hasNext()) {
+				if(itThat.value() == itThis.value()) {
+					itThat.next();
+					final int to = getIndex(itThis.getDataIndexAndIncrement());
+					that._dict.addToEntry(ret, 0, to, nCol);
+				}
+				else if(itThat.value() < itThis.value()) {
+					itThat.next();
+					that._dict.addToEntry(ret, 0, defThis, nCol);
+				}
+				else
+					itThis.next();
+			}
+		}
+		else {
+			throw new NotImplementedException();
+		}
+
+		return ret;
 	}
 
 }
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 85aab80..14afdf4 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
@@ -565,7 +565,29 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	@Override
 	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
-		throw new NotImplementedException();
+		final AIterator itThat = that._indexes.getIterator();
+		final AIterator itThis = _indexes.getIterator();
+		final int nCol = that._colIndexes.length;
+		// final int defThat = that.getNumValues() * nCol - nCol;
+
+		if(preModified) {
+			while(itThat.hasNext() && itThis.hasNext()) {
+				if(itThat.value() == itThis.value()) {
+					final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
+					that._dict.addToEntry(ret, fr, 1, nCol);
+				}
+				else if(itThat.value() < itThis.value())
+					itThat.next();
+				else{
+					itThis.next();
+					// that._dict.addToEntry(ret, defThat, 0, nCol);
+				}
+			}
+		}
+		else {
+			throw new NotImplementedException();
+		}
+		return ret;
 	}
 
 	@Override
@@ -600,10 +622,6 @@ public class ColGroupSDCSingle extends ColGroupValue {
 					itThis.next();
 			}
 
-			// while(itThat.hasNext()) {
-			// 	final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
-			// 	that._dict.addToEntry(ret, fr, 1, nCol);
-			// }
 			return ret;
 		}
 		else {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index 223c3a0..dc52ca1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -828,13 +828,23 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 		if(sameIndexStructure(lhs)) {
 			int[] agI = getCounts();
-			for(int a = 0, off = 0; a < nvL; a++, off += nvL + 1)
-				leftMultDictEntry(agI[a], off, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, result);
+			for(int i = 0; i < agI.length; i++) {
+				for(int l = 0; l < lCol; l++) {
+					final int leftOff = lhs._colIndexes[l] * numCols;
+					final double lhV = lhValues[i * lCol + l] * agI[i];
+					if(lhV != 0)
+						for(int r = 0; r < rCol; r++) {
+							final double rhV = rhValues[i * rCol + r];
+							final double va = lhV * rhV;
+							result[leftOff + this._colIndexes[r]] += va;
+						}
+				}
+			}
 		}
 		else if(lhs instanceof ColGroupConst || this instanceof ColGroupConst) {
-			double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
-			double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
-			vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, result, numCols);
+			// double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
+			// double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
+			// vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, result, numCols);
 		}
 		else {
 			int[] countsRight = getCounts();
@@ -848,9 +858,9 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 			if(skipRight > threshold && percentageRight > percentageLeft && !(this instanceof ColGroupDDC)) {
 				double[] mct = this._dict.getMostCommonTuple(this.getCounts(), rCol);
-				double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
-				if(mct != null)
-					vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, result, numCols);
+				// double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
+				// if(mct != null)
+				// 	vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, result, numCols);
 
 				ColGroupValue thisM = (mct != null) ? (ColGroupValue) this
 					.copyAndSet(this._dict.subtractTuple(mct)) : this;
@@ -860,9 +870,9 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			}
 			else if(skipLeft > threshold && !(lhs instanceof ColGroupDDC)) {
 				double[] mct = lhs._dict.getMostCommonTuple(lhs.getCounts(), lCol);
-				double[] thisColSum = this._dict.colSum(getCounts(), rCol);
-				if(mct != null)
-					vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, result, numCols);
+				// double[] thisColSum = this._dict.colSum(getCounts(), rCol);
+				// if(mct != null)
+				// 	vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, result, numCols);
 
 				ColGroupValue lhsM = (mct != null) ? (ColGroupValue) lhs.copyAndSet(lhs._dict.subtractTuple(mct)) : lhs;
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhsM, true);
@@ -882,41 +892,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		}
 	}
 
-	// private void leftMultMapPreAggregate(final int nvL, final int lCol, final int rCol, final ColGroupValue lhs,
-	// final int numCols, double[] lhValues, double[] rhValues, double[] c, MapPreAggregate agM) {
-	// final int[] map = agM.getMap();
-	// final int aggSize = agM.getSize();
-	// for(int k = 0; k < aggSize; k += 2)
-	// leftMultDictEntry(map[k + 1], map[k], nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-	// leftMultDictEntry(agM.getMapFreeValue(), 0, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-	// }
-
-	// private void leftMultArrayPreAggregate(final int nvL, final int nvR, final int lCol, final int rCol,
-	// final ColGroupValue lhs, final int numCols, double[] lhValues, double[] rhValues, double[] c, int[] arr) {
-	// for(int a = 0; a < nvL * nvR; a++)
-	// leftMultDictEntry(arr[a], a, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-	// }
-
-	private void leftMultDictEntry(final int m, final int a, final int nvL, final int lCol, final int rCol,
-		final ColGroupValue lhs, final int numCols, final double[] lhValues, final double[] rhValues,
-		final double[] c) {
-
-		if(m > 0) {
-			final int lhsRowOffset = (a % nvL) * lCol;
-			final int rhsRowOffset = (a / nvL) * rCol;
-
-			for(int j = 0; j < lCol; j++) {
-				final int resultOff = lhs._colIndexes[j] * numCols;
-				final double lh = lhValues[lhsRowOffset + j] * m;
-				if(lh != 0)
-					for(int i = 0; i < rCol; i++) {
-						double rh = rhValues[rhsRowOffset + i];
-						c[resultOff + _colIndexes[i]] += lh * rh;
-					}
-			}
-		}
-	}
-
 	@Override
 	public void tsmm(double[] result, int numColumns) {
 		int[] counts = getCounts();
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 1aeda85..352e96b 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
@@ -22,6 +22,8 @@ package org.apache.sysds.runtime.compress.colgroup.dictionary;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.Bitmap;
 import org.apache.sysds.runtime.compress.utils.BitmapLossy;
@@ -35,7 +37,7 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
  */
 public abstract class ADictionary {
 
-	// private static final Log LOG = LogFactory.getLog(ADictionary.class.getName());
+	protected static final Log LOG = LogFactory.getLog(ADictionary.class.getName());
 
 	/**
 	 * Get all the values contained in the dictionary as a linearized double array.
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 b4f843f..5a32823 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
@@ -24,8 +24,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
@@ -39,7 +37,6 @@ import org.apache.sysds.utils.MemoryEstimates;
  */
 public class Dictionary extends ADictionary {
 
-	protected static final Log LOG = LogFactory.getLog(Dictionary.class.getName());
 	private final double[] _values;
 
 	public Dictionary(double[] values) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index a8559db..7b01fb5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -24,8 +24,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.utils.BitmapLossy;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Divide;
@@ -42,7 +40,6 @@ import org.apache.sysds.utils.MemoryEstimates;
  */
 public class QDictionary extends ADictionary {
 
-	protected static final Log LOG = LogFactory.getLog(QDictionary.class.getName());
 	protected double _scale;
 	protected byte[] _values;
 
@@ -121,14 +118,13 @@ public class QDictionary extends ADictionary {
 		return ret;
 	}
 
-
 	@Override
-	public double[] aggregateTuples(Builtin fn, final int nCol){
+	public double[] aggregateTuples(Builtin fn, final int nCol) {
 		if(nCol == 1)
 			return getValues();
 		final int nRows = _values.length / nCol;
 		double[] res = new double[nRows];
-		for(int i = 0; i < nRows; i++){
+		for(int i = 0; i < nRows; i++) {
 			final int off = i * nCol;
 			res[i] = _values[off];
 			for(int j = off + 1; j < off + nCol; j++)
@@ -258,7 +254,6 @@ public class QDictionary extends ADictionary {
 		return new QDictionary(ret, _scale);
 	}
 
-
 	@Override
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
@@ -314,20 +309,20 @@ public class QDictionary extends ADictionary {
 		}
 	}
 
-
 	@Override
-	public double[] colSum(int[] counts, int nCol){
+	public double[] colSum(int[] counts, int nCol) {
 		throw new NotImplementedException("Not Implemented");
 		// final double[] res = new double[counts.length];
 		// int idx = 0;
 		// for(int k = 0; k< _values.length / counts.length; k++){
-		// 	final int cntk = counts[k];
-		// 	for(int j = 0; j< counts.length; j++){
-		// 		res[j] += _values[idx++] * cntk;
-		// 	}
+		// final int cntk = counts[k];
+		// for(int j = 0; j< counts.length; j++){
+		// res[j] += _values[idx++] * cntk;
+		// }
 		// }
 		// return res;
 	}
+
 	@Override
 	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
 		throw new NotImplementedException("Not Implemented");
@@ -412,7 +407,7 @@ public class QDictionary extends ADictionary {
 		}
 	}
 
-	public String getString( int colIndexes) {
+	public String getString(int colIndexes) {
 		StringBuilder sb = new StringBuilder();
 		for(int i = 0; i < size(); i++) {
 			sb.append(_values[i]);
@@ -453,32 +448,31 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public boolean containsValue(double pattern){
+	public boolean containsValue(double pattern) {
 		if(Double.isNaN(pattern) || Double.isInfinite(pattern))
 			return false;
 		throw new NotImplementedException("Not contains value on Q Dictionary");
 	}
 
 	@Override
-	public long getNumberNonZeros(int[] counts, int nCol){
-		long nnz =  0;
+	public long getNumberNonZeros(int[] counts, int nCol) {
+		long nnz = 0;
 		final int nRow = _values.length / nCol;
-		for(int i = 0; i < nRow; i++){
+		for(int i = 0; i < nRow; i++) {
 			long rowCount = 0;
-			final int off = i * nCol; 
-			for(int j = off; j < off + nCol; j++){
+			final int off = i * nCol;
+			for(int j = off; j < off + nCol; j++) {
 				if(_values[j] != 0)
-					rowCount ++;
+					rowCount++;
 			}
 			nnz += rowCount * counts[i];
 		}
 		return nnz;
 	}
 
-
 	@Override
-	public void addToEntry(Dictionary d, int fr, int to, int nCol){
-		throw new NotImplementedException("Not implemented yet");	
+	public void addToEntry(Dictionary d, int fr, int to, int nCol) {
+		throw new NotImplementedException("Not implemented yet");
 	}
 
 	@Override
@@ -487,9 +481,9 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public long getNumberNonZerosContained(){
+	public long getNumberNonZerosContained() {
 		long count = 0;
-		for(double v : _values){
+		for(double v : _values) {
 			if(v != 0.0)
 				count++;
 		}
@@ -497,12 +491,12 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public double[] getMostCommonTuple(int[] counts, int nCol){
+	public double[] getMostCommonTuple(int[] counts, int nCol) {
 		return null;
 	}
 
 	@Override
-	public ADictionary subtractTuple(double[] tuple){
+	public ADictionary subtractTuple(double[] tuple) {
 		throw new NotImplementedException();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java
new file mode 100644
index 0000000..3a400f3
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java
@@ -0,0 +1,218 @@
+/*
+ * 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.dictionary;
+
+import org.apache.sysds.runtime.functionobjects.Builtin;
+import org.apache.sysds.runtime.functionobjects.ValueFunction;
+import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
+
+/**
+ * A sparse dictionary implementation, use if the tuples are sparse.
+ */
+public class SparseDictionary extends ADictionary {
+
+	@Override
+	public double[] getValues() {
+		LOG.warn("Inefficient materialization of sparse Dictionary.");
+
+		return null;
+	}
+
+	@Override
+	public double getValue(int i) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public int hasZeroTuple(int nCol) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public long getInMemorySize() {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public double aggregate(double init, Builtin fn) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public double[] aggregateTuples(Builtin fn, int nCol) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public int size() {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public ADictionary apply(ScalarOperator op) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary clone() {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary cloneAndExtend(int len) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public boolean isLossy() {
+		// TODO Auto-generated method stub
+		return false;
+	}
+
+	@Override
+	public int getNumberOfValues(int ncol) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public double[] sumAllRowsToDouble(boolean square, int nrColumns) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public double sumRow(int k, boolean square, int nrColumns) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public double[] colSum(int[] counts, int nCol) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public double sum(int[] counts, int ncol) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public double sumsq(int[] counts, int ncol) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public String getString(int colIndexes) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public void addMaxAndMin(double[] ret, int[] colIndexes) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary reExpandColumns(int max) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public boolean containsValue(double pattern) {
+		// TODO Auto-generated method stub
+		return false;
+	}
+
+	@Override
+	public long getNumberNonZeros(int[] counts, int nCol) {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public long getNumberNonZerosContained() {
+		// TODO Auto-generated method stub
+		return 0;
+	}
+
+	@Override
+	public void addToEntry(Dictionary d, int fr, int to, int nCol) {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Override
+	public double[] getMostCommonTuple(int[] counts, int nCol) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+	@Override
+	public ADictionary subtractTuple(double[] tuple) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
index 5332060..4902cb8 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
@@ -42,19 +42,19 @@ public abstract class CompressedSizeEstimator {
 	protected static final Log LOG = LogFactory.getLog(CompressedSizeEstimator.class.getName());
 
 	/** The Matrix Block to extract the compression estimates from */
-	protected MatrixBlock _data;
+	final protected MatrixBlock _data;
 	/** The number of rows in the matrix block, extracted to a field because the matrix could be transposed */
-	protected int _numRows;
+	final protected int _numRows;
 	/** The number of columns in the matrix block, extracted to a field because the matrix could be transposed */
-	protected int _numCols;
+	final protected int _numCols;
 	/** The compression settings to use, for estimating the size, and compress the ColGroups. */
-	protected final CompressionSettings _compSettings;
+	final protected CompressionSettings _compSettings;
 
 	/**
 	 * Boolean specifying if the _data is in transposed format. This is used to select the correct readers for the
 	 * extraction of bitmaps for the columns.
 	 */
-	protected boolean _transposed = false;
+	protected boolean _transposed;
 
 	/**
 	 * Main Constructor for Compression Estimator.
@@ -72,6 +72,15 @@ public abstract class CompressedSizeEstimator {
 		_compSettings = compSettings;
 	}
 
+
+	public int getNumRows(){
+		return _numRows;
+	}
+
+	public int getNumColumns(){
+		return _numCols;
+	}
+
 	/**
 	 * Multi threaded version of extracting Compression Size info
 	 * 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
index 0c83202..af94473 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -41,6 +41,8 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	private static final int FORCE_TRANSPOSE_ON_SAMPLE_THRESHOLD = 8000;
 
 	private final int[] _sampleRows;
+
+	private final MatrixBlock _sample;
 	private HashMap<Integer, Double> _solveCache = null;
 
 	/**
@@ -56,7 +58,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		super(data, compSettings, transposed);
 		_sampleRows = sampleRows;
 		_solveCache = new HashMap<>();
-		_data = sampleData(data, compSettings, sampleRows, transposed);
+		_sample = sampleData(data, compSettings, sampleRows, transposed);
 	}
 
 	protected MatrixBlock sampleData(MatrixBlock data, CompressionSettings compSettings, int[] sampleRows,
@@ -75,8 +77,6 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 			sampledMatrixBlock = LibMatrixReorg.transposeInPlace(sampledMatrixBlock, 16);
 		}
 		else {
-
-			// Override the _data Matrix block with the sampled matrix block.
 			MatrixBlock select = (transposed) ? new MatrixBlock(data.getNumColumns(), 1,
 				true) : new MatrixBlock(data.getNumRows(), 1, true);
 			for(int i = 0; i < sampleRows.length; i++)
@@ -106,7 +106,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		// final int numCols = colIndexes.length;
 
 		// extract statistics from sample
-		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, _data, _transposed);
+		final ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, _sample, _transposed);
 		final EstimationFactors fact = EstimationFactors.computeSizeEstimationFactors(ubm, false, _numRows, colIndexes);
 		final int numZerosInSample = ubm.getZeroCounts();
 		final boolean lossy = ubm.getType() == BitmapType.Lossy;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
index 7b9bb8d..d25f280 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -144,6 +144,14 @@ public class CompressedSizeInfoColGroup {
 		return _cardinalityRatio;
 	}
 
+	public double getMostCommonFraction(){
+		return (double) _facts.largestOff / _facts.numRows;
+	}
+
+	public double getTupleSparsity(){
+		return _facts.tupleSparsity;
+	}
+
 	private static Map<CompressionType, Long> calculateCompressionSizes(EstimationFactors fact,
 		Set<CompressionType> validCompressionTypes) {
 		Map<CompressionType, Long> res = new HashMap<>();

[systemds] 05/07: [SYSTEMDS-2997] CLA MatrixBlock Dictionary Update

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

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

commit 0f8e302bcf90434269712a13617728fdaa69f119
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Tue May 18 22:35:51 2021 +0200

    [SYSTEMDS-2997] CLA MatrixBlock Dictionary Update
    
    This commit change the dictionary of the column groups to support
    MatrixBlocks, this further enforce the previous design of using already
    implemented kernels, and allow for sparse dictionary exploitation in
    operations.
---
 .../runtime/compress/CompressedMatrixBlock.java    |  39 +-
 .../compress/cocode/CoCodeCostMatrixMult.java      |   2 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |  64 +---
 .../runtime/compress/colgroup/ColGroupConst.java   |  14 +-
 .../runtime/compress/colgroup/ColGroupDDC.java     |  44 ++-
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   6 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     |  12 +-
 .../runtime/compress/colgroup/ColGroupRLE.java     |  12 +-
 .../runtime/compress/colgroup/ColGroupSDC.java     |  10 +-
 .../compress/colgroup/ColGroupSDCSingle.java       |  71 ++--
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  30 +-
 .../compress/colgroup/ColGroupSDCZeros.java        |  11 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  17 +-
 .../runtime/compress/colgroup/ColGroupValue.java   | 399 ++++++++++++++-------
 .../compress/colgroup/dictionary/ADictionary.java  |  41 ++-
 .../compress/colgroup/dictionary/Dictionary.java   |  43 ++-
 .../colgroup/dictionary/DictionaryFactory.java     |  34 +-
 .../colgroup/dictionary/MatrixBlockDictionary.java | 195 ++++++++--
 .../compress/colgroup/dictionary/QDictionary.java  |  31 +-
 .../sysds/runtime/compress/lib/BitmapEncoder.java  |   1 +
 .../runtime/compress/lib/CLALibBinaryCellOp.java   |  11 +-
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |   2 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     | 358 +++++++++---------
 .../runtime/compress/lib/CLALibRelationalOp.java   |   9 +-
 .../sysds/runtime/compress/lib/CLALibScalar.java   |   1 -
 .../readers/ReaderCompressedSelection.java         |   6 +-
 .../sysds/runtime/compress/utils/DblArray.java     |  11 +-
 .../sysds/runtime/matrix/data/LibMatrixMult.java   |   2 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |  10 +
 .../component/compress/CompressedMatrixTest.java   |   7 +-
 .../component/compress/CompressedTestBase.java     |  40 ++-
 .../sysds/test/component/compress/TestBase.java    |  14 +-
 32 files changed, 958 insertions(+), 589 deletions(-)

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 249ad3e..27d77ed 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -62,7 +62,6 @@ import org.apache.sysds.runtime.compress.lib.CLALibReExpand;
 import org.apache.sysds.runtime.compress.lib.CLALibRightMultBy;
 import org.apache.sysds.runtime.compress.lib.CLALibScalar;
 import org.apache.sysds.runtime.compress.lib.CLALibSquash;
-import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.controlprogram.caching.CacheBlock;
 import org.apache.sysds.runtime.controlprogram.caching.MatrixObject.UpdateType;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
@@ -198,13 +197,17 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 		Timing time = new Timing(true);
 
+		long nnz = getNonZeros() == -1 ? recomputeNonZeros() : nonZeros;
+		if(isEmpty())
+			return new MatrixBlock(rlen, clen, true, 0);
+
 		// preallocation sparse rows to avoid repeated reallocations
 		MatrixBlock ret = getUncompressedColGroupAndRemoveFromListOfColGroups();
 		if(ret != null && getColGroups().size() == 0)
 			return ret;
 		else if(ret == null)
 			ret = new MatrixBlock(rlen, clen, false, -1);
-
+		ret.setNonZeros(nnz);
 		ret.allocateDenseBlock();
 		decompress(ret);
 
@@ -220,7 +223,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	private MatrixBlock decompress(MatrixBlock ret) {
 
 		for(AColGroup grp : _colGroups)
-			grp.decompressToBlockUnSafe(ret, 0, rlen, 0, grp.getValues());
+			grp.decompressToBlockUnSafe(ret, 0, rlen, 0);
 
 		if(ret.isInSparseFormat())
 			ret.sortSparseRows();
@@ -229,10 +232,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			ret.recomputeNonZeros();
 			ret.examSparsity();
 		}
-		else if(nonZeros == -1)
-			ret.setNonZeros(this.recomputeNonZeros());
-		else
-			ret.setNonZeros(nonZeros);
 
 		return ret;
 	}
@@ -289,10 +288,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			ret.recomputeNonZeros();
 			ret.examSparsity();
 		}
-		else if(nonZeros == -1)
-			ret.setNonZeros(this.recomputeNonZeros());
-		else
-			ret.setNonZeros(nonZeros);
+
 		return ret;
 	}
 
@@ -332,7 +328,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			long nnz = 0;
 			for(AColGroup g : _colGroups)
 				nnz += g.getNumberNonZeros();
-
 			nonZeros = nnz;
 		}
 		return nonZeros;
@@ -673,21 +668,17 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		// check for transpose type
 		if(tstype != MMTSJType.LEFT) // right not supported yet
 			throw new DMLRuntimeException("Invalid MMTSJ type '" + tstype.toString() + "'.");
-
+		if(isEmptyBlock())
+			return new MatrixBlock(clen, clen, true);
 		// create output matrix block
 		if(out == null)
 			out = new MatrixBlock(clen, clen, false);
 		else
 			out.reset(clen, clen, false);
 		out.allocateDenseBlock();
-
-		if(!isEmptyBlock(false)) {
-			// compute matrix mult
-			CLALibLeftMultBy.leftMultByTransposeSelf(_colGroups, out, k, getNumColumns(), getMaxNumValues(),
-				isOverlapping());
-			// post-processing
-			out.setNonZeros(LinearAlgebraUtils.copyUpperToLowerTriangle(out));
-		}
+		// compute matrix mult
+		CLALibLeftMultBy.leftMultByTransposeSelf(_colGroups, out, k, getNumColumns(), getMaxNumValues(),
+			isOverlapping());
 		return out;
 	}
 
@@ -767,7 +758,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 			// decompress row partition
 			for(AColGroup grp : _colGroups)
-				grp.decompressToBlock(_ret, _rl, _ru, grp.getValues(), false);
+				grp.decompressToBlock(_ret, _rl, _ru, false);
 
 			// post processing (sort due to append)
 			if(_ret.isInSparseFormat())
@@ -927,7 +918,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public boolean isEmptyBlock(boolean safe) {
-		return(_colGroups == null || getNonZeros() == 0);
+		return _colGroups == null || nonZeros == 0;
 	}
 
 	@Override
@@ -1002,7 +993,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		AColGroup grp = _colGroups.get(0);
 		MatrixBlock vals = grp.getValuesAsBlock();
 		if(grp instanceof ColGroupValue) {
-			MatrixBlock counts = getCountsAsBlock(((ColGroupValue) grp).getCounts());
+			MatrixBlock counts = getCountsAsBlock( ((ColGroupValue) grp).getCounts());
 			if(counts.isEmpty())
 				return vals.cmOperations(op);
 			return vals.cmOperations(op, counts);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
index 836e4d0..0d39b47 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
@@ -113,7 +113,7 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 			final double postScalingCost = (nCols > 1 && elm.getTupleSparsity() > 0.4) ? numberTuples *
 				nCols : numberTuples * nCols * tupleSparsity;
 
-			this.cost = preAggregateCost + postScalingCost;
+			this.cost = preAggregateCost + postScalingCost ;
 		}
 
 		@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
index 6b74136..c1daeb9 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
@@ -207,7 +207,7 @@ public abstract class AColGroup implements Serializable {
 	 * @param ru     row upper
 	 */
 	public void decompressToBlock(MatrixBlock target, int rl, int ru) {
-		decompressToBlock(target, rl, ru, rl, getValues(), true);
+		decompressToBlock(target, rl, ru, rl, true);
 	}
 
 	/**
@@ -219,23 +219,9 @@ public abstract class AColGroup implements Serializable {
 	 * @param offT   The rowOffset into target to decompress to.
 	 */
 	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlock(target, rl, ru, offT, getValues(), true);
+		decompressToBlock(target, rl, ru, offT, true);
 	}
 
-	/**
-	 * Decompress the contents of this column group into the target matrixBlock using the values provided as replacement
-	 * of the dictionary values, it is assumed that the target matrix Block have the same number of columns and at least
-	 * the number of rows ru.
-	 * 
-	 * @param target The target matrixBlock to decompress into
-	 * @param rl     The row to start at
-	 * @param ru     The row to end at
-	 * @param values The dictionary values materialized.
-	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
-	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, double[] values, boolean safe) {
-		decompressToBlock(target, rl, ru, rl, values, safe);
-	}
 
 	/**
 	 * Decompress the contents of this column group into the target matrixBlock, it is assumed that the target matrix
@@ -247,25 +233,9 @@ public abstract class AColGroup implements Serializable {
 	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
 	 */
 	public void decompressToBlock(MatrixBlock target, int rl, int ru, boolean safe) {
-		decompressToBlock(target, rl, ru, rl, getValues(), safe);
+		decompressToBlock(target, rl, ru, rl, safe);
 	}
 
-	/**
-	 * Decompress the contents of this column group into the target matrixBlock with an offset of the indexes, it is
-	 * assumed that the target matrix Block have the same number of columns and at least the number of rows ru.
-	 * 
-	 * The offset of indexes makes it possible to decompress parts of the compressed column group like say rows 10 to
-	 * 20, into row 0 to 10 in the target matrix.
-	 * 
-	 * @param target The target matrixBlock to decompress into
-	 * @param rl     The row to start at
-	 * @param ru     The row to end at
-	 * @param offT   The offset into the target to decompress to.
-	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
-	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT, boolean safe) {
-		decompressToBlock(target, rl, ru, offT, getValues(), safe);
-	}
 
 	/**
 	 * Decompress the contents of this column group into the target matrixBlock with an offset of the indexes using the
@@ -279,14 +249,13 @@ public abstract class AColGroup implements Serializable {
 	 * @param rl     The row to start at
 	 * @param ru     The row to end at
 	 * @param offT   The offset into the target to decompress to.
-	 * @param values The dictionary values materialized.
 	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
 	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT, double[] values, boolean safe) {
+	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT,  boolean safe) {
 		if(safe)
-			decompressToBlockSafe(target, rl, ru, offT, values);
+			decompressToBlockSafe(target, rl, ru, offT);
 		else
-			decompressToBlockUnSafe(target, rl, ru, offT, values);
+			decompressToBlockUnSafe(target, rl, ru, offT);
 	}
 
 	/**
@@ -297,9 +266,8 @@ public abstract class AColGroup implements Serializable {
 	 * @param rl     row lower
 	 * @param ru     row upper
 	 * @param offT   Offset into target to assign from
-	 * @param values The Values materialized in the dictionary
 	 */
-	public abstract void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values);
+	public abstract void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT);
 
 	/**
 	 * Decompress the contents of the columngroup unsafely, meaning that it does not count nonzero values.
@@ -308,22 +276,8 @@ public abstract class AColGroup implements Serializable {
 	 * @param rl     row lower
 	 * @param ru     row upper
 	 * @param offT   Offset into target to assign from
-	 * @param values The Values materialized in the dictionary
-	 */
-	public abstract void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values);
-
-	/**
-	 * Decompress the contents of this column group into the specified full matrix block.
-	 * 
-	 * @param target a matrix block where the columns covered by this column group have not yet been filled in.
-	 * @param rl     row lower
-	 * @param ru     row upper
-	 * @param offT   The offset into the target matrix block to decompress to.
-	 * @param values The Values materialized in the dictionary
 	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockSafe(target, rl, ru, offT, values);
-	}
+	public abstract void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT);
 
 	/**
 	 * Decompress the contents of this column group into uncompressed packed columns
@@ -401,7 +355,7 @@ public abstract class AColGroup implements Serializable {
 	 */
 	public static void decompressColumnToBlockUnSafe(MatrixBlock target, int rl, int ru, List<AColGroup> colGroups) {
 		for(AColGroup g : colGroups)
-			g.decompressToBlockUnSafe(target, rl, ru, rl, g.getValues());
+			g.decompressToBlockUnSafe(target, rl, ru, rl);
 	}
 
 	/**
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 e3c2965..d439c4e 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
@@ -113,14 +113,15 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(_colIndexes.length * target.getNumRows() + target.getNonZeros());
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		double[] c = target.getDenseBlockValues();
+		double[] values = getValues();
 		offT = offT * target.getNumColumns();
 		for(int i = rl; i < ru; i++, offT += target.getNumColumns())
 			for(int j = 0; j < _colIndexes.length; j++)
@@ -205,9 +206,10 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, double[] values, int rl, int ru) {
+	public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, int rl, int ru) {
 		final double[] cV = c.getDenseBlockValues();
-		if(a.isEmpty())
+		final double[] values = getValues();
+		if(values == null  || a.isEmpty())
 			return;
 		else if(a.isInSparseFormat()) {
 			SparseBlock sb = a.getSparseBlock();
@@ -241,7 +243,7 @@ public class ColGroupConst extends ColGroupValue {
 
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-		return new ColGroupConst(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, true, left));
+		return new ColGroupConst(_colIndexes, _numRows, applyBinaryRowOp(op, v, true, left));
 	}
 
 	@Override
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 fa967da..42cf04b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
@@ -27,6 +27,7 @@ import java.util.Arrays;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
+import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
@@ -64,16 +65,50 @@ public class ColGroupDDC extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(target.getNonZeros() + _numRows * _colIndexes.length);
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
 		final double[] c = target.getDenseBlockValues();
+		if(_dict instanceof MatrixBlockDictionary) {
+			MatrixBlock dmb = ((MatrixBlockDictionary) _dict).getMatrixBlock();
+			if(dmb.isEmpty())
+				return;
+			else if(dmb.isInSparseFormat())
+				decompressToBlockUnsafeSparse(c, rl, ru, offT, dmb.getSparseBlock(), tCol, nCol);
+			else
+				decompressToBlockUnsafeDense(c, rl, ru, offT, dmb.getDenseBlockValues(), tCol, nCol);
+		}
+		else
+			decompressToBlockUnsafeDense(c, rl, ru, offT, getValues(), tCol, nCol);
+
+	}
+
+	private void decompressToBlockUnsafeSparse(double[] c, int rl, int ru, int offT, SparseBlock sb, int tCol,
+		int nCol) {
+		offT = offT * tCol;
+		for(int i = rl; i < ru; i++, offT += tCol) {
+			final int rowIndex = _data.getIndex(i);
+			if(sb.isEmpty(rowIndex))
+				continue;
+			final int apos = sb.pos(rowIndex);
+			final int alen = sb.size(rowIndex) + apos;
+			final double[] avals = sb.values(rowIndex);
+			final int[] aix = sb.indexes(rowIndex);
+			for(int j = apos; j < alen; j++)
+				c[offT + _colIndexes[aix[j]]] += avals[j];
+
+		}
+	}
+
+	private void decompressToBlockUnsafeDense(double[] c, int rl, int ru, int offT, double[] values, int tCol,
+		int nCol) {
+		// final double[] values = getValues();
 		offT = offT * tCol;
 
 		for(int i = rl; i < ru; i++, offT += tCol) {
@@ -81,7 +116,6 @@ public class ColGroupDDC extends ColGroupValue {
 			for(int j = 0; j < nCol; j++)
 				c[offT + _colIndexes[j]] += values[rowIndex + j];
 		}
-
 	}
 
 	@Override
@@ -567,7 +601,7 @@ public class ColGroupDDC extends ColGroupValue {
 
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-		ADictionary aDict = applyBinaryRowOp(op.fn, v, true, left);
+		ADictionary aDict = applyBinaryRowOp(op, v, true, left);
 		return new ColGroupDDC(_colIndexes, _numRows, aDict, _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 be33491..11adb66 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
@@ -85,12 +85,12 @@ public class ColGroupEmpty extends ColGroupCompressed {
 
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
 		// do nothing.
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		// do nothing.
 	}
 
@@ -138,7 +138,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		if(sparseSafe)
 			return this;
 		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[_colIndexes.length]).applyBinaryRowOp(op.fn, v, sparseSafe, _colIndexes, left));
+			new Dictionary(new double[_colIndexes.length]).applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left));
 	}
 
 	@Override
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 bb4f325..712a574 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
@@ -70,11 +70,12 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
 
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
+		final double[] values = getValues();
 
 		// cache blocking config and position array
 		int[] apos = skipScan(numVals, rl);
@@ -112,13 +113,14 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
 		final int offOut = (rl - offT);
 		final int targetCols = target.getNumColumns();
+		final double[] values = getValues();
 
 		// cache blocking config and position array
 		int[] apos = skipScan(numVals, rl);
@@ -363,7 +365,7 @@ public class ColGroupOLE extends ColGroupOffset {
 		// fast path: sparse-safe operations
 		// Note that bitmaps don't change and are shallow-copied
 		if(sparseSafe) {
-			return new ColGroupOLE(_colIndexes, _numRows, _zeros, applyBinaryRowOp(op.fn, v, sparseSafe, left), _data,
+			return new ColGroupOLE(_colIndexes, _numRows, _zeros, applyBinaryRowOp(op, v, sparseSafe, left), _data,
 				_ptr, getCachedCounts());
 		}
 
@@ -372,10 +374,10 @@ public class ColGroupOLE extends ColGroupOffset {
 		boolean[] lind = computeZeroIndicatorVector();
 		int[] loff = computeOffsets(lind);
 		if(loff.length == 0) { // empty offset list: go back to fast path
-			return new ColGroupOLE(_colIndexes, _numRows, false, applyBinaryRowOp(op.fn, v, true, left), _data, _ptr,
+			return new ColGroupOLE(_colIndexes, _numRows, false, applyBinaryRowOp(op, v, true, left), _data, _ptr,
 				getCachedCounts());
 		}
-		ADictionary rvalues = applyBinaryRowOp(op.fn, v, sparseSafe, left);
+		ADictionary rvalues = applyBinaryRowOp(op, v, sparseSafe, left);
 		char[] lbitmap = genOffsetBitmap(loff, loff.length);
 		char[] rbitmaps = Arrays.copyOf(_data, _data.length + lbitmap.length);
 		System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
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 1649591..1c81fca 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
@@ -68,10 +68,11 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
+		final double[] values = getValues();
 
 		// position and start offset arrays
 		int[] astart = new int[numVals];
@@ -111,10 +112,11 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
+		final double[] values = getValues();
 
 		// position and start offset arrays
 		int[] astart = new int[numVals];
@@ -623,7 +625,7 @@ public class ColGroupRLE extends ColGroupOffset {
 		// 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.fn, v, sparseSafe, left), _data,
+			return new ColGroupRLE(_colIndexes, _numRows, _zeros, applyBinaryRowOp(op, v, sparseSafe, left), _data,
 				_ptr, getCachedCounts());
 		}
 
@@ -632,11 +634,11 @@ public class ColGroupRLE extends ColGroupOffset {
 		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.fn, v, true, left), _data, _ptr,
+			return new ColGroupRLE(_colIndexes, _numRows, false, applyBinaryRowOp(op, v, true, left), _data, _ptr,
 				getCachedCounts());
 		}
 
-		ADictionary rvalues = applyBinaryRowOp(op.fn, v, sparseSafe, left);
+		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);
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 d2cebf5..769b2fe 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
@@ -97,17 +97,19 @@ public class ColGroupSDC extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(getNumberNonZeros());
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
+		final double[] values = getValues();
 		final int offsetToDefault = values.length - nCol;
+
 		double[] c = target.getDenseBlockValues();
 		offT = offT * tCol;
 		int i = rl;
@@ -343,7 +345,7 @@ public class ColGroupSDC extends ColGroupValue {
 
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-		return new ColGroupSDC(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, true, left), _indexes, _data,
+		return new ColGroupSDC(_colIndexes, _numRows, applyBinaryRowOp(op, v, true, left), _indexes, _data,
 			getCachedCounts());
 	}
 
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 1424072..493afac 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
@@ -86,17 +86,19 @@ public class ColGroupSDCSingle extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(_numRows * _colIndexes.length + target.getNonZeros());
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
+		final double[] values = getValues();
 		final int offsetToDefault = values.length - nCol;
+
 		double[] c = target.getDenseBlockValues();
 		offT = offT * tCol;
 		int i = rl;
@@ -105,17 +107,17 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		for(; i < ru && it.hasNext(); i++, offT += tCol) {
 			if(it.value() == i) {
 				for(int j = 0; j < nCol; j++)
-					c[offT + _colIndexes[j]] += values[offsetToDefault + j];
+					c[offT + _colIndexes[j]] += values[j];
 				it.next();
 			}
 			else
 				for(int j = 0; j < nCol; j++)
-					c[offT + _colIndexes[j]] += values[j];
+					c[offT + _colIndexes[j]] += values[offsetToDefault + j];
 		}
 
 		for(; i < ru; i++, offT += tCol)
 			for(int j = 0; j < nCol; j++)
-				c[offT + _colIndexes[j]] += values[j];
+				c[offT + _colIndexes[j]] += values[offsetToDefault + j];
 	}
 
 	@Override
@@ -166,14 +168,14 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		for(; i < ru && it.hasNext(); i++, offT++) {
 			if(it.value() == i) {
 				it.next();
-				c[offT] += values[offsetToDefault + colpos];
+				c[offT] += values[colpos];
 			}
 			else
-				c[offT] += values[colpos];
+				c[offT] += values[offsetToDefault + colpos];
 		}
 
 		for(; i < ru; i++, offT++)
-			c[offT] += values[colpos];
+			c[offT] += values[offsetToDefault + colpos];
 	}
 
 	@Override
@@ -186,9 +188,9 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		AIterator it = _indexes.getIterator();
 		it.skipTo(r);
 		if(it.value() == r)
-			return _dict.getValue(_colIndexes.length + ix);
+			return _dict.getValue(ix + c);
 		else
-			return _dict.getValue(ix);
+			return _dict.getValue(_colIndexes.length + c);
 
 	}
 
@@ -208,14 +210,14 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		it.skipTo(rl);
 		for(; rix < ru && it.hasNext(); rix++) {
 			if(it.value() != rix)
-				c[rix] += vals[0];
-			else {
 				c[rix] += vals[1];
+			else {
+				c[rix] += vals[0];
 				it.next();
 			}
 		}
 		for(; rix < ru; rix++) {
-			c[rix] += vals[0];
+			c[rix] += vals[1];
 		}
 	}
 
@@ -230,14 +232,14 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 		for(; rix < ru && it.hasNext(); rix++) {
 			if(it.value() != rix)
-				c[rix] = builtin.execute(c[rix], vals[0]);
-			else {
 				c[rix] = builtin.execute(c[rix], vals[1]);
+			else {
+				c[rix] = builtin.execute(c[rix], vals[0]);
 				it.next();
 			}
 		}
 		for(; rix < ru; rix++) {
-			c[rix] = builtin.execute(c[rix], vals[0]);
+			c[rix] = builtin.execute(c[rix], vals[1]);
 		}
 	}
 
@@ -273,21 +275,25 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		if(row > 0) {
 			int offA = _numRows * row;
 			for(; i < _numRows && it.hasNext(); i++, offA++)
-				if(it.value() == i)
-					vals[1] += a[offA];
-				else
+				if(it.value() == i){
+					it.next();
 					vals[0] += a[offA];
+				}
+				else
+					vals[1] += a[offA];
 			for(; i < _numRows; i++, offA++)
-				vals[0] += a[offA];
+				vals[1] += a[offA];
 		}
 		else {
 			for(; i < _numRows && it.hasNext(); i++)
-				if(it.value() == i)
-					vals[1] += a[i];
-				else
+				if(it.value() == i){
+					it.next();
 					vals[0] += a[i];
+				}
+				else
+					vals[1] += a[i];
 			for(; i < _numRows; i++)
-				vals[0] += a[i];
+				vals[1] += a[i];
 		}
 
 		return vals;
@@ -326,7 +332,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-		return new ColGroupSDCSingle(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, true, left), _indexes,
+		return new ColGroupSDCSingle(_colIndexes, _numRows, applyBinaryRowOp(op, v, true, left), _indexes,
 			getCachedCounts());
 	}
 
@@ -382,11 +388,12 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		for(; i < this._numRows && it.hasNext(); i++) {
 			int col = lhs._data.getIndex(i);
 			if(it.value() == i) {
-				row = 1;
+				row = 0;
 				it.next();
 			}
 			else
-				row = 0;
+				row = 1;
+
 			if(col < lhs.getNumValues())
 				ag.increment(col + row * nCol);
 		}
@@ -420,11 +427,11 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			else
 				col = defL;
 			if(rIt.value() == i) {
-				row = 1;
+				row = 0;
 				rIt.next();
 			}
 			else
-				row = 0;
+				row = 1;
 			ag.increment(col + row * nCol);
 		}
 
@@ -444,11 +451,11 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			col = defL;
 			for(; i < this._numRows && rIt.hasNext(); i++) {
 				if(rIt.value() == i) {
-					row = 1;
+					row = 0;
 					rIt.next();
 				}
 				else
-					row = 0;
+					row = 1;
 				ag.increment(col + row * nCol);
 			}
 		}
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 997c42a..7e68e17 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
@@ -88,15 +88,16 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(_indexes.getSize() * _colIndexes.length + target.getNonZeros());
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
+		final double[] values = getValues();
 		final int offTCorrected = offT - rl;
 		final double[] c = target.getDenseBlockValues();
 
@@ -108,6 +109,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 			for(int j = 0; j < nCol; j++) {
 				c[rc + _colIndexes[j]] += values[j];
 			}
+			it.next();
 		}
 	}
 
@@ -280,7 +282,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		if(isSparseSafeOp)
 			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, applyScalarOp(op), _indexes, getCachedCounts());
 		else {
-			ADictionary aDictionary = swapEntries(applyScalarOp(op, val0, getNumCols()));
+			ADictionary aDictionary = applyScalarOp(op, val0, getNumCols());// swapEntries();
 			// ADictionary aDictionary = applyScalarOp(op, val0, getNumCols());
 			return new ColGroupSDCSingle(_colIndexes, _numRows, aDictionary, _indexes, null);
 		}
@@ -289,22 +291,22 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
 		if(sparseSafe)
-			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, sparseSafe, left),
+			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, applyBinaryRowOp(op, v, sparseSafe, left),
 				_indexes, getCachedCounts());
 		else {
-			ADictionary aDictionary = applyBinaryRowOp(op.fn, v, sparseSafe, left);
+			ADictionary aDictionary = applyBinaryRowOp(op, v, sparseSafe, left);
 			return new ColGroupSDCSingle(_colIndexes, _numRows, aDictionary, _indexes, getCachedCounts());
 		}
 	}
 
-	private ADictionary swapEntries(ADictionary aDictionary) {
-		double[] values = aDictionary.getValues().clone();
-		double[] swap = new double[_colIndexes.length];
-		System.arraycopy(values, 0, swap, 0, _colIndexes.length);
-		System.arraycopy(values, _colIndexes.length, values, 0, _colIndexes.length);
-		System.arraycopy(swap, 0, values, _colIndexes.length, _colIndexes.length);
-		return new Dictionary(values);
-	}
+	// private ADictionary swapEntries(ADictionary aDictionary) {
+	// 	double[] values = aDictionary.getValues().clone();
+	// 	double[] swap = new double[_colIndexes.length];
+	// 	System.arraycopy(values, 0, swap, 0, _colIndexes.length);
+	// 	System.arraycopy(values, _colIndexes.length, values, 0, _colIndexes.length);
+	// 	System.arraycopy(swap, 0, values, _colIndexes.length, _colIndexes.length);
+	// 	return new Dictionary(values);
+	// }
 
 	@Override
 	public void write(DataOutput out) throws IOException {
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 77cf6e2..f04f1a3 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
@@ -105,15 +105,16 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
-		decompressToBlockUnSafe(target, rl, ru, offT, values);
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
 		target.setNonZeros(getNumberNonZeros());
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
+		final double[] values = getValues();
 		final int offTCorrected = offT - rl;
 		final double[] c = target.getDenseBlockValues();
 
@@ -306,10 +307,10 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
 		if(sparseSafe)
-			return new ColGroupSDCZeros(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, sparseSafe, left), _indexes,
+			return new ColGroupSDCZeros(_colIndexes, _numRows, applyBinaryRowOp(op, v, sparseSafe, left), _indexes,
 				_data, getCachedCounts());
 		else
-			return new ColGroupSDC(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, sparseSafe, left), _indexes, _data,
+			return new ColGroupSDC(_colIndexes, _numRows, applyBinaryRowOp(op, v, sparseSafe, left), _indexes, _data,
 				getCachedCounts());
 	}
 
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 c8f9bf6..562e4d8 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
@@ -157,7 +157,7 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
 		double[] c = target.getDenseBlockValues();
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
@@ -180,7 +180,7 @@ public class ColGroupUncompressed extends AColGroup {
 			}
 		}
 		else {
-			values = _data.getDenseBlockValues();
+			double[] values = _data.getDenseBlockValues();
 			offT = offT * tCol;
 			int offS = rl * nCol;
 			for(int row = rl; row < ru; row++, offT += tCol, offS += nCol) {
@@ -197,7 +197,7 @@ public class ColGroupUncompressed extends AColGroup {
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		double[] c = target.getDenseBlockValues();
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
@@ -218,7 +218,7 @@ public class ColGroupUncompressed extends AColGroup {
 			}
 		}
 		else {
-			values = _data.getDenseBlockValues();
+			double[] values = _data.getDenseBlockValues();
 			offT = offT * tCol;
 			int offS = rl * nCol;
 			for(int row = rl; row < ru; row++, offT += tCol, offS += nCol)
@@ -521,7 +521,8 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public void tsmm(double[] result, int numColumns) {
-		MatrixBlock tmp = new MatrixBlock(_colIndexes.length, _colIndexes.length, true);
+		final int tCol = _colIndexes.length;
+		MatrixBlock tmp = new MatrixBlock(tCol, tCol, true);
 		LibMatrixMult.matrixMultTransposeSelf(_data, tmp, true, false);
 		if(tmp.getDenseBlock() == null && tmp.getSparseBlock() == null)
 			return;
@@ -530,9 +531,9 @@ public class ColGroupUncompressed extends AColGroup {
 		}
 		else {
 			double[] tmpV = tmp.getDenseBlockValues();
-			for(int i = 0, offD = 0, offT = 0; i < numColumns; i++, offD += numColumns, offT += _colIndexes.length)
-				for(int j = i; j < numColumns; j++)
-					result[offD + _colIndexes[j]] += tmpV[offT + j];
+			for(int row = 0, offRet = 0, offTmp = 0; row < tCol; row++, offRet += numColumns, offTmp += tCol)
+				for(int col = row; col < tCol; col++)
+					result[offRet + _colIndexes[col]] += tmpV[offTmp + col];
 		}
 
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index a0c127a..42bf407 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -41,10 +41,10 @@ import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
-import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 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.ScalarOperator;
 
 /**
@@ -90,11 +90,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlock(target, rl, ru, offT, getValues());
-	}
-
-	@Override
 	public final int getNumValues() {
 		return _dict.getNumberOfValues(_colIndexes.length);
 	}
@@ -119,12 +114,13 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	@Override
 	public MatrixBlock getValuesAsBlock() {
-		final double[] values = getValues();
-		int vlen = values.length;
-		int rlen = _zeros ? vlen + 1 : vlen;
-		MatrixBlock ret = new MatrixBlock(rlen, 1, false);
-		for(int i = 0; i < vlen; i++)
-			ret.quickSetValue(i, 0, values[i]);
+		_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
+		MatrixBlock ret = ((MatrixBlockDictionary) _dict).getMatrixBlock();
+		if(_zeros) {
+			MatrixBlock tmp = new MatrixBlock();
+			ret.append(new MatrixBlock(1, _colIndexes.length, 0), tmp, false);
+			return tmp;
+		}
 		return ret;
 	}
 
@@ -379,9 +375,9 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param left       Specify which side the operation is executed on.
 	 * @return The new Dictionary with values.
 	 */
-	public ADictionary applyBinaryRowOp(ValueFunction fn, double[] v, boolean sparseSafe, boolean left) {
-		return sparseSafe ? _dict.clone().applyBinaryRowOp(fn, v, sparseSafe, _colIndexes, left) : _dict
-			.applyBinaryRowOp(fn, v, sparseSafe, _colIndexes, left);
+	public ADictionary applyBinaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
+		return sparseSafe ? _dict.clone().applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left) : _dict
+			.applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left);
 	}
 
 	protected void setandExecute(double[] c, boolean square, double val, int rix) {
@@ -446,7 +442,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		sb.append(" Is Lossy: " + _dict.isLossy() + " num Rows: " + getNumRows() + " contain zero row:" + _zeros);
 		sb.append(super.toString());
 		if(_dict != null) {
-			sb.append(String.format("\n%15s%5d ", "Values:", _dict.getValues().length));
+			sb.append(String.format("\n%15s ", "Values: " + _dict.getClass().getSimpleName()));
 			sb.append(_dict.getString(_colIndexes.length));
 		}
 		return sb.toString();
@@ -468,9 +464,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	@Override
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
-
 		out.writeBoolean(_zeros);
-		out.writeBoolean(_dict.isLossy());
+
 		_dict.write(out);
 
 	}
@@ -802,8 +797,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	private void leftMultByColGroupValue(ColGroupValue lhs, MatrixBlock result) {
 		final int nvL = lhs.getNumValues();
 		final int nvR = this.getNumValues();
-		final double[] lhValues = lhs.getValues();
-		final double[] rhValues = this.getValues();
+		// final double[] lhValues = lhs.getValues();
+		// final double[] rhValues = this.getValues();
 		final int lCol = lhs._colIndexes.length;
 		final int rCol = this._colIndexes.length;
 		final double[] resV = result.getDenseBlockValues();
@@ -812,25 +807,20 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		final double threshold = 0.2;
 
 		if(sameIndexStructure(lhs)) {
-			int[] agI = getCounts();
-			for(int i = 0; i < agI.length; i++) {
-				if(i < nvL)
-					for(int l = 0; l < lCol; l++) {
-						final int leftOff = lhs._colIndexes[l] * numCols;
-						final double lhV = lhValues[i * lCol + l] * agI[i];
-						if(lhV != 0)
-							for(int r = 0; r < rCol; r++) {
-								final double rhV = rhValues[i * rCol + r];
-								final double va = lhV * rhV;
-								resV[leftOff + this._colIndexes[r]] += va;
-							}
-					}
+			if(this._dict == lhs._dict) {
+				tsmmDictionaryWithScaling(_dict, getCounts(), lhs._colIndexes, this._colIndexes, resV, numCols);
 			}
+			else
+				matrixMultDictionariesAndOutputToColIndexesWithScaling(lhs._dict, this._dict, lhs._colIndexes,
+					this._colIndexes, resV, numCols, getCounts());
+
 		}
 		else if(lhs instanceof ColGroupConst || this instanceof ColGroupConst) {
-			double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
-			double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
-			vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, resV, numCols);
+			ADictionary r = this instanceof ColGroupConst ? this._dict : new Dictionary(
+				this._dict.colSum(getCounts(), rCol));
+			ADictionary l = lhs instanceof ColGroupConst ? lhs._dict : new Dictionary(
+				lhs._dict.colSum(lhs.getCounts(), lCol));
+			matrixMultDictionariesAndOutputToColIndexes(l, r, lhs._colIndexes, this._colIndexes, resV, numCols);
 		}
 		else {
 			int[] countsRight = getCounts();
@@ -846,53 +836,90 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				double[] mct = this._dict.getMostCommonTuple(this.getCounts(), rCol);
 				double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
 				if(mct != null)
-					vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, resV, numCols);
+					outerProduct(lhsSum, lhs._colIndexes, mct, this._colIndexes, resV, numCols);
 
 				ColGroupValue thisM = (mct != null) ? (ColGroupValue) this
 					.copyAndSet(this._dict.subtractTuple(mct)) : this;
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(thisM, true);
-				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
-					this._colIndexes, resV, numCols);
+				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes, resV,
+					numCols);
 			}
 			else if(skipLeft > threshold && !(lhs instanceof ColGroupDDC)) {
 				double[] mct = lhs._dict.getMostCommonTuple(lhs.getCounts(), lCol);
 				double[] thisColSum = this._dict.colSum(getCounts(), rCol);
 				if(mct != null)
-					vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, resV, numCols);
+					outerProduct(mct, lhs._colIndexes, thisColSum, this._colIndexes, resV, numCols);
 
 				ColGroupValue lhsM = (mct != null) ? (ColGroupValue) lhs.copyAndSet(lhs._dict.subtractTuple(mct)) : lhs;
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhsM, true);
-				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
-					this._colIndexes, resV, numCols);
+				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes, resV,
+					numCols);
 			}
 			else if(nvR * rCol < nvL * lCol) {
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(this, false);
-				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
-					this._colIndexes, resV, numCols);
+				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes, resV,
+					numCols);
 			}
 			else {
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhs, false);
-				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
-					this._colIndexes, resV, numCols);
+				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes, resV,
+					numCols);
 			}
 		}
 	}
 
 	@Override
 	public void tsmm(double[] result, int numColumns) {
-		int[] counts = getCounts();
-		double[] values = getValues();
-		int[] columns = getColIndices();
+
+		// final int[] counts = getCounts();
+
+		// _dict = _dict.getAsMatrixBlockDictionary(_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);
+		// 	else
+		// 		tsmmDense(result, numColumns, mb.getDenseBlockValues(), counts);
+		// }
+		// else
+		// 	tsmmDense(result, numColumns, getValues(), counts);
+
+	}
+
+	private void tsmmDense(double[] result, int numColumns, double[] values, int[] counts) {
 		if(values == null)
 			return;
-		for(int i = 0; i < columns.length; i++) {
-			final int y = columns[i] * numColumns;
-			for(int j = i; j < columns.length; j++) {
-				final int x = columns[j];
-				for(int h = 0; h < values.length / columns.length; h++) {
-					double a = values[h * columns.length + i];
-					double b = values[h * columns.length + j];
-					result[x + y] += a * b * counts[h];
+		final int nCol = _colIndexes.length;
+		final int nRow = values.length / _colIndexes.length;
+		for(int k = 0; k < nRow; k++) {
+			final int offTmp = nCol * k;
+			final int scale = counts[k];
+			for(int i = 0; i < nCol; i++) {
+				final int offRet = numColumns * _colIndexes[i];
+				final double v = values[offTmp + i] * scale;
+				if(v != 0)
+					for(int j = i; j < nCol; j++)
+						result[offRet + _colIndexes[j]] += v * values[offTmp + j];
+			}
+		}
+	}
+
+	private void tsmmSparse(double[] result, int numColumns, SparseBlock sb, int[] counts) {
+		for(int row = 0; row < sb.numRows(); row++) {
+			if(sb.isEmpty(row))
+				continue;
+			final int apos = sb.pos(row);
+			final int alen = sb.size(row);
+			final int[] aix = sb.indexes(row);
+			final double[] avals = sb.values(row);
+			for(int i = apos; i < apos + alen; i++) {
+				final int offRet = _colIndexes[aix[i]] * numColumns;
+				final double val = avals[i] * counts[row];
+				for(int j = i; j < apos + alen; j++) {
+					result[offRet + _colIndexes[aix[j]]] += val * avals[j];
 				}
 			}
 		}
@@ -909,19 +936,71 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return _dict.getNumberNonZeros(counts, _colIndexes.length);
 	}
 
-	private static void vectorVectorMultiply(final double[] left, final int[] leftRows, final double[] right,
+	private static void matrixMultDictionariesAndOutputToColIndexesWithScaling(final ADictionary left,
+		final ADictionary right, final int[] leftRows, final int[] rightColumns, final double[] result,
+		final int outCols, final int[] counts) {
+		final boolean modifyRight = right.getInMemorySize() > left.getInMemorySize();
+		ADictionary rightM = modifyRight ? right.scaleTuples(counts, rightColumns.length) : right;
+		ADictionary leftM = modifyRight ? left : left.scaleTuples(counts, leftRows.length);
+
+		matrixMultDictionariesAndOutputToColIndexes(leftM, rightM, leftRows, rightColumns, result, outCols);
+
+	}
+
+	private static void tsmmDictionaryWithScaling(final ADictionary dict, final int[] counts, final int[] rows,
+		final int[] cols, final double[] res, final int outCols) {
+
+		if(dict instanceof MatrixBlockDictionary) {
+			MatrixBlockDictionary mbd = (MatrixBlockDictionary) dict;
+			MatrixBlock mb = mbd.getMatrixBlock();
+			if(mb.isEmpty())
+				return;
+			else if(mb.isInSparseFormat()) {
+				SparseBlock sb = mb.getSparseBlock();
+				for(int row = 0; row < sb.numRows(); row++) {
+					if(sb.isEmpty(row))
+						continue;
+					final int apos = sb.pos(row);
+					final int alen = sb.size(row);
+					final int[] aix = sb.indexes(row);
+					final double[] avals = sb.values(row);
+					for(int i = apos; i < apos + alen; i++) {
+						final int offRet = rows[aix[i]] * outCols;
+						final double val = avals[i] * counts[row];
+						for(int j = i; j < apos + alen; j++) {
+							res[offRet + cols[aix[j]]] += val * avals[j];
+						}
+					}
+				}
+			}
+			else {
+				throw new NotImplementedException();
+			}
+		}
+		else {
+			double[] values = dict.getValues();
+			for(int row = 0; row < rows.length; row++) {
+				final int offTmp = cols.length * row;
+				final int offRet = outCols * rows[row];
+				for(int col = 0; col < cols.length; col++) {
+					final double v = values[offTmp + col] * counts[row];
+					if(v != 0)
+						for(int j = col; j < cols.length; j++)
+							res[offRet + cols[col]] += v * values[offTmp + j];
+				}
+			}
+		}
+	}
+
+	private static void outerProduct(final double[] left, final int[] leftRows, final double[] right,
 		final int[] rightColumns, final double[] result, final int outCols) {
-		if(left.length != leftRows.length) {
-			// LOG.error(Arrays.toString(left));
-			// LOG.error(Arrays.toString(right));
-			// LOG.error(Arrays.toString(leftRows));
-			// LOG.error(Arrays.toString(rightColumns));
+		if(left.length != leftRows.length)
 			throw new DMLCompressionException(
 				"Error left length " + left.length + " not equal columns length" + leftRows.length);
-		}
+
 		if(right.length != rightColumns.length)
 			throw new DMLCompressionException(
-				"Error right not equal length " + right.length + "  " + rightColumns.length);
+				"Error right not equal length " + right.length + " " + rightColumns.length);
 		for(int row = 0; row < leftRows.length; row++) {
 			final int outputRowOffset = leftRows[row] * outCols;
 			final double vLeft = left[row];
@@ -937,64 +1016,137 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * 
 	 * making the multiplication a: t(left) %*% right
 	 * 
-	 * @param left      The left side matrix, transposed linearized row major
-	 * @param right     The right hand side linearized row major
+	 * @param left      The left side dictionary
+	 * @param right     The right side dictionary
 	 * @param rowsLeft  The number of rows and the row indexes on the left hand side
 	 * @param colsRight The number of columns and the column indexes on the right hand side
 	 * @param result    The result matrix to put the results into, linearized row major
 	 * @param outCols   The output columns count, to know how much to offset into with results.
 	 */
-	private static void matrixMultDictionariesAndOutputToColIndexes(double[] left, double[] right, int[] rowsLeft,
+	private static void matrixMultDictionariesAndOutputToColIndexes(ADictionary left, ADictionary right, int[] rowsLeft,
 		int[] colsRight, double[] result, int outCols) {
 
 		try {
-			final int rows = left.length / rowsLeft.length;
-			if(rows != right.length / colsRight.length)
-				throw new DMLCompressionException(
-					"Not equal number of rows: " + rows + " " + right.length / colsRight.length);
-			for(int k = 0; k < rows; k++) {
-				final int offL = k * rowsLeft.length;
-				final int offR = k * colsRight.length;
-				// final int offL = k * colsRight.length;
-				// final int offR = k * rowsLeft.length;
-				// if(offR < right.length && offL < left.length)
-				for(int i = 0; i < rowsLeft.length; i++) {
-					final int offOut = rowsLeft[i] * outCols;
-					final double vl = left[offL + i];
-					if(vl != 0)
-						for(int j = 0; j < colsRight.length; j++) {
-							final double vr = right[offR + j];
-							result[offOut + colsRight[j]] += vl * vr;
-						}
+			double[] leftV = null;
+			double[] rightV = null;
+
+			if(left instanceof MatrixBlockDictionary) {
+				MatrixBlockDictionary leftD = left.getAsMatrixBlockDictionary(rowsLeft.length);
+				MatrixBlock leftMB = leftD.getMatrixBlock();
+				if(leftMB.isEmpty())
+					return;
+				else if(right instanceof MatrixBlockDictionary) {
+					MatrixBlockDictionary rightD = right.getAsMatrixBlockDictionary(colsRight.length);
+					MatrixBlock rightMB = rightD.getMatrixBlock();
+					if(rightMB.isEmpty())
+						return;
+					else if(rightMB.isInSparseFormat() && leftMB.isInSparseFormat()) {
+						throw new NotImplementedException("Not Supported sparse sparse dictionary multiplication");
+					}
+				}
+				else if(leftMB.isInSparseFormat()) {
+					matrixMultDictionariesAndOutputToColIndecesSparseDense(leftMB.getSparseBlock(), right.getValues(),
+						rowsLeft, colsRight, result, outCols);
+					return;
 				}
 			}
+			else {
+				leftV = left.getValues();
+			}
+
+			if(right instanceof MatrixBlockDictionary) {
+				MatrixBlockDictionary rightD = right.getAsMatrixBlockDictionary(colsRight.length);
+				MatrixBlock rightMB = rightD.getMatrixBlock();
+
+				if(rightMB.isEmpty())
+					return;
+				else if(rightMB.isInSparseFormat()) {
+					matrixMultDictionariesAndOutputToColIndecesDenseSparse(leftV, rightMB.getSparseBlock(), rowsLeft,
+						colsRight, result, outCols);
+					return;
+				}
+			}
+			else {
+				rightV = right.getValues();
+			}
+
+			if(leftV != null && rightV != null) {
+				// default if there was not sparse found;
+				LOG.warn("Inefficient forced dense values");
+				matrixMultDictionariesAndOutputToColIndexesDenseDense(leftV, rightV, rowsLeft, colsRight, result,
+					outCols);
+			}
+
 		}
 		catch(Exception e) {
-
 			if(logMM) {
-				StringBuilder sb = new StringBuilder();
-				sb.append("\nLeft (transposed):\n");
-				for(int i = 0; i < rowsLeft.length; i++) {
-					for(int j = i * rowsLeft.length; j < (i + 1) * rowsLeft.length; j++)
-						sb.append(left[j] + ", ");
-					sb.append("\n");
-				}
-				LOG.error(sb);
-
-				sb = new StringBuilder();
-				sb.append("\nRight:\n");
-				for(int i = 0; i < colsRight.length; i++) {
-					for(int j = i * colsRight.length; j < (i + 1) * colsRight.length; j++)
-						sb.append(right[j] + ", ");
-					sb.append("\n");
-				}
-				LOG.error(sb);
+				LOG.error("\nLeft (transposed):\n" + left + "\nRight:\n" + right);
 				logMM = false;
 			}
 			throw new DMLCompressionException("MM of pre aggregated colGroups failed", e);
 		}
 	}
 
+	private static void matrixMultDictionariesAndOutputToColIndexesDenseDense(double[] left, double[] right,
+		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		final int commonDim = Math.min(left.length / rowsLeft.length, right.length / colsRight.length);
+		for(int k = 0; k < commonDim; k++) {
+			final int offL = k * rowsLeft.length;
+			final int offR = k * colsRight.length;
+			for(int i = 0; i < rowsLeft.length; i++) {
+				final int offOut = rowsLeft[i] * outCols;
+				final double vl = left[offL + i];
+				if(vl != 0)
+					for(int j = 0; j < colsRight.length; j++) {
+						final double vr = right[offR + j];
+						result[offOut + colsRight[j]] += vl * vr;
+					}
+			}
+		}
+	}
+
+	private static void matrixMultDictionariesAndOutputToColIndecesSparseDense(SparseBlock left, double[] right,
+		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		final int commonDim = Math.min(left.numRows(), right.length / colsRight.length);
+		for(int i = 0; i < commonDim; i++) {
+			if(left.isEmpty(i))
+				continue;
+			final int apos = left.pos(i);
+			final int alen = left.size(i) + apos;
+			final int[] aix = left.indexes(i);
+			final double[] leftVals = left.values(i);
+			final int offRight = i * colsRight.length;
+			for(int k = apos; k < alen; k++) {
+				final int offOut = rowsLeft[aix[k]] * outCols;
+				final double v = leftVals[k];
+				for(int j = 0; j < colsRight.length; j++)
+					result[offOut + colsRight[j]] += v * right[offRight + j];
+			}
+		}
+	}
+
+	private static void matrixMultDictionariesAndOutputToColIndecesDenseSparse(double[] left, SparseBlock right,
+		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		final int commonDim = Math.min(left.length / rowsLeft.length, right.numRows());
+		for(int i = 0; i < commonDim; i++) {
+			if(right.isEmpty(i))
+				continue;
+			final int apos = right.pos(i);
+			final int alen = right.size(i) + apos;
+			final int[] aix = right.indexes(i);
+			final double[] rightVals = right.values(i);
+			final int offLeft = i * rowsLeft.length;
+			for(int j = 0; j < rowsLeft.length; j++) {
+				final int offOut = rowsLeft[j] * outCols;
+				final double v = left[offLeft + j];
+				if(v != 0)
+					for(int k = apos; k < alen; k++) {
+						result[offOut + colsRight[aix[k]]] += v * rightVals[k];
+					}
+			}
+		}
+	}
+
 	@Override
 	public boolean isDense() {
 		return !_zeros;
@@ -1009,7 +1161,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param rl     The row to start the matrix multiplication from
 	 * @param ru     The row to stop the matrix multiplication at.
 	 */
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, double[] values, int rl, int ru) {
+	@Override
+	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
 		final int numVals = getNumValues();
 		if(!(_dict instanceof MatrixBlockDictionary))
 			_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
@@ -1026,37 +1179,39 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				preAgg.setNonZeros(numVals);
 				// LOG.error("PreAgg Sparsity " + preAgg.getSparsity() + " nnz " + preAgg.getNonZeros());
 				LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
-				addToResult(tmpRes, result, i);
+				addVectorToResult(tmpRes, result, i);
 				tmpRes.reset();
 			}
 		}
 	}
 
-	private void addToResult(MatrixBlock tmp, MatrixBlock result, int row) {
+	private void addVectorToResult(MatrixBlock tmp, MatrixBlock result, int row) {
 		if(tmp.isEmpty())
 			return;
-		else if(tmp.isInSparseFormat()) {
-			throw new NotImplementedException();
+		final double[] retV = result.getDenseBlockValues();
+		final int nColRet = result.getNumColumns();
+		final int offR = row * nColRet;
+		if(tmp.isInSparseFormat()) {
+			final SparseBlock sb = tmp.getSparseBlock();
+			if(sb.isEmpty(0))
+				return;
+			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 = apos; i < apos + alen; i++)
+				retV[offR + _colIndexes[aix[i]]] += avals[i];
+
 		}
 		else {
 			final double[] tmpV = tmp.getDenseBlockValues();
-			final double[] retV = result.getDenseBlockValues();
-			final int nColRet = result.getNumColumns();
 			// final int nColTmp = tmp.getNumColumns();
-			final int offR = row * nColRet;
 			// for(int row = rl, offT = 0, offR = rl * nColRet; row < ru; row++, offT += nColTmp, offR += nColRet) {
-			for(int col = 0; col < _colIndexes.length; col++) {
-				final int colOffset = _colIndexes[col];
-				retV[offR + colOffset] += tmpV[col];
-			}
+			for(int col = 0; col < _colIndexes.length; col++)
+				retV[offR + _colIndexes[col]] += tmpV[col];
+
 			// }
 		}
-
-	}
-
-	@Override
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
-		leftMultByMatrix(matrix, result, getValues(), rl, ru);
 	}
 
 	public AColGroup rightMultByMatrix(MatrixBlock right) {
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 e5d2a15..e1d4152 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
@@ -25,7 +25,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.functionobjects.Builtin;
-import org.apache.sysds.runtime.functionobjects.ValueFunction;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 
 /**
@@ -113,16 +113,16 @@ public abstract class ADictionary {
 	 */
 	public abstract ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols);
 
-	public ADictionary applyBinaryRowOp(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes,
+	public ADictionary applyBinaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes,
 		boolean left) {
-		return (left) ? applyBinaryRowOpLeft(fn, v, sparseSafe, colIndexes) : applyBinaryRowOpRight(fn, v, sparseSafe,
+		return (left) ? applyBinaryRowOpLeft(op, v, sparseSafe, colIndexes) : applyBinaryRowOpRight(op, v, sparseSafe,
 			colIndexes);
 	}
 
-	public abstract ADictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe,
+	public abstract ADictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe,
 		int[] colIndexes);
 
-	public abstract ADictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe,
+	public abstract ADictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe,
 		int[] colIndexes);
 
 	/**
@@ -148,18 +148,14 @@ public abstract class ADictionary {
 	 * @param out the output sink to write the dictionary to.
 	 * @throws IOException if the sink fails.
 	 */
-	public void write(DataOutput out) throws IOException {
-		out.writeBoolean(isLossy());
-	}
+	public abstract void write(DataOutput out) throws IOException;
 
 	/**
 	 * Calculate the space consumption if the dictionary is stored on disk.
 	 * 
 	 * @return the long count of bytes to store the dictionary.
 	 */
-	public long getExactSizeOnDisk() {
-		return 1;
-	}
+	public abstract long getExactSizeOnDisk();
 
 	/**
 	 * Specify if the Dictionary is lossy.
@@ -275,7 +271,19 @@ public abstract class ADictionary {
 	 * @param counts The counts of the individual tuples contained, managed by the column group.
 	 * @return a new double array containing the most common value
 	 */
-	public abstract double[] getMostCommonTuple(int[] counts, int nCol);
+	public double[] getMostCommonTuple(int[] counts, int nCol) {
+		int maxIndex = 0;
+		int maxCount = 0;
+		for(int i = 0; i < counts.length; i++) {
+			if(counts[i] > maxCount) {
+				maxCount = counts[i];
+				maxIndex = i;
+			}
+		}
+		return getTuple(maxIndex, nCol);
+	}
+
+	public abstract double[] getTuple(int index, int nCol);
 
 	/**
 	 * Allocate a new dictionary where the tuple given is subtracted from all tuples in the previous dictionary.
@@ -293,4 +301,13 @@ public abstract class ADictionary {
 	 * @return A Dictionary containing a MatrixBlock.
 	 */
 	public abstract MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol);
+
+	/**
+	 * Scale all tuples contained in the dictionary by the scaling factor given in the int list.
+	 * 
+	 * @param scaling The ammout to multiply the given tuples with
+	 * @param nCol    The number of columns contained in this column group.
+	 * @return A New dictionary (since we don't want to modify the underlying dictionary)
+	 */
+	public abstract ADictionary scaleTuples(int[] scaling, int nCol);
 }
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 595c250..770557b 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
@@ -30,6 +30,7 @@ import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.utils.MemoryEstimates;
 
@@ -119,15 +120,16 @@ public class Dictionary extends ADictionary {
 	public Dictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
 		// allocate new array just once because we need to add the newVal.
 		double[] values = new double[_values.length + numCols];
-		for(int i = 0; i < _values.length; i++) {
+		for(int i = 0; i < _values.length; i++)
 			values[i] = op.executeScalar(_values[i]);
-		}
+		
 		Arrays.fill(values, _values.length, _values.length + numCols, newVal);
 		return new Dictionary(values);
 	}
 
 	@Override
-	public Dictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+	public Dictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+		ValueFunction fn = op.fn;
 		final int len = size();
 		final int lenV = colIndexes.length;
 		if(sparseSafe) {
@@ -150,7 +152,8 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public Dictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+	public Dictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+		ValueFunction fn = op.fn;
 		final int len = size();
 		final int lenV = colIndexes.length;
 		if(sparseSafe) {
@@ -194,6 +197,7 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public void write(DataOutput out) throws IOException {
+		out.writeByte(DictionaryFactory.Type.FP64_DICT.ordinal());
 		out.writeInt(size());
 		for(int i = 0; i < size(); i++)
 			out.writeDouble(_values[i]);
@@ -201,7 +205,7 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public long getExactSizeOnDisk() {
-		return 4 + 8 * size();
+		return 1 + 4 + 8 * size();
 	}
 
 	public int size() {
@@ -446,18 +450,11 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public double[] getMostCommonTuple(int[] counts, int nCol) {
-		int maxIndex = 0;
-		int maxCount = 0;
-		for(int i = 0; i < counts.length; i++) {
-			if(counts[i] > maxCount) {
-				maxCount = counts[i];
-				maxIndex = i;
-			}
-		}
+	public double[] getTuple(int index, int nCol) {
+
 		final double[] tuple = new double[nCol];
 		boolean allZero = true;
-		for(int i = maxIndex * nCol, off = 0; i < (maxIndex + 1) * nCol && i < _values.length; i++, off++) {
+		for(int i = index * nCol, off = 0; i < (index + 1) * nCol && i < _values.length; i++, off++) {
 			final double v = _values[i];
 			if(v != 0) {
 				tuple[off] = _values[i];
@@ -493,6 +490,20 @@ public class Dictionary extends ADictionary {
 		for(int k = 0; k < vlen; k++)
 			for(int j = 0, valOff = k * ncol; j < ncol; j++)
 				c[colIndexes[j]] = fn.execute(c[colIndexes[j]], getValue(valOff + j));
-		
+
+	}
+
+	@Override
+	public ADictionary scaleTuples(int[] scaling, int nCol) {
+		final double[] scaledValues = new double[_values.length];
+		int off = 0;
+		for(int tuple = 0; tuple < _values.length / nCol; tuple++) {
+			final int scale = scaling[tuple];
+			for(int v = 0; v < nCol; v++) {
+				scaledValues[off] = _values[off] * scale;
+				off++;
+			}
+		}
+		return new Dictionary(scaledValues);
 	}
 }
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 bab32b0..1fd47d8 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
@@ -25,6 +25,7 @@ import java.io.IOException;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.Bitmap;
 import org.apache.sysds.runtime.compress.utils.BitmapLossy;
@@ -36,26 +37,23 @@ public class DictionaryFactory {
 
 	protected static final Log LOG = LogFactory.getLog(DictionaryFactory.class.getName());
 
+	public enum Type {
+		FP64_DICT, MATRIX_BLOCK_DICT, INT8_DICT
+	}
+
 	public static ADictionary read(DataInput in) throws IOException {
-		boolean lossy = in.readBoolean();
-		if(lossy) {
-
-			double scale = in.readDouble();
-			int numVals = in.readInt();
-			// read distinct values
-			byte[] values = numVals == 0 ? null : new byte[numVals];
-			for(int i = 0; i < numVals; i++)
-				values[i] = in.readByte();
-			return new QDictionary(values, scale);
-		}
-		else {
-			int numVals = in.readInt();
-			// read distinct values
-			double[] values = new double[numVals];
-			for(int i = 0; i < numVals; i++)
-				values[i] = in.readDouble();
-			return new Dictionary(values);
+		Type type = Type.values()[in.readByte()];
+		switch(type) {
+			case FP64_DICT:
+				return Dictionary.read(in);
+			case MATRIX_BLOCK_DICT:
+				return MatrixBlockDictionary.read(in);
+			case INT8_DICT:
+				return QDictionary.read(in);
+			default:
+				throw new DMLCompressionException("Unsupported type of dictionary : " + type);
 		}
+
 	}
 
 	public static long getInMemorySize(int nrValues, int nrColumns, double tupleSparsity, boolean lossy) {
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 322e56b..aae66e6 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
@@ -1,12 +1,18 @@
 package org.apache.sysds.runtime.compress.colgroup.dictionary;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.data.DenseBlockFP64;
 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.ValueFunction;
+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.matrix.operators.ScalarOperator;
 
 public class MatrixBlockDictionary extends ADictionary {
@@ -24,7 +30,8 @@ public class MatrixBlockDictionary extends ADictionary {
     @Override
     public double[] getValues() {
         LOG.warn("Inefficient force dense format.");
-        _data.sparseToDense();
+        if(_data.isInSparseFormat())
+            _data.sparseToDense();
         return _data.getDenseBlockValues();
     }
 
@@ -39,15 +46,13 @@ public class MatrixBlockDictionary extends ADictionary {
     public int hasZeroTuple(int nCol) {
         if(_data.isInSparseFormat()) {
             SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(sb.isEmpty(i)) {
+            for(int i = 0; i < _data.getNumRows(); i++)
+                if(sb.isEmpty(i))
                     return i;
-                }
-            }
         }
-        else {
+        else
             throw new NotImplementedException();
-        }
+
         return -1;
     }
 
@@ -163,18 +168,56 @@ public class MatrixBlockDictionary extends ADictionary {
     @Override
     public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
         MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
-        MatrixBlock res2 = res.append(new MatrixBlock(1, 1, newVal), new MatrixBlock());
-        return new MatrixBlockDictionary(res2);
+        final int lastRow = res.getNumRows();
+        MatrixBlock res2 = new MatrixBlock(lastRow + 1, res.getNumColumns(), true);
+        if(res.isEmpty()) {
+            for(int i = 0; i < numCols; i++)
+                res2.appendValue(lastRow, i, newVal);
+            return new MatrixBlockDictionary(res2);
+        }
+        else {
+            res.append(new MatrixBlock(1, numCols, newVal), res2, false);
+            return new MatrixBlockDictionary(res2);
+        }
     }
 
     @Override
-    public ADictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
-        throw new NotImplementedException();
+    public ADictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+        MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
+        for(int i = 0; i < colIndexes.length; i++)
+            rowVector.quickSetValue(0, i, v[colIndexes[i]]);
+        MatrixBlock res = new MatrixBlock();
+        if(sparseSafe) {
+            rowVector.binaryOperations(op, _data, res);
+        }
+        else {
+            if(!_data.isInSparseFormat())
+                LOG.warn("Inefficient binary row op allocating Matrix multiple times");
+            MatrixBlock tmp = new MatrixBlock();
+            tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
+            rowVector.binaryOperations(op, tmp, res);
+
+        }
+        return new MatrixBlockDictionary(res);
     }
 
     @Override
-    public ADictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
-        throw new NotImplementedException();
+    public ADictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+        MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
+        for(int i = 0; i < colIndexes.length; i++)
+            rowVector.quickSetValue(0, i, v[colIndexes[i]]);
+        MatrixBlock res = new MatrixBlock();
+        if(sparseSafe) {
+            _data.binaryOperations(op, rowVector, res);
+        }
+        else {
+            if(!_data.isInSparseFormat())
+                LOG.warn("Inefficient binary row op allocating Matrix multiple times");
+            MatrixBlock tmp = new MatrixBlock();
+            tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
+            tmp.binaryOperations(op, rowVector, res);
+        }
+        return new MatrixBlockDictionary(res);
     }
 
     @Override
@@ -382,7 +425,8 @@ public class MatrixBlockDictionary extends ADictionary {
 
     @Override
     public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
-        throw new NotImplementedException();
+        MatrixBlock retBlock = _data.slice(0, _data.getNumRows() - 1, idxStart, idxEnd - 1);
+        return new MatrixBlockDictionary(retBlock);
     }
 
     @Override
@@ -399,11 +443,12 @@ public class MatrixBlockDictionary extends ADictionary {
     public long getNumberNonZeros(int[] counts, int nCol) {
         if(_data.isEmpty())
             return 0;
+
         long nnz = 0;
         if(_data.isInSparseFormat()) {
             SparseBlock sb = _data.getSparseBlock();
             for(int i = 0; i < _data.getNumRows(); i++)
-                if(sb.isEmpty(i))
+                if(!sb.isEmpty(i))
                     nnz += sb.size(i) * counts[i];
 
         }
@@ -430,17 +475,67 @@ public class MatrixBlockDictionary extends ADictionary {
 
     @Override
     public void addToEntry(Dictionary d, int fr, int to, int nCol) {
-        throw new NotImplementedException();
+        double[] v = d.getValues();
+        if(_data.isEmpty())
+            return;
+        else if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            if(sb.isEmpty(fr))
+                return;
+            final int apos = sb.pos(fr);
+            final int alen = sb.size(fr) + apos;
+            final int[] aix = sb.indexes(fr);
+            final double[] avals = sb.values(fr);
+            final int offsetTo = nCol * to;
+            for(int j = apos; j < alen; j++) {
+                v[offsetTo + aix[j]] += avals[j];
+            }
+        }
+        else {
+            final int sf = nCol * fr; // start from
+            final int ef = sf + nCol; // end from
+            final double[] thisV = _data.getDenseBlockValues();
+            for(int i = sf, j = nCol * to; i < ef; i++, j++) {
+                v[j] += thisV[i];
+            }
+        }
     }
 
     @Override
-    public double[] getMostCommonTuple(int[] counts, int nCol) {
-        throw new NotImplementedException();
+    public double[] getTuple(int index, int nCol) {
+        if(_data.isEmpty() || index >= _data.getNumRows())
+            return null;
+        else if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            if(sb.isEmpty(index))
+                return null;
+            double[] tuple = new double[nCol];
+            final int apos = sb.pos(index);
+            final int alen = sb.size(index) + apos;
+            final int[] aix = sb.indexes(index);
+            final double[] avals = sb.values(index);
+            for(int j = apos; j < alen; j++) {
+                tuple[aix[j]] = avals[j];
+            }
+            return tuple;
+        }
+        else {
+            double[] tuple = new double[nCol];
+            double[] values = _data.getDenseBlockValues();
+            int offset = index * nCol;
+            for(int i = 0; i < nCol; i++, offset++)
+                tuple[i] = values[offset];
+            return tuple;
+        }
     }
 
     @Override
     public ADictionary subtractTuple(double[] tuple) {
-        throw new NotImplementedException();
+        DenseBlockFP64 b = new DenseBlockFP64(new int[] {1, tuple.length}, tuple);
+        MatrixBlock rowVector = new MatrixBlock(1, tuple.length, b);
+        MatrixBlock res = new MatrixBlock(_data.getNumColumns(), _data.getNumRows(), _data.isInSparseFormat());
+        _data.binaryOperations(new BinaryOperator(Minus.getMinusFnObject()), rowVector, res);
+        return new MatrixBlockDictionary(res);
     }
 
     @Override
@@ -453,4 +548,64 @@ public class MatrixBlockDictionary extends ADictionary {
     public String toString() {
         return "MatrixBlock Dictionary :" + _data.toString();
     }
+
+    @Override
+    public ADictionary scaleTuples(int[] scaling, int nCol) {
+        if(_data.isEmpty()) {
+            throw new NotImplementedException("could return null here? or empty DictionaryMatrixBlock...");
+        }
+        else if(_data.isInSparseFormat()) {
+            MatrixBlock retBlock = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), true);
+            retBlock.allocateSparseRowsBlock(true);
+            SparseBlock sbRet = retBlock.getSparseBlock();
+            SparseBlock sbThis = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sbThis.isEmpty(i)) {
+                    sbRet.set(i, sbThis.get(i), true);
+
+                    final int count = scaling[i];
+                    final int apos = sbRet.pos(i);
+                    final int alen = sbRet.size(i) + apos;
+                    final double[] avals = sbRet.values(i);
+                    for(int j = apos; j < alen; j++)
+                        avals[j] = count * avals[j];
+                }
+            }
+            retBlock.setNonZeros(_data.getNonZeros());
+            return new MatrixBlockDictionary(retBlock);
+        }
+        else {
+            final double[] _values = _data.getDenseBlockValues();
+            final double[] scaledValues = new double[_values.length];
+            int off = 0;
+            for(int tuple = 0; tuple < _values.length / nCol; tuple++) {
+                final int scale = scaling[tuple];
+                for(int v = 0; v < nCol; v++) {
+                    scaledValues[off] = _values[off] * scale;
+                    off++;
+                }
+            }
+            DenseBlockFP64 db = new DenseBlockFP64(new int[] {_data.getNumRows(), _data.getNumColumns()}, scaledValues);
+            MatrixBlock retBlock = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), db);
+            retBlock.setNonZeros(_data.getNonZeros());
+            return new MatrixBlockDictionary(retBlock);
+        }
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeByte(DictionaryFactory.Type.MATRIX_BLOCK_DICT.ordinal());
+        _data.write(out);
+    }
+
+    public static MatrixBlockDictionary read(DataInput in) throws IOException {
+        MatrixBlock ret = new MatrixBlock();
+        ret.readFields(in);
+        return new MatrixBlockDictionary(ret);
+    }
+
+    @Override
+    public long getExactSizeOnDisk() {
+        return 1 + _data.getExactSizeOnDisk();
+    }
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index 70836ca..4c3f9f9 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -19,6 +19,7 @@
 
 package org.apache.sysds.runtime.compress.colgroup.dictionary;
 
+import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
@@ -30,6 +31,7 @@ import org.apache.sysds.runtime.functionobjects.Divide;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.utils.MemoryEstimates;
 
@@ -193,8 +195,8 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public QDictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
-
+	public QDictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+		ValueFunction fn = op.fn;
 		if(_values == null) {
 			if(sparseSafe) {
 				return new QDictionary(null, 1);
@@ -234,7 +236,7 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public QDictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+	public QDictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
 		throw new NotImplementedException("Not Implemented yet");
 	}
 
@@ -256,16 +258,26 @@ public class QDictionary extends ADictionary {
 
 	@Override
 	public void write(DataOutput out) throws IOException {
-		super.write(out);
+		out.writeByte(DictionaryFactory.Type.INT8_DICT.ordinal());
 		out.writeDouble(_scale);
 		out.writeInt(_values.length);
 		for(int i = 0; i < _values.length; i++)
 			out.writeByte(_values[i]);
 	}
 
+	public static QDictionary read(DataInput in) throws IOException {
+		double scale = in.readDouble();
+		int numVals = in.readInt();
+		byte[] values = new byte[numVals];
+		for(int i = 0; i < numVals; i++) {
+			values[i] = in.readByte();
+		}
+		return new QDictionary(values, scale);
+	}
+
 	@Override
 	public long getExactSizeOnDisk() {
-		return 8 + 4 + size();
+		return 1 + 8 + 4 + size();
 	}
 
 	@Override
@@ -491,7 +503,7 @@ public class QDictionary extends ADictionary {
 	}
 
 	@Override
-	public double[] getMostCommonTuple(int[] counts, int nCol) {
+	public double[] getTuple(int index, int nCol) {
 		return null;
 	}
 
@@ -507,6 +519,11 @@ public class QDictionary extends ADictionary {
 
 	@Override
 	public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
-		throw new NotImplementedException();	
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public ADictionary scaleTuples(int[] scaling, int nCol) {
+		throw new NotImplementedException();
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
index 64b0bdd..9a52348 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
@@ -196,6 +196,7 @@ public class BitmapEncoder {
 
 		while((cellVals = rowReader.nextRow()) != null) {
 			if(cellVals.getData() != null) {
+				cellVals.resetHash();
 				IntArrayList lstPtr = distinctVals.get(cellVals);
 				if(lstPtr == null) {
 					// create new objects only on demand
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
index d454a7f..48c2810 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibBinaryCellOp.java
@@ -177,7 +177,7 @@ public class CLALibBinaryCellOp {
 		}
 
 		List<AColGroup> newColGroups = new ArrayList<>(oldColGroups.size());
-		int k = OptimizerUtils.getConstrainedNumThreads(-1);
+		int k = op.getNumThreads();
 		ExecutorService pool = CommonThreadPool.get(k);
 		ArrayList<BinaryMVRowTask> tasks = new ArrayList<>();
 		try {
@@ -238,8 +238,7 @@ public class CLALibBinaryCellOp {
 		boolean foundConst = false;
 		for(AColGroup grp : m1.getColGroups()) {
 			if(!m2.isEmpty() && !foundConst && grp instanceof ColGroupConst) {
-				ADictionary newDict = ((ColGroupValue) grp).applyBinaryRowOp(op.fn, m2.getDenseBlockValues(), false,
-					left);
+				ADictionary newDict = ((ColGroupValue) grp).applyBinaryRowOp(op, m2.getDenseBlockValues(), false, left);
 				newColGroups.add(new ColGroupConst(grp.getColIndices(), m1.getNumRows(), newDict));
 				foundConst = true;
 			}
@@ -251,7 +250,7 @@ public class CLALibBinaryCellOp {
 			int[] colIndexes = oldColGroups.get(0).getColIndices();
 			double[] v = m2.getDenseBlockValues();
 			ADictionary newDict = new Dictionary(new double[colIndexes.length]);
-			newDict = newDict.applyBinaryRowOp(op.fn, v, true, colIndexes, left);
+			newDict = newDict.applyBinaryRowOp(op, v, true, colIndexes, left);
 			newColGroups.add(new ColGroupConst(colIndexes, m1.getNumRows(), newDict));
 		}
 		ret.allocateColGroupList(newColGroups);
@@ -354,7 +353,7 @@ public class CLALibBinaryCellOp {
 		public Integer call() {
 			// unsafe decompress, since we count nonzeros afterwards.
 			for(AColGroup g : _m1.getColGroups())
-				g.decompressToBlock(_ret, _rl, _ru, g.getValues(), false);
+				g.decompressToBlock(_ret, _rl, _ru, false);
 
 			if(_m2.isInSparseFormat())
 				throw new NotImplementedException("Not Implemented sparse Format execution for MM.");
@@ -398,7 +397,7 @@ public class CLALibBinaryCellOp {
 		public Integer call() {
 			// unsafe decompress, since we count nonzeros afterwards.
 			for(AColGroup g : _m1.getColGroups())
-				g.decompressToBlock(_ret, _rl, _ru, g.getValues(), false);
+				g.decompressToBlock(_ret, _rl, _ru, false);
 
 			if(_m2.isInSparseFormat())
 				throw new NotImplementedException("Not Implemented sparse Format execution for MM.");
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
index f6c734c..c0ac099 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
@@ -496,7 +496,7 @@ public class CLALibCompAgg {
 		private MatrixBlock decompressToTemp() {
 			MatrixBlock tmp = getTmp();
 			for(AColGroup g : _m1.getColGroups())
-				g.decompressToBlockUnSafe(tmp, _rl, _ru, 0, g.getValues());
+				g.decompressToBlockUnSafe(tmp, _rl, _ru, 0);
 			tmp.setNonZeros(_rl + _ru);
 			return tmp;
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index ba22642..1fff010 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -32,9 +32,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
-import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
+import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.CommonThreadPool;
@@ -85,29 +84,16 @@ public class CLALibLeftMultBy {
 		return ret;
 	}
 
-	// public static void leftMultByTransposeSelf(CompressedMatrixBlock mb, MatrixBlock result, int k) {
-
-	// }
-
 	public static void leftMultByTransposeSelf(List<AColGroup> groups, MatrixBlock result, int k, int numColumns,
 		Pair<Integer, int[]> v, boolean overlapping) {
 
 		result.allocateDenseBlock();
 
-		if(!overlapping && groups.get(0).getNumCols() == numColumns) {
-			leftMultBySelfDiagonalColGroup(groups, result, numColumns);
-			return;
-		}
-
 		if(overlapping) {
 			LOG.warn("Inefficient TSMM with overlapping matrix could be implemented multi-threaded but is not yet.");
 			leftMultByCompressedTransposedMatrix(groups, groups, result);
-
-			result.recomputeNonZeros();
-			return;
 		}
-
-		if(k <= 1) {
+		else if(k <= 1) {
 			for(int i = 0; i < groups.size(); i++)
 				leftMultByCompressedTransposedMatrix(groups.get(i), groups, result, i, groups.size());
 		}
@@ -128,35 +114,37 @@ public class CLALibLeftMultBy {
 				throw new DMLRuntimeException(e);
 			}
 		}
-
+		// LOG.error(result);
+		// Move values in the lower part of the matrix to the upper part
 		copyToUpperTriangle(result.getDenseBlockValues(), numColumns);
-		result.recomputeNonZeros();
+
+		// LOG.error(result);
+		// calculate the number of non zeros, and allocate all value locations by copying upper triangle back to bottom.
+		long nnz = LinearAlgebraUtils.copyUpperToLowerTriangle(result);
+		result.setNonZeros(nnz);
+		// Evaluate if the output should be sparsely allocated.
+		result.examSparsity(false);
+		result.setNonZeros(nnz);
+		// LOG.error(result);
 	}
 
 	private static void copyToUpperTriangle(final double[] c, final int cols) {
 		for(int i = 0, offC = 0; i < cols; i++, offC += cols)
 			for(int j = i, offR = i * cols; j < cols; j++, offR += cols) {
-				final double d = c[i + offR];
-				if(d != 0)
-					c[offC + j] = d;
+				final double prev = c[offC + j];
+				if(prev == 0)
+					c[offC + j] = c[i + offR];
 			}
 
 	}
 
-	private static void leftMultBySelfDiagonalColGroup(List<AColGroup> groups, MatrixBlock result, int numColumns) {
-		double[] outValues = result.getDenseBlockValues();
-		for(AColGroup g : groups)
-			g.tsmm(outValues, numColumns);
-
-	}
-
 	private static MatrixBlock leftMultByCompressedTransposedMatrix(List<AColGroup> colGroups,
 		CompressedMatrixBlock that, MatrixBlock ret, int k, int numColumns, Pair<Integer, int[]> v,
 		boolean overlapping) {
 
 		ret.allocateDenseBlock();
 		List<AColGroup> thatCGs = that.getColGroups();
-		Pair<Integer, int[]> thatV = that.getMaxNumValues();
+		// Pair<Integer, int[]> thatV = that.getMaxNumValues();
 
 		if(k <= 1 || overlapping || that.isOverlapping()) {
 			if(overlapping || that.isOverlapping())
@@ -170,14 +158,14 @@ public class CLALibLeftMultBy {
 				ArrayList<Callable<Object>> tasks = new ArrayList<>();
 				for(int i = 0; i < thatCGs.size(); i++) {
 
-					if(thatCGs.get(i).getNumCols() > 1 || thatCGs.get(i).getCompType() == CompressionType.CONST)
-						tasks.add(new LeftMultByCompressedTransposedMatrixTask(colGroups, thatCGs.get(i), ret, 0,
-							colGroups.size()));
-					else {
-						int row = thatCGs.get(i).getColIndices()[0];
-						tasks.add(new LeftMultByCompressedTransposedMatrixTask2(colGroups, thatCGs, ret, v, thatV, row,
-							row + 1, overlapping, 1));
-					}
+					// if(thatCGs.get(i).getNumCols() > 1 || thatCGs.get(i).getCompType() == CompressionType.CONST)
+					tasks.add(new LeftMultByCompressedTransposedMatrixTask(colGroups, thatCGs.get(i), ret, 0,
+						colGroups.size()));
+					// else {
+					// int row = thatCGs.get(i).getColIndices()[0];
+					// tasks.add(new LeftMultByCompressedTransposedMatrixTask2(colGroups, thatCGs, ret, v, thatV, row,
+					// row + 1, overlapping, 1));
+					// }
 				}
 
 				for(Future<Object> tret : pool.invokeAll(tasks))
@@ -211,7 +199,7 @@ public class CLALibLeftMultBy {
 		@Override
 		public Object call() {
 			try {
-				IPreAggregate.setupThreadLocalMemory(1024);
+				// IPreAggregate.setupThreadLocalMemory(1024);
 				leftMultByCompressedTransposedMatrix(_left, _groups, _ret, _start, _end);
 
 			}
@@ -232,8 +220,13 @@ public class CLALibLeftMultBy {
 	private static void leftMultByCompressedTransposedMatrix(AColGroup lhs, List<AColGroup> thisCG, MatrixBlock ret,
 		int colGroupStart, int colGroupEnd) {
 
-		for(; colGroupStart < colGroupEnd; colGroupStart++)
-			thisCG.get(colGroupStart).leftMultByAColGroup(lhs, ret);
+		for(; colGroupStart < colGroupEnd; colGroupStart++) {
+			AColGroup rhs = thisCG.get(colGroupStart);
+			if(rhs != lhs)
+				rhs.leftMultByAColGroup(lhs, ret);
+			else
+				rhs.tsmm(ret.getDenseBlockValues(), ret.getNumColumns());
+		}
 
 	}
 
@@ -265,14 +258,14 @@ public class CLALibLeftMultBy {
 				else {
 					for(AColGroup g : colGroups) {
 						// if(g instanceof ColGroupDDC) {
-						// 	tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, 0, that.getNumRows(), v));
+						// tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, 0, that.getNumRows(), v));
 						// }
 						// else {
 
-							for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
-								tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, blo,
-									Math.min(blo + rowBlockSize, that.getNumRows()), v));
-							}
+						for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
+							tasks.add(new LeftMatrixColGroupMultTask(g, that, ret, blo,
+								Math.min(blo + rowBlockSize, that.getNumRows()), v));
+						}
 						// }
 					}
 				}
@@ -291,93 +284,94 @@ public class CLALibLeftMultBy {
 		return ret;
 	}
 
-	private static void leftMultByCompressedTransposeRowSection(List<AColGroup> thisGroups, List<AColGroup> thatGroups,
-		MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl, int ru, boolean overlapping,
-		int k) {
-		if(k > 1 && !overlapping)
-			leftMultByCompressedTransposeRowSectionParallel(thisGroups, thatGroups, result, v, thatV, rl, ru, k);
-		else
-			leftMultByCompressedTransposeRowSectionSingleThread(thisGroups, thatGroups, result, v, thatV, rl, ru);
+	// private static void leftMultByCompressedTransposeRowSection(List<AColGroup> thisGroups, List<AColGroup>
+	// thatGroups,
+	// MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl, int ru, boolean overlapping,
+	// int k) {
+	// if(k > 1 && !overlapping)
+	// leftMultByCompressedTransposeRowSectionParallel(thisGroups, thatGroups, result, v, thatV, rl, ru, k);
+	// else
+	// leftMultByCompressedTransposeRowSectionSingleThread(thisGroups, thatGroups, result, v, thatV, rl, ru);
 
-	}
+	// }
 
-	private static void leftMultByCompressedTransposeRowSectionParallel(List<AColGroup> thisGroups,
-		List<AColGroup> thatGroups, MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl,
-		int ru, int k) {
+	// private static void leftMultByCompressedTransposeRowSectionParallel(List<AColGroup> thisGroups,
+	// List<AColGroup> thatGroups, MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl,
+	// int ru, int k) {
 
-		// preallocated dense tmp matrix blocks
-		MatrixBlock lhs = new MatrixBlock(1, thisGroups.get(0).getNumRows(), false);
-		MatrixBlock tmpret = new MatrixBlock(1, result.getNumColumns(), false);
-		lhs.allocateDenseBlock();
-		tmpret.allocateDenseBlock();
+	// // preallocated dense tmp matrix blocks
+	// MatrixBlock lhs = new MatrixBlock(1, thisGroups.get(0).getNumRows(), false);
+	// MatrixBlock tmpret = new MatrixBlock(1, result.getNumColumns(), false);
+	// lhs.allocateDenseBlock();
+	// tmpret.allocateDenseBlock();
 
-		ExecutorService pool = CommonThreadPool.get(k);
-		ArrayList<leftMultByVectorTransposeTask> tasks = new ArrayList<>();
-		for(int j = rl; j < ru; j++) {
-			AColGroup.decompressColumnToBlock(lhs, j, thatGroups);
-			if(!lhs.isEmptyBlock(false)) {
+	// ExecutorService pool = CommonThreadPool.get(k);
+	// ArrayList<leftMultByVectorTransposeTask> tasks = new ArrayList<>();
+	// for(int j = rl; j < ru; j++) {
+	// AColGroup.decompressColumnToBlock(lhs, j, thatGroups);
+	// if(!lhs.isEmptyBlock(false)) {
 
-				try {
-					int groupBatch = Math.max(thisGroups.size() / k, 1);
+	// try {
+	// int groupBatch = Math.max(thisGroups.size() / k, 1);
 
-					for(int i = 0; i * groupBatch < thisGroups.size(); i++) {
-						tasks.add(new leftMultByVectorTransposeTask(thisGroups, lhs, tmpret, i * groupBatch,
-							Math.min(thisGroups.size(), (i + 1) * groupBatch), v));
-					}
-					for(Future<Object> future : pool.invokeAll(tasks))
-						future.get();
-				}
-				catch(InterruptedException | ExecutionException e) {
-					throw new DMLRuntimeException(e);
-				}
+	// for(int i = 0; i * groupBatch < thisGroups.size(); i++) {
+	// tasks.add(new leftMultByVectorTransposeTask(thisGroups, lhs, tmpret, i * groupBatch,
+	// Math.min(thisGroups.size(), (i + 1) * groupBatch), v));
+	// }
+	// for(Future<Object> future : pool.invokeAll(tasks))
+	// future.get();
+	// }
+	// catch(InterruptedException | ExecutionException e) {
+	// throw new DMLRuntimeException(e);
+	// }
 
-				double[] tmpRetValues = tmpret.getDenseBlockValues();
-				double[] resultValues = result.getDenseBlockValues();
-				int offset = tmpret.getNumColumns() * j;
-				for(int i = 0; i < tmpret.getNumColumns(); i++, offset++) {
-					resultValues[offset] += tmpRetValues[i];
-					tmpRetValues[i] = 0;
-				}
-			}
-			lhs.reset();
-			tasks.clear();
-		}
-		pool.shutdown();
+	// double[] tmpRetValues = tmpret.getDenseBlockValues();
+	// double[] resultValues = result.getDenseBlockValues();
+	// int offset = tmpret.getNumColumns() * j;
+	// for(int i = 0; i < tmpret.getNumColumns(); i++, offset++) {
+	// resultValues[offset] += tmpRetValues[i];
+	// tmpRetValues[i] = 0;
+	// }
+	// }
+	// lhs.reset();
+	// tasks.clear();
+	// }
+	// pool.shutdown();
 
-		// post processing
-		ColGroupValue.cleanupThreadLocalMemory();
-	}
+	// // post processing
+	// ColGroupValue.cleanupThreadLocalMemory();
+	// }
 
-	private static void leftMultByCompressedTransposeRowSectionSingleThread(List<AColGroup> thisGroups,
-		List<AColGroup> thatGroups, MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl,
-		int ru) {
-		final int numRows = thisGroups.get(0).getNumRows();
-
-		// preallocated dense tmp matrix blocks
-		MatrixBlock lhs = new MatrixBlock(1, numRows, false);
-		MatrixBlock tmpret = new MatrixBlock(1, result.getNumColumns(), false);
-
-		lhs.allocateDenseBlock();
-		tmpret.allocateDenseBlock();
-
-		for(int j = rl; j < ru; j++) {
-			AColGroup.decompressColumnToBlock(lhs, j, thatGroups);
-			if(!lhs.isEmptyBlock(false)) {
-				for(AColGroup grp : thisGroups)
-					grp.leftMultByMatrix(lhs, tmpret);
-
-				double[] tmpRetValues = tmpret.getDenseBlockValues();
-				double[] resultValues = result.getDenseBlockValues();
-				int offset = tmpret.getNumColumns() * j;
-				for(int i = 0; i < tmpret.getNumColumns(); i++, offset++) {
-					resultValues[offset] += tmpRetValues[i];
-					tmpRetValues[i] = 0;
-				}
-			}
-			lhs.reset();
-		}
+	// private static void leftMultByCompressedTransposeRowSectionSingleThread(List<AColGroup> thisGroups,
+	// List<AColGroup> thatGroups, MatrixBlock result, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl,
+	// int ru) {
+	// final int numRows = thisGroups.get(0).getNumRows();
+
+	// // preallocated dense tmp matrix blocks
+	// MatrixBlock lhs = new MatrixBlock(1, numRows, false);
+	// MatrixBlock tmpret = new MatrixBlock(1, result.getNumColumns(), false);
+
+	// lhs.allocateDenseBlock();
+	// tmpret.allocateDenseBlock();
+
+	// for(int j = rl; j < ru; j++) {
+	// AColGroup.decompressColumnToBlock(lhs, j, thatGroups);
+	// if(!lhs.isEmptyBlock(false)) {
+	// for(AColGroup grp : thisGroups)
+	// grp.leftMultByMatrix(lhs, tmpret);
+
+	// double[] tmpRetValues = tmpret.getDenseBlockValues();
+	// double[] resultValues = result.getDenseBlockValues();
+	// int offset = tmpret.getNumColumns() * j;
+	// for(int i = 0; i < tmpret.getNumColumns(); i++, offset++) {
+	// resultValues[offset] += tmpRetValues[i];
+	// tmpRetValues[i] = 0;
+	// }
+	// }
+	// lhs.reset();
+	// }
 
-	}
+	// }
 
 	private static class LeftMatrixMatrixMultTask implements Callable<Object> {
 		private final List<AColGroup> _group;
@@ -400,7 +394,7 @@ public class CLALibLeftMultBy {
 		@Override
 		public Object call() {
 			try {
-				ColGroupValue.setupThreadLocalMemory(_v.getLeft() * (_ru - _rl));
+				ColGroupValue.setupThreadLocalMemory(_v.getLeft());
 				for(int j = 0; j < _group.size(); j++)
 					_group.get(j).leftMultByMatrix(_that, _ret, _rl, _ru);
 			}
@@ -433,7 +427,7 @@ public class CLALibLeftMultBy {
 		public Object call() {
 
 			try {
-				ColGroupValue.setupThreadLocalMemory(_v.getLeft() * (_ru - _rl));
+				ColGroupValue.setupThreadLocalMemory(_v.getLeft());
 				_group.leftMultByMatrix(_that, _ret, _rl, _ru);
 			}
 			catch(Exception e) {
@@ -443,65 +437,65 @@ public class CLALibLeftMultBy {
 		}
 	}
 
-	private static class LeftMultByCompressedTransposedMatrixTask2 implements Callable<Object> {
-		private final List<AColGroup> _groups;
-		private final List<AColGroup> _thatGroups;
-		private final MatrixBlock _ret;
-		private final int _rl;
-		private final int _ru;
-		private final Pair<Integer, int[]> _v;
-		private final Pair<Integer, int[]> _thatV;
-		private final boolean _overlapping;
-		private final int _extraThreads;
-
-		protected LeftMultByCompressedTransposedMatrixTask2(List<AColGroup> thisGroups, List<AColGroup> thatGroups,
-			MatrixBlock ret, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl, int ru, boolean overlapping,
-			int extraThreads) {
-			_groups = thisGroups;
-			_thatGroups = thatGroups;
-			_ret = ret;
-			_rl = rl;
-			_ru = ru;
-			_v = v;
-			_thatV = thatV;
-			_overlapping = overlapping;
-			_extraThreads = extraThreads;
-		}
-
-		@Override
-		public Object call() {
-			ColGroupValue.setupThreadLocalMemory(Math.max(_v.getLeft(), _thatV.getLeft()) + 1);
-			leftMultByCompressedTransposeRowSection(_groups, _thatGroups, _ret, _v, _thatV, _rl, _ru, _overlapping,
-				_extraThreads);
-			return null;
-		}
-	}
+	// private static class LeftMultByCompressedTransposedMatrixTask2 implements Callable<Object> {
+	// private final List<AColGroup> _groups;
+	// private final List<AColGroup> _thatGroups;
+	// private final MatrixBlock _ret;
+	// private final int _rl;
+	// private final int _ru;
+	// private final Pair<Integer, int[]> _v;
+	// private final Pair<Integer, int[]> _thatV;
+	// private final boolean _overlapping;
+	// private final int _extraThreads;
+
+	// protected LeftMultByCompressedTransposedMatrixTask2(List<AColGroup> thisGroups, List<AColGroup> thatGroups,
+	// MatrixBlock ret, Pair<Integer, int[]> v, Pair<Integer, int[]> thatV, int rl, int ru, boolean overlapping,
+	// int extraThreads) {
+	// _groups = thisGroups;
+	// _thatGroups = thatGroups;
+	// _ret = ret;
+	// _rl = rl;
+	// _ru = ru;
+	// _v = v;
+	// _thatV = thatV;
+	// _overlapping = overlapping;
+	// _extraThreads = extraThreads;
+	// }
 
-	private static class leftMultByVectorTransposeTask implements Callable<Object> {
-		private final List<AColGroup> _grps;
-		private final MatrixBlock _rowVector;
-		private final MatrixBlock _result;
-		private final int _gl;
-		private final int _gu;
-		private final Pair<Integer, int[]> _v;
+	// @Override
+	// public Object call() {
+	// ColGroupValue.setupThreadLocalMemory(Math.max(_v.getLeft(), _thatV.getLeft()));
+	// leftMultByCompressedTransposeRowSection(_groups, _thatGroups, _ret, _v, _thatV, _rl, _ru, _overlapping,
+	// _extraThreads);
+	// return null;
+	// }
+	// }
 
-		protected leftMultByVectorTransposeTask(List<AColGroup> grps, MatrixBlock rowVector, MatrixBlock result, int gl,
-			int gu, Pair<Integer, int[]> v) {
-			_grps = grps;
-			_rowVector = rowVector;
-			_result = result;
-			_gl = gl;
-			_gu = gu;
-			_v = v;
-		}
+	// private static class leftMultByVectorTransposeTask implements Callable<Object> {
+	// private final List<AColGroup> _grps;
+	// private final MatrixBlock _rowVector;
+	// private final MatrixBlock _result;
+	// private final int _gl;
+	// private final int _gu;
+	// private final Pair<Integer, int[]> _v;
+
+	// protected leftMultByVectorTransposeTask(List<AColGroup> grps, MatrixBlock rowVector, MatrixBlock result, int gl,
+	// int gu, Pair<Integer, int[]> v) {
+	// _grps = grps;
+	// _rowVector = rowVector;
+	// _result = result;
+	// _gl = gl;
+	// _gu = gu;
+	// _v = v;
+	// }
 
-		@Override
-		public Object call() {
-			ColGroupValue.setupThreadLocalMemory(_v.getLeft() + 1);
-			for(int i = _gl; i < _gu; i++) {
-				_grps.get(i).leftMultByMatrix(_rowVector, _result);
-			}
-			return null;
-		}
-	}
+	// @Override
+	// public Object call() {
+	// ColGroupValue.setupThreadLocalMemory(_v.getLeft());
+	// for(int i = _gl; i < _gu; i++) {
+	// _grps.get(i).leftMultByMatrix(_rowVector, _result);
+	// }
+	// return null;
+	// }
+	// }
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
index af4e679..ce6b3d3 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRelationalOp.java
@@ -156,7 +156,7 @@ public class CLALibRelationalOp {
 			MatrixBlock tmp = new MatrixBlock(blkz, cols, false, -1).allocateBlock();
 			for(int i = 0; i * blkz < outRows; i++) {
 				for(MinMaxGroup mmg : minMax) {
-					mmg.g.decompressToBlock(tmp, i * blkz, Math.min((i + 1) * blkz, rows), 0, mmg.values);
+					mmg.g.decompressToBlock(tmp, i * blkz, Math.min((i + 1) * blkz, rows), 0);
 				}
 				for(int row = 0; row < blkz && row < rows - i * blkz; row++) {
 					int off = (row + i * blkz);
@@ -260,11 +260,8 @@ public class CLALibRelationalOp {
 			}
 
 			for(MinMaxGroup mmg : _minMax) {
-				mmg.g.decompressToBlockUnSafe(tmp,
-					_i * _blkz,
-					Math.min((_i + 1) * _blkz, mmg.g.getNumRows()),
-					0,
-					mmg.values);
+				if(mmg.g.getNumberNonZeros() != 0)
+					mmg.g.decompressToBlockUnSafe(tmp, _i * _blkz, Math.min((_i + 1) * _blkz, mmg.g.getNumRows()), 0);
 			}
 
 			for(int row = 0, off = _i * _blkz; row < _blkz && row < _rows - _i * _blkz; row++, off++) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
index e3ef846..d1b766e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
@@ -100,7 +100,6 @@ public class CLALibScalar {
 		}
 
 		ret.recomputeNonZeros();
-
 		return ret;
 
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
index 3dafebe..47941ab 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/readers/ReaderCompressedSelection.java
@@ -53,11 +53,7 @@ public class ReaderCompressedSelection extends ReaderColumnSelection {
 			// decompress into the tmpBlock.
 			currentBlock = _lastRow / decompressRowCount;
 			for(AColGroup g : compressedOverlap.getColGroups()) {
-				g.decompressToBlockUnSafe(_tmp,
-					_lastRow,
-					Math.min(_lastRow + decompressRowCount, g.getNumRows()),
-					0,
-					g.getValues());
+				g.decompressToBlockUnSafe(_tmp, _lastRow, Math.min(_lastRow + decompressRowCount, g.getNumRows()), 0);
 			}
 		}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArray.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArray.java
index 0810e8d..6385697 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArray.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArray.java
@@ -27,6 +27,8 @@ import java.util.Arrays;
 public class DblArray {
 	private double[] _arr;
 
+	private int hash = 0;
+
 	public DblArray() {
 		_arr = null;
 	}
@@ -43,9 +45,16 @@ public class DblArray {
 		return _arr;
 	}
 
+	public void resetHash(){
+		hash = 0;
+	}
+
 	@Override
 	public int hashCode() {
-		return _arr == null ? 0 : Arrays.hashCode(_arr);
+		if(hash != 0)
+			return hash;
+		hash = _arr == null ? 0 : Arrays.hashCode(_arr);
+		return hash;
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixMult.java b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixMult.java
index 819cd6b..3ff1bb9 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/LibMatrixMult.java
@@ -1230,7 +1230,7 @@ public class LibMatrixMult
 				if( a.isEmpty(i) ) continue; 
 				int apos = a.pos(i);
 				int alen = a.size(i);
-				int[] aix = a.indexes(i);
+				// int[] aix = a.indexes(i);
 				double[] avals = a.values(i);
 				double[] cvals = c.values(i);
 				int cix = c.pos(i);
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
index 4031a67..fe25916 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
@@ -1078,6 +1078,16 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		return evalSparseFormatOnDisk(rlen, clen, nonZeros);
 	}
 	
+	/**
+	 * Evaluates if this matrix block should be in sparse format in
+	 * memory. Depending on the current representation, the state of the
+	 * matrix block is changed to the right representation if necessary. 
+	 * Note that this consumes for the time of execution memory for both 
+	 * representations.
+	 * 
+	 * Allowing CSR format is default for this operation.
+	 * 
+	 */
 	public void examSparsity() {
 		examSparsity(true);
 	}
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
index 4cad36e..de88ad6 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
@@ -30,6 +30,7 @@ import java.io.DataOutputStream;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.CompressionSettingsBuilder;
 import org.apache.sysds.runtime.compress.CompressionStatistics;
+import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.PartitionerType;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.matrix.data.LibMatrixCountDistinct;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -181,7 +182,8 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 	@Test
 	public void testCompressionRatio() {
 		try {
-			if(!(cmb instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock) ||
+				compressionSettings.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
 				return;
 			CompressionStatistics cStat = cmbStats;
 			if(cStat != null)
@@ -196,7 +198,8 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 	@Test
 	public void testCompressionEstimationVSCompression() {
 		try {
-			if(!(cmb instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock) ||
+				compressionSettings.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
 				return;
 			CompressionStatistics cStat = cmbStats;
 			if(cStat != null) {
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
index 7b9d871..d6563d8 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
@@ -75,17 +75,18 @@ import org.junit.runners.Parameterized.Parameters;
 public abstract class CompressedTestBase extends TestBase {
 	protected static final Log LOG = LogFactory.getLog(CompressedTestBase.class.getName());
 
-	protected static SparsityType[] usedSparsityTypes = new SparsityType[] {SparsityType.FULL, SparsityType.SPARSE,};
+	protected static SparsityType[] usedSparsityTypes = new SparsityType[] {SparsityType.FULL, SparsityType.SPARSE,
+		SparsityType.ULTRA_SPARSE};
 
 	protected static ValueType[] usedValueTypes = new ValueType[] {ValueType.RAND_ROUND, ValueType.OLE_COMPRESSIBLE,
 		ValueType.RLE_COMPRESSIBLE};
 
-	protected static ValueRange[] usedValueRanges = new ValueRange[] {ValueRange.SMALL, ValueRange.NEGATIVE,
-		ValueRange.BYTE};
+	protected static ValueRange[] usedValueRanges = new ValueRange[] {ValueRange.BOOLEAN, ValueRange.SMALL,
+		ValueRange.NEGATIVE};
 
 	protected static OverLapping[] overLapping = new OverLapping[] {
 		// OverLapping.COL,
-		OverLapping.PLUS, OverLapping.MATRIX, OverLapping.NONE, OverLapping.APPEND_EMPTY, OverLapping.APPEND_CONST,
+		OverLapping.PLUS, OverLapping.MATRIX, OverLapping.NONE, OverLapping.APPEND_CONST, OverLapping.APPEND_EMPTY
 		// OverLapping.MATRIX_PLUS,
 		// OverLapping.SQUASH,
 		// OverLapping.MATRIX_MULT_NEGATIVE
@@ -115,10 +116,11 @@ public abstract class CompressedTestBase extends TestBase {
 
 		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("false")
 			.setInvestigateEstimate(true),
-		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
-			.setColumnPartitioner(PartitionerType.BIN_PACKING).setInvestigateEstimate(true),
-		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
-			.setColumnPartitioner(PartitionerType.STATIC).setInvestigateEstimate(true),
+
+		// new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
+		// .setColumnPartitioner(PartitionerType.BIN_PACKING).setInvestigateEstimate(true),
+		// new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
+		// .setColumnPartitioner(PartitionerType.STATIC).setInvestigateEstimate(true),
 
 		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
 			.setColumnPartitioner(PartitionerType.COST_MATRIX_MULT).setInvestigateEstimate(true),
@@ -155,7 +157,8 @@ public abstract class CompressedTestBase extends TestBase {
 	};
 
 	protected static MatrixTypology[] usedMatrixTypology = new MatrixTypology[] { // Selected Matrix Types
-		MatrixTypology.SMALL, MatrixTypology.FEW_COL,
+		// MatrixTypology.SMALL,
+		MatrixTypology.FEW_COL,
 		// MatrixTypology.FEW_ROW,
 		MatrixTypology.LARGE,
 		// // MatrixTypology.SINGLE_COL,
@@ -263,7 +266,7 @@ public abstract class CompressedTestBase extends TestBase {
 				}
 				if(ov == OverLapping.PLUS) {
 					// LOG.error(cmb.slice(0,10,0,10));
-					ScalarOperator sop = new LeftScalarOperator(Plus.getPlusFnObject(), 15);
+					ScalarOperator sop = new LeftScalarOperator(Plus.getPlusFnObject(), 5);
 					mb = mb.scalarOperations(sop, new MatrixBlock());
 					cmb = cmb.scalarOperations(sop, new MatrixBlock());
 					// LOG.error(cmb.slice(0,10,0,10));
@@ -296,10 +299,16 @@ public abstract class CompressedTestBase extends TestBase {
 		for(SparsityType st : usedSparsityTypes)
 			for(ValueType vt : usedValueTypes)
 				for(ValueRange vr : usedValueRanges)
-					for(CompressionSettingsBuilder cs : usedCompressionSettings)
-						for(MatrixTypology mt : usedMatrixTypology)
-							for(OverLapping ov : overLapping)
-								tests.add(new Object[] {st, vt, vr, cs, mt, ov});
+					if((st == SparsityType.ULTRA_SPARSE && vr == ValueRange.LARGE) || st != SparsityType.ULTRA_SPARSE)
+						for(CompressionSettingsBuilder cs : usedCompressionSettings)
+							for(MatrixTypology mt : usedMatrixTypology)
+								for(OverLapping ov : overLapping)
+									if((ov == OverLapping.APPEND_CONST || ov == OverLapping.APPEND_EMPTY)) {
+										if(vr == ValueRange.BOOLEAN)
+											tests.add(new Object[] {st, vt, vr, cs, mt, ov});
+									}
+									else
+										tests.add(new Object[] {st, vt, vr, cs, mt, ov});
 		for(CompressionSettingsBuilder cs : usedCompressionSettings)
 			for(MatrixTypology mt : usedMatrixTypology)
 				for(OverLapping ov : overLapping) {
@@ -707,6 +716,8 @@ public abstract class CompressedTestBase extends TestBase {
 				// matrix-vector compressed
 				MatrixBlock ret2 = cmb.transposeSelfMatrixMultOperations(new MatrixBlock(), mType, _k);
 
+				// LOG.error("actual : " + ret1);
+				// LOG.error("compressed : " + ret2);
 				// compare result with input
 				compareResultMatrices(ret1, ret2, 100);
 			}
@@ -1063,7 +1074,6 @@ public abstract class CompressedTestBase extends TestBase {
 		catch(Exception e) {
 			// e.printStackTrace();
 			throw new DMLRuntimeException(this.toString() + "\n" + e.getMessage(), e);
-
 		}
 	}
 
diff --git a/src/test/java/org/apache/sysds/test/component/compress/TestBase.java b/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
index 29a2a87..abdce93 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
@@ -52,7 +52,6 @@ public class TestBase {
 	protected OverLapping overlappingType;
 
 	// Input
-	protected double[][] input;
 	protected MatrixBlock mb;
 
 	public TestBase(SparsityType sparType, ValueType valType, ValueRange valueRange,
@@ -66,28 +65,29 @@ public class TestBase {
 			this.max = TestConstants.getMaxRangeValue(valueRange);
 			this.min = TestConstants.getMinRangeValue(valueRange);
 			this.overlappingType = ov;
+			double[][] input;
 			switch(valType) {
 				case CONST:
 					this.min = this.max;
 					// Do not Break, utilize the RAND afterwards.
 				case RAND:
-					this.input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed);
+					input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed);
 					break;
 				case RAND_ROUND:
-					this.input = TestUtils.round(TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed));
+					input = TestUtils.round(TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed));
 					break;
 				case OLE_COMPRESSIBLE:
 					// Note the Compressible Input generator, generates an already Transposed input
 					// normally, therefore last argument is true, to build a non transposed matrix.
-					this.input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.OLE,
+					input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.OLE,
 						(max - min), max, min, sparsity, seed, true);
 					break;
 				case RLE_COMPRESSIBLE:
-					this.input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.RLE,
+					input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.RLE,
 						(max - min), max, min, sparsity, seed, true);
 					break;
 				case ONE_HOT_ENCODED:
-					this.input = CompressibleInputGenerator.getInputOneHotMatrix(rows, cols, seed);
+					input = CompressibleInputGenerator.getInputOneHotMatrix(rows, cols, seed);
 					break;
 				default:
 					throw new NotImplementedException("Not Implemented Test Value type input generator");
@@ -97,7 +97,7 @@ public class TestBase {
 			this.valType = valType;
 			this.compressionSettings = compressionSettings.create();
 
-			mb = DataConverter.convertToMatrixBlock(this.input);
+			mb = DataConverter.convertToMatrixBlock(input);
 
 		}
 		catch(Exception e) {

[systemds] 04/07: [SYSTEMDS-2992] CLA init workload cost functions

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

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

commit e8da72fa0c607fe191265a962f39b79420d06611
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Fri May 14 10:13:40 2021 +0200

    [SYSTEMDS-2992] CLA init workload cost functions
    
    Initial version of cost based co-coding functions.
    This commit adds costs for compressed tsmm and compressed mm.
---
 .../runtime/compress/CompressedMatrixBlock.java    |   2 +
 .../compress/CompressedMatrixBlockFactory.java     |  17 +-
 .../runtime/compress/CompressionSettings.java      |   8 +-
 .../compress/CompressionSettingsBuilder.java       |  15 +-
 .../runtime/compress/CompressionStatistics.java    |  18 +-
 .../runtime/compress/cocode/AColumnCoCoder.java    |   6 +-
 .../runtime/compress/cocode/CoCodeBinPacking.java  |   4 +-
 .../sysds/runtime/compress/cocode/CoCodeCost.java  |   4 +-
 .../compress/cocode/CoCodeCostMatrixMult.java      |  72 ++--
 .../runtime/compress/cocode/CoCodeStatic.java      |   4 +-
 .../runtime/compress/cocode/PlanningCoCoder.java   |  16 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |  16 +-
 .../compress/colgroup/ColGroupCompressed.java      |   8 +-
 .../runtime/compress/colgroup/ColGroupConst.java   |   5 -
 .../runtime/compress/colgroup/ColGroupDDC.java     |   5 +-
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   4 -
 .../runtime/compress/colgroup/ColGroupFactory.java | 102 ++---
 .../runtime/compress/colgroup/ColGroupOLE.java     |   7 +-
 .../runtime/compress/colgroup/ColGroupOffset.java  |  10 +-
 .../runtime/compress/colgroup/ColGroupSDC.java     |   2 +-
 .../compress/colgroup/ColGroupSDCSingle.java       |   2 +-
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  20 +-
 .../compress/colgroup/ColGroupSDCZeros.java        |  16 +-
 .../runtime/compress/colgroup/ColGroupSizes.java   |  57 ++-
 .../compress/colgroup/ColGroupUncompressed.java    |   5 +
 .../runtime/compress/colgroup/ColGroupValue.java   |  66 +--
 .../compress/colgroup/dictionary/ADictionary.java  |  36 +-
 .../compress/colgroup/dictionary/Dictionary.java   |  22 +
 .../colgroup/dictionary/DictionaryFactory.java     | 135 +++++-
 .../colgroup/dictionary/MatrixBlockDictionary.java | 456 +++++++++++++++++++++
 .../compress/colgroup/dictionary/QDictionary.java  |  10 +
 .../colgroup/dictionary/SparseDictionary.java      | 218 ----------
 .../compress/estim/CompressedSizeEstimator.java    |  87 +++-
 .../estim/CompressedSizeEstimatorExact.java        |   3 +-
 .../estim/CompressedSizeEstimatorFactory.java      |  41 +-
 .../estim/CompressedSizeEstimatorSample.java       |  38 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |  27 +-
 .../sysds/runtime/compress/lib/BitmapEncoder.java  |  56 +--
 .../runtime/compress/lib/BitmapLossyEncoder.java   |  30 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |   2 +-
 .../runtime/compress/lib/CLALibRightMultBy.java    |   6 +-
 .../sysds/runtime/compress/lib/CLALibSquash.java   |  11 +-
 .../sysds/runtime/compress/utils/Bitmap.java       |  29 +-
 .../utils/{Bitmap.java => MultiColBitmap.java}     |  43 +-
 .../org/apache/sysds/utils/MemoryEstimates.java    |   2 +
 .../compress/AbstractCompressedUnaryTests.java     |   3 -
 .../component/compress/CompressedTestBase.java     |   7 +-
 .../compress/CompressibleInputGenerator.java       |  15 +
 .../sysds/test/component/compress/TestBase.java    |  13 +-
 .../test/component/compress/TestConstants.java     |   1 +
 .../compress/colgroup/JolEstimateTest.java         |  74 ++--
 .../compress/estim/SampleEstimatorTest.java        | 119 ++++++
 52 files changed, 1314 insertions(+), 661 deletions(-)

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 bb7781e..249ad3e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -208,6 +208,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		ret.allocateDenseBlock();
 		decompress(ret);
 
+		ret.examSparsity();
 		if(DMLScript.STATISTICS || LOG.isDebugEnabled()) {
 			double t = time.stop();
 			LOG.debug("decompressed block w/ k=" + 1 + " in " + t + "ms.");
@@ -256,6 +257,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		ret.allocateDenseBlock();
 		decompress(ret, k);
 
+		ret.examSparsity();
 		if(DMLScript.STATISTICS || LOG.isDebugEnabled()) {
 			double t = time.stop();
 			LOG.debug("decompressed block w/ k=" + k + " in " + time.stop() + "ms.");
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 bb0cf8a..1756018 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -28,6 +28,7 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder;
+import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.PartitionerType;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupEmpty;
@@ -161,18 +162,18 @@ public class CompressedMatrixBlockFactory {
 			_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
 
 		logPhase();
-		long memoryEstimate = sizeInfos.memoryEstimate();
 
-		if(memoryEstimate < _stats.originalSize)
+		if(_stats.estimatedSizeCols < _stats.originalSize || compSettings.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
 			coCodePhase(sizeEstimator, sizeInfos, mb.getNumRows());
 		else {
-			LOG.info("Estimated Size of singleColGroups: " + memoryEstimate);
+			LOG.info("Estimated Size of singleColGroups: " + _stats.estimatedSizeCols);
 			LOG.info("Original size                    : " + _stats.originalSize);
 		}
 	}
 
 	private void coCodePhase(CompressedSizeEstimator sizeEstimator, CompressedSizeInfo sizeInfos, int numRows) {
 		coCodeColGroups = PlanningCoCoder.findCoCodesByPartitioning(sizeEstimator, sizeInfos, numRows, k, compSettings);
+		_stats.estimatedSizeCoCoded = coCodeColGroups.memoryEstimate();
 		logPhase();
 	}
 
@@ -206,6 +207,7 @@ public class CompressedMatrixBlockFactory {
 
 	private void compressPhase() {
 		res.allocateColGroupList(ColGroupFactory.compressColGroups(mb, coCodeColGroups, compSettings, k));
+		_stats.compressedInitialSize = res.getInMemorySize();
 		logPhase();
 	}
 
@@ -229,6 +231,7 @@ public class CompressedMatrixBlockFactory {
 			o.add(combineConst(c));
 
 		res.allocateColGroupList(o);
+
 		logPhase();
 	}
 
@@ -276,12 +279,11 @@ public class CompressedMatrixBlockFactory {
 	private void cleanupPhase() {
 
 		res.cleanupBlock(true, true);
-		mb.cleanupBlock(true, true);
 
 		_stats.size = res.estimateCompressedSizeInMemory();
 		
 		final double ratio = _stats.getRatio();
-		if(ratio < 1) {
+		if(ratio < 1 && compSettings.columnPartitioner != PartitionerType.COST_MATRIX_MULT)  {
 			LOG.info("--dense size:        " + _stats.denseSize);
 			LOG.info("--original size:     " + _stats.originalSize);
 			LOG.info("--compressed size:   " + _stats.size);
@@ -291,6 +293,8 @@ public class CompressedMatrixBlockFactory {
 			return;
 		}
 
+		mb.cleanupBlock(true, true);
+
 		_stats.setColGroupsCounts(res.getColGroups());
 
 		logPhase();
@@ -311,10 +315,12 @@ public class CompressedMatrixBlockFactory {
 			switch(phase) {
 				case 0:
 					LOG.debug("--compression phase " + phase + " Classify  : " + getLastTimePhase());
+					LOG.debug("--Individual Columns Estimated Compression: " + _stats.estimatedSizeCols);
 					break;
 				case 1:
 					LOG.debug("--compression phase " + phase + " Grouping  : " + getLastTimePhase());
 					LOG.debug("Grouping using: " + compSettings.columnPartitioner);
+					LOG.debug("--Cocoded Columns estimated Compression:" + _stats.estimatedSizeCoCoded);
 					break;
 				case 2:
 					LOG.debug("--compression phase " + phase + " Transpose : " + getLastTimePhase());
@@ -324,6 +330,7 @@ public class CompressedMatrixBlockFactory {
 					LOG.debug("--compression phase " + phase + " Compress  : " + getLastTimePhase());
 					LOG.debug("--compression Hash collisions:" + DblArrayIntListHashMap.hashMissCount);
 					DblArrayIntListHashMap.hashMissCount = 0;
+					LOG.debug("--compressed initial actual size:" + _stats.compressedInitialSize);
 					break;
 				case 4:
 					LOG.debug("--compression phase " + phase + " Share     : " + getLastTimePhase());
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 ddbc60b..895600d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -97,10 +97,15 @@ public class CompressionSettings {
 	 */
 	public final EnumSet<CompressionType> validCompressions;
 
+	/**
+	 * The minimum size of the sample extracted.
+	 */
+	public final int minimumSampleSize;
+
 	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput,
 		boolean skipList, int seed, boolean investigateEstimate, boolean lossy,
 		EnumSet<CompressionType> validCompressions, boolean sortValuesByLength, PartitionerType columnPartitioner,
-		int maxColGroupCoCode, double coCodePercentage) {
+		int maxColGroupCoCode, double coCodePercentage, int minimumSampleSize) {
 		this.samplingRatio = samplingRatio;
 		this.allowSharedDictionary = allowSharedDictionary;
 		this.transposeInput = transposeInput;
@@ -113,6 +118,7 @@ public class CompressionSettings {
 		this.columnPartitioner = columnPartitioner;
 		this.maxColGroupCoCode = maxColGroupCoCode;
 		this.coCodePercentage = coCodePercentage;
+		this.minimumSampleSize = minimumSampleSize;
 		LOG.debug(this);
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index 216267e..83d01e5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -43,6 +43,7 @@ public class CompressionSettingsBuilder {
 	private PartitionerType columnPartitioner;
 	private int maxStaticColGroupCoCode = 10000;
 	private double coCodePercentage = 0.01;
+	private int minimumSampleSize = 2000;
 
 	private final static double defaultSampleRate = 0.01;
 
@@ -255,6 +256,18 @@ public class CompressionSettingsBuilder {
 	}
 
 	/**
+	 * Set the minimum sample size to extract from a given matrix, this overrules the sample percentage if the sample
+	 * percentage extracted is lower than this minimum bound.
+	 * 
+	 * @param minimumSampleSize The minimum sample size to extract
+	 * @return The CompressionSettingsBuilder
+	 */
+	public CompressionSettingsBuilder setMinimumSampleSize(int minimumSampleSize) {
+		this.minimumSampleSize = minimumSampleSize;
+		return this;
+	}
+
+	/**
 	 * Create the CompressionSettings object to use in the compression.
 	 * 
 	 * @return The CompressionSettings
@@ -262,6 +275,6 @@ public class CompressionSettingsBuilder {
 	public CompressionSettings create() {
 		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, skipList, seed,
 			investigateEstimate, lossy, validCompressions, sortValuesByLength, columnPartitioner,
-			maxStaticColGroupCoCode, coCodePercentage);
+			maxStaticColGroupCoCode, coCodePercentage, minimumSampleSize);
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
index 5c7d815..466953a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
@@ -31,10 +31,16 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup;
  */
 public class CompressionStatistics {
 
+	// sizes while compressing
+	public long estimatedSizeCoCoded;
+	public long estimatedSizeCols;
+	public long compressedInitialSize;
+
+	// sizes before compression
 	public long originalSize;
 	public long denseSize;
-	public long estimatedSizeColGroups;
-	public long estimatedSizeCols;
+
+	// compressed size
 	public long size;
 
 	private Map<String, int[]> colGroupCounts;
@@ -74,20 +80,20 @@ public class CompressionStatistics {
 
 		for(String ctKey : colGroupCounts.keySet())
 			sb.append(ctKey + ":" + colGroupCounts.get(ctKey)[0] + " ");
-		
+
 		return sb.toString();
 	}
 
 	public String getGroupsSizesString() {
 		StringBuilder sb = new StringBuilder();
 
-		for(String ctKey : colGroupCounts.keySet()) 
+		for(String ctKey : colGroupCounts.keySet())
 			sb.append(ctKey + ":" + colGroupCounts.get(ctKey)[1] + " ");
-		
+
 		return sb.toString();
 	}
 
-	public double getRatio(){
+	public double getRatio() {
 		return (double) originalSize / size;
 	}
 
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 1ede652..124e3a4 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
@@ -32,12 +32,10 @@ public abstract class AColumnCoCoder {
 
 	final protected CompressedSizeEstimator _est;
 	final protected CompressionSettings _cs;
-	// final protected int _numRows;
 
-	protected AColumnCoCoder(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
+	protected AColumnCoCoder(CompressedSizeEstimator sizeEstimator, CompressionSettings cs) {
 		_est = sizeEstimator;
 		_cs = cs;
-		// _numRows = numRows;
 	}
 
 	/**
@@ -57,7 +55,7 @@ public abstract class AColumnCoCoder {
 	protected CompressedSizeInfoColGroup joinWithAnalysis(CompressedSizeInfoColGroup lhs,
 		CompressedSizeInfoColGroup rhs) {
 		int[] joined = Util.join(lhs.getColumns(), rhs.getColumns());
-		return _est.estimateCompressedColGroupSize(joined);
+		return _est.estimateCompressedColGroupSize(joined,( lhs.getNumVals() + 1) * (rhs.getNumVals() + 1));
 	}
 
 	protected CompressedSizeInfoColGroup joinWithoutAnalysis(CompressedSizeInfoColGroup lhs,
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 0116edc..e6dace1 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
@@ -46,8 +46,8 @@ public class CoCodeBinPacking extends AColumnCoCoder {
 	 */
 	public static double BIN_CAPACITY = 0.000032;
 
-	protected CoCodeBinPacking(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
-		super(sizeEstimator, cs, numRows);
+	protected CoCodeBinPacking(CompressedSizeEstimator sizeEstimator, CompressionSettings cs) {
+		super(sizeEstimator, cs);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
index 66ad209..3f7185a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
@@ -49,8 +49,8 @@ public class CoCodeCost extends AColumnCoCoder {
 
 	private final static int toSmallForAnalysis = 64;
 
-	protected CoCodeCost(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
-		super(sizeEstimator, cs, numRows);
+	protected CoCodeCost(CompressedSizeEstimator sizeEstimator, CompressionSettings cs) {
+		super(sizeEstimator, cs);
 		largestDistinct = Math.min(4096, Math.max(256, (int) (sizeEstimator.getNumRows() * cs.coCodePercentage)));
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
index 910c94a..836e4d0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
@@ -26,28 +26,35 @@ import java.util.PriorityQueue;
 import java.util.Queue;
 
 import org.apache.sysds.runtime.compress.CompressionSettings;
-import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorSample;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 
-/**
- * Column group partitioning by number of distinct items estimated. This allows us to join columns based on the worst
- * case estimate of the joined sizes. Then once we decide to join, if the worst case is okay, we then analyze the actual
- * cardinality of the join.
- * 
- * This method allows us to compress many more columns than the BinPacking
- * 
- */
 public class CoCodeCostMatrixMult extends AColumnCoCoder {
 
-	protected CoCodeCostMatrixMult(CompressedSizeEstimator e, CompressionSettings cs, int numRows) {
-		super(e, cs, numRows);
+	protected CoCodeCostMatrixMult(CompressedSizeEstimator e, CompressionSettings cs) {
+		super(e, cs);
 	}
 
 	@Override
 	protected CompressedSizeInfo coCodeColumns(CompressedSizeInfo colInfos, int k) {
-		colInfos.setInfo(join(colInfos.getInfo()));
+
+		List<CompressedSizeInfoColGroup> joinRes = join(colInfos.getInfo());
+
+		if(_cs.samplingRatio < 0.1 && _est instanceof CompressedSizeEstimatorSample) {
+			LOG.debug("Performing second join with double sample rate");
+			CompressedSizeEstimatorSample estS = (CompressedSizeEstimatorSample) _est;
+			estS.sampleData(estS.getSample().getNumRows() * 2);
+			List<int[]> colG = new ArrayList<>(joinRes.size());
+			for(CompressedSizeInfoColGroup g : joinRes)
+				colG.add(g.getColumns());
+
+			joinRes = join(estS.computeCompressedSizeInfos(colG, k));
+		}
+
+		colInfos.setInfo(joinRes);
+
 		return colInfos;
 	}
 
@@ -66,6 +73,8 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 					final CostOfJoin r = que.poll();
 					final double costIndividual = (l.cost + r.cost);
 					final CostOfJoin g = new CostOfJoin(joinWithAnalysis(l.elm, r.elm));
+					if(LOG.isDebugEnabled())
+						LOG.debug("\nl:      " + l + "\nr:      " + r + "\njoined: " + g);
 					if(g.cost < costIndividual)
 						que.add(g);
 					else {
@@ -81,6 +90,7 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 			else
 				break;
 		}
+
 		for(CostOfJoin g : que)
 			ret.add(g.elm);
 
@@ -94,30 +104,16 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 		protected CostOfJoin(CompressedSizeInfoColGroup elm) {
 			this.elm = elm;
 
-			final double constantOverheadForColGroup = 5;
-			final double nCols = elm.getColumns().length;
+			final int nCols = elm.getColumns().length;
 			final double nRows = _est.getNumRows();
-			if(elm.getBestCompressionType() == CompressionType.UNCOMPRESSED)
-				this.cost = nRows * nCols * 2 + constantOverheadForColGroup;
-			else {
-				final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-
-				// LOG.error(constantOverheadForColGroup);
-				final double commonFraction = elm.getMostCommonFraction();
-				final double rowsCost = commonFraction > 0.2 ? nRows * (1 - commonFraction) : nRows;
-				// this.cost = rowsToProcess + elm.getNumVals() * nCols + constantOverheadForColGroup;
-				// this.cost = rowsToProcess + elm.getNumVals() * nCols * (1 - commonFraction) +
-				// constantOverheadForColGroup;
-				// final double sparsity_tuple_effect = elm.getTupleSparsity() > 0.4 ? 1 -
-				// Math.min(elm.getTupleSparsity(), 0.9) : 1;
-				final int numberTuples = elm.getNumVals();
-				final double tuplesCost = (numberTuples < blksz) ? numberTuples : numberTuples * 2;
-
-				// this.cost = elementsCost;
-				// this.cost = rowsCost + tuplesCost * sparsity_tuple_effect + constantOverheadForColGroup;
-
-				this.cost = rowsCost + tuplesCost + constantOverheadForColGroup;
-			}
+			final double preAggregateCost = nRows;
+
+			final int numberTuples = elm.getNumVals();
+			final double tupleSparsity = elm.getTupleSparsity();
+			final double postScalingCost = (nCols > 1 && elm.getTupleSparsity() > 0.4) ? numberTuples *
+				nCols : numberTuples * nCols * tupleSparsity;
+
+			this.cost = preAggregateCost + postScalingCost;
 		}
 
 		@Override
@@ -128,10 +124,14 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 		@Override
 		public String toString() {
 			StringBuilder sb = new StringBuilder();
-			sb.append("\n");
 			sb.append(cost);
 			sb.append(" - ");
+			sb.append(elm.getBestCompressionType());
+			sb.append(" nrVals: ");
+			sb.append(elm.getNumVals());
+			sb.append(" ");
 			sb.append(Arrays.toString(elm.getColumns()));
+
 			return sb.toString();
 		}
 	}
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 06e01b4..674a819 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
@@ -29,8 +29,8 @@ import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
  */
 public class CoCodeStatic extends AColumnCoCoder {
 
-	protected CoCodeStatic(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
-		super(sizeEstimator, cs, numRows);
+	protected CoCodeStatic(CompressedSizeEstimator sizeEstimator, CompressionSettings cs) {
+		super(sizeEstimator, cs);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
index 94572c9..3bae0e5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
@@ -67,7 +67,7 @@ public class PlanningCoCoder {
 			List<CompressedSizeInfoColGroup> newGroups = new ArrayList<>();
 			mem = new Memorizer();
 			for(CompressedSizeInfoColGroup g : colInfos.getInfo()) {
-				if(g.getBestCompressionType() == CompressionType.CONST)
+				if(g.getBestCompressionType(cs) == CompressionType.CONST)
 					constantGroups.add(g);
 				else {
 					mem.put(g);
@@ -93,13 +93,13 @@ public class PlanningCoCoder {
 		CompressionSettings cs, int numRows) {
 		switch(type) {
 			case BIN_PACKING:
-				return new CoCodeBinPacking(est, cs, numRows);
+				return new CoCodeBinPacking(est, cs);
 			case STATIC:
-				return new CoCodeStatic(est, cs, numRows);
+				return new CoCodeStatic(est, cs);
 			case COST:
-				return new CoCodeCost(est, cs, numRows);
+				return new CoCodeCost(est, cs);
 			case COST_MATRIX_MULT:
-				return new CoCodeCostMatrixMult(est, cs, numRows);
+				return new CoCodeCostMatrixMult(est, cs);
 			default:
 				throw new RuntimeException("Unsupported column group partitioner: " + type.toString());
 		}
@@ -186,7 +186,7 @@ public class PlanningCoCoder {
 				break;
 		}
 
-		LOG.error(mem.stats());
+		LOG.debug(mem.stats());
 		mem.resetStats();
 
 		List<CompressedSizeInfoColGroup> ret = new ArrayList<>(workset.size());
@@ -226,9 +226,9 @@ public class PlanningCoCoder {
 			if(g == null) {
 				final CompressedSizeInfoColGroup left = mem.get(new ColIndexes(c1));
 				final CompressedSizeInfoColGroup right = mem.get(new ColIndexes(c2));
-				final boolean leftConst = left.getBestCompressionType() == CompressionType.CONST &&
+				final boolean leftConst = left.getBestCompressionType(cs) == CompressionType.CONST &&
 					left.getNumOffs() == 0;
-				final boolean rightConst = right.getBestCompressionType() == CompressionType.CONST &&
+				final boolean rightConst = right.getBestCompressionType(cs) == CompressionType.CONST &&
 					right.getNumOffs() == 0;
 				if(leftConst)
 					g = CompressedSizeInfoColGroup.addConstGroup(c, right, cs.validCompressions);
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 ddc6195..6b74136 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
@@ -32,6 +32,7 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysds.utils.MemoryEstimates;
 
 import edu.emory.mathcs.backport.java.util.Arrays;
 
@@ -143,6 +144,15 @@ public abstract class AColGroup implements Serializable {
 	public abstract int getNumRows();
 
 	/**
+	 * Obtain number of distinct tuples in contained sets of values associated with this column group.
+	 * 
+	 * If the column group is uncompressed the number or rows is returned.
+	 * 
+	 * @return the number of distinct sets of values associated with the bitmaps in this column group
+	 */
+	public abstract int getNumValues();
+
+	/**
 	 * Obtain the number of columns in this column group.
 	 * 
 	 * @return number of columns in this column group
@@ -183,7 +193,11 @@ public abstract class AColGroup implements Serializable {
 	 * 
 	 * @return an upper bound on the number of bytes used to store this ColGroup in memory.
 	 */
-	public abstract long estimateInMemorySize();
+	public long estimateInMemorySize(){
+		long size = 16; // object header
+		size += MemoryEstimates.intArrayCost(_colIndexes.length);
+		return size;
+	}
 
 	/**
 	 * Decompress the contents of this column group into the specified full matrix block.
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
index c5d29ba..3b598e6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
@@ -58,8 +58,6 @@ public abstract class ColGroupCompressed extends AColGroup {
 		_numRows = numRows;
 	}
 
-	public abstract int getNumValues();
-
 	public abstract double[] getValues();
 
 	public abstract void addMinMax(double[] ret);
@@ -147,4 +145,10 @@ public abstract class ColGroupCompressed extends AColGroup {
 		return _numRows;
 	}
 
+	@Override
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		size += 4;
+		return size;
+	}
 }
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 8f6299c..e3c2965 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
@@ -113,11 +113,6 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public long estimateInMemorySize() {
-		return ColGroupSizes.estimateInMemorySizeCONST(getNumCols(), getNumValues(), isLossy());
-	}
-
-	@Override
 	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
 		decompressToBlockUnSafe(target, rl, ru, offT, values);
 		target.setNonZeros(_colIndexes.length * target.getNumRows() + target.getNonZeros());
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 df45f56..fa967da 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
@@ -555,8 +555,9 @@ public class ColGroupDDC extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		return ColGroupSizes.estimateInMemorySizeDDC(getNumCols(), getNumValues(), _numRows, isLossy());
-
+		long size = super.estimateInMemorySize();
+		size += _data.getInMemorySize();
+		return size;
 	}
 
 	@Override
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 c873b1a..be33491 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
@@ -83,10 +83,6 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		return ColGroupType.EMPTY;
 	}
 
-	@Override
-	public long estimateInMemorySize() {
-		return ColGroupSizes.estimateInMemorySizeEMPTY(getNumCols());
-	}
 
 	@Override
 	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
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 62acf71..ffcc2fb 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
@@ -34,9 +34,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.PartitionerType;
 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.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.tree.AInsertionSorter;
@@ -47,7 +48,6 @@ import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.compress.lib.BitmapEncoder;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
-import org.apache.sysds.runtime.compress.utils.Bitmap;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -155,8 +155,9 @@ public class ColGroupFactory {
 
 	private static Collection<AColGroup> compressColGroup(MatrixBlock in, int[] colIndexes,
 		CompressionSettings compSettings) {
-		if(in.isInSparseFormat() && compSettings.transposed) {
-
+		if(in.isEmpty())
+			return Collections.singletonList(new ColGroupEmpty(colIndexes, compSettings.transposed ? in.getNumColumns(): in.getNumRows()));
+		else if(in.isInSparseFormat() && compSettings.transposed) {
 			final SparseBlock sb = in.getSparseBlock();
 			for(int col : colIndexes)
 				if(sb.isEmpty(col))
@@ -191,7 +192,7 @@ public class ColGroupFactory {
 
 	private static AColGroup compressColGroupForced(MatrixBlock in, int[] colIndexes,
 		CompressionSettings compSettings) {
-			
+
 		ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, in, compSettings.transposed);
 
 		CompressedSizeEstimator estimator = new CompressedSizeEstimatorExact(in, compSettings);
@@ -200,7 +201,8 @@ public class ColGroupFactory {
 			estimator.estimateCompressedColGroupSize(ubm, colIndexes), compSettings.validCompressions);
 
 		int numRows = compSettings.transposed ? in.getNumColumns() : in.getNumRows();
-		return compress(colIndexes, numRows, ubm, sizeInfo.getBestCompressionType(), compSettings, in);
+		return compress(colIndexes, numRows, ubm, sizeInfo.getBestCompressionType(compSettings), compSettings, in,
+			sizeInfo.getTupleSparsity());
 	}
 
 	// private static AColGroup compressColGroupCorrecting(MatrixBlock in,
@@ -275,17 +277,21 @@ public class ColGroupFactory {
 	 * @param compType       The CompressionType selected
 	 * @param cs             The compression Settings used for the given compression
 	 * @param rawMatrixBlock The copy of the original input (maybe transposed) MatrixBlock
+	 * @param tupleSparsity  The sparsity of the ubs entries.
 	 * @return A Compressed ColGroup
 	 */
 	public static AColGroup compress(int[] colIndexes, int rlen, ABitmap ubm, CompressionType compType,
-		CompressionSettings cs, MatrixBlock rawMatrixBlock) {
+		CompressionSettings cs, MatrixBlock rawMatrixBlock, double tupleSparsity) {
+
+		if(compType == CompressionType.UNCOMPRESSED && cs.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
+			compType = CompressionType.DDC;
 
 		final IntArrayList[] of = ubm.getOffsetList();
 
 		if(of == null)
 			return new ColGroupEmpty(colIndexes, rlen);
 		else if(of.length == 1 && of[0].size() == rlen)
-			return new ColGroupConst(colIndexes, rlen, ADictionary.getDictionary(ubm));
+			return new ColGroupConst(colIndexes, rlen, DictionaryFactory.create(ubm));
 
 		if(LOG.isTraceEnabled())
 			LOG.trace("compressing to: " + compType);
@@ -295,13 +301,13 @@ public class ColGroupFactory {
 
 			switch(compType) {
 				case DDC:
-					return compressDDC(colIndexes, rlen, ubm, cs);
+					return compressDDC(colIndexes, rlen, ubm, cs, tupleSparsity);
 				case RLE:
-					return compressRLE(colIndexes, rlen, ubm, cs);
+					return compressRLE(colIndexes, rlen, ubm, cs, tupleSparsity);
 				case OLE:
-					return compressOLE(colIndexes, rlen, ubm, cs);
+					return compressOLE(colIndexes, rlen, ubm, cs, tupleSparsity);
 				case SDC:
-					return compressSDC(colIndexes, rlen, ubm, cs);
+					return compressSDC(colIndexes, rlen, ubm, cs, tupleSparsity);
 				case UNCOMPRESSED:
 					return new ColGroupUncompressed(colIndexes, rawMatrixBlock, cs.transposed);
 				default:
@@ -316,7 +322,8 @@ public class ColGroupFactory {
 		}
 	}
 
-	private static AColGroup compressSDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+	private static AColGroup compressSDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
+		double tupleSparsity) {
 
 		int numZeros = (int) ((long) rlen - (int) ubm.getNumOffsets());
 		int largestOffset = 0;
@@ -330,17 +337,17 @@ public class ColGroupFactory {
 			index++;
 		}
 		AColGroup cg;
-		ADictionary dict = new Dictionary(((Bitmap) ubm).getValues());
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
 		if(numZeros >= largestOffset && ubm.getOffsetList().length == 1)
 			cg = new ColGroupSDCSingleZeros(colIndexes, rlen, dict, ubm.getOffsetList()[0].extractValues(true), null);
 		else if(ubm.getOffsetList().length == 1) {// todo
-			dict = moveFrequentToLastDictionaryEntry(dict, ubm, rlen, largestIndex);
+			dict = DictionaryFactory.moveFrequentToLastDictionaryEntry(dict, ubm, rlen, largestIndex);
 			cg = setupSingleValueSDCColGroup(colIndexes, rlen, ubm, dict);
 		}
 		else if(numZeros >= largestOffset)
 			cg = setupMultiValueZeroColGroup(colIndexes, ubm, rlen, dict);
 		else {
-			dict = moveFrequentToLastDictionaryEntry(dict, ubm, rlen, largestIndex);
+			dict = DictionaryFactory.moveFrequentToLastDictionaryEntry(dict, ubm, rlen, largestIndex);
 			cg = setupMultiValueColGroup(colIndexes, numZeros, largestOffset, ubm, rlen, largestIndex, dict);
 		}
 		return cg;
@@ -397,62 +404,17 @@ public class ColGroupFactory {
 		return new ColGroupSDCSingle(colIndexes, numRows, dict, _indexes, null);
 	}
 
-	private static ADictionary moveFrequentToLastDictionaryEntry(ADictionary dict, ABitmap ubm, int numRows,
-		int largestIndex) {
-		final double[] dictValues = dict.getValues();
-		final int zeros = numRows - (int) ubm.getNumOffsets();
-		final int nCol = ubm.getNumColumns();
-		final int offsetToLargest = largestIndex * nCol;
-
-		if(zeros == 0) {
-			final double[] swap = new double[nCol];
-			System.arraycopy(dictValues, offsetToLargest, swap, 0, nCol);
-			for(int i = offsetToLargest; i < dictValues.length - nCol; i++) {
-				dictValues[i] = dictValues[i + nCol];
-			}
-			System.arraycopy(swap, 0, dictValues, dictValues.length - nCol, nCol);
-			return dict;
-		}
-
-		final int largestIndexSize = ubm.getOffsetsList(largestIndex).size();
-		final double[] newDict = new double[dictValues.length + nCol];
-
-		if(zeros > largestIndexSize)
-			System.arraycopy(dictValues, 0, newDict, 0, dictValues.length);
-		else {
-			System.arraycopy(dictValues, 0, newDict, 0, offsetToLargest);
-			System.arraycopy(dictValues, offsetToLargest + nCol, newDict, offsetToLargest,
-				dictValues.length - offsetToLargest - nCol);
-			System.arraycopy(dictValues, offsetToLargest, newDict, newDict.length - nCol, nCol);
-		}
-		return new Dictionary(newDict);
-	}
-
-	private static AColGroup compressDDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+	private static AColGroup compressDDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
+		double tupleSparsity) {
 
 		boolean _zeros = ubm.getNumOffsets() < (long) rlen;
-		ADictionary dict = ADictionary.getDictionary(ubm);
-		double[] values = dict.getValues();
-		if(_zeros) {
-			double[] appendedZero = new double[values.length + colIndexes.length];
-			System.arraycopy(values, 0, appendedZero, 0, values.length);
-			dict = new Dictionary(appendedZero);
-		}
-		else
-			dict = new Dictionary(values);
-
+		ADictionary dict = (_zeros) ? DictionaryFactory.createWithAppendedZeroTuple(ubm,
+			tupleSparsity) : DictionaryFactory.create(ubm, tupleSparsity);
 		int numVals = ubm.getNumValues();
 		AMapToData _data = MapToFactory.create(rlen, numVals + (_zeros ? 1 : 0));
 		if(_zeros)
 			_data.fill(numVals);
 
-		// for(int i = 0; i < numVals; i++) {
-		// int[] tmpList = ubm.getOffsetsList(i).extractValues();
-		// int tmpListSize = ubm.getNumOffsets(i);
-		// for(int k = 0; k < tmpListSize; k++)
-		// _data[tmpList[k]] = (char) i;
-		// }
-
 		for(int i = 0; i < numVals; i++) {
 			IntArrayList tmpList = ubm.getOffsetsList(i);
 			final int sz = tmpList.size();
@@ -464,9 +426,10 @@ public class ColGroupFactory {
 
 	}
 
-	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
+		double tupleSparsity) {
 
-		ADictionary dict = ADictionary.getDictionary(ubm);
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
 		ColGroupOLE ole = new ColGroupOLE(rlen);
 
 		final int numVals = ubm.getNumValues();
@@ -485,9 +448,10 @@ public class ColGroupFactory {
 		return ole;
 	}
 
-	private static AColGroup compressRLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+	private static AColGroup compressRLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
+		double tupleSparsity) {
 
-		ADictionary dict = ADictionary.getDictionary(ubm);
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity);
 		ColGroupRLE rle = new ColGroupRLE(rlen);
 		// compress the bitmaps
 		final int numVals = ubm.getNumValues();
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 2df2878..bb4f325 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
@@ -329,11 +329,6 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public long estimateInMemorySize() {
-		return ColGroupSizes.estimateInMemorySizeOLE(getNumCols(), getNumValues(), _data.length, _numRows, isLossy());
-	}
-
-	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		double val0 = op.executeScalar(0);
 
@@ -1209,7 +1204,7 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
index d33e816..7b7c595 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
@@ -27,6 +27,7 @@ import java.util.Arrays;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.functionobjects.Builtin;
+import org.apache.sysds.utils.MemoryEstimates;
 
 /**
  * Base class for column groups encoded with various types of bitmap encoding.
@@ -75,11 +76,10 @@ public abstract class ColGroupOffset extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		// Could use a ternary operator, but it looks odd with our code formatter here.
-
-		return ColGroupSizes.estimateInMemorySizeOffset(getNumCols(), getNumValues(), _ptr.length, _data.length,
-			isLossy());
-
+		long size = super.estimateInMemorySize();
+		size += MemoryEstimates.intArrayCost(_ptr.length);
+		size += MemoryEstimates.charArrayCost(_data.length);
+		return size;
 	}
 
 	protected final void sumAllValues(double[] b, double[] c) {
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 e449259..d2cebf5 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
@@ -330,7 +330,7 @@ public class ColGroupSDC extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(), isLossy());
+		long size = super.estimateInMemorySize();
 		size += _indexes.getInMemorySize();
 		size += _data.getInMemorySize();
 		return size;
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 58649c3..1424072 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
@@ -314,7 +314,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(), isLossy());
+		long size = super.estimateInMemorySize();
 		size += _indexes.getInMemorySize();
 		return size;
 	}
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 94be81c..997c42a 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
@@ -187,10 +187,18 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		final double vals = _dict.aggregateTuples(builtin, _colIndexes.length)[0];
 		final AIterator it = _indexes.getIterator();
 		it.skipTo(rl);
-		while(it.hasNext() && it.value() < ru) {
-			final int idx = it.value();
-			it.next();
-			c[idx] = builtin.execute(c[idx], vals);
+		int rix = rl;
+		for(; rix < ru && it.hasNext(); rix++) {
+			if(it.value() != rix)
+				c[rix] = builtin.execute(c[rix], 0);
+			else {
+				c[rix] = builtin.execute(c[rix], vals);
+				it.next();
+			}
+		}
+
+		for(; rix < ru; rix++) {
+			c[rix] = builtin.execute(c[rix], 0);
 		}
 	}
 
@@ -260,7 +268,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(), isLossy());
+		long size = super.estimateInMemorySize();
 		size += _indexes.getInMemorySize();
 		return size;
 	}
@@ -450,7 +458,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
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 23b06a9..77cf6e2 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
@@ -198,9 +198,17 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		final double[] vals = _dict.aggregateTuples(builtin, _colIndexes.length);
 		final AIterator it = _indexes.getIterator();
 		it.skipTo(rl);
-		while(it.hasNext() && it.value() < ru) {
-			final int idx = it.value();
-			c[idx] = builtin.execute(c[idx], vals[getIndex(it.getDataIndexAndIncrement())]);
+
+		int rix = rl;
+		for(; rix < ru && it.hasNext(); rix++) {
+			if(it.value() != rix)
+				c[rix] = builtin.execute(c[rix], 0);
+			else
+				c[rix] = builtin.execute(c[rix], vals[_data.getIndex(it.getDataIndexAndIncrement())]);
+		}
+
+		for(; rix < ru; rix++) {
+			c[rix] = builtin.execute(c[rix], 0);
 		}
 	}
 
@@ -277,7 +285,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(), isLossy());
+		long size = super.estimateInMemorySize();
 		size += _indexes.getInMemorySize();
 		size += _data.getInMemorySize();
 		return size;
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 3c8c50a..faa3bb3 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
@@ -32,67 +32,68 @@ public class ColGroupSizes {
 	protected static final Log LOG = LogFactory.getLog(ColGroupSizes.class.getName());
 
 	public static long estimateInMemorySizeGroup(int nrColumns) {
-		long size = 0;
-		size += 16; // Object header
+		long size = 16; // Object header
 		size += MemoryEstimates.intArrayCost(nrColumns);
 		return size;
 	}
 
-	public static long estimateInMemorySizeGroupValue(int nrColumns, int nrValues, boolean lossy) {
-		long size = estimateInMemorySizeGroup(nrColumns);
+	public static long estimateInMoemorySizeCompressedColumn(int nrColumns) {
+		return estimateInMemorySizeGroup(nrColumns) + 4; // 4 for num Rows;
+	}
+
+	public static long estimateInMemorySizeGroupValue(int nrColumns, int nrValues, double tupleSparsity,
+		boolean lossy) {
+		long size = estimateInMoemorySizeCompressedColumn(nrColumns);
 		// LOG.error("MemorySize Group Value: " + nrColumns + " " + nrValues + " " + lossy);
 		size += 8; // Dictionary Reference.
 		size += 8; // Counts reference
-		size += 4; // int numRows
 		size += 1; // _zeros boolean reference
 		size += 1; // _lossy boolean reference
 		size += 2; // padding
-		size += DictionaryFactory.getInMemorySize(nrValues, nrColumns, lossy);
+		size += DictionaryFactory.getInMemorySize(nrValues, nrColumns, tupleSparsity, lossy);
 		return size;
 	}
 
-	public static long estimateInMemorySizeDDC(int nrCols, int numTuples, int dataLength, boolean lossy) {
+	public static long estimateInMemorySizeDDC(int nrCols, int numTuples, int dataLength, double tupleSparsity,
+		boolean lossy) {
 		// LOG.error("Arguments for DDC memory Estimate " + nrCols + " " + numTuples + " " + dataLength + " " + lossy);
-		long size = estimateInMemorySizeGroupValue(nrCols, numTuples, lossy);
-		size += 8; // Map toFactory reference;
+		long size = estimateInMemorySizeGroupValue(nrCols, numTuples, tupleSparsity, lossy);
 		size += MapToFactory.estimateInMemorySize(dataLength, numTuples);
 		return size;
 	}
 
 	public static long estimateInMemorySizeOffset(int nrColumns, int nrValues, int pointers, int offsetLength,
-		boolean lossy) {
+		double tupleSparsity, boolean lossy) {
 		// LOG.error("Offset Size: " + nrColumns + " " + nrValues + " " + pointers + " " + offsetLength);
-		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, lossy);
+		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, tupleSparsity, lossy);
 		size += MemoryEstimates.intArrayCost(pointers);
 		size += MemoryEstimates.charArrayCost(offsetLength);
 		return size;
 	}
 
 	public static long estimateInMemorySizeOLE(int nrColumns, int nrValues, int offsetLength, int nrRows,
-		boolean lossy) {
+		double tupleSparsity, boolean lossy) {
 		// LOG.error(nrColumns + " " + nrValues + " " + offsetLength + " " + nrRows + " " + lossy);
 		nrColumns = nrColumns > 0 ? nrColumns : 1;
 		offsetLength += (nrRows / CompressionSettings.BITMAP_BLOCK_SZ) * 2;
-		long size = 0;
-		size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues / nrColumns) + 1, offsetLength, lossy);
-		if(nrRows > CompressionSettings.BITMAP_BLOCK_SZ * 2) {
-			size += MemoryEstimates.intArrayCost((int) nrValues / nrColumns);
-		}
+		long size = estimateInMemorySizeOffset(nrColumns, nrValues, nrValues  + 1, offsetLength,
+			tupleSparsity, lossy);
 		return size;
 	}
 
-	public static long estimateInMemorySizeRLE(int nrColumns, int nrValues, int nrRuns, int nrRows, boolean lossy) {
+	public static long estimateInMemorySizeRLE(int nrColumns, int nrValues, int nrRuns, int nrRows,
+		double tupleSparsity, boolean lossy) {
 		// LOG.error("RLE Size: " + nrColumns + " " + nrValues + " " + nrRuns + " " + nrRows);
 		int offsetLength = (nrRuns) * 2;
-		long size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues) + 1, offsetLength, lossy);
+		long size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues) + 1, offsetLength, tupleSparsity, lossy);
 
 		return size;
 	}
 
 	public static long estimateInMemorySizeSDC(int nrColumns, int nrValues, int nrRows, int largestOff,
-		boolean largestOffIsZero, boolean containNoZeroValues, boolean lossy) {
+		boolean largestOffIsZero, boolean containNoZeroValues, double tupleSparsity, boolean lossy) {
 		long size = estimateInMemorySizeGroupValue(nrColumns,
-			nrValues + (largestOffIsZero || containNoZeroValues ? 0 : 1), lossy);
+			nrValues + (largestOffIsZero || containNoZeroValues ? 0 : 1), tupleSparsity, lossy);
 		// LOG.error("SDC Estimation values: " + nrColumns + " " + nrValues + " " + nrRows + " " + largestOff);
 		size += OffsetFactory.estimateInMemorySize(nrRows - largestOff - 1, nrRows);
 		if(nrValues > 1)
@@ -101,22 +102,20 @@ public class ColGroupSizes {
 	}
 
 	public static long estimateInMemorySizeSDCSingle(int nrColumns, int nrValues, int nrRows, int largestOff,
-		boolean largestOffIsZero, boolean containNoZeroValues, boolean lossy) {
+		boolean largestOffIsZero, boolean containNoZeroValues, double tupleSparsity, boolean lossy) {
 		long size = estimateInMemorySizeGroupValue(nrColumns,
-			nrValues + (largestOffIsZero || containNoZeroValues ? 0 : 1), lossy);
+			nrValues + (largestOffIsZero || containNoZeroValues ? 0 : 1), tupleSparsity, lossy);
 		size += OffsetFactory.estimateInMemorySize(nrRows - largestOff, nrRows);
 		return size;
 	}
 
-	public static long estimateInMemorySizeCONST(int nrColumns, int nrValues, boolean lossy) {
-		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, lossy);
+	public static long estimateInMemorySizeCONST(int nrColumns, int nrValues, double tupleSparsity, boolean lossy) {
+		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, tupleSparsity, lossy);
 		return size;
 	}
 
 	public static long estimateInMemorySizeEMPTY(int nrColumns) {
-		long size = estimateInMemorySizeGroup(nrColumns);
-		size += 8; // null pointer to _dict
-		return size;
+		return estimateInMoemorySizeCompressedColumn(nrColumns);
 	}
 
 	public static long estimateInMemorySizeUncompressed(int nrRows, int nrColumns, double sparsity) {
@@ -124,7 +123,7 @@ public class ColGroupSizes {
 		// Since the Object is a col group the overhead from the Memory Size group is added
 		size += estimateInMemorySizeGroup(nrColumns);
 		size += 8; // reference to MatrixBlock.
-		size += MatrixBlock.estimateSizeInMemory(nrRows, nrColumns, (nrColumns > 1) ?  sparsity : 1);
+		size += MatrixBlock.estimateSizeInMemory(nrRows, nrColumns, (nrColumns > 1) ? sparsity : 1);
 		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 3cec6fb..c8f9bf6 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
@@ -660,4 +660,9 @@ public class ColGroupUncompressed extends AColGroup {
 		LibMatrixMult.matrixMult(_data, right, out, InfrastructureAnalyzer.getLocalParallelism());
 		return new ColGroupUncompressed(outputCols, out, false);
 	}
+
+	@Override
+	public int getNumValues() {
+		return _data.getNumRows();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index fc7da24..a0c127a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -33,6 +33,7 @@ import org.apache.sysds.runtime.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.colgroup.pre.ArrPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
@@ -93,11 +94,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		decompressToBlock(target, rl, ru, offT, getValues());
 	}
 
-	/**
-	 * Obtain number of distinct sets of values associated with the bitmaps in this column group.
-	 * 
-	 * @return the number of distinct sets of values associated with the bitmaps in this column group
-	 */
+	@Override
 	public final int getNumValues() {
 		return _dict.getNumberOfValues(_colIndexes.length);
 	}
@@ -250,22 +247,21 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return aggregateColumns;
 	}
 
-	private Pair<int[], double[]> preaggValuesFromDense(final int numVals, final double[] b, double[] dictVals,
-		final int cl, final int cu, final int cut) {
+	private Pair<int[], double[]> preaggValuesFromDense(final int numVals, final double[] b, final int cl, final int cu,
+		final int cut) {
 
-		int[] aggregateColumns = getAggregateColumnsSetDense(b, cl, cu, cut);
-		double[] ret = new double[numVals * aggregateColumns.length];
+		final int[] aggregateColumns = getAggregateColumnsSetDense(b, cl, cu, cut);
+		final double[] ret = new double[numVals * aggregateColumns.length];
 
 		for(int k = 0, off = 0;
 			k < numVals * _colIndexes.length;
 			k += _colIndexes.length, off += aggregateColumns.length) {
 			for(int h = 0; h < _colIndexes.length; h++) {
 				int idb = _colIndexes[h] * cut;
-				double v = dictVals[k + h];
-				for(int i = 0; i < aggregateColumns.length; i++) {
-					ret[off + i] += v * b[idb + aggregateColumns[i]];
-				}
-
+				double v = _dict.getValue(k + h);
+				if(v != 0)
+					for(int i = 0; i < aggregateColumns.length; i++)
+						ret[off + i] += v * b[idb + aggregateColumns[i]];
 			}
 		}
 
@@ -290,8 +286,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return aggregateColumns;
 	}
 
-	private Pair<int[], double[]> preaggValuesFromSparse(int numVals, SparseBlock b, double[] dictVals, int cl, int cu,
-		int cut) {
+	private Pair<int[], double[]> preaggValuesFromSparse(int numVals, SparseBlock b, int cl, int cu, int cut) {
 
 		int[] aggregateColumns = getAggregateColumnsSetSparse(b);
 
@@ -310,7 +305,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 						for(int j = 0, offOrg = h;
 							j < numVals * aggregateColumns.length;
 							j += aggregateColumns.length, offOrg += _colIndexes.length) {
-							ret[j + retIdx] += dictVals[offOrg] * sValues[i];
+							ret[j + retIdx] += _dict.getValue(offOrg) * sValues[i];
 						}
 				}
 			}
@@ -318,18 +313,18 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return new ImmutablePair<>(aggregateColumns, ret);
 	}
 
-	public Pair<int[], double[]> preaggValues(int numVals, MatrixBlock b, double[] dictVals, int cl, int cu, int cut) {
-		return b.isInSparseFormat() ? preaggValuesFromSparse(numVals, b.getSparseBlock(), dictVals, cl, cu,
-			cut) : preaggValuesFromDense(numVals, b.getDenseBlockValues(), dictVals, cl, cu, cut);
+	public Pair<int[], double[]> preaggForRightMultiplyValues(int numVals, MatrixBlock b, int cl, int cu, int cut) {
+		return b.isInSparseFormat() ? preaggValuesFromSparse(numVals, b.getSparseBlock(), cl, cu,
+			cut) : preaggValuesFromDense(numVals, b.getDenseBlockValues(), cl, cu, cut);
 	}
 
-	protected static double[] sparsePreaggValues(int numVals, double v, boolean allocNew, double[] dictVals) {
-		double[] ret = allocNew ? new double[numVals + 1] : allocDVector(numVals + 1, true);
+	// protected static double[] sparsePreaggValues(int numVals, double v, boolean allocNew, ADictionary dict) {
+	// double[] ret = allocNew ? new double[numVals + 1] : allocDVector(numVals + 1, true);
 
-		for(int k = 0; k < numVals; k++)
-			ret[k] = dictVals[k] * v;
-		return ret;
-	}
+	// for(int k = 0; k < numVals; k++)
+	// ret[k] = dictVals[k] * v;
+	// return ret;
+	// }
 
 	protected double computeMxx(double c, Builtin builtin) {
 		if(_zeros)
@@ -1016,9 +1011,10 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 */
 	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, double[] values, int rl, int ru) {
 		final int numVals = getNumValues();
+		if(!(_dict instanceof MatrixBlockDictionary))
+			_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
 
-		DenseBlock dictV = new DenseBlockFP64(new int[] {numVals, _colIndexes.length}, values);
-		MatrixBlock dictM = new MatrixBlock(numVals, _colIndexes.length, dictV);
+		MatrixBlock dictM = ((MatrixBlockDictionary) _dict).getMatrixBlock();
 		dictM.examSparsity();
 		MatrixBlock tmpRes = new MatrixBlock(1, _colIndexes.length, false);
 		for(int i = rl; i < ru; i++) {
@@ -1064,10 +1060,22 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	public AColGroup rightMultByMatrix(MatrixBlock right) {
-		Pair<int[], double[]> pre = preaggValues(getNumValues(), right, getValues(), 0, right.getNumColumns(),
+		Pair<int[], double[]> pre = preaggForRightMultiplyValues(getNumValues(), right, 0, right.getNumColumns(),
 			right.getNumColumns());
 		if(pre.getLeft().length > 0)
 			return copyAndSet(pre.getLeft(), pre.getRight());
 		return null;
 	}
+
+	@Override
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		size += 8; // Dictionary Reference.
+		size += 8; // Counts reference
+		size += 1; // _zeros boolean reference
+		size += 1; // _lossy boolean reference
+		size += 2; // padding
+		size += _dict.getInMemorySize();
+		return size;
+	}
 }
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 352e96b..e5d2a15 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
@@ -24,9 +24,6 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysds.runtime.compress.utils.ABitmap;
-import org.apache.sysds.runtime.compress.utils.Bitmap;
-import org.apache.sysds.runtime.compress.utils.BitmapLossy;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
@@ -143,13 +140,7 @@ public abstract class ADictionary {
 	 * @param fn         The function to apply to individual columns
 	 * @param colIndexes The mapping to the target columns from the individual columns
 	 */
-	public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
-		int ncol = colIndexes.length;
-		int vlen = size() / ncol;
-		for(int k = 0; k < vlen; k++)
-			for(int j = 0, valOff = k * ncol; j < ncol; j++)
-				c[colIndexes[j]] = fn.execute(c[colIndexes[j]], getValue(valOff + j));
-	}
+	public abstract void aggregateCols(double[] c, Builtin fn, int[] colIndexes);
 
 	/**
 	 * Write the dictionary to a DataOutput.
@@ -178,7 +169,7 @@ public abstract class ADictionary {
 	public abstract boolean isLossy();
 
 	/**
-	 * Get the number of values given that the column group has n columns
+	 * Get the number of distinct tuples given that the column group has n columns
 	 * 
 	 * @param ncol The number of Columns in the ColumnGroup.
 	 * @return the number of value tuples contained in the dictionary.
@@ -254,6 +245,14 @@ public abstract class ADictionary {
 
 	public abstract boolean containsValue(double pattern);
 
+	/**
+	 * Calculate the number of non zeros in the dictionary. The number of non zeros should be scaled with the counts
+	 * given
+	 * 
+	 * @param counts The counts of each dictionary entry
+	 * @param nCol   The number of columns in this dictionary
+	 * @return The nonZero count
+	 */
 	public abstract long getNumberNonZeros(int[] counts, int nCol);
 
 	public abstract long getNumberNonZerosContained();
@@ -268,13 +267,6 @@ public abstract class ADictionary {
 	 */
 	public abstract void addToEntry(Dictionary d, int fr, int to, int nCol);
 
-	public static ADictionary getDictionary(ABitmap ubm) {
-		if(ubm instanceof BitmapLossy)
-			return new QDictionary((BitmapLossy) ubm).makeDoubleDictionary();
-		else
-			return new Dictionary(((Bitmap) ubm).getValues());
-	}
-
 	/**
 	 * Get the most common tuple element contained in the dictionary
 	 * 
@@ -293,4 +285,12 @@ public abstract class ADictionary {
 	 * @return a new instance of dictionary with the tuple subtracted.
 	 */
 	public abstract ADictionary subtractTuple(double[] tuple);
+
+	/**
+	 * Get this dictionary as a matrixBlock dictionary. This allows us to use optimized kernels coded elsewhere in the
+	 * system, such as matrix multiplication.
+	 * 
+	 * @return A Dictionary containing a MatrixBlock.
+	 */
+	public abstract MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol);
 }
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 5a32823..595c250 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
@@ -25,8 +25,11 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.sysds.runtime.DMLCompressionException;
+import org.apache.sysds.runtime.data.DenseBlock;
+import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.ValueFunction;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.utils.MemoryEstimates;
 
@@ -473,4 +476,23 @@ public class Dictionary extends ADictionary {
 		}
 		return new Dictionary(newValues);
 	}
+
+	@Override
+	public MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol) {
+		final int nRow = _values.length / nCol;
+		DenseBlock dictV = new DenseBlockFP64(new int[] {nRow, nCol}, _values);
+		MatrixBlock dictM = new MatrixBlock(nRow, nCol, dictV);
+		dictM.examSparsity();
+		return new MatrixBlockDictionary(dictM);
+	}
+
+	@Override
+	public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
+		int ncol = colIndexes.length;
+		int vlen = size() / ncol;
+		for(int k = 0; k < vlen; k++)
+			for(int j = 0, valOff = k * ncol; j < ncol; j++)
+				c[colIndexes[j]] = fn.execute(c[colIndexes[j]], getValue(valOff + j));
+		
+	}
 }
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 328e145..bab32b0 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
@@ -22,8 +22,20 @@ package org.apache.sysds.runtime.compress.colgroup.dictionary;
 import java.io.DataInput;
 import java.io.IOException;
 
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.runtime.compress.utils.ABitmap;
+import org.apache.sysds.runtime.compress.utils.Bitmap;
+import org.apache.sysds.runtime.compress.utils.BitmapLossy;
+import org.apache.sysds.runtime.compress.utils.MultiColBitmap;
+import org.apache.sysds.runtime.data.SparseBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+
 public class DictionaryFactory {
 
+	protected static final Log LOG = LogFactory.getLog(DictionaryFactory.class.getName());
+
 	public static ADictionary read(DataInput in) throws IOException {
 		boolean lossy = in.readBoolean();
 		if(lossy) {
@@ -46,10 +58,131 @@ public class DictionaryFactory {
 		}
 	}
 
-	public static long getInMemorySize(int nrValues, int nrColumns, boolean lossy) {
+	public static long getInMemorySize(int nrValues, int nrColumns, double tupleSparsity, boolean lossy) {
 		if(lossy)
 			return QDictionary.getInMemorySize(nrValues * nrColumns);
+		else if(nrColumns > 1 && tupleSparsity < 0.4)
+			return MatrixBlockDictionary.getInMemorySize(nrValues, nrColumns, tupleSparsity);
 		else
 			return Dictionary.getInMemorySize(nrValues * nrColumns);
 	}
+
+	public static ADictionary create(ABitmap ubm) {
+		return create(ubm, 1.0);
+	}
+
+	public static ADictionary create(ABitmap ubm, double sparsity) {
+		if(ubm instanceof BitmapLossy)
+			return new QDictionary((BitmapLossy) ubm);
+		else if(ubm instanceof Bitmap)
+			return new Dictionary(((Bitmap) ubm).getValues());
+		else if(sparsity < 0.4 && ubm instanceof MultiColBitmap) {
+			final int nCols = ubm.getNumColumns();
+			final int nRows = ubm.getNumValues();
+			final MultiColBitmap mcbm = (MultiColBitmap) ubm;
+
+			final MatrixBlock m = new MatrixBlock(nRows, nCols, true);
+			m.allocateSparseRowsBlock();
+			final SparseBlock sb = m.getSparseBlock();
+
+			final int nVals = ubm.getNumValues();
+			for(int i = 0; i < nVals; i++) {
+				final double[] tuple = mcbm.getValues(i);
+				for(int col = 0; col < nCols; col++)
+					sb.append(i, col, tuple[col]);
+			}
+			m.recomputeNonZeros();
+			return new MatrixBlockDictionary(m);
+		}
+		else if(ubm instanceof MultiColBitmap) {
+			MultiColBitmap mcbm = (MultiColBitmap) ubm;
+			final int nCol = ubm.getNumColumns();
+			final int nVals = ubm.getNumValues();
+			double[] resValues = new double[nVals * nCol];
+			for(int i = 0; i < nVals; i++)
+				System.arraycopy(mcbm.getValues(i), 0, resValues, i * nCol, nCol);
+
+			return new Dictionary(resValues);
+		}
+		throw new NotImplementedException(
+			"Not implemented creation of bitmap type : " + ubm.getClass().getSimpleName());
+	}
+
+	public static ADictionary createWithAppendedZeroTuple(ABitmap ubm) {
+		return createWithAppendedZeroTuple(ubm, 1.0);
+	}
+
+	public static ADictionary createWithAppendedZeroTuple(ABitmap ubm, double sparsity) {
+		// Log.warn("Inefficient creation of dictionary, to then allocate again.");
+		final int nRows = ubm.getNumValues() + 1;
+		final int nCols = ubm.getNumColumns();
+		if(ubm instanceof Bitmap) {
+			Bitmap bm = (Bitmap) ubm;
+			double[] resValues = new double[ubm.getNumValues() + 1];
+			double[] from = bm.getValues();
+			System.arraycopy(from, 0, resValues, 0, from.length);
+			return new Dictionary(resValues);
+		}
+		else if(sparsity < 0.4 && ubm instanceof MultiColBitmap) {
+			final MultiColBitmap mcbm = (MultiColBitmap) ubm;
+			final MatrixBlock m = new MatrixBlock(nRows, nCols, true);
+			m.allocateSparseRowsBlock();
+			final SparseBlock sb = m.getSparseBlock();
+
+			final int nVals = ubm.getNumValues();
+			for(int i = 0; i < nVals; i++) {
+				final double[] tuple = mcbm.getValues(i);
+				for(int col = 0; col < nCols; col++)
+					sb.append(i, col, tuple[col]);
+			}
+			m.recomputeNonZeros();
+			return new MatrixBlockDictionary(m);
+		}
+		else if(ubm instanceof MultiColBitmap) {
+			MultiColBitmap mcbm = (MultiColBitmap) ubm;
+			final int nVals = ubm.getNumValues();
+			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);
+		}
+		else {
+			throw new NotImplementedException(
+				"Not implemented creation of bitmap type : " + ubm.getClass().getSimpleName());
+		}
+
+	}
+
+	public static ADictionary moveFrequentToLastDictionaryEntry(ADictionary dict, ABitmap ubm, int numRows,
+		int largestIndex) {
+		LOG.warn("Inefficient creation of dictionary, to then allocate again to move one entry to end.");
+		final double[] dictValues = dict.getValues();
+		final int zeros = numRows - (int) ubm.getNumOffsets();
+		final int nCol = ubm.getNumColumns();
+		final int offsetToLargest = largestIndex * nCol;
+
+		if(zeros == 0) {
+			final double[] swap = new double[nCol];
+			System.arraycopy(dictValues, offsetToLargest, swap, 0, nCol);
+			for(int i = offsetToLargest; i < dictValues.length - nCol; i++) {
+				dictValues[i] = dictValues[i + nCol];
+			}
+			System.arraycopy(swap, 0, dictValues, dictValues.length - nCol, nCol);
+			return dict;
+		}
+
+		final int largestIndexSize = ubm.getOffsetsList(largestIndex).size();
+		final double[] newDict = new double[dictValues.length + nCol];
+
+		if(zeros > largestIndexSize)
+			System.arraycopy(dictValues, 0, newDict, 0, dictValues.length);
+		else {
+			System.arraycopy(dictValues, 0, newDict, 0, offsetToLargest);
+			System.arraycopy(dictValues, offsetToLargest + nCol, newDict, offsetToLargest,
+				dictValues.length - offsetToLargest - nCol);
+			System.arraycopy(dictValues, offsetToLargest, newDict, newDict.length - nCol, nCol);
+		}
+		return new Dictionary(newDict);
+	}
 }
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
new file mode 100644
index 0000000..322e56b
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/MatrixBlockDictionary.java
@@ -0,0 +1,456 @@
+package org.apache.sysds.runtime.compress.colgroup.dictionary;
+
+import org.apache.commons.lang.NotImplementedException;
+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.ValueFunction;
+import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
+
+public class MatrixBlockDictionary extends ADictionary {
+
+    private MatrixBlock _data;
+
+    public MatrixBlockDictionary(MatrixBlock data) {
+        _data = data;
+    }
+
+    public MatrixBlock getMatrixBlock() {
+        return _data;
+    }
+
+    @Override
+    public double[] getValues() {
+        LOG.warn("Inefficient force dense format.");
+        _data.sparseToDense();
+        return _data.getDenseBlockValues();
+    }
+
+    @Override
+    public double getValue(int i) {
+        final int nCol = _data.getNumColumns();
+        LOG.warn("inefficient get value at index");
+        return _data.quickGetValue(i / nCol, i % nCol);
+    }
+
+    @Override
+    public int hasZeroTuple(int nCol) {
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(sb.isEmpty(i)) {
+                    return i;
+                }
+            }
+        }
+        else {
+            throw new NotImplementedException();
+        }
+        return -1;
+    }
+
+    @Override
+    public long getInMemorySize() {
+        return 8 + _data.estimateSizeInMemory();
+    }
+
+    public static long getInMemorySize(int numberValues, int numberColumns, double sparsity) {
+        return 8 + MatrixBlock.estimateSizeInMemory(numberValues, numberColumns, sparsity);
+    }
+
+    @Override
+    public double aggregate(double init, Builtin fn) {
+        if(fn.getBuiltinCode() == BuiltinCode.MAX)
+            return fn.execute(init, _data.max());
+        else if(fn.getBuiltinCode() == BuiltinCode.MIN)
+            return fn.execute(init, _data.min());
+        else
+            throw new NotImplementedException();
+    }
+
+    @Override
+    public double[] aggregateTuples(Builtin fn, int nCol) {
+        double[] ret = new double[_data.getNumRows()];
+        if(_data.isEmpty())
+            return ret;
+        else if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final double[] avals = sb.values(i);
+                    ret[i] = avals[apos];
+                    for(int j = apos + 1; j < alen; j++)
+                        ret[i] = fn.execute(ret[i], avals[j]);
+
+                    if(sb.size(i) < _data.getNumColumns())
+                        ret[i] = fn.execute(ret[i], 0);
+                }
+                else
+                    ret[i] = fn.execute(ret[i], 0);
+            }
+        }
+        else if(nCol == 1)
+            return _data.getDenseBlockValues();
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                ret[k] = values[off++];
+                for(int j = 1; j < _data.getNumColumns(); j++)
+                    ret[k] = fn.execute(ret[k], values[off++]);
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
+        if(_data.isEmpty()) {
+            for(int j = 0; j < colIndexes.length; j++) {
+                final int idx = colIndexes[j];
+                c[idx] = fn.execute(c[idx], 0);
+            }
+        }
+        else if(_data.isInSparseFormat()) {
+            MatrixBlock t = LibMatrixReorg.transposeInPlace(_data, 1);
+            if(!t.isInSparseFormat()) {
+                throw new NotImplementedException();
+            }
+            SparseBlock sbt = t.getSparseBlock();
+
+            for(int i = 0; i < _data.getNumColumns(); i++) {
+                final int idx = colIndexes[i];
+                if(!sbt.isEmpty(i)) {
+                    final int apos = sbt.pos(i);
+                    final int alen = sbt.size(i) + apos;
+                    final double[] avals = sbt.values(i);
+                    for(int j = apos; j < alen; j++)
+                        c[idx] = fn.execute(c[idx], avals[j]);
+                    if(avals.length != _data.getNumRows())
+                        c[idx] = fn.execute(c[idx], 0);
+                }
+                else
+                    c[idx] = fn.execute(c[idx], 0);
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final int idx = colIndexes[j];
+                    c[idx] = fn.execute(c[idx], values[off++]);
+                }
+            }
+        }
+    }
+
+    @Override
+    public int size() {
+        return (int) _data.getNonZeros();
+    }
+
+    @Override
+    public ADictionary apply(ScalarOperator op) {
+        MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
+        return new MatrixBlockDictionary(res);
+    }
+
+    @Override
+    public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
+        MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
+        MatrixBlock res2 = res.append(new MatrixBlock(1, 1, newVal), new MatrixBlock());
+        return new MatrixBlockDictionary(res2);
+    }
+
+    @Override
+    public ADictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public ADictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public ADictionary clone() {
+        MatrixBlock ret = new MatrixBlock();
+        ret.copy(_data);
+        return new MatrixBlockDictionary(ret);
+    }
+
+    @Override
+    public ADictionary cloneAndExtend(int len) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public boolean isLossy() {
+        return false;
+    }
+
+    @Override
+    public int getNumberOfValues(int ncol) {
+        return _data.getNumRows();
+    }
+
+    @Override
+    public double[] sumAllRowsToDouble(boolean square, int nrColumns) {
+        double[] ret = new double[_data.getNumRows()];
+
+        if(_data.isEmpty())
+            return ret;
+        else if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final double[] avals = sb.values(i);
+                    for(int j = apos; j < alen; j++) {
+                        ret[i] += (square) ? avals[j] * avals[j] : avals[j];
+                    }
+                }
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final double v = values[off++];
+                    ret[k] += (square) ? v * v : v;
+                }
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public double sumRow(int k, boolean square, int nrColumns) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public double[] colSum(int[] counts, int nCol) {
+        if(_data.isEmpty())
+            return null;
+        double[] ret = new double[nCol];
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    // double tmpSum = 0;
+                    final int count = counts[i];
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final int[] aix = sb.indexes(i);
+                    final double[] avals = sb.values(i);
+                    for(int j = apos; j < alen; j++) {
+                        ret[aix[j]] += count * avals[j];
+                    }
+                }
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                final int countK = counts[k];
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final double v = values[off++];
+                    ret[j] += v * countK;
+                }
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
+        if(_data.isEmpty())
+            return;
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    // double tmpSum = 0;
+                    final int count = counts[i];
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final int[] aix = sb.indexes(i);
+                    final double[] avals = sb.values(i);
+                    for(int j = apos; j < alen; j++) {
+                        c[colIndexes[aix[j]]] += square ? count * avals[j] * avals[j] : count * avals[j];
+                    }
+                }
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                final int countK = counts[k];
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final double v = values[off++];
+                    c[colIndexes[j]] += square ? v * v * countK : v * countK;
+                }
+            }
+        }
+    }
+
+    @Override
+    public double sum(int[] counts, int ncol) {
+        double tmpSum = 0;
+        if(_data.isEmpty())
+            return tmpSum;
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    final int count = counts[i];
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final double[] avals = sb.values(i);
+                    for(int j = apos; j < alen; j++) {
+                        tmpSum += count * avals[j];
+                    }
+                }
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                final int countK = counts[k];
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final double v = values[off++];
+                    tmpSum += v * countK;
+                }
+            }
+        }
+        return tmpSum;
+    }
+
+    @Override
+    public double sumsq(int[] counts, int ncol) {
+        double tmpSum = 0;
+        if(_data.isEmpty())
+            return tmpSum;
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                if(!sb.isEmpty(i)) {
+                    final int count = counts[i];
+                    final int apos = sb.pos(i);
+                    final int alen = sb.size(i) + apos;
+                    final double[] avals = sb.values(i);
+                    for(int j = apos; j < alen; j++) {
+                        tmpSum += count * avals[j] * avals[j];
+                    }
+                }
+            }
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int k = 0; k < _data.getNumRows(); k++) {
+                final int countK = counts[k];
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    final double v = values[off++];
+                    tmpSum += v * v * countK;
+                }
+            }
+        }
+        return tmpSum;
+    }
+
+    @Override
+    public String getString(int colIndexes) {
+        return _data.toString();
+    }
+
+    @Override
+    public void addMaxAndMin(double[] ret, int[] colIndexes) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public ADictionary reExpandColumns(int max) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public boolean containsValue(double pattern) {
+        return _data.containsValue(pattern);
+    }
+
+    @Override
+    public long getNumberNonZeros(int[] counts, int nCol) {
+        if(_data.isEmpty())
+            return 0;
+        long nnz = 0;
+        if(_data.isInSparseFormat()) {
+            SparseBlock sb = _data.getSparseBlock();
+            for(int i = 0; i < _data.getNumRows(); i++)
+                if(sb.isEmpty(i))
+                    nnz += sb.size(i) * counts[i];
+
+        }
+        else {
+            double[] values = _data.getDenseBlockValues();
+            int off = 0;
+            for(int i = 0; i < _data.getNumRows(); i++) {
+                int countThisTuple = 0;
+                for(int j = 0; j < _data.getNumColumns(); j++) {
+                    double v = values[off++];
+                    if(v != 0)
+                        countThisTuple++;
+                }
+                nnz += countThisTuple * counts[i];
+            }
+        }
+        return nnz;
+    }
+
+    @Override
+    public long getNumberNonZerosContained() {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public void addToEntry(Dictionary d, int fr, int to, int nCol) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public double[] getMostCommonTuple(int[] counts, int nCol) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public ADictionary subtractTuple(double[] tuple) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol) {
+        // Simply return this.
+        return this;
+    }
+
+    @Override
+    public String toString() {
+        return "MatrixBlock Dictionary :" + _data.toString();
+    }
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index 7b01fb5..70836ca 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -499,4 +499,14 @@ public class QDictionary extends ADictionary {
 	public ADictionary subtractTuple(double[] tuple) {
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
+		throw new NotImplementedException();	
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java
deleted file mode 100644
index 3a400f3..0000000
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/SparseDictionary.java
+++ /dev/null
@@ -1,218 +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.dictionary;
-
-import org.apache.sysds.runtime.functionobjects.Builtin;
-import org.apache.sysds.runtime.functionobjects.ValueFunction;
-import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
-
-/**
- * A sparse dictionary implementation, use if the tuples are sparse.
- */
-public class SparseDictionary extends ADictionary {
-
-	@Override
-	public double[] getValues() {
-		LOG.warn("Inefficient materialization of sparse Dictionary.");
-
-		return null;
-	}
-
-	@Override
-	public double getValue(int i) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public int hasZeroTuple(int nCol) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public long getInMemorySize() {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public double aggregate(double init, Builtin fn) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public double[] aggregateTuples(Builtin fn, int nCol) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public int size() {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public ADictionary apply(ScalarOperator op) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary applyBinaryRowOpLeft(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary applyBinaryRowOpRight(ValueFunction fn, double[] v, boolean sparseSafe, int[] colIndexes) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary clone() {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary cloneAndExtend(int len) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public boolean isLossy() {
-		// TODO Auto-generated method stub
-		return false;
-	}
-
-	@Override
-	public int getNumberOfValues(int ncol) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public double[] sumAllRowsToDouble(boolean square, int nrColumns) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public double sumRow(int k, boolean square, int nrColumns) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public double[] colSum(int[] counts, int nCol) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public double sum(int[] counts, int ncol) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public double sumsq(int[] counts, int ncol) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public String getString(int colIndexes) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public void addMaxAndMin(double[] ret, int[] colIndexes) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary reExpandColumns(int max) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public boolean containsValue(double pattern) {
-		// TODO Auto-generated method stub
-		return false;
-	}
-
-	@Override
-	public long getNumberNonZeros(int[] counts, int nCol) {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public long getNumberNonZerosContained() {
-		// TODO Auto-generated method stub
-		return 0;
-	}
-
-	@Override
-	public void addToEntry(Dictionary d, int fr, int to, int nCol) {
-		// TODO Auto-generated method stub
-
-	}
-
-	@Override
-	public double[] getMostCommonTuple(int[] counts, int nCol) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-	@Override
-	public ADictionary subtractTuple(double[] tuple) {
-		// TODO Auto-generated method stub
-		return null;
-	}
-
-}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
index d4b2a63..f46ee53 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimator.java
@@ -21,6 +21,7 @@ package org.apache.sysds.runtime.compress.estim;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -91,6 +92,52 @@ public abstract class CompressedSizeEstimator {
 		return new CompressedSizeInfo(sizeInfos);
 	}
 
+	/**
+	 * Multi threaded version of extracting Compression Size info from list of specified columns
+	 * 
+	 * @return
+	 */
+
+	/**
+	 * Multi threaded version of extracting Compression Size info from list of specified columns
+	 * 
+	 * @param columnLists The specified columns to extract.
+	 * @param k           The parallelization degree
+	 * @return The Compression information from the specified column groups.
+	 */
+	public List<CompressedSizeInfoColGroup> computeCompressedSizeInfos(Collection<int[]> columnLists, int k) {
+		if(k == 1)
+			return computeCompressedSizeInfos(columnLists);
+		try {
+			ExecutorService pool = CommonThreadPool.get(k);
+			ArrayList<SizeEstimationTask> tasks = new ArrayList<>();
+			for(int[] g : columnLists)
+				tasks.add(new SizeEstimationTask(this, g));
+			List<Future<CompressedSizeInfoColGroup>> rtask = pool.invokeAll(tasks);
+			ArrayList<CompressedSizeInfoColGroup> ret = new ArrayList<>();
+			for(Future<CompressedSizeInfoColGroup> lrtask : rtask)
+				ret.add(lrtask.get());
+			pool.shutdown();
+			return ret;
+		}
+		catch(InterruptedException | ExecutionException e) {
+			return computeCompressedSizeInfos(columnLists);
+		}
+	}
+
+	/**
+	 * Compression Size info from list of specified columns
+	 * 
+	 * @param columnLists The specified columns to extract.
+	 * @return The Compression information from the specified column groups.
+	 */
+	public List<CompressedSizeInfoColGroup> computeCompressedSizeInfos(Collection<int[]> columnLists) {
+		ArrayList<CompressedSizeInfoColGroup> ret = new ArrayList<>();
+		for(int[] g : columnLists)
+			ret.add(estimateCompressedColGroupSize(g));
+		return ret;
+	}
+
 	private CompressedSizeInfoColGroup[] estimateIndividualColumnGroupSizes(int k) {
 		return (k > 1) ? CompressedSizeInfoColGroup(_numCols, k) : CompressedSizeInfoColGroup(_numCols);
 	}
@@ -106,19 +153,36 @@ public abstract class CompressedSizeEstimator {
 	}
 
 	/**
-	 * Abstract method for extracting Compressed Size Info of specified columns, together in a single ColGroup
+	 * Method for extracting Compressed Size Info of specified columns, together in a single ColGroup
 	 * 
-	 * @param colIndexes The Colums to group together inside a ColGroup
+	 * @param colIndexes The columns to group together inside a ColGroup
 	 * @return The CompressedSizeInformation associated with the selected ColGroups.
 	 */
-	public abstract CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes);
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
+		return estimateCompressedColGroupSize(colIndexes, Integer.MAX_VALUE);
+	}
+
+	/**
+	 * A method to extract the Compressed Size Info for a given list of columns, This method further limits the
+	 * estimated number of unique values, since in some cases the estimated number of uniques is estimated higher than
+	 * the number estimated in sub groups of the given colIndexes.
+	 * 
+	 * @param colIndexes         The columns to extract compression information from
+	 * @param nrUniqueUpperBound The upper bound of unique elements allowed in the estimate, can be calculated from the
+	 *                           number of unique elements estimated in sub columns multiplied together. This is
+	 *                           flexible in the sense that if the sample is small then this unique can be manually
+	 *                           edited like in CoCodeCostMatrixMult.
+	 * 
+	 * @return The CompressedSizeInfoColGroup fro the given column indexes.
+	 */
+	public abstract CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int nrUniqueUpperBound);
 
 	/**
 	 * Method used to extract the CompressedSizeEstimationFactors from an constructed UncompressedBitmap. Note this
 	 * method works both for the sample based estimator and the exact estimator, since the bitmap, can be extracted from
 	 * a sample or from the entire dataset.
 	 * 
-	 * @param ubm        The UncompressedBitmap, either extracted from a sample or from the entier dataset
+	 * @param ubm        The UncompressedBitmap, either extracted from a sample or from the entire dataset
 	 * @param colIndexes The columns that is compressed together.
 	 * @return The size factors estimated from the Bit Map.
 	 */
@@ -154,16 +218,21 @@ public abstract class CompressedSizeEstimator {
 
 	private static class SizeEstimationTask implements Callable<CompressedSizeInfoColGroup> {
 		private final CompressedSizeEstimator _estimator;
-		private final int _col;
+		private final int[] _cols;
 
 		protected SizeEstimationTask(CompressedSizeEstimator estimator, int col) {
 			_estimator = estimator;
-			_col = col;
+			_cols = new int[] {col};
+		}
+
+		protected SizeEstimationTask(CompressedSizeEstimator estimator, int[] cols) {
+			_estimator = estimator;
+			_cols = cols;
 		}
 
 		@Override
 		public CompressedSizeInfoColGroup call() {
-			return _estimator.estimateCompressedColGroupSize(new int[] {_col});
+			return _estimator.estimateCompressedColGroupSize(_cols);
 		}
 	}
 
@@ -174,8 +243,4 @@ public abstract class CompressedSizeEstimator {
 		}
 		return colIndexes;
 	}
-
-	public MatrixBlock getSample() {
-		return _data;
-	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
index 40677e9..5fdf8e2 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -34,7 +34,8 @@ public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int nrUniqueUpperBound) {
+		// exact estimator can ignore upper bound.
 		ABitmap entireBitMap = BitmapEncoder.extractBitmap(colIndexes, _data, _transposed);
 		return new CompressedSizeInfoColGroup(estimateCompressedColGroupSize(entireBitMap, colIndexes),
 			_cs.validCompressions);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
index cf5437e..ebeef16 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
@@ -27,23 +27,42 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 public class CompressedSizeEstimatorFactory {
 	protected static final Log LOG = LogFactory.getLog(CompressedSizeEstimatorFactory.class.getName());
 
-	public static final int minimumSampleSize = 2000;
-
 	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, CompressionSettings cs) {
 
-		final long nRows = cs.transposed ? data.getNumColumns() : data.getNumRows();
+		final int nRows = cs.transposed ? data.getNumColumns() : data.getNumRows();
+		final int nCols = cs.transposed ? data.getNumRows() : data.getNumColumns();
+		final int nnzRows = (int) Math.ceil(data.getNonZeros() / nCols);
 		
-		// Calculate the sample size.
-		// If the sample size is very small, set it to the minimum size
-		final int sampleSize = Math.max((int) Math.ceil(nRows * cs.samplingRatio), minimumSampleSize);
+		final double sampleRatio = cs.samplingRatio;
+		final int sampleSize = getSampleSize(sampleRatio, nRows, cs.minimumSampleSize);
 
-		CompressedSizeEstimator est;
-		if(cs.samplingRatio >= 1.0 || nRows < minimumSampleSize || sampleSize > nRows)
-			est = new CompressedSizeEstimatorExact(data, cs);
-		else
-			est = new CompressedSizeEstimatorSample(data, cs, sampleSize);
+		final CompressedSizeEstimator est = (shouldUseExactEstimator(cs, nRows, sampleSize,
+			nnzRows)) ? new CompressedSizeEstimatorExact(data,
+				cs) : tryToMakeSampleEstimator(data, cs, sampleRatio, sampleSize, nRows, nnzRows);
 
 		LOG.debug("Estimating using: " + est);
 		return est;
 	}
+
+	private static CompressedSizeEstimator tryToMakeSampleEstimator(MatrixBlock data, CompressionSettings cs,
+		double sampleRatio, int sampleSize, int nRows, int nnzRows) {
+		CompressedSizeEstimatorSample estS = new CompressedSizeEstimatorSample(data, cs, sampleSize);
+		while(estS.getSample() == null) {
+			LOG.warn("Doubling sample size");
+			sampleSize = sampleSize * 2;
+			if(shouldUseExactEstimator(cs, nRows, sampleSize, nnzRows))
+				return new CompressedSizeEstimatorExact(data, cs);
+			else
+				estS.sampleData(sampleSize);
+		}
+		return estS;
+	}
+
+	private static boolean shouldUseExactEstimator(CompressionSettings cs, int nRows, int sampleSize, int nnzRows) {
+		return cs.samplingRatio >= 1.0 || nRows < cs.minimumSampleSize || sampleSize > nnzRows;
+	}
+
+	private static int getSampleSize(double sampleRatio, int nRows, int minimumSampleSize) {
+		return Math.max((int) Math.ceil(nRows * sampleRatio), minimumSampleSize);
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
index 3098394..9fe57d7 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -21,7 +21,6 @@ package org.apache.sysds.runtime.compress.estim;
 
 import java.util.HashMap;
 
-import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.estim.sample.HassAndStokes;
@@ -38,8 +37,8 @@ import org.apache.sysds.runtime.util.UtilFunctions;
 
 public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 
-	private final int[] _sampleRows;
-	private final MatrixBlock _sample;
+	private int[] _sampleRows;
+	private MatrixBlock _sample;
 	private HashMap<Integer, Double> _solveCache = null;
 
 	/**
@@ -51,12 +50,16 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	 */
 	public CompressedSizeEstimatorSample(MatrixBlock data, CompressionSettings cs, int sampleSize) {
 		super(data, cs);
-		_sampleRows = CompressedSizeEstimatorSample.getSortedUniformSample(_numRows, sampleSize, _cs.seed);
-		_solveCache = new HashMap<>();
-		_sample = sampleData();
+		_sample = sampleData(sampleSize);
 	}
 
-	protected MatrixBlock sampleData() {
+	public MatrixBlock getSample() {
+		return _sample;
+	}
+
+	public MatrixBlock sampleData(int sampleSize) {
+		_sampleRows = CompressedSizeEstimatorSample.getSortedUniformSample(_numRows, sampleSize, _cs.seed);
+		_solveCache = new HashMap<>();
 		MatrixBlock sampledMatrixBlock;
 		if(_data.isInSparseFormat() && !_cs.transposed) {
 			sampledMatrixBlock = new MatrixBlock(_sampleRows.length, _data.getNumColumns(), true);
@@ -77,18 +80,17 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 				select.appendValue(_sampleRows[i], 0, 1);
 
 			sampledMatrixBlock = _data.removeEmptyOperations(new MatrixBlock(), !_cs.transposed, true, select);
-
 		}
 
 		if(sampledMatrixBlock.isEmpty())
-			throw new DMLCompressionException("Empty sample block");
-
-		return sampledMatrixBlock;
+			return null;
+		else
+			return sampledMatrixBlock;
 
 	}
 
 	@Override
-	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes, int nrUniqueUpperBound) {
 		final int sampleSize = _sampleRows.length;
 		// final int numCols = colIndexes.length;
 		final double scalingFactor = ((double) _numRows / sampleSize);
@@ -109,7 +111,7 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		// Estimate number of distinct values (incl fixes for anomalies w/ large sample fraction)
 		int totalCardinality = getNumDistinctValues(ubm, _numRows, sampleSize, _solveCache);
 		// Number of unique is trivially bounded by the sampled number of uniques and the number of rows.
-		totalCardinality = Math.min(Math.max(totalCardinality, fact.numVals), _numRows);
+		totalCardinality = Math.min(Math.min(Math.max(totalCardinality, fact.numVals), _numRows), nrUniqueUpperBound);
 
 		// estimate number of non-zeros (conservatively round up)
 		final double C = Math.max(1 - (double) fact.numSingle / sampleSize, (double) sampleSize / _numRows);
@@ -127,7 +129,6 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		for(IntArrayList a : ubm.getOffsetList())
 			if(a.size() > largestInstanceCount)
 				largestInstanceCount = a.size();
-		
 
 		final boolean zeroIsMostFrequent = largestInstanceCount == numZerosInSample;
 
@@ -307,12 +308,12 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	/**
 	 * Returns a sorted array of n integers, drawn uniformly from the range [0,range).
 	 * 
-	 * @param range    the range
-	 * @param smplSize sample size
+	 * @param range      the range
+	 * @param sampleSize sample size
 	 * @return sorted array of integers
 	 */
-	private static int[] getSortedUniformSample(int range, int smplSize, long seed) {
-		return UtilFunctions.getSortedSampleIndexes(range, smplSize, seed);
+	private static int[] getSortedUniformSample(int range, int sampleSize, long seed) {
+		return UtilFunctions.getSortedSampleIndexes(range, sampleSize, seed);
 	}
 
 	@Override
@@ -329,4 +330,5 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 		sb.append(_numRows);
 		return sb.toString();
 	}
+
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
index 50aeb7e..8b984ff 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -27,6 +27,8 @@ import java.util.Set;
 import org.apache.commons.lang.NotImplementedException;
 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.cocode.PlanningCoCoder.PartitionerType;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupSizes;
 
@@ -59,7 +61,7 @@ public class CompressedSizeInfoColGroup {
 		_cardinalityRatio = (double) numVals / numRows;
 		_sizes = null;
 		_bestCompressionType = null;
-		_minSize = ColGroupSizes.estimateInMemorySizeDDC(columns.length, numVals, numRows, false);
+		_minSize = ColGroupSizes.estimateInMemorySizeDDC(columns.length, numVals, numRows, 1.0, false);
 	}
 
 	/**
@@ -106,6 +108,15 @@ public class CompressedSizeInfoColGroup {
 		return _sizes.get(ct);
 	}
 
+	public CompressionType getBestCompressionType(CompressionSettings cs) {
+		if(cs.columnPartitioner == PartitionerType.COST_MATRIX_MULT) {
+			// if(_sizes.get(CompressionType.SDC) * 0.8 < _sizes.get(_bestCompressionType))
+			if(getMostCommonFraction() > 0.4)
+				return CompressionType.SDC;
+		}
+		return _bestCompressionType;
+	}
+
 	public CompressionType getBestCompressionType() {
 		return _bestCompressionType;
 	}
@@ -171,26 +182,28 @@ public class CompressedSizeInfoColGroup {
 			case DDC:
 				// + 1 if the column contains zero
 				return ColGroupSizes.estimateInMemorySizeDDC(numCols,
-					fact.numVals + (fact.numOffs < fact.numRows ? 1 : 0), fact.numRows, fact.lossy);
+					fact.numVals + (fact.numOffs < fact.numRows ? 1 : 0), fact.numRows, fact.tupleSparsity, fact.lossy);
 			case RLE:
 				return ColGroupSizes.estimateInMemorySizeRLE(numCols, fact.numVals, fact.numRuns, fact.numRows,
-					fact.lossy);
+					fact.tupleSparsity, fact.lossy);
 			case OLE:
 				return ColGroupSizes.estimateInMemorySizeOLE(numCols, fact.numVals, fact.numOffs + fact.numVals,
-					fact.numRows, fact.lossy);
+					fact.numRows, fact.tupleSparsity, fact.lossy);
 			case UNCOMPRESSED:
 				return ColGroupSizes.estimateInMemorySizeUncompressed(fact.numRows, numCols, fact.overAllSparsity);
 			case SDC:
 				if(fact.numOffs <= 1)
 					return ColGroupSizes.estimateInMemorySizeSDCSingle(numCols, fact.numVals, fact.numRows,
-						fact.largestOff, fact.zeroIsMostFrequent, fact.containNoZeroValues, fact.lossy);
+						fact.largestOff, fact.zeroIsMostFrequent, fact.containNoZeroValues, fact.tupleSparsity,
+						fact.lossy);
 				return ColGroupSizes.estimateInMemorySizeSDC(numCols, fact.numVals, fact.numRows, fact.largestOff,
-					fact.zeroIsMostFrequent, fact.containNoZeroValues, fact.lossy);
+					fact.zeroIsMostFrequent, fact.containNoZeroValues, fact.tupleSparsity, fact.lossy);
 			case CONST:
 				if(fact.numOffs == 0)
 					return ColGroupSizes.estimateInMemorySizeEMPTY(numCols);
 				else if(fact.numOffs == fact.numRows && fact.numVals == 1)
-					return ColGroupSizes.estimateInMemorySizeCONST(numCols, fact.numVals, fact.lossy);
+					return ColGroupSizes.estimateInMemorySizeCONST(numCols, fact.numVals, fact.tupleSparsity,
+						fact.lossy);
 				else
 					return -1;
 			default:
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
index 310b236..64b0bdd 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapEncoder.java
@@ -34,6 +34,7 @@ import org.apache.sysds.runtime.compress.utils.DblArrayIntListHashMap.DArrayILis
 import org.apache.sysds.runtime.compress.utils.DoubleIntListHashMap;
 import org.apache.sysds.runtime.compress.utils.DoubleIntListHashMap.DIListEntry;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
+import org.apache.sysds.runtime.compress.utils.MultiColBitmap;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
@@ -52,29 +53,28 @@ public class BitmapEncoder {
 	 * @param transposed Boolean specifying if the rawblock was transposed.
 	 * @return uncompressed bitmap representation of the columns
 	 */
-	public static Bitmap extractBitmap(int[] colIndices, MatrixBlock rawBlock, boolean transposed) {
-
-		Bitmap res = null;
-		if(colIndices.length == 1) {
-			res = extractBitmap(colIndices[0], rawBlock, transposed);
-		}
-		// multiple column selection (general case)
-		else {
+	public static ABitmap extractBitmap(int[] colIndices, MatrixBlock rawBlock, boolean transposed) {
+		try {
 			final int numRows = transposed ? rawBlock.getNumColumns() : rawBlock.getNumRows();
-			try {
-				res = extractBitmap(colIndices, ReaderColumnSelection.createReader(rawBlock, colIndices, transposed), numRows);
-			}
-			catch(Exception e) {
-				throw new DMLRuntimeException("Failed to extract bitmap", e);
+			if(rawBlock.isEmpty() && colIndices.length == 1)
+				return new Bitmap(null, null, numRows);
+			else if(colIndices.length == 1)
+				return extractBitmap(colIndices[0], rawBlock, transposed);
+			else if(rawBlock.isEmpty())
+				return new MultiColBitmap(colIndices.length, null, null, numRows);
+			else {
+				ReaderColumnSelection reader = ReaderColumnSelection.createReader(rawBlock, colIndices, transposed);
+				return extractBitmap(colIndices, reader, numRows);
 			}
 		}
-		return res;
-
+		catch(Exception e) {
+			throw new DMLRuntimeException("Failed to extract bitmap", e);
+		}
 	}
 
 	public static ABitmap extractBitmap(int[] colIndices, int rows, BitSet rawBlock, CompressionSettings compSettings) {
 		ReaderColumnSelection reader = new ReaderColumnSelectionBitSet(rawBlock, rows, colIndices);
-		Bitmap res = extractBitmap(colIndices, reader, rows);
+		ABitmap res = extractBitmap(colIndices, reader, rows);
 		return res;
 	}
 
@@ -88,7 +88,7 @@ public class BitmapEncoder {
 	 * @param transposed Boolean specifying if the rawBlock is transposed or not.
 	 * @return Bitmap containing the Information of the column.
 	 */
-	private static Bitmap extractBitmap(int colIndex, MatrixBlock rawBlock, boolean transposed) {
+	private static ABitmap extractBitmap(int colIndex, MatrixBlock rawBlock, boolean transposed) {
 		DoubleIntListHashMap hashMap = transposed ? extractHashMapTransposed(colIndex,
 			rawBlock) : extractHashMap(colIndex, rawBlock);
 		return makeBitmap(hashMap, transposed ? rawBlock.getNumColumns() : rawBlock.getNumRows());
@@ -187,7 +187,7 @@ public class BitmapEncoder {
 	 * @param numRows    The number of contained rows
 	 * @return The Bitmap
 	 */
-	protected static Bitmap extractBitmap(int[] colIndices, ReaderColumnSelection rowReader, int numRows) {
+	protected static ABitmap extractBitmap(int[] colIndices, ReaderColumnSelection rowReader, int numRows) {
 		// probe map for distinct items (for value or value groups)
 		DblArrayIntListHashMap distinctVals = new DblArrayIntListHashMap();
 
@@ -216,24 +216,24 @@ public class BitmapEncoder {
 	 * @param numCols      Number of columns
 	 * @return The Bitmap.
 	 */
-	private static Bitmap makeBitmap(DblArrayIntListHashMap distinctVals, int numRows, int numCols) {
+	private static ABitmap makeBitmap(DblArrayIntListHashMap distinctVals, int numRows, int numCols) {
 		// added for one pass bitmap construction
 		// Convert inputs to arrays
 		ArrayList<DArrayIListEntry> mapEntries = distinctVals.extractValues();
 		if(!mapEntries.isEmpty()) {
 			int numVals = distinctVals.size();
-			double[] values = new double[numVals * numCols];
+			double[][] values = new double[numVals][];
 			IntArrayList[] offsetsLists = new IntArrayList[numVals];
 			int bitmapIx = 0;
 			for(DArrayIListEntry val : mapEntries) {
-				System.arraycopy(val.key.getData(), 0, values, bitmapIx * numCols, numCols);
+				values[bitmapIx] = val.key.getData();
 				offsetsLists[bitmapIx++] = val.value;
 			}
 
-			return new Bitmap(numCols, offsetsLists, values, numRows);
+			return new MultiColBitmap(numCols, offsetsLists, values, numRows);
 		}
 		else
-			return new Bitmap(numCols, null, null, numRows);
+			return new MultiColBitmap(numCols, null, null, numRows);
 
 	}
 
@@ -248,7 +248,7 @@ public class BitmapEncoder {
 		// added for one pass bitmap construction
 		// Convert inputs to arrays
 		int numVals = distinctVals.size();
-		if(numVals > 0){
+		if(numVals > 0) {
 
 			double[] values = new double[numVals];
 			IntArrayList[] offsetsLists = new IntArrayList[numVals];
@@ -257,11 +257,11 @@ public class BitmapEncoder {
 				values[bitmapIx] = val.key;
 				offsetsLists[bitmapIx++] = val.value;
 			}
-	
-			return new Bitmap(1, offsetsLists, values, numRows);
+
+			return new Bitmap(offsetsLists, values, numRows);
 		}
-		else{
-			return new Bitmap(1, null, null, numRows);
+		else {
+			return new Bitmap(null, null, numRows);
 		}
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapLossyEncoder.java b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapLossyEncoder.java
index 81789f1..4e74d69 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapLossyEncoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/BitmapLossyEncoder.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 
+import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
@@ -68,20 +69,21 @@ public class BitmapLossyEncoder {
 	 * @param numRows The number of rows contained in the ubm.
 	 * @return A bitmap.
 	 */
-	public static ABitmap makeBitmapLossy(Bitmap ubm, int numRows) {
-		final double[] fp = ubm.getValues();
-		if(fp.length == 0) {
-			return ubm;
-		}
-		Stats stats = new Stats(fp);
-		// TODO make better decisions than just a 8 Bit encoding.
-		if(Double.isInfinite(stats.max) || Double.isInfinite(stats.min)) {
-			LOG.warn("Defaulting to incompressable colGroup");
-			return ubm;
-		}
-		else {
-			return make8BitLossy(ubm, stats, numRows);
-		}
+	public static ABitmap makeBitmapLossy(ABitmap ubm, int numRows) {
+		throw new NotImplementedException();
+		// final double[] fp = ubm.getValues();
+		// if(fp.length == 0) {
+		// 	return ubm;
+		// }
+		// Stats stats = new Stats(fp);
+		// // TODO make better decisions than just a 8 Bit encoding.
+		// if(Double.isInfinite(stats.max) || Double.isInfinite(stats.min)) {
+		// 	LOG.warn("Defaulting to incompressable colGroup");
+		// 	return ubm;
+		// }
+		// else {
+		// 	return make8BitLossy(ubm, stats, numRows);
+		// }
 	}
 
 	/**
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index 9c73f7c..ba22642 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -255,7 +255,7 @@ public class CLALibLeftMultBy {
 				ExecutorService pool = CommonThreadPool.get(k);
 				// compute remaining compressed column groups in parallel
 				ArrayList<Callable<Object>> tasks = new ArrayList<>();
-				int rowBlockSize = 8;
+				int rowBlockSize = 1;
 				if(overlapping) {
 					for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
 						tasks.add(new LeftMatrixMatrixMultTask(colGroups, that, ret, blo,
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
index 5b0a105..61e275c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
@@ -141,14 +141,14 @@ public class CLALibRightMultBy {
 			ret.setOverlapping(true);
 
 		if(containsNull) {
-			ColGroupEmpty cge = findEmptyColumnsAndMakeEmptyColGroup(retCg, ret.getNumColumns());
+			ColGroupEmpty cge = findEmptyColumnsAndMakeEmptyColGroup(retCg, ret.getNumColumns(), ret.getNumRows());
 			if(cge != null)
 				retCg.add(cge);
 		}
 		return ret;
 	}
 
-	private static ColGroupEmpty findEmptyColumnsAndMakeEmptyColGroup(List<AColGroup> colGroups, int nCols) {
+	private static ColGroupEmpty findEmptyColumnsAndMakeEmptyColGroup(List<AColGroup> colGroups, int nCols, int nRows) {
 		Set<Integer> emptyColumns = new HashSet<Integer>(nCols);
 		for(int i = 0; i < nCols; i++)
 			emptyColumns.add(i);
@@ -159,7 +159,7 @@ public class CLALibRightMultBy {
 
 		if(emptyColumns.size() != 0) {
 			int[] emptyColumnsFinal = emptyColumns.stream().mapToInt(Integer::intValue).toArray();
-			return new ColGroupEmpty(emptyColumnsFinal, colGroups.get(0).getNumRows());
+			return new ColGroupEmpty(emptyColumnsFinal, nRows);
 		}
 		else
 			return null;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSquash.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSquash.java
index aacea4a..fecd08e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSquash.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibSquash.java
@@ -39,7 +39,6 @@ import org.apache.sysds.runtime.compress.colgroup.ColGroupFactory;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
 import org.apache.sysds.runtime.compress.readers.ReaderColumnSelection;
 import org.apache.sysds.runtime.compress.utils.ABitmap;
-import org.apache.sysds.runtime.compress.utils.Bitmap;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class CLALibSquash {
@@ -123,18 +122,16 @@ public class CLALibSquash {
 			map = extractBitmap(columnIds, m);
 		}
 		else
-			map = BitmapLossyEncoder.extractMapFromCompressedSingleColumn(m,
-				columnIds[0],
-				minMaxes[columnIds[0] * 2],
+			map = BitmapLossyEncoder.extractMapFromCompressedSingleColumn(m, columnIds[0], minMaxes[columnIds[0] * 2],
 				minMaxes[columnIds[0] * 2 + 1], m.getNumRows());
 
-		AColGroup newGroup = ColGroupFactory.compress(columnIds, m.getNumRows(), map, CompressionType.DDC, cs, m);
+		AColGroup newGroup = ColGroupFactory.compress(columnIds, m.getNumRows(), map, CompressionType.DDC, cs, m, 1);
 		return newGroup;
 	}
 
 	private static ABitmap extractBitmap(int[] colIndices, CompressedMatrixBlock compressedBlock) {
-		Bitmap x = BitmapEncoder.extractBitmap(colIndices,
-			ReaderColumnSelection.createCompressedReader(compressedBlock, colIndices),  compressedBlock.getNumRows());
+		ABitmap x = BitmapEncoder.extractBitmap(colIndices,
+			ReaderColumnSelection.createCompressedReader(compressedBlock, colIndices), compressedBlock.getNumRows());
 		return BitmapLossyEncoder.makeBitmapLossy(x, compressedBlock.getNumRows());
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
index 09354f6..db0c2b6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
@@ -34,8 +34,8 @@ public final class Bitmap extends ABitmap {
 	 */
 	private double[] _values;
 
-	public Bitmap(int numCols, IntArrayList[] offsetsLists, double[] values, int rows) {
-		super(numCols, offsetsLists, rows);
+	public Bitmap(IntArrayList[] offsetsLists, double[] values, int rows) {
+		super(1, offsetsLists, rows);
 		_values = values;
 	}
 
@@ -48,33 +48,16 @@ public final class Bitmap extends ABitmap {
 		return _values;
 	}
 
-	/**
-	 * Obtain tuple of column values associated with index.
-	 * 
-	 * @param ix index of a particular distinct value
-	 * @return the tuple of column values associated with the specified index
-	 */
-	public double[] getValues(int ix) {
-		return Arrays.copyOfRange(_values, ix * _numCols, (ix + 1) * _numCols);
-	}
-
 	public int getNumNonZerosInOffset(int idx){
-		if(_numCols == 1)
-			return  _values[0] != 0 ? 1 : 0;
-		int nz = 0;
-		for(int i = idx * _numCols; i < (idx+1) * _numCols; i++)
-			nz += _values[i] == 0 ? 0 : 1;
-		
-		return nz;
+		return  _values[idx] != 0 ? 1 : 0;
 	}
 
 	public int getNumValues() {
-		return (_values == null) ? 0: _values.length / _numCols;
+		return (_values == null) ? 0: _values.length;
 	}
 
 	public void sortValuesByFrequency() {
 		int numVals = getNumValues();
-		int numCols = getNumColumns();
 
 		double[] freq = new double[numVals];
 		int[] pos = new int[numVals];
@@ -90,10 +73,10 @@ public final class Bitmap extends ABitmap {
 		ArrayUtils.reverse(pos);
 
 		// create new value and offset list arrays
-		double[] lvalues = new double[numVals * numCols];
+		double[] lvalues = new double[numVals];
 		IntArrayList[] loffsets = new IntArrayList[numVals];
 		for(int i = 0; i < numVals; i++) {
-			System.arraycopy(_values, pos[i] * numCols, lvalues, i * numCols, numCols);
+			lvalues[i]  = _values[pos[i]];
 			loffsets[i] = _offsetsLists[pos[i]];
 		}
 		_values = lvalues;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/MultiColBitmap.java
similarity index 70%
copy from src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
copy to src/main/java/org/apache/sysds/runtime/compress/utils/MultiColBitmap.java
index 09354f6..80d98df 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/Bitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/MultiColBitmap.java
@@ -27,14 +27,12 @@ import org.apache.sysds.runtime.util.SortUtils;
 /**
  * Uncompressed representation of one or more columns in bitmap format.
  */
-public final class Bitmap extends ABitmap {
+public final class MultiColBitmap extends ABitmap {
 
-	/**
-	 * Distinct values that appear in the column. Linearized as value groups <v11 v12> <v21 v22>.
-	 */
-	private double[] _values;
+	/** Distinct tuples that appear in the columnGroup */
+	private double[][] _values;
 
-	public Bitmap(int numCols, IntArrayList[] offsetsLists, double[] values, int rows) {
+	public MultiColBitmap(int numCols, IntArrayList[] offsetsLists, double[][] values, int rows) {
 		super(numCols, offsetsLists, rows);
 		_values = values;
 	}
@@ -44,7 +42,7 @@ public final class Bitmap extends ABitmap {
 	 * 
 	 * @return dictionary of value tuples
 	 */
-	public double[] getValues() {
+	public double[][] getValues() {
 		return _values;
 	}
 
@@ -55,29 +53,26 @@ public final class Bitmap extends ABitmap {
 	 * @return the tuple of column values associated with the specified index
 	 */
 	public double[] getValues(int ix) {
-		return Arrays.copyOfRange(_values, ix * _numCols, (ix + 1) * _numCols);
+		return _values[ix];
 	}
 
-	public int getNumNonZerosInOffset(int idx){
-		if(_numCols == 1)
-			return  _values[0] != 0 ? 1 : 0;
+	public int getNumNonZerosInOffset(int idx) {
 		int nz = 0;
-		for(int i = idx * _numCols; i < (idx+1) * _numCols; i++)
-			nz += _values[i] == 0 ? 0 : 1;
-		
+		for(double v : getValues(idx))
+			nz += v == 0 ? 0 : 1;
+
 		return nz;
 	}
 
 	public int getNumValues() {
-		return (_values == null) ? 0: _values.length / _numCols;
+		return (_values == null) ? 0 : _values.length;
 	}
 
 	public void sortValuesByFrequency() {
-		int numVals = getNumValues();
-		int numCols = getNumColumns();
-
-		double[] freq = new double[numVals];
-		int[] pos = new int[numVals];
+		final int numVals = getNumValues();
+		
+		final double[] freq = new double[numVals];
+		final int[] pos = new int[numVals];
 
 		// populate the temporary arrays
 		for(int i = 0; i < numVals; i++) {
@@ -90,10 +85,10 @@ public final class Bitmap extends ABitmap {
 		ArrayUtils.reverse(pos);
 
 		// create new value and offset list arrays
-		double[] lvalues = new double[numVals * numCols];
+		double[][] lvalues = new double[numVals][];
 		IntArrayList[] loffsets = new IntArrayList[numVals];
 		for(int i = 0; i < numVals; i++) {
-			System.arraycopy(_values, pos[i] * numCols, lvalues, i * numCols, numCols);
+			lvalues[i]  = _values[pos[i]];
 			loffsets[i] = _offsetsLists[pos[i]];
 		}
 		_values = lvalues;
@@ -104,7 +99,9 @@ public final class Bitmap extends ABitmap {
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
 		sb.append(super.toString());
-		sb.append("\nValues: " + Arrays.toString(_values));
+		sb.append("\nValues:");
+		for(double[] vv : _values)
+			sb.append("\n" + Arrays.toString(vv));
 		return sb.toString();
 	}
 
diff --git a/src/main/java/org/apache/sysds/utils/MemoryEstimates.java b/src/main/java/org/apache/sysds/utils/MemoryEstimates.java
index 8401d1a..8e0bac8 100644
--- a/src/main/java/org/apache/sysds/utils/MemoryEstimates.java
+++ b/src/main/java/org/apache/sysds/utils/MemoryEstimates.java
@@ -29,6 +29,8 @@ package org.apache.sysds.utils;
  */
 public class MemoryEstimates {
 
+	// private static final Log LOG = LogFactory.getLog(MemoryEstimates.class.getName());
+
 	/**
 	 * Get the worst case memory usage of an java.util.BitSet java object.
 	 * 
diff --git a/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java b/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
index 94ba730..80df1e5 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
@@ -228,9 +228,6 @@ public abstract class AbstractCompressedUnaryTests extends CompressedTestBase {
 			}
 
 		}
-		catch(NotImplementedException e) {
-			throw e;
-		}
 		catch(Exception e) {
 			e.printStackTrace();
 			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
index 1167512..7b9d871 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
@@ -78,7 +78,7 @@ public abstract class CompressedTestBase extends TestBase {
 	protected static SparsityType[] usedSparsityTypes = new SparsityType[] {SparsityType.FULL, SparsityType.SPARSE,};
 
 	protected static ValueType[] usedValueTypes = new ValueType[] {ValueType.RAND_ROUND, ValueType.OLE_COMPRESSIBLE,
-		ValueType.RLE_COMPRESSIBLE,};
+		ValueType.RLE_COMPRESSIBLE};
 
 	protected static ValueRange[] usedValueRanges = new ValueRange[] {ValueRange.SMALL, ValueRange.NEGATIVE,
 		ValueRange.BYTE};
@@ -120,6 +120,9 @@ public abstract class CompressedTestBase extends TestBase {
 		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
 			.setColumnPartitioner(PartitionerType.STATIC).setInvestigateEstimate(true),
 
+		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setTransposeInput("true")
+			.setColumnPartitioner(PartitionerType.COST_MATRIX_MULT).setInvestigateEstimate(true),
+
 		// Forced Uncompressed tests
 		new CompressionSettingsBuilder().setValidCompressions(EnumSet.of(CompressionType.UNCOMPRESSED)),
 
@@ -302,6 +305,8 @@ public abstract class CompressedTestBase extends TestBase {
 				for(OverLapping ov : overLapping) {
 					tests.add(new Object[] {SparsityType.EMPTY, ValueType.RAND, ValueRange.BOOLEAN, cs, mt, ov});
 					tests.add(new Object[] {SparsityType.FULL, ValueType.CONST, ValueRange.LARGE, cs, mt, ov});
+					tests.add(
+						new Object[] {SparsityType.FULL, ValueType.ONE_HOT_ENCODED, ValueRange.BOOLEAN, cs, mt, ov});
 				}
 		return tests;
 	}
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressibleInputGenerator.java b/src/test/java/org/apache/sysds/test/component/compress/CompressibleInputGenerator.java
index 4666327..a8a2a3e 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressibleInputGenerator.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressibleInputGenerator.java
@@ -82,6 +82,21 @@ public class CompressibleInputGenerator {
 		return output;
 	}
 
+	public static double[][] getInputOneHotMatrix(int rows, int cols, int seed) {
+		double[][] variations = new double[cols][];
+		for(int i = 0; i < cols; i++) {
+			variations[i] = new double[cols];
+			variations[i][i] = 1;
+		}
+
+		double[][] matrix = new double[rows][];
+		Random r = new Random(seed);
+		for(int i = 0; i < rows; i++)
+			matrix[i] = variations[r.nextInt(cols)];
+
+		return matrix;
+	}
+
 	private static double[][] rle(int rows, int cols, int nrUnique, int max, int min, double sparsity, int seed,
 		boolean transpose) {
 
diff --git a/src/test/java/org/apache/sysds/test/component/compress/TestBase.java b/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
index f2ebbba..29a2a87 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/TestBase.java
@@ -35,6 +35,7 @@ import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
 import org.apache.sysds.test.component.compress.TestConstants.ValueType;
 
 public class TestBase {
+	// private static final Log LOG = LogFactory.getLog(TestBase.class.getName());
 
 	protected ValueType valType;
 	protected ValueRange valRange;
@@ -70,20 +71,23 @@ public class TestBase {
 					this.min = this.max;
 					// Do not Break, utilize the RAND afterwards.
 				case RAND:
-					this.input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, 7);
+					this.input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed);
 					break;
 				case RAND_ROUND:
-					this.input = TestUtils.round(TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, 7));
+					this.input = TestUtils.round(TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, seed));
 					break;
 				case OLE_COMPRESSIBLE:
 					// Note the Compressible Input generator, generates an already Transposed input
 					// normally, therefore last argument is true, to build a non transposed matrix.
 					this.input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.OLE,
-						(max - min), max, min, sparsity, 7, true);
+						(max - min), max, min, sparsity, seed, true);
 					break;
 				case RLE_COMPRESSIBLE:
 					this.input = CompressibleInputGenerator.getInputDoubleMatrix(rows, cols, CompressionType.RLE,
-						(max - min), max, min, sparsity, 7, true);
+						(max - min), max, min, sparsity, seed, true);
+					break;
+				case ONE_HOT_ENCODED:
+					this.input = CompressibleInputGenerator.getInputOneHotMatrix(rows, cols, seed);
 					break;
 				default:
 					throw new NotImplementedException("Not Implemented Test Value type input generator");
@@ -94,6 +98,7 @@ public class TestBase {
 			this.compressionSettings = compressionSettings.create();
 
 			mb = DataConverter.convertToMatrixBlock(this.input);
+
 		}
 		catch(Exception e) {
 			e.printStackTrace();
diff --git a/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java b/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
index c936c1b..649e96c 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
@@ -38,6 +38,7 @@ public class TestConstants {
 		RAND_ROUND, // Values rounded to nearest whole numbers.
 		OLE_COMPRESSIBLE, // Ideal inputs for OLE Compression.
 		RLE_COMPRESSIBLE, // Ideal inputs for RLE Compression.
+		ONE_HOT_ENCODED,
 	}
 
 	public enum MatrixTypology {
diff --git a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
index c32ccbe..b370451 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/colgroup/JolEstimateTest.java
@@ -58,7 +58,7 @@ public abstract class JolEstimateTest {
 	public abstract CompressionType getCT();
 
 	private final long actualSize;
-	// The actual compressed column group
+	private final int actualNumberUnique;
 	private final AColGroup cg;
 
 	public JolEstimateTest(MatrixBlock mbt) {
@@ -72,8 +72,9 @@ public abstract class JolEstimateTest {
 				.setValidCompressions(EnumSet.of(getCT())).create();
 			cs.transposed = true;
 			ABitmap ubm = BitmapEncoder.extractBitmap(colIndexes, mbt, true);
-			cg = ColGroupFactory.compress(colIndexes, mbt.getNumColumns(), ubm, getCT(), cs, mbt);
+			cg = ColGroupFactory.compress(colIndexes, mbt.getNumColumns(), ubm, getCT(), cs, mbt, 1);
 			actualSize = cg.estimateInMemorySize();
+			actualNumberUnique = cg.getNumValues();
 		}
 		catch(Exception e) {
 			e.printStackTrace();
@@ -83,21 +84,7 @@ public abstract class JolEstimateTest {
 
 	@Test
 	public void compressedSizeInfoEstimatorExact() {
-		try {
-			CompressionSettings cs = new CompressionSettingsBuilder().setSamplingRatio(1.0)
-				.setValidCompressions(EnumSet.of(getCT())).setSeed(seed).create();
-			cs.transposed = true;
-
-			final long estimateCSI = CompressedSizeEstimatorFactory.getSizeEstimator(mbt, cs)
-				.estimateCompressedColGroupSize().getCompressionSize(cg.getCompType());
-
-			boolean res = Math.abs(estimateCSI - actualSize) <= 0;
-			assertTrue("CSI estimate " + estimateCSI + " should be exactly " + actualSize + "\n" + cg.toString(), res);
-		}
-		catch(Exception e) {
-			e.printStackTrace();
-			assertTrue("Failed exact test " + getCT(), false);
-		}
+		compressedSizeInfoEstimatorSample(1.0, 1.0);
 	}
 
 	@Test
@@ -107,48 +94,59 @@ public abstract class JolEstimateTest {
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_50() {
-		compressedSizeInfoEstimatorSample(0.5, 0.90);
+		compressedSizeInfoEstimatorSample(0.5, 0.8);
 	}
 
 	@Test
 	public void compressedSizeInfoEstimatorSample_20() {
-		compressedSizeInfoEstimatorSample(0.2, 0.8);
+		compressedSizeInfoEstimatorSample(0.2, 0.7);
 	}
 
-	@Test
-	public void compressedSizeInfoEstimatorSample_10() {
-		compressedSizeInfoEstimatorSample(0.1, 0.75);
-	}
+	// @Test
+	// public void compressedSizeInfoEstimatorSample_10() {
+	// 	compressedSizeInfoEstimatorSample(0.1, 0.6);
+	// }
 
-	@Test
-	public void compressedSizeInfoEstimatorSample_5() {
-		compressedSizeInfoEstimatorSample(0.05, 0.7);
-	}
+	// @Test
+	// public void compressedSizeInfoEstimatorSample_5() {
+	// 	compressedSizeInfoEstimatorSample(0.05, 0.5);
+	// }
 
-	@Test
-	public void compressedSizeInfoEstimatorSample_1() {
-		compressedSizeInfoEstimatorSample(0.01, 0.6);
-	}
+	// @Test
+	// public void compressedSizeInfoEstimatorSample_1() {
+	// 	compressedSizeInfoEstimatorSample(0.01, 0.4);
+	// }
 
 	public void compressedSizeInfoEstimatorSample(double ratio, double tolerance) {
 		try {
-			if(mbt.getNumColumns() < CompressedSizeEstimatorFactory.minimumSampleSize)
-				return; // Skip the tests that anyway wouldn't use the sample based approach.
 
 			CompressionSettings cs = new CompressionSettingsBuilder().setSamplingRatio(ratio)
-				.setValidCompressions(EnumSet.of(getCT())).setSeed(seed).create();
+				.setValidCompressions(EnumSet.of(getCT())).setMinimumSampleSize(100).setSeed(seed).create();
 			cs.transposed = true;
 
 			final CompressedSizeInfoColGroup cgsi = CompressedSizeEstimatorFactory.getSizeEstimator(mbt, cs)
 				.estimateCompressedColGroupSize();
+
+			if(cg.getCompType() != CompressionType.UNCOMPRESSED && actualNumberUnique > 10) {
+
+				final int estimateNUniques = cgsi.getNumVals();
+				final double minToleranceNUniques = actualNumberUnique * tolerance;
+				final double maxToleranceNUniques = actualNumberUnique / tolerance;
+				final String uniqueString = minToleranceNUniques + " < " + estimateNUniques + " < "
+					+ maxToleranceNUniques;
+				final boolean withinToleranceOnNUniques = minToleranceNUniques <= actualNumberUnique &&
+					actualNumberUnique <= maxToleranceNUniques;
+				assertTrue("CSI Sampled estimate of number of unique values not in range " + uniqueString + "\n" + cgsi,
+					withinToleranceOnNUniques);
+			}
+
 			final long estimateCSI = cgsi.getCompressionSize(cg.getCompType());
 			final double minTolerance = actualSize * tolerance;
 			final double maxTolerance = actualSize / tolerance;
 			final String rangeString = minTolerance + " < " + estimateCSI + " < " + maxTolerance;
-			boolean res = minTolerance < estimateCSI && estimateCSI < maxTolerance;
-			assertTrue(
-				"CSI Sampled estimate is not in tolerance range " + rangeString + "\n" + cgsi + "\n" + cg.toString(),
-				res);
+			final boolean withinToleranceOnSize = minTolerance <= estimateCSI && estimateCSI <= maxTolerance;
+			assertTrue("CSI Sampled estimate is not in tolerance range " + rangeString + " Actual number uniques:"
+				+ actualNumberUnique + "\n" + cgsi, withinToleranceOnSize);
 
 		}
 		catch(Exception e) {
diff --git a/src/test/java/org/apache/sysds/test/component/compress/estim/SampleEstimatorTest.java b/src/test/java/org/apache/sysds/test/component/compress/estim/SampleEstimatorTest.java
new file mode 100644
index 0000000..fba1088
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/component/compress/estim/SampleEstimatorTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.component.compress.estim;
+
+import static org.junit.Assert.assertTrue;
+
+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.CompressionSettingsBuilder;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorFactory;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.util.DataConverter;
+import org.apache.sysds.test.TestUtils;
+import org.junit.Test;
+
+public class SampleEstimatorTest {
+
+	protected static final Log LOG = LogFactory.getLog(SampleEstimatorTest.class.getName());
+
+	private static final int seed = 1512314;
+
+	final MatrixBlock mbt;
+
+	public SampleEstimatorTest() {
+		// matrix block 2 columns
+		mbt = DataConverter
+			.convertToMatrixBlock(TestUtils.round(TestUtils.generateTestMatrix(2, 500000, 0, 299, 1.0, seed + 1)));
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorFull() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(1.0, 1.0);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_90() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.9, 0.9);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_50() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.5, 0.90);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_20() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.2, 0.8);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_10() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.1, 0.75);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_5() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.05, 0.7);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_1() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.01, 0.6);
+	}
+
+	@Test
+	public void compressedSizeInfoEstimatorSample_p1() {
+		testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(0.001, 0.5);
+	}
+
+	/**
+	 * This test verify that the estimated number or unique values in individual columns is adhered to when analyzing
+	 * multi columns.
+	 * 
+	 * The important part here is not if the number of unique elements is estimated correctly, but that the relation
+	 * between observations is preserved.
+	 * 
+	 * @param ratio     Ratio to sample
+	 * @param tolerance A percentage tolerance in number of unique element estimated
+	 */
+	private void testSampleEstimateIsAtMaxEstimatedElementsInEachColumnsProduct(double ratio, double tolerance) {
+
+		final CompressionSettings cs_estimate = new CompressionSettingsBuilder().setMinimumSampleSize(100)
+			.setSamplingRatio(ratio).setSeed(seed).create();
+
+		cs_estimate.transposed = true;
+
+		final CompressedSizeEstimator estimate = CompressedSizeEstimatorFactory.getSizeEstimator(mbt, cs_estimate);
+		final int estimate_1 = estimate.estimateCompressedColGroupSize(new int[] {0}).getNumVals() + 1;
+		final int estimate_2 = estimate.estimateCompressedColGroupSize(new int[] {1}).getNumVals() + 1;
+
+		final int estimate_full = estimate.estimateCompressedColGroupSize(new int[] {0, 1}, estimate_1 * estimate_2)
+			.getNumVals();
+		assertTrue(
+			"Estimate of all columns should be upper bounded by distinct of each column multiplied: " + estimate_full
+				+ " * " + tolerance + " <= " + estimate_1 * estimate_2,
+			estimate_full * tolerance <= estimate_1 * estimate_2);
+
+	}
+
+}

[systemds] 06/07: [SYSTEMDS-2998] CLA Offset and Mapping Tests

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

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

commit 9bfd7ff26a4e840805c209b7becc13498f10b61d
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Mon May 24 12:21:09 2021 +0200

    [SYSTEMDS-2998] CLA Offset and Mapping Tests
    
    Add various tests compression
    
    - InsertionSorterTests
    - OffsetTests
    - MappingTests
    
    Minor bug fixes and better mapping test.
    
    Better compression ration on SDC with 3 distinct elements since the
    dictionary contains number of distinct elements -1 for SDC. Therefore if
    the SDC contain 3 distinct values, it only need 2 distinct identifiers
    in the dictionary.
---
 .../runtime/compress/CompressedMatrixBlock.java    |   86 +-
 .../compress/CompressedMatrixBlockFactory.java     |   27 +-
 .../runtime/compress/CompressionSettings.java      |    6 +-
 .../compress/CompressionSettingsBuilder.java       |   38 +-
 .../runtime/compress/CompressionStatistics.java    |   14 +-
 .../compress/cocode/CoCodeCostMatrixMult.java      |    6 +-
 .../runtime/compress/cocode/CoCodeCostTSMM.java    |  188 +++
 .../runtime/compress/cocode/PlanningCoCoder.java   |   14 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |  271 +----
 .../compress/colgroup/ColGroupCompressed.java      |   39 +-
 .../runtime/compress/colgroup/ColGroupConst.java   |  306 ++---
 .../runtime/compress/colgroup/ColGroupDDC.java     |  576 +++++-----
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   48 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |   84 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     |  658 +++++------
 .../runtime/compress/colgroup/ColGroupRLE.java     |  630 +++++-----
 .../runtime/compress/colgroup/ColGroupSDC.java     |  701 +++++------
 .../compress/colgroup/ColGroupSDCSingle.java       |  630 +++++-----
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  377 +++---
 .../compress/colgroup/ColGroupSDCZeros.java        |  473 ++++----
 .../runtime/compress/colgroup/ColGroupSizes.java   |   15 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  298 +++--
 .../runtime/compress/colgroup/ColGroupValue.java   |  716 ++++++------
 .../compress/colgroup/dictionary/ADictionary.java  |  137 ++-
 .../compress/colgroup/dictionary/Dictionary.java   |   77 +-
 .../colgroup/dictionary/DictionaryFactory.java     |  101 +-
 .../colgroup/dictionary/MatrixBlockDictionary.java | 1213 ++++++++++----------
 .../compress/colgroup/dictionary/QDictionary.java  |   36 +-
 .../colgroup/insertionsort/AInsertionSorter.java   |   68 ++
 .../insertionsort/InsertionSorterFactory.java      |   47 +
 .../{tree => insertionsort}/MaterializeSort.java   |   69 +-
 .../{tree => insertionsort}/MergeSort.java         |  104 +-
 .../compress/colgroup/mapping/MapToBit.java        |   24 +-
 .../compress/colgroup/mapping/MapToByte.java       |    4 +-
 .../compress/colgroup/mapping/MapToChar.java       |    4 +-
 .../compress/colgroup/mapping/MapToFactory.java    |   45 +-
 .../compress/colgroup/mapping/MapToInt.java        |    4 +-
 .../runtime/compress/colgroup/offset/AOffset.java  |   15 +-
 .../compress/colgroup/offset/OffsetByte.java       |    6 +-
 .../compress/colgroup/offset/OffsetChar.java       |    7 +-
 .../compress/colgroup/offset/OffsetFactory.java    |   30 +-
 .../compress/colgroup/pre/IPreAggregate.java       |   79 --
 .../compress/colgroup/pre/MapPreAggregate.java     |   62 -
 .../compress/colgroup/pre/PreAggregateFactory.java |   41 -
 .../compress/colgroup/tree/AInsertionSorter.java   |   95 --
 .../colgroup/tree/InsertionSorterFactory.java      |   33 -
 .../runtime/compress/colgroup/tree/Naive.java      |  136 ---
 .../compress/estim/CompressedSizeEstimator.java    |    8 +-
 .../estim/CompressedSizeEstimatorSample.java       |    3 +-
 .../sysds/runtime/compress/lib/BitmapEncoder.java  |    2 +-
 .../runtime/compress/lib/BitmapLossyEncoder.java   |   54 +-
 .../runtime/compress/lib/CLALibBinaryCellOp.java   |   48 +-
 .../runtime/compress/lib/CLALibLeftMultBy.java     |  189 +--
 .../runtime/compress/lib/CLALibRelationalOp.java   |    6 +-
 .../sysds/runtime/compress/lib/CLALibSquash.java   |   13 +-
 .../compress/readers/ReaderColumnSelection.java    |    3 +-
 .../readers/ReaderColumnSelectionBitSet.java       |   16 +-
 .../ReaderColumnSelectionDenseMultiBlock.java      |    8 +-
 ...erColumnSelectionDenseMultiBlockTransposed.java |   12 +-
 .../ReaderColumnSelectionDenseSingleBlock.java     |   11 +-
 ...rColumnSelectionDenseSingleBlockTransposed.java |   18 +-
 .../readers/ReaderColumnSelectionSparse.java       |    4 +-
 .../ReaderColumnSelectionSparseTransposed.java     |    5 +-
 .../sysds/runtime/compress/utils/DblArray.java     |    4 +
 .../runtime/matrix/data/RandomMatrixGenerator.java |    2 +-
 src/test/java/org/apache/sysds/test/TestUtils.java |   33 +-
 .../compress/AbstractCompressedUnaryTests.java     |   13 +-
 .../component/compress/CompressedMatrixTest.java   |  281 ++++-
 .../component/compress/CompressedTestBase.java     |  323 ++++--
 .../component/compress/CompressedVectorTest.java   |   11 +-
 .../compress/ParCompressedMatrixTest.java          |   30 +-
 .../sysds/test/component/compress/TestBase.java    |   86 +-
 .../test/component/compress/TestConstants.java     |   28 +-
 .../compress/colgroup/JolEstimateTest.java         |   35 +-
 .../insertionsort/TestInsertionSorters.java        |  128 +++
 .../compress/insertionsorter/MergeSortTest.java    |  276 -----
 .../component/compress/mapping/MappingTests.java   |  182 +++
 .../compress/offset/OffsetNegativeTests.java       |   90 ++
 .../compress/offset/OffsetSingleTests.java}        |   38 +-
 .../component/compress/offset/OffsetTests.java     |  132 ++-
 80 files changed, 5562 insertions(+), 5168 deletions(-)

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 27d77ed..aff61b8 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -159,15 +159,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	public CompressedMatrixBlock(CompressedMatrixBlock that) {
 		super(that.getNumRows(), that.getNumColumns(), true);
-		sparseBlock = null;
-		denseBlock = null;
-		nonZeros = that.getNonZeros();
-
-		_colGroups = new ArrayList<>();
-		for(AColGroup cg : that._colGroups)
-			_colGroups.add(cg.copy());
-
-		overlappingColGroups = that.overlappingColGroups;
+		this.copyCompressedMatrix(that);
 	}
 
 	public boolean isSingleUncompressedGroup() {
@@ -175,6 +167,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			_colGroups.get(0).getCompType() == CompressionType.UNCOMPRESSED);
 	}
 
+	/**
+	 * Allocate the given column group and remove all references to old column groups.
+	 * 
+	 * This is done by simply allocating a ned _colGroups list and adding the given column group
+	 * 
+	 * @param cg The column group to use after.
+	 */
 	public void allocateColGroup(AColGroup cg) {
 		_colGroups = new ArrayList<>(1);
 		_colGroups.add(cg);
@@ -305,7 +304,8 @@ public class CompressedMatrixBlock extends MatrixBlock {
 					ColGroupUncompressed guc = (ColGroupUncompressed) g;
 					MatrixBlock gMB = guc.getData();
 					// Make sure that it is the correct dimensions
-					if(gMB.getNumColumns() == this.getNumColumns() && gMB.getNumRows() == this.getNumRows()) {
+					if(gMB.getNumColumns() == this.getNumColumns() && gMB.getNumRows() == this.getNumRows() &&
+						!gMB.isEmpty() && !gMB.isInSparseFormat()) {
 						_colGroups.remove(i);
 						return gMB;
 					}
@@ -666,20 +666,23 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	@Override
 	public MatrixBlock transposeSelfMatrixMultOperations(MatrixBlock out, MMTSJType tstype, int k) {
 		// check for transpose type
-		if(tstype != MMTSJType.LEFT) // right not supported yet
+		if(tstype == MMTSJType.LEFT) {
+			if(isEmptyBlock()) {
+				return new MatrixBlock(clen, clen, true);
+			}
+			// create output matrix block
+			if(out == null)
+				out = new MatrixBlock(clen, clen, false);
+			else
+				out.reset(clen, clen, false);
+			out.allocateDenseBlock();
+			CLALibLeftMultBy.leftMultByTransposeSelf(_colGroups, out, k, getNumColumns(), getMaxNumValues(),
+				isOverlapping());
+			return out;
+		}
+		else {
 			throw new DMLRuntimeException("Invalid MMTSJ type '" + tstype.toString() + "'.");
-		if(isEmptyBlock())
-			return new MatrixBlock(clen, clen, true);
-		// create output matrix block
-		if(out == null)
-			out = new MatrixBlock(clen, clen, false);
-		else
-			out.reset(clen, clen, false);
-		out.allocateDenseBlock();
-		// compute matrix mult
-		CLALibLeftMultBy.leftMultByTransposeSelf(_colGroups, out, k, getNumColumns(), getMaxNumValues(),
-			isOverlapping());
-		return out;
+		}
 	}
 
 	@Override
@@ -758,7 +761,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 			// decompress row partition
 			for(AColGroup grp : _colGroups)
-				grp.decompressToBlock(_ret, _rl, _ru, false);
+				grp.decompressToBlockUnSafe(_ret, _rl, _ru);
 
 			// post processing (sort due to append)
 			if(_ret.isInSparseFormat())
@@ -771,7 +774,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
-		sb.append("\nCompressed Matrix:");
+		sb.append("CompressedMatrixBlock:");
 		sb.append("\nCols:" + getNumColumns() + " Rows:" + getNumRows() + " Overlapping: " + isOverlapping() + " nnz: "
 			+ nonZeros);
 		if(_colGroups != null)
@@ -811,7 +814,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			// and it is not inclusive in decompression, and construction of MatrixBlock.
 			tmp = new MatrixBlock(ru + 1 - rl, getNumColumns(), false).allocateDenseBlock();
 			for(AColGroup g : getColGroups())
-				g.decompressToBlock(tmp, rl, ru + 1, 0);
+				g.decompressToBlockUnSafe(tmp, rl, ru + 1, 0);
 			tmp.recomputeNonZeros();
 			return tmp;
 		}
@@ -882,25 +885,25 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public double max() {
-		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uamax", -1);
+		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uamax", 1);
 		return aggregateUnaryOperations(op, null, 1000, null).getValue(0, 0);
 	}
 
 	@Override
 	public double min() {
-		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uamin", -1);
+		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uamin", 1);
 		return aggregateUnaryOperations(op, null, 1000, null).getValue(0, 0);
 	}
 
 	@Override
 	public double sum() {
-		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uak+", -1);
+		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uak+", 1);
 		return aggregateUnaryOperations(op, null, 1000, null).getValue(0, 0);
 	}
 
 	@Override
 	public double sumSq() {
-		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uasqk+", -1);
+		AggregateUnaryOperator op = InstructionUtils.parseBasicAggregateUnaryOperator("uasqk+", 1);
 		return aggregateUnaryOperations(op, null, 1000, null).getValue(0, 0);
 	}
 
@@ -993,7 +996,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		AColGroup grp = _colGroups.get(0);
 		MatrixBlock vals = grp.getValuesAsBlock();
 		if(grp instanceof ColGroupValue) {
-			MatrixBlock counts = getCountsAsBlock( ((ColGroupValue) grp).getCounts());
+			MatrixBlock counts = getCountsAsBlock(((ColGroupValue) grp).getCounts());
 			if(counts.isEmpty())
 				return vals.cmOperations(op);
 			return vals.cmOperations(op, counts);
@@ -1224,6 +1227,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed) {
+		LOG.info("Inplace rand ops not on CompressedMatrix");
 		MatrixBlock ret = new MatrixBlock(getNumRows(), getNumColumns(), true);
 		LibMatrixDatagen.generateRandomMatrix(ret, rgen, bigrand, bSeed);
 		return ret;
@@ -1231,6 +1235,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed, int k) {
+		LOG.info("Inplace rand ops not on CompressedMatrix");
 		MatrixBlock ret = new MatrixBlock(getNumRows(), getNumColumns(), true);
 		LibMatrixDatagen.generateRandomMatrix(ret, rgen, bigrand, bSeed, k);
 		return ret;
@@ -1284,37 +1289,36 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public void copy(MatrixValue thatValue) {
-		CompressedMatrixBlock that = checkType(thatValue);
-		if(this == that) // prevent data loss (e.g., on sparse-dense conversion)
-			throw new RuntimeException("Copy must not overwrite itself!");
-
+		copy(thatValue, false);
 	}
 
 	private static CompressedMatrixBlock checkType(MatrixValue thatValue) {
-		if(thatValue == null || !(thatValue instanceof CompressedMatrixBlock)) {
+		if(thatValue == null || !(thatValue instanceof CompressedMatrixBlock))
 			throw new DMLRuntimeException("Invalid call to copy, requre a compressed MatrixBlock to copy to");
-		}
+
 		return (CompressedMatrixBlock) thatValue;
 	}
 
 	@Override
 	public void copy(MatrixValue thatValue, boolean sp) {
 		CompressedMatrixBlock that = checkType(thatValue);
-
+		if(this == that) // prevent data loss (e.g., on sparse-dense conversion)
+			throw new RuntimeException("Copy must not overwrite itself!");
 		copyCompressedMatrix(that);
 	}
 
 	private void copyCompressedMatrix(CompressedMatrixBlock that) {
-		if(this == that) // prevent data loss (e.g., on sparse-dense conversion)
-			throw new RuntimeException("Copy must not overwrite itself!");
 		this.rlen = that.rlen;
 		this.clen = that.clen;
-
+		this.sparseBlock = null;
+		this.denseBlock = null;
 		this.nonZeros = that.getNonZeros();
+
 		this._colGroups = new ArrayList<>();
 		for(AColGroup cg : that._colGroups)
 			_colGroups.add(cg.copy());
 
 		overlappingColGroups = that.overlappingColGroups;
 	}
+
 }
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 1756018..914a707 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -122,9 +122,9 @@ public class CompressedMatrixBlockFactory {
 	 */
 	public static CompressedMatrixBlock createConstant(int numRows, int numCols, double value) {
 		CompressedMatrixBlock block = new CompressedMatrixBlock(numRows, numCols);
-		ColGroupConst cg = ColGroupConst.genColGroupConst(numRows, numCols, value);
+		AColGroup cg = ColGroupFactory.genColGroupConst(numRows, numCols, value);
 		block.allocateColGroup(cg);
-		block.setNonZeros(value == 0.0 ? 0 : numRows * numCols);
+		block.recomputeNonZeros();
 		return block;
 	}
 
@@ -157,13 +157,11 @@ public class CompressedMatrixBlockFactory {
 	private void classifyPhase() {
 		CompressedSizeEstimator sizeEstimator = CompressedSizeEstimatorFactory.getSizeEstimator(mb, compSettings);
 		CompressedSizeInfo sizeInfos = sizeEstimator.computeCompressedSizeInfos(k);
-
-		if(compSettings.investigateEstimate)
-			_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
-
+		_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
 		logPhase();
-
-		if(_stats.estimatedSizeCols < _stats.originalSize || compSettings.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
+		
+		if(_stats.estimatedSizeCols < _stats.originalSize ||
+			compSettings.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
 			coCodePhase(sizeEstimator, sizeInfos, mb.getNumRows());
 		else {
 			LOG.info("Estimated Size of singleColGroups: " + _stats.estimatedSizeCols);
@@ -281,9 +279,9 @@ public class CompressedMatrixBlockFactory {
 		res.cleanupBlock(true, true);
 
 		_stats.size = res.estimateCompressedSizeInMemory();
-		
+
 		final double ratio = _stats.getRatio();
-		if(ratio < 1 && compSettings.columnPartitioner != PartitionerType.COST_MATRIX_MULT)  {
+		if(ratio < 1 && compSettings.columnPartitioner != PartitionerType.COST_MATRIX_MULT) {
 			LOG.info("--dense size:        " + _stats.denseSize);
 			LOG.info("--original size:     " + _stats.originalSize);
 			LOG.info("--compressed size:   " + _stats.size);
@@ -304,7 +302,7 @@ public class CompressedMatrixBlockFactory {
 	private Pair<MatrixBlock, CompressionStatistics> abortCompression() {
 		LOG.warn("Compression aborted at phase: " + phase);
 		if(compSettings.transposed)
-			LibMatrixReorg.transposeInPlace(mb,k);
+			LibMatrixReorg.transposeInPlace(mb, k);
 		return new ImmutablePair<>(mb, _stats);
 	}
 
@@ -346,10 +344,9 @@ public class CompressedMatrixBlockFactory {
 					LOG.debug("--compression ratio: " + _stats.getRatio());
 					int[] lengths = new int[res.getColGroups().size()];
 					int i = 0;
-					for(AColGroup colGroup : res.getColGroups()) {
-						if(colGroup.getValues() != null)
-							lengths[i++] = colGroup.getValues().length / colGroup.getColIndices().length;
-					}
+					for(AColGroup colGroup : res.getColGroups())
+						lengths[i++] = colGroup.getNumValues();
+
 					LOG.debug("--compressed colGroup dictionary sizes: " + Arrays.toString(lengths));
 					if(LOG.isTraceEnabled()) {
 						for(AColGroup colGroup : res.getColGroups()) {
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 895600d..aeabc92 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -73,9 +73,6 @@ public class CompressionSettings {
 	/** If the seed is -1 then the system used system millisecond time and class hash for seeding. */
 	public final int seed;
 
-	/** Boolean specifying if the compression strategy should be investigated and monitored. */
-	public final boolean investigateEstimate;
-
 	/** True if lossy compression is enabled */
 	public final boolean lossy;
 
@@ -103,7 +100,7 @@ public class CompressionSettings {
 	public final int minimumSampleSize;
 
 	protected CompressionSettings(double samplingRatio, boolean allowSharedDictionary, String transposeInput,
-		boolean skipList, int seed, boolean investigateEstimate, boolean lossy,
+		boolean skipList, int seed, boolean lossy,
 		EnumSet<CompressionType> validCompressions, boolean sortValuesByLength, PartitionerType columnPartitioner,
 		int maxColGroupCoCode, double coCodePercentage, int minimumSampleSize) {
 		this.samplingRatio = samplingRatio;
@@ -111,7 +108,6 @@ public class CompressionSettings {
 		this.transposeInput = transposeInput;
 		this.skipList = skipList;
 		this.seed = seed;
-		this.investigateEstimate = investigateEstimate;
 		this.validCompressions = validCompressions;
 		this.lossy = lossy;
 		this.sortValuesByLength = sortValuesByLength;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index 83d01e5..3ec42a0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -36,12 +36,11 @@ public class CompressionSettingsBuilder {
 	private String transposeInput;
 	private boolean skipList = true;
 	private int seed = -1;
-	private boolean investigateEstimate = true;
 	private boolean lossy = false;
 	private EnumSet<CompressionType> validCompressions;
 	private boolean sortValuesByLength = true;
 	private PartitionerType columnPartitioner;
-	private int maxStaticColGroupCoCode = 10000;
+	private int maxColGroupCoCode = 10000;
 	private double coCodePercentage = 0.01;
 	private int minimumSampleSize = 2000;
 
@@ -74,9 +73,15 @@ public class CompressionSettingsBuilder {
 		this.samplingRatio = that.samplingRatio;
 		this.allowSharedDictionary = that.allowSharedDictionary;
 		this.transposeInput = that.transposeInput;
+		this.skipList = that.skipList;
 		this.seed = that.seed;
-		this.investigateEstimate = that.investigateEstimate;
+		this.lossy = that.lossy;
 		this.validCompressions = EnumSet.copyOf(that.validCompressions);
+		this.sortValuesByLength = that.sortValuesByLength;
+		this.columnPartitioner = that.columnPartitioner;
+		this.maxColGroupCoCode = that.maxColGroupCoCode;
+		this.coCodePercentage = that.coCodePercentage;
+		this.minimumSampleSize = that.minimumSampleSize;
 		return this;
 	}
 
@@ -170,17 +175,6 @@ public class CompressionSettingsBuilder {
 	}
 
 	/**
-	 * Set if the compression should be investigated while compressing.
-	 * 
-	 * @param investigateEstimate A boolean specifying it the input should be estimated.
-	 * @return The CompressionSettingsBuilder
-	 */
-	public CompressionSettingsBuilder setInvestigateEstimate(boolean investigateEstimate) {
-		this.investigateEstimate = investigateEstimate;
-		return this;
-	}
-
-	/**
 	 * Set the valid compression strategies used for the compression.
 	 * 
 	 * @param validCompressions An EnumSet of CompressionTypes to use in the compression
@@ -230,14 +224,14 @@ public class CompressionSettingsBuilder {
 	}
 
 	/**
-	 * Set the maximum number of columns to CoCode together in the static CoCoding strategy. Compression time increase
-	 * with higher numbers.
+	 * Set the maximum number of columns to CoCode together in the CoCoding strategy. Compression time increase with
+	 * higher numbers.
 	 * 
-	 * @param maxStaticColGroupCoCode The max selected.
+	 * @param maxColGroupCoCode The max selected.
 	 * @return The CompressionSettingsBuilder
 	 */
-	public CompressionSettingsBuilder setmaxStaticColGroupCoCode(int maxStaticColGroupCoCode) {
-		this.maxStaticColGroupCoCode = maxStaticColGroupCoCode;
+	public CompressionSettingsBuilder setMaxColGroupCoCode(int maxColGroupCoCode) {
+		this.maxColGroupCoCode = maxColGroupCoCode;
 		return this;
 	}
 
@@ -273,8 +267,8 @@ public class CompressionSettingsBuilder {
 	 * @return The CompressionSettings
 	 */
 	public CompressionSettings create() {
-		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, skipList, seed,
-			investigateEstimate, lossy, validCompressions, sortValuesByLength, columnPartitioner,
-			maxStaticColGroupCoCode, coCodePercentage, minimumSampleSize);
+		return new CompressionSettings(samplingRatio, allowSharedDictionary, transposeInput, skipList, seed, lossy,
+			validCompressions, sortValuesByLength, columnPartitioner, maxColGroupCoCode, coCodePercentage,
+			minimumSampleSize);
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
index 466953a..f35d417 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
@@ -94,15 +94,21 @@ public class CompressionStatistics {
 	}
 
 	public double getRatio() {
-		return (double) originalSize / size;
+		return size == 0.0 ? Double.POSITIVE_INFINITY : (double) originalSize / size;
 	}
 
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
-		sb.append("Compression Statistics:\n");
-		sb.append("\t" + getGroupsTypesString() + "\n");
-		sb.append("\t" + getGroupsSizesString() + "\n");
+		sb.append("CompressionStatistics:\n");
+		sb.append("Dense Size       : " + denseSize);
+		sb.append("Original Size    : " + originalSize);
+		sb.append("Compressed Size  : " + size);
+		sb.append("CompressionRatio : " + getRatio());
+		if(colGroupCounts != null){
+			sb.append("\t" + getGroupsTypesString() + "\n");
+			sb.append("\t" + getGroupsSizesString() + "\n");
+		}
 		return sb.toString();
 	}
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
index 0d39b47..53f3243 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostMatrixMult.java
@@ -110,10 +110,10 @@ public class CoCodeCostMatrixMult extends AColumnCoCoder {
 
 			final int numberTuples = elm.getNumVals();
 			final double tupleSparsity = elm.getTupleSparsity();
-			final double postScalingCost = (nCols > 1 && elm.getTupleSparsity() > 0.4) ? numberTuples *
-				nCols : numberTuples * nCols * tupleSparsity;
+			final double postScalingCost = (nCols > 1 && tupleSparsity > 0.4) ? numberTuples * nCols : numberTuples *
+				nCols * tupleSparsity;
 
-			this.cost = preAggregateCost + postScalingCost ;
+			this.cost = preAggregateCost + postScalingCost;
 		}
 
 		@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostTSMM.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostTSMM.java
new file mode 100644
index 0000000..f31c53f
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCostTSMM.java
@@ -0,0 +1,188 @@
+/*
+ * 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.cocode;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorSample;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
+
+public class CoCodeCostTSMM extends AColumnCoCoder {
+
+	protected CoCodeCostTSMM(CompressedSizeEstimator e, CompressionSettings cs) {
+		super(e, cs);
+	}
+
+	@Override
+	protected CompressedSizeInfo coCodeColumns(CompressedSizeInfo colInfos, int k) {
+
+		List<CompressedSizeInfoColGroup> joinRes = join(colInfos.getInfo());
+
+		if(_cs.samplingRatio < 0.1 && _est instanceof CompressedSizeEstimatorSample) {
+			LOG.debug("Performing second join with double sample rate");
+			CompressedSizeEstimatorSample estS = (CompressedSizeEstimatorSample) _est;
+			estS.sampleData(estS.getSample().getNumRows() * 2);
+			List<int[]> colG = new ArrayList<>(joinRes.size());
+			for(CompressedSizeInfoColGroup g : joinRes)
+				colG.add(g.getColumns());
+
+			joinRes = join(estS.computeCompressedSizeInfos(colG, k));
+		}
+
+		colInfos.setInfo(joinRes);
+
+		return colInfos;
+	}
+
+	private List<CompressedSizeInfoColGroup> join(List<CompressedSizeInfoColGroup> currentGroups) {
+
+		Queue<CompressedSizeInfoColGroup> que = new PriorityQueue<>(currentGroups.size(),
+			new Comparator<CompressedSizeInfoColGroup>() {
+				@Override
+				public int compare(CompressedSizeInfoColGroup a, CompressedSizeInfoColGroup b) {
+					final int aNV = a.getNumVals();
+					final int bNV = b.getNumVals();
+					if(aNV == bNV)
+						return 0;
+					else if(aNV > bNV)
+						return 1;
+					else
+						return -1;
+				}
+			});
+
+		List<CompressedSizeInfoColGroup> ret = new ArrayList<>();
+		for(CompressedSizeInfoColGroup g : currentGroups)
+			que.add(g);
+
+		double currentCost = getCost(que, ret);
+		while(true) {
+			if(que.peek() != null) {
+				final CompressedSizeInfoColGroup l = que.poll();
+				if(que.peek() != null) {
+					final CompressedSizeInfoColGroup r = que.poll();
+					final CompressedSizeInfoColGroup g = joinWithAnalysis(l, r);
+					final double newCost = getCost(que, ret, g);
+					if(newCost < currentCost) {
+						currentCost = newCost;
+						que.add(g);
+					}
+					else {
+						ret.add(l);
+						que.add(r);
+					}
+				}
+				else {
+					ret.add(l);
+					break;
+				}
+			}
+			else
+				break;
+		}
+
+		for(CompressedSizeInfoColGroup g : que)
+			ret.add(g);
+
+		return ret;
+	}
+
+	private double getCost(Queue<CompressedSizeInfoColGroup> que, List<CompressedSizeInfoColGroup> ret) {
+		CompressedSizeInfoColGroup[] queValues = que.toArray(new CompressedSizeInfoColGroup[que.size()]);
+		return getCost(queValues, ret);
+	}
+
+	private double getCost(Queue<CompressedSizeInfoColGroup> que, List<CompressedSizeInfoColGroup> ret,
+		CompressedSizeInfoColGroup g) {
+		CompressedSizeInfoColGroup[] queValues = que.toArray(new CompressedSizeInfoColGroup[que.size()]);
+		double cost = getCost(queValues, ret);
+		cost += getCostOfSelfTSMM(g);
+		for(int i = 0; i < queValues.length; i++)
+			cost += getCostOfLeftTransposedMM(queValues[i], g);
+
+		for(int i = 0; i < ret.size(); i++)
+			cost += getCostOfLeftTransposedMM(ret.get(i), g);
+		return cost;
+	}
+
+	private double getCost(CompressedSizeInfoColGroup[] queValues, List<CompressedSizeInfoColGroup> ret) {
+		double cost = 0;
+		for(int i = 0; i < queValues.length; i++) {
+			cost += getCostOfSelfTSMM(queValues[i]);
+			for(int j = i + 1; j < queValues.length; j++)
+				cost += getCostOfLeftTransposedMM(queValues[i], queValues[j]);
+
+			for(CompressedSizeInfoColGroup g : ret)
+				cost += getCostOfLeftTransposedMM(queValues[i], g);
+
+		}
+		for(int i = 0; i < ret.size(); i++) {
+			cost += getCostOfSelfTSMM(ret.get(i));
+			for(int j = i + 1; j < ret.size(); j++)
+				cost += getCostOfLeftTransposedMM(ret.get(i), ret.get(j));
+
+		}
+		return cost;
+	}
+
+	private double getCostOfSelfTSMM(CompressedSizeInfoColGroup g) {
+		double cost = 0;
+		final int nCol = g.getColumns().length;
+		cost += g.getNumVals() * (nCol * (nCol + 1)) / 2;
+		return cost;
+	}
+
+	private double getCostOfLeftTransposedMM(CompressedSizeInfoColGroup gl, CompressedSizeInfoColGroup gr) {
+		final int nRows = _est.getNumRows();
+		final int nColsL = gl.getColumns().length;
+		final int nColsR = gl.getColumns().length;
+
+		// final double preAggLeft = (nRows / (1 - gl.getMostCommonFraction())) * nColsL;
+		// final double preAggRight = (nRows / (1 - gr.getMostCommonFraction())) * nColsR;
+
+		final double preAggLeft = nRows;
+		final double preAggRight = nRows;
+
+		final double tsL = gl.getTupleSparsity();
+		final double tsR = gr.getTupleSparsity();
+
+		// final double tsL = 1;
+		// final double tsR = 1;
+
+		final int nvL = gl.getNumVals();
+		final int nvR = gr.getNumVals();
+
+		final double postScaleLeft = nColsL > 1 && tsL > 0.4 ? nvL * nColsL : nvL * nColsL * tsL;
+		final double postScaleRight = nColsR > 1 && tsR > 0.4 ? nvR * nColsR : nvR * nColsR * tsR;
+
+		final double costLeft = preAggLeft + postScaleLeft * 5;
+		final double costRight = preAggRight + postScaleRight * 5;
+
+		return Math.min(costLeft, costRight);
+	}
+
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
index 3bae0e5..6074d1d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCoder.java
@@ -41,7 +41,17 @@ public class PlanningCoCoder {
 	 * The Valid coCoding techniques
 	 */
 	public enum PartitionerType {
-		BIN_PACKING, STATIC, COST, COST_MATRIX_MULT;
+		BIN_PACKING, STATIC, COST, COST_MATRIX_MULT, COST_TSMM;
+
+		public static boolean isCostBased( PartitionerType pt) {
+			switch(pt) {
+				case COST_MATRIX_MULT:
+				case COST_TSMM:
+					return true;
+				default:
+					return false;
+			}
+		}
 	}
 
 	/**
@@ -100,6 +110,8 @@ public class PlanningCoCoder {
 				return new CoCodeCost(est, cs);
 			case COST_MATRIX_MULT:
 				return new CoCodeCostMatrixMult(est, cs);
+			case COST_TSMM:
+				return new CoCodeCostTSMM(est, cs);
 			default:
 				throw new RuntimeException("Unsupported column group partitioner: " + type.toString());
 		}
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 c1daeb9..0e68d0a 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
@@ -23,19 +23,17 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.utils.MemoryEstimates;
 
-import edu.emory.mathcs.backport.java.util.Arrays;
-
 /**
  * Abstract Class that is the lowest class type for the Compression framework.
  * 
@@ -58,34 +56,6 @@ public abstract class AColGroup implements Serializable {
 	 */
 	protected enum ColGroupType {
 		UNCOMPRESSED, RLE, OLE, DDC, CONST, EMPTY, SDC, SDCSingle, SDCSingleZeros, SDCZeros;
-
-		/**
-		 * Get the super type of the specific ColGroup Type used.
-		 * 
-		 * @param c The concrete ColGroupType
-		 * @return The super CompressionType.
-		 */
-		public static CompressionType getSuperType(ColGroupType c) {
-			switch(c) {
-				case RLE:
-					return CompressionType.RLE;
-				case OLE:
-					return CompressionType.OLE;
-				case DDC:
-					return CompressionType.DDC;
-				case CONST:
-					return CompressionType.CONST;
-				case EMPTY:
-					return CompressionType.EMPTY;
-				case SDC:
-				case SDCSingle:
-				case SDCSingleZeros:
-				case SDCZeros:
-					return CompressionType.SDC;
-				default:
-					return CompressionType.UNCOMPRESSED;
-			}
-		}
 	}
 
 	/** The ColGroup Indexes contained in the ColGroup */
@@ -101,10 +71,6 @@ public abstract class AColGroup implements Serializable {
 	 * @param colIndices offsets of the columns in the matrix block that make up the group
 	 */
 	protected AColGroup(int[] colIndices) {
-		if(colIndices == null)
-			throw new DMLRuntimeException("null input to ColGroup is invalid");
-		if(colIndices.length == 0)
-			throw new DMLRuntimeException("0 is an invalid number of columns in a ColGroup");
 		_colIndexes = colIndices;
 	}
 
@@ -118,16 +84,6 @@ public abstract class AColGroup implements Serializable {
 	}
 
 	/**
-	 * Obtain a column index value.
-	 * 
-	 * @param colNum column number
-	 * @return column index value
-	 */
-	public int getColIndex(int colNum) {
-		return _colIndexes[colNum];
-	}
-
-	/**
 	 * Set the column indexes of the column group.
 	 * 
 	 * @param colIndexes
@@ -183,7 +139,7 @@ public abstract class AColGroup implements Serializable {
 	 * 
 	 * @param offset The offset to move all columns
 	 */
-	public void shiftColIndices(int offset) {
+	public final void shiftColIndices(int offset) {
 		for(int i = 0; i < _colIndexes.length; i++)
 			_colIndexes[i] += offset;
 	}
@@ -193,81 +149,33 @@ public abstract class AColGroup implements Serializable {
 	 * 
 	 * @return an upper bound on the number of bytes used to store this ColGroup in memory.
 	 */
-	public long estimateInMemorySize(){
+	public long estimateInMemorySize() {
 		long size = 16; // object header
 		size += MemoryEstimates.intArrayCost(_colIndexes.length);
 		return size;
 	}
 
 	/**
-	 * Decompress the contents of this column group into the specified full matrix block.
+	 * Decompress the contents of this column group into the specified full matrix block while managing the number of
+	 * non zeros.
 	 * 
 	 * @param target a matrix block where the columns covered by this column group have not yet been filled in.
 	 * @param rl     row lower
 	 * @param ru     row upper
+	 * @param offT   Offset into target to assign from
 	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru) {
-		decompressToBlock(target, rl, ru, rl, true);
-	}
-
-	/**
-	 * Decompress the contents of this column group into the specified full matrix block.
-	 * 
-	 * @param target a matrix block where the columns covered by this column group have not yet been filled in.
-	 * @param rl     The row to start at
-	 * @param ru     The row to end at
-	 * @param offT   The rowOffset into target to decompress to.
-	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlock(target, rl, ru, offT, true);
-	}
-
-
-	/**
-	 * Decompress the contents of this column group into the target matrixBlock, it is assumed that the target matrix
-	 * Block have the same number of columns and at least the number of rows ru.
-	 * 
-	 * @param target The target matrixBlock to decompress into
-	 * @param rl     The row to start at
-	 * @param ru     The row to end at
-	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
-	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, boolean safe) {
-		decompressToBlock(target, rl, ru, rl, safe);
-	}
-
-
-	/**
-	 * Decompress the contents of this column group into the target matrixBlock with an offset of the indexes using the
-	 * values provided as replacement of the dictionary values, it is assumed that the target matrix Block have the same
-	 * number of columns and at least the number of rows ru.
-	 * 
-	 * The offset of indexes makes it possible to decompress parts of the compressed column group like say rows 10 to
-	 * 20, into row 0 to 10 in the target matrix.
-	 * 
-	 * @param target The target matrixBlock to decompress into
-	 * @param rl     The row to start at
-	 * @param ru     The row to end at
-	 * @param offT   The offset into the target to decompress to.
-	 * @param safe   Boolean specifying if the operation should be safe, aka counting nnz.
-	 */
-	public void decompressToBlock(MatrixBlock target, int rl, int ru, int offT,  boolean safe) {
-		if(safe)
-			decompressToBlockSafe(target, rl, ru, offT);
-		else
-			decompressToBlockUnSafe(target, rl, ru, offT);
-	}
+	public abstract void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT);
 
 	/**
-	 * Decompress the contents of this column group into the specified full matrix block while managing the number of
-	 * non zeros.
+	 * Decompress the contents of the columngroup unsafely, meaning that it does not count nonzero values.
 	 * 
 	 * @param target a matrix block where the columns covered by this column group have not yet been filled in.
 	 * @param rl     row lower
 	 * @param ru     row upper
-	 * @param offT   Offset into target to assign from
 	 */
-	public abstract void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT);
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru) {
+		decompressToBlockUnSafe(target, rl, ru, rl);
+	}
 
 	/**
 	 * Decompress the contents of the columngroup unsafely, meaning that it does not count nonzero values.
@@ -279,136 +187,31 @@ public abstract class AColGroup implements Serializable {
 	 */
 	public abstract void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT);
 
-	/**
-	 * Decompress the contents of this column group into uncompressed packed columns
-	 * 
-	 * @param target          a dense matrix block. The block must have enough space to hold the contents of this column
-	 *                        group.
-	 * @param colIndexTargets array that maps column indices in the original matrix block to columns of target.
-	 */
-	public abstract void decompressToBlock(MatrixBlock target, int[] colIndexTargets);
-
-	/**
-	 * Decompress an entire column into the target matrix block. This decompression maintain the number of non zeros.
-	 * This method assumes that the Matrix block that is decompressed into has a column for the values to decompress
-	 * into.
-	 * 
-	 * @param target    Target matrix block to decompress into.
-	 * @param colIndex  The column index to decompress.
-	 * @param colGroups The list of column groups to decompress.
-	 */
-	public static void decompressColumnToBlock(MatrixBlock target, int colIndex, List<AColGroup> colGroups) {
-		for(AColGroup g : colGroups) {
-			int groupColIndex = Arrays.binarySearch(g._colIndexes, colIndex);
-			if(groupColIndex >= 0) {
-				g.decompressColumnToBlock(target, groupColIndex);
-			}
-		}
-	}
-
-	/**
-	 * Find all column groups with the given index and decompress them into the target double array summing the values.
-	 * 
-	 * If the column is not found nothing is decompressed.
-	 * 
-	 * @param target    The target column array to decompress into
-	 * @param colIndex  The Column index to find in the list of column groups
-	 * @param colGroups The column Groups to search in.
-	 */
-	public static void decompressColumnToArray(double[] target, int colIndex, List<AColGroup> colGroups) {
-		for(AColGroup g : colGroups) {
-			int groupColIndex = Arrays.binarySearch(g._colIndexes, colIndex);
-			if(groupColIndex >= 0) {
-				g.decompressColumnToBlock(target, groupColIndex, 0, g.getNumRows());
-			}
-		}
-	}
+	// /**
+	//  * Decompress the contents of this column group into uncompressed packed columns
+	//  * 
+	//  * @param target          a dense matrix block. The block must have enough space to hold the contents of this column
+	//  *                        group.
+	//  * @param colIndexTargets array that maps column indices in the original matrix block to columns of target.
+	//  */
+	// public abstract void decompressToBlock(MatrixBlock target, int[] colIndexTargets);
 
 	/**
 	 * Decompress part of the col groups into the target matrix block, this decompression maintain the number of non
 	 * zeros.
 	 * 
 	 * @param target    The Target matrix block to decompress into
-	 * @param colIndex  The column index to decompress.
 	 * @param rl        The row to start the decompression from
 	 * @param ru        The row to end the decompression at
 	 * @param colGroups The list of column groups to decompress.
 	 */
-	public static void decompressColumnToBlock(MatrixBlock target, int colIndex, int rl, int ru,
+	public final static void decompressColumnToBlockUnSafe(MatrixBlock target, int rl, int ru,
 		List<AColGroup> colGroups) {
-		for(AColGroup g : colGroups) {
-			int groupColIndex = Arrays.binarySearch(g._colIndexes, colIndex);
-			if(groupColIndex >= 0) {
-				g.decompressColumnToBlock(target, groupColIndex, rl, ru);
-			}
-		}
-	}
-
-	/**
-	 * Decompress part of the col groups into the target matrix block, this decompression maintain the number of non
-	 * zeros.
-	 * 
-	 * @param target    The Target matrix block to decompress into
-	 * @param rl        The row to start the decompression from
-	 * @param ru        The row to end the decompression at
-	 * @param colGroups The list of column groups to decompress.
-	 */
-	public static void decompressColumnToBlockUnSafe(MatrixBlock target, int rl, int ru, List<AColGroup> colGroups) {
 		for(AColGroup g : colGroups)
 			g.decompressToBlockUnSafe(target, rl, ru, rl);
 	}
 
 	/**
-	 * Decompress part of the col groups into the target dense double array. This assumes that the double array is a row
-	 * linearized matrix double array.
-	 * 
-	 * This is much faster than decompressing into a target matrix block since nnz is not managed.
-	 * 
-	 * @param target    Target double array to decompress into
-	 * @param colIndex  The column index to decompress.
-	 * @param rl        The row to start decompression from
-	 * @param ru        The row to end the decompression at
-	 * @param colGroups The list of column groups to decompress.
-	 */
-	public static void decompressColumnToBlock(double[] target, int colIndex, int rl, int ru,
-		List<AColGroup> colGroups) {
-		for(AColGroup g : colGroups) {
-			int groupColIndex = Arrays.binarySearch(g._colIndexes, colIndex);
-			if(groupColIndex >= 0) {
-				g.decompressColumnToBlock(target, groupColIndex, rl, ru);
-			}
-		}
-	}
-
-	/**
-	 * Decompress to block.
-	 * 
-	 * @param target dense output vector
-	 * @param colpos column to decompress, error if larger or equal numCols
-	 */
-	public abstract void decompressColumnToBlock(MatrixBlock target, int colpos);
-
-	/**
-	 * Decompress to block.
-	 * 
-	 * @param target dense output vector
-	 * @param colpos column to decompress, error if larger or equal numCols
-	 * @param rl     the Row to start decompression from
-	 * @param ru     the Row to end decompression at
-	 */
-	public abstract void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru);
-
-	/**
-	 * Decompress to dense array.
-	 * 
-	 * @param target dense output vector double array.
-	 * @param colpos column to decompress, error if larger or equal numCols
-	 * @param rl     the Row to start decompression from
-	 * @param ru     the Row to end decompression at
-	 */
-	public abstract void decompressColumnToBlock(double[] target, int colpos, int rl, int ru);
-
-	/**
 	 * Serializes column group to data output.
 	 * 
 	 * @param out data output
@@ -477,7 +280,7 @@ public abstract class AColGroup implements Serializable {
 	public abstract AColGroup rightMultByMatrix(MatrixBlock right);
 
 	/**
-	 * Do a transposed self matrix multiplication, but only with this column group.
+	 * Do a transposed self matrix multiplication on the left side t(x) %*% x. but only with this column group.
 	 * 
 	 * This gives better performance since there is no need to iterate through all the rows of the matrix, but the
 	 * execution can be limited to its number of distinct values.
@@ -490,13 +293,29 @@ public abstract class AColGroup implements Serializable {
 	public abstract void tsmm(double[] result, int numColumns);
 
 	/**
+	 * Do a transposed self matrix multiplication on the left side t(x) %*% x. but only with this column group, and only
+	 * on a subset of the columns contained in this columnGroup.
+	 * 
+	 * This gives better performance since there is no need to iterate through all the rows of the matrix, but the
+	 * execution can be limited to its number of distinct values.
+	 * 
+	 * Note it only calculate the upper triangle
+	 * 
+	 * @param result     A row major dense allocation of a matrixBlock, of size [numColumns x numColumns]
+	 * @param numColumns The number of columns in the row major result matrix.
+	 * @param idxStart   The starting index in the _colIndexes.
+	 * @param idxEnd     The ending index in the _colIndexes.
+	 */
+	public abstract void tsmm(double[] result, int numColumns, int idxStart, int idxEnd);
+
+	/**
 	 * Left multiply with this column group
 	 * 
 	 * @param matrix The matrix to multiply with on the left
 	 * @param result The result to output the values into, always dense for the purpose of the column groups
 	 *               parallelizing
 	 */
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result) {
+	public final void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result) {
 		leftMultByMatrix(matrix, result, 0, matrix.getNumRows());
 	}
 
@@ -570,14 +389,6 @@ public abstract class AColGroup implements Serializable {
 	public abstract void countNonZerosPerRow(int[] rnnz, int rl, int ru);
 
 	/**
-	 * Base class for column group row iterators. We do not implement the default Iterator interface in order to avoid
-	 * unnecessary value copies per group.
-	 */
-	protected abstract class ColGroupRowIterator {
-		public abstract void next(double[] buff, int rowIx, int segIx, boolean last);
-	}
-
-	/**
 	 * Is Lossy
 	 * 
 	 * @return returns if the ColGroup is compressed in a lossy manner.
@@ -603,7 +414,7 @@ public abstract class AColGroup implements Serializable {
 	 * @return A cloned Column Group, with a copied pointer to the old column groups index structure, but reduced
 	 *         dictionary and _columnIndexes correctly aligned with the expected sliced compressed matrix.
 	 */
-	public AColGroup sliceColumns(int cl, int cu) {
+	public final AColGroup sliceColumns(int cl, int cu) {
 		AColGroup ret = (cu - cl == 1) ? sliceColumn(cl) : sliceMultiColumns(cl, cu);
 		return ret;
 	}
@@ -615,7 +426,7 @@ public abstract class AColGroup implements Serializable {
 	 * @return A new column group that is a single column, if the column requested is not in this column group null is
 	 *         returned.
 	 */
-	public AColGroup sliceColumn(int col) {
+	public final AColGroup sliceColumn(int col) {
 		int idx = Arrays.binarySearch(_colIndexes, col);
 		if(idx >= 0)
 			return sliceSingleColumn(idx);
@@ -631,7 +442,7 @@ public abstract class AColGroup implements Serializable {
 	 * @return A column group of this containing the columns specified, returns null if the columns specified is not
 	 *         contained in the column group
 	 */
-	protected AColGroup sliceMultiColumns(int cl, int cu) {
+	protected final AColGroup sliceMultiColumns(int cl, int cu) {
 		int idStart = 0;
 		int idEnd = 0;
 		for(int i = 0; i < _colIndexes.length; i++) {
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
index 3b598e6..a643c45 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
@@ -24,7 +24,6 @@ import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
 import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
-import org.apache.sysds.runtime.functionobjects.Mean;
 import org.apache.sysds.runtime.functionobjects.Plus;
 import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.functionobjects.ReduceCol;
@@ -64,20 +63,13 @@ public abstract class ColGroupCompressed extends AColGroup {
 
 	public abstract boolean isLossy();
 
-	/**
-	 * if -1 is returned it means false, otherwise it returns an index where the zero tuple can be found.
-	 * 
-	 * @return A Index where the zero tuple can be found.
-	 */
-	protected abstract int containsAllZeroTuple();
-
 	protected abstract double computeMxx(double c, Builtin builtin);
 
 	protected abstract void computeColMxx(double[] c, Builtin builtin);
 
 	protected abstract void computeSum(double[] c, boolean square);
 
-	protected abstract void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean);
+	protected abstract void computeRowSums(double[] c, boolean square, int rl, int ru);
 
 	protected abstract void computeColSums(double[] c, boolean square);
 
@@ -102,30 +94,31 @@ public abstract class ColGroupCompressed extends AColGroup {
 
 	@Override
 	public final void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int rl, int ru) {
-		// sum and sumsq (reduceall/reducerow over tuples and counts)
 		if(op.aggOp.increOp.fn instanceof Plus || op.aggOp.increOp.fn instanceof KahanPlus ||
 			op.aggOp.increOp.fn instanceof KahanPlusSq) {
 			boolean square = op.aggOp.increOp.fn instanceof KahanPlusSq;
-			boolean mean = op.aggOp.increOp.fn instanceof Mean;
 			if(op.indexFn instanceof ReduceAll)
 				computeSum(c, square);
 			else if(op.indexFn instanceof ReduceCol)
-				computeRowSums(c, square, rl, ru, mean);
+				computeRowSums(c, square, rl, ru);
 			else if(op.indexFn instanceof ReduceRow)
 				computeColSums(c, square);
 		}
-		// min and max (reduceall/reducerow over tuples only)
-		else if(op.aggOp.increOp.fn instanceof Builtin &&
-			(((Builtin) op.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MAX ||
-				((Builtin) op.aggOp.increOp.fn).getBuiltinCode() == BuiltinCode.MIN)) {
-			Builtin builtin = (Builtin) op.aggOp.increOp.fn;
+		else if(op.aggOp.increOp.fn instanceof Builtin) {
+			Builtin bop = (Builtin) op.aggOp.increOp.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);
+				else if(op.indexFn instanceof ReduceRow)
+					computeColMxx(c, bop);
+			}
+			else {
+				throw new DMLScriptException("unsupported builtin type: " + bop);
+			}
 
-			if(op.indexFn instanceof ReduceAll)
-				c[0] = computeMxx(c[0], builtin);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowMxx(c, builtin, rl, ru);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColMxx(c, builtin);
 		}
 		else {
 			throw new DMLScriptException("Unknown UnaryAggregate operator on CompressedMatrixBlock");
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 d439c4e..019c6e1 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
@@ -25,8 +25,6 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.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.pre.ArrPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -46,20 +44,6 @@ public class ColGroupConst extends ColGroupValue {
 		super(numRows);
 	}
 
-	public static ColGroupConst genColGroupConst(int numRows, int numCols, double value) {
-
-		int[] colIndices = new int[numCols];
-		for(int i = 0; i < numCols; i++)
-			colIndices[i] = i;
-
-		double[] values = new double[numCols];
-		for(int i = 0; i < numCols; i++)
-			values[i] = value;
-
-		ADictionary dict = new Dictionary(values);
-		return new ColGroupConst(colIndices, numRows, dict);
-	}
-
 	/**
 	 * Constructs an Constant Colum Group, that contains only one tuple, with the given value.
 	 * 
@@ -84,18 +68,13 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		double vals = _dict.sumAllRowsToDouble(square, _colIndexes.length)[0];
 		for(int rix = rl; rix < ru; rix++)
 			c[rix] += vals;
 	}
 
 	@Override
-	protected void computeColSums(double[] c, boolean square) {
-		_dict.colSum(c, getCounts(), _colIndexes, square);
-	}
-
-	@Override
 	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru) {
 		double value = _dict.aggregateTuples(builtin, _colIndexes.length)[0];
 		for(int i = rl; i < ru; i++)
@@ -113,15 +92,9 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(_colIndexes.length * target.getNumRows() + target.getNonZeros());
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		double[] c = target.getDenseBlockValues();
-		double[] values = getValues();
 		offT = offT * target.getNumColumns();
 		for(int i = rl; i < ru; i++, offT += target.getNumColumns())
 			for(int j = 0; j < _colIndexes.length; j++)
@@ -129,113 +102,156 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		int ncol = getNumCols();
-		double[] values = getValues();
-		for(int i = 0; i < _numRows; i++)
-			for(int colIx = 0; colIx < ncol; colIx++) {
-				int origMatrixColIx = getColIndex(colIx);
-				int col = colIndexTargets[origMatrixColIx];
-				double cellVal = values[colIx];
-				target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
-			}
-
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock values) {
+		throw new NotImplementedException();
 	}
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colPos) {
-		double[] c = target.getDenseBlockValues();
-		double v = _dict.getValue(colPos);
-		if(v != 0)
-			for(int i = 0; i < c.length; i++)
-				c[i] += v;
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	int ncol = getNumCols();
+	// 	double[] values = getValues();
+	// 	for(int i = 0; i < _numRows; i++)
+	// 		for(int colIx = 0; colIx < ncol; colIx++) {
+	// 			int origMatrixColIx = _colIndexes[colIx];
+	// 			int col = colIndexTargets[origMatrixColIx];
+	// 			double cellVal = values[colIx];
+	// 			target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
+	// 		}
 
-		target.setNonZeros(_numRows);
+	// }
 
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colPos) {
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double v = _dict.getValue(colPos);
+	// 	if(v != 0)
+	// 		for(int i = 0; i < c.length; i++)
+	// 			c[i] += v;
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colPos, int rl, int ru) {
-		double[] c = target.getDenseBlockValues();
-		double v = _dict.getValue(colPos);
-		final int length = ru - rl;
-		if(v != 0)
-			for(int i = 0; i < length; i++)
-				c[i] += v;
+	// 	target.setNonZeros(_numRows);
 
-		target.setNonZeros(_numRows);
-	}
+	// }
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colPos, int rl, int ru) {
-		double v = _dict.getValue(colPos);
-		final int length = ru - rl;
-		if(v != 0)
-			for(int i = 0; i < length; i++)
-				c[i] += v;
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colPos, int rl, int ru) {
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double v = _dict.getValue(colPos);
+	// 	final int length = ru - rl;
+	// 	if(v != 0)
+	// 		for(int i = 0; i < length; i++)
+	// 			c[i] += v;
 
-	}
+	// 	target.setNonZeros(_numRows);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colPos, int rl, int ru) {
+	// 	double v = _dict.getValue(colPos);
+	// 	final int length = ru - rl;
+	// 	if(v != 0)
+	// 		for(int i = 0; i < length; i++)
+	// 			c[i] += v;
+
+	// }
 
 	@Override
 	public double get(int r, int c) {
 		return _dict.getValue(Arrays.binarySearch(_colIndexes, c));
 	}
 
-	public double[] preAggregate(double[] a, int row) {
-		return new double[] {preAggregateSingle(a, row)};
-	}
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// return new double[] {preAggregateSingle(a, row)};
+	// }
 
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
-		return new double[] {preAggregateSparseSingle(sb, row)};
-	}
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// return new double[] {preAggregateSparseSingle(sb, row)};
+	// }
 
-	public double preAggregateSparseSingle(SparseBlock sb, int row) {
-		double v = 0;
-		double[] sparseV = sb.values(row);
-		for(int i = sb.pos(row); i < sb.pos(row) + sb.size(row); i++) {
-			v += sparseV[i];
-		}
-		return v;
-	}
-
-	private double preAggregateSingle(double[] a, int row) {
-		double vals = 0;
-		for(int off = _numRows * row; off < _numRows * row + _numRows; off++)
-			vals += a[off];
-		return vals;
+	@Override
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
+		else
+			preAggregateDense(m, preAgg, rl, ru);
 	}
 
-	@Override
-	public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, int rl, int ru) {
-		final double[] cV = c.getDenseBlockValues();
-		final double[] values = getValues();
-		if(values == null  || a.isEmpty())
-			return;
-		else if(a.isInSparseFormat()) {
-			SparseBlock sb = a.getSparseBlock();
-			for(int i = rl; i < ru; i++) {
-
-				if(!sb.isEmpty(i)) {
-					double v = preAggregateSparseSingle(sb, i);
-					int offC = i * c.getNumColumns();
-					for(int j = 0; j < _colIndexes.length; j++)
-						cV[offC + _colIndexes[j]] += v * values[j];
-
-				}
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			for(int rc = 0, offLeft = rowLeft * _numRows; rc < _numRows; rc++, offLeft++) {
+				preAV[offOut] += mV[offLeft];
 			}
 		}
-		else {
-			double[] aV = a.getDenseBlockValues();
-			for(int i = rl; i < ru; i++) {
-				double preAggVals = preAggregateSingle(aV, i);
-				int offC = i * c.getNumColumns();
-				for(int j = 0; j < _colIndexes.length; j++)
-					cV[offC + _colIndexes[j]] += preAggVals * values[j];
+	}
 
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final double[] avals = sb.values(rowLeft);
+			for(int j = apos; j < alen; j++) {
+				preAV[offOut] += avals[j];
 			}
 		}
 	}
 
+
+	// public double preAggregateSparseSingle(SparseBlock sb, int row) {
+	// double v = 0;
+	// double[] sparseV = sb.values(row);
+	// for(int i = sb.pos(row); i < sb.pos(row) + sb.size(row); i++) {
+	// v += sparseV[i];
+	// }
+	// return v;
+	// }
+
+	// private double preAggregateSingle(double[] a, int row) {
+	// double vals = 0;
+	// for(int off = _numRows * row; off < _numRows * row + _numRows; off++)
+	// vals += a[off];
+	// return vals;
+	// }
+
+	// @Override
+	// public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, int rl, int ru) {
+	// final double[] cV = c.getDenseBlockValues();
+	// final double[] values = getValues();
+	// if(values == null || a.isEmpty())
+	// return;
+	// else if(a.isInSparseFormat()) {
+	// SparseBlock sb = a.getSparseBlock();
+	// for(int i = rl; i < ru; i++) {
+
+	// if(!sb.isEmpty(i)) {
+	// double v = preAggregateSparseSingle(sb, i);
+	// int offC = i * c.getNumColumns();
+	// for(int j = 0; j < _colIndexes.length; j++)
+	// cV[offC + _colIndexes[j]] += v * values[j];
+
+	// }
+	// }
+	// }
+	// else {
+	// double[] aV = a.getDenseBlockValues();
+	// for(int i = rl; i < ru; i++) {
+	// double preAggVals = preAggregateSingle(aV, i);
+	// int offC = i * c.getNumColumns();
+	// for(int j = 0; j < _colIndexes.length; j++)
+	// cV[offC + _colIndexes[j]] += preAggVals * values[j];
+
+	// }
+	// }
+	// }
+
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		return new ColGroupConst(_colIndexes, _numRows, applyScalarOp(op));
@@ -264,40 +280,40 @@ public class ColGroupConst extends ColGroupValue {
 		throw new NotImplementedException("This function should not be called");
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		return new ArrPreAggregate(lhs.getCounts());
-	}
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	return new ArrPreAggregate(lhs.getCounts());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -325,17 +341,7 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	protected int containsAllZeroTuple() {
-		return -1;
-	}
-
-	@Override
 	protected boolean sameIndexStructure(ColGroupCompressed that) {
 		return that instanceof ColGroupEmpty || that instanceof ColGroupConst;
 	}
-
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
 }
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 42cf04b..6cdbe4e 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
@@ -24,17 +24,11 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
-import org.apache.sysds.runtime.compress.colgroup.dictionary.MatrixBlockDictionary;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
-import org.apache.sysds.runtime.data.DenseBlock;
-import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -65,32 +59,10 @@ public class ColGroupDDC extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(target.getNonZeros() + _numRows * _colIndexes.length);
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
-		final int nCol = _colIndexes.length;
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock sb) {
 		final int tCol = target.getNumColumns();
 		final double[] c = target.getDenseBlockValues();
-		if(_dict instanceof MatrixBlockDictionary) {
-			MatrixBlock dmb = ((MatrixBlockDictionary) _dict).getMatrixBlock();
-			if(dmb.isEmpty())
-				return;
-			else if(dmb.isInSparseFormat())
-				decompressToBlockUnsafeSparse(c, rl, ru, offT, dmb.getSparseBlock(), tCol, nCol);
-			else
-				decompressToBlockUnsafeDense(c, rl, ru, offT, dmb.getDenseBlockValues(), tCol, nCol);
-		}
-		else
-			decompressToBlockUnsafeDense(c, rl, ru, offT, getValues(), tCol, nCol);
-
-	}
-
-	private void decompressToBlockUnsafeSparse(double[] c, int rl, int ru, int offT, SparseBlock sb, int tCol,
-		int nCol) {
 		offT = offT * tCol;
 		for(int i = rl; i < ru; i++, offT += tCol) {
 			final int rowIndex = _data.getIndex(i);
@@ -106,9 +78,12 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 	}
 
-	private void decompressToBlockUnsafeDense(double[] c, int rl, int ru, int offT, double[] values, int tCol,
-		int nCol) {
-		// final double[] values = getValues();
+	@Override
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
+		final int nCol = _colIndexes.length;
+		final int tCol = target.getNumColumns();
+		final double[] c = target.getDenseBlockValues();
 		offT = offT * tCol;
 
 		for(int i = rl; i < ru; i++, offT += tCol) {
@@ -118,67 +93,67 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 	}
 
-	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		int ncol = getNumCols();
-		double[] dictionary = getValues();
-		for(int i = 0; i < _numRows; i++) {
-			int rowIndex = _data.getIndex(i) * ncol;
-			for(int colIx = 0; colIx < ncol; colIx++) {
-				int origMatrixColIx = getColIndex(colIx);
-				int col = colIndexTargets[origMatrixColIx];
-				double cellVal = dictionary[rowIndex + colIx];
-				target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
-			}
-
-		}
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		int ncol = getNumCols();
-		double[] c = target.getDenseBlockValues();
-		double[] values = getValues();
-		int nnz = 0;
-		for(int i = 0; i < _numRows; i++) {
-			int index = _data.getIndex(i);
-			if(index < getNumValues())
-				nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
-			else
-				nnz++;
-
-		}
-		target.setNonZeros(nnz);
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		int ncol = getNumCols();
-		double[] c = target.getDenseBlockValues();
-		double[] values = getValues();
-		final int numValues = getNumValues();
-		int nnz = 0;
-		for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-			int index = _data.getIndex(r);
-			if(index < numValues)
-				nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
-			else
-				nnz++;
-		}
-		target.setNonZeros(nnz);
-	}
-
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		int ncol = getNumCols();
-		double[] values = getValues();
-		final int numValues = getNumValues();
-		for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-			int index = _data.getIndex(r);
-			if(index < numValues)
-				c[i] += values[(index) * ncol + colpos];
-		}
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	int ncol = getNumCols();
+	// 	double[] dictionary = getValues();
+	// 	for(int i = 0; i < _numRows; i++) {
+	// 		int rowIndex = _data.getIndex(i) * ncol;
+	// 		for(int colIx = 0; colIx < ncol; colIx++) {
+	// 			int origMatrixColIx = _colIndexes[colIx];
+	// 			int col = colIndexTargets[origMatrixColIx];
+	// 			double cellVal = dictionary[rowIndex + colIx];
+	// 			target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
+	// 		}
+
+	// 	}
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	int ncol = getNumCols();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double[] values = getValues();
+	// 	int nnz = 0;
+	// 	for(int i = 0; i < _numRows; i++) {
+	// 		int index = _data.getIndex(i);
+	// 		if(index < getNumValues())
+	// 			nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
+	// 		else
+	// 			nnz++;
+
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	int ncol = getNumCols();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double[] values = getValues();
+	// 	final int numValues = getNumValues();
+	// 	int nnz = 0;
+	// 	for(int i = 0, r = rl; i < ru - rl; i++, r++) {
+	// 		int index = _data.getIndex(r);
+	// 		if(index < numValues)
+	// 			nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
+	// 		else
+	// 			nnz++;
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	int ncol = getNumCols();
+	// 	double[] values = getValues();
+	// 	final int numValues = getNumValues();
+	// 	for(int i = 0, r = rl; i < ru - rl; i++, r++) {
+	// 		int index = _data.getIndex(r);
+	// 		if(index < numValues)
+	// 			c[i] += values[(index) * ncol + colpos];
+	// 	}
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -214,7 +189,7 @@ public class ColGroupDDC extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		double[] vals = _dict.sumAllRowsToDouble(square, _colIndexes.length);
 		for(int rix = rl; rix < ru; rix++)
 			c[rix] += vals[_data.getIndex(rix)];
@@ -242,54 +217,89 @@ public class ColGroupDDC extends ColGroupValue {
 		return counts;
 	}
 
-	@Override
-	public double[] preAggregate(double[] a, int row) {
-		double[] vals = allocDVector(getNumValues(), true);
-		if(row > 0)
-			for(int i = 0, off = _numRows * row; i < _numRows; i++, off++)
-				vals[_data.getIndex(i)] += a[off];
-		else
-			for(int i = 0; i < _numRows; i++)
-				vals[_data.getIndex(i)] += a[i];
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// double[] vals = allocDVector(getNumValues(), true);
+	// if(row > 0)
+	// for(int i = 0, off = _numRows * row; i < _numRows; i++, off++)
+	// vals[_data.getIndex(i)] += a[off];
+	// else
+	// for(int i = 0; i < _numRows; i++)
+	// vals[_data.getIndex(i)] += a[i];
 
-		return vals;
-	}
+	// return vals;
+	// }
 
-	@Override
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
 
-		double[] vals = allocDVector(getNumValues(), true);
-		int[] indexes = sb.indexes(row);
-		double[] sparseV = sb.values(row);
-		for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++)
-			vals[_data.getIndex(indexes[i])] += sparseV[i];
-		return vals;
+	// double[] vals = allocDVector(getNumValues(), true);
+	// int[] indexes = sb.indexes(row);
+	// double[] sparseV = sb.values(row);
+	// for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++)
+	// vals[_data.getIndex(indexes[i])] += sparseV[i];
+	// return vals;
 
-	}
+	// }
 
 	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-
-		final int retCols = getNumValues();
-		final int retRows = ru - rl;
-		final double[] vals = allocDVector(retRows * retCols, true);
-		final DenseBlock retB = new DenseBlockFP64(new int[] {retRows, retCols}, vals);
-		final MatrixBlock ret = new MatrixBlock(retRows, retCols, retB);
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
+		else
+			preAggregateDense(m, preAgg, rl, ru);
+	}
 
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
 		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			for(int rc = 0, offLeft = rowLeft * _numRows; rc < _numRows; rc++, offLeft++) {
+				preAV[offOut + _data.getIndex(rc)] += mV[offLeft];
+			}
+		}
+	}
 
-		ret.setNonZeros(retRows * retCols);
-		for(int k = rl; k < ru; k++) {
-			final int offT = ret.getNumColumns() * k;
-			final int offM = m.getNumColumns() * k;
-			for(int i = 0; i < _numRows; i++) {
-				int index = _data.getIndex(i);
-				vals[offT + index] += mV[offM + i];
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final int[] aix = sb.indexes(rowLeft);
+			final double[] avals = sb.values(rowLeft);
+			for(int j = apos; j < alen; j++) {
+				preAV[offOut + _data.getIndex(aix[j])] += avals[j];
 			}
 		}
-		return ret;
 	}
 
+	// @Override
+	// public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+
+	// final int retCols = getNumValues();
+	// final int retRows = ru - rl;
+	// final double[] vals = allocDVector(retRows * retCols, true);
+	// final DenseBlock retB = new DenseBlockFP64(new int[] {retRows, retCols}, vals);
+	// final MatrixBlock ret = new MatrixBlock(retRows, retCols, retB);
+
+	// final double[] mV = m.getDenseBlockValues();
+
+	// ret.setNonZeros(retRows * retCols);
+	// for(int k = rl; k < ru; k++) {
+	// final int offT = ret.getNumColumns() * k;
+	// final int offM = m.getNumColumns() * k;
+	// for(int i = 0; i < _numRows; i++) {
+	// int index = _data.getIndex(i);
+	// vals[offT + index] += mV[offM + i];
+	// }
+	// }
+	// return ret;
+	// }
+
 	/**
 	 * Generic get value for byte-length-agnostic access to first column.
 	 * 
@@ -325,158 +335,158 @@ public class ColGroupDDC extends ColGroupValue {
 		_data.set(r, code);
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		// int[] m = _data.materializeMultiplied(nCol);
-		for(int i = 0; i < this._numRows; i++)
-			ag.increment(lhs._data.getIndex(i) + this._data.getIndex(i) * nCol);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		AIterator lIt = lhs._indexes.getIterator();
-		final int offsetToDefault = nCol - 1;
-
-		int i = 0;
-
-		int col;
-		for(; i < this._numRows && lIt.hasNext(); i++) {
-			int row = this._data.getIndex(i);
-			if(lIt.value() == i)
-				col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
-
-			else
-				col = offsetToDefault;
-			ag.increment(col + row * nCol);
-		}
-		col = offsetToDefault;
-		for(; i < this._numRows; i++) {
-			int row = this._data.getIndex(i);
-			ag.increment(col + row * nCol);
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator lIt = lhs._indexes.getIterator();
-
-		int i = 0;
-
-		int col;
-		for(; i < this._numRows && lIt.hasNext(); i++) {
-			int row = this._data.getIndex(i);
-			if(lIt.value() == i) {
-				col = 1;
-				lIt.next();
-			}
-			else
-				col = 0;
-			ag.increment(col + row * nCol);
-		}
-
-		for(; i < this._numRows; i++)
-			ag.increment(this._data.getIndex(i) * nCol);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator lIt = lhs._indexes.getIterator();
-
-		while(lIt.hasNext()) {
-			int row = this._data.getIndex(lIt.value());
-			int col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
-			ag.increment(col + row * nCol);
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		final AIterator lIt = lhs._indexes.getIterator();
-
-		while(lIt.hasNext()) {
-			int row = this._data.getIndex(lIt.value());
-			lIt.next();
-			ag.increment(row);
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kl = 0; kl < NVL; kl++) {
-			final int bOffL = lhs._ptr[kl];
-			final int bLenL = lhs.len(kl);
-			for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-				sLenL = lhs._data[bOffL + bixL];
-				for(int i = 1; i <= sLenL; i++) {
-					int idx = this._data.getIndex(offL + lhs._data[bOffL + bixL + i]);
-					ag.increment(kl + idx * NVL);
-				}
-			}
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kl = 0; kl < NVL; kl++) {
-			final int boffL = lhs._ptr[kl];
-			final int blenL = lhs.len(kl);
-			for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-				startL += lhs._data[boffL + bixL];
-				lenL = lhs._data[boffL + bixL + 1];
-				final int endL = startL + lenL;
-				for(int i = startL; i < endL; i++) {
-					int kr = _data.getIndex(i) * NVL;
-					ag.increment(kl + kr);
-				}
-			}
-		}
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	// int[] m = _data.materializeMultiplied(nCol);
+	// 	for(int i = 0; i < this._numRows; i++)
+	// 		ag.increment(lhs._data.getIndex(i) + this._data.getIndex(i) * nCol);
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	AIterator lIt = lhs._indexes.getIterator();
+	// 	final int offsetToDefault = nCol - 1;
+
+	// 	int i = 0;
+
+	// 	int col;
+	// 	for(; i < this._numRows && lIt.hasNext(); i++) {
+	// 		int row = this._data.getIndex(i);
+	// 		if(lIt.value() == i)
+	// 			col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
+
+	// 		else
+	// 			col = offsetToDefault;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+	// 	col = offsetToDefault;
+	// 	for(; i < this._numRows; i++) {
+	// 		int row = this._data.getIndex(i);
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+
+	// 	int i = 0;
+
+	// 	int col;
+	// 	for(; i < this._numRows && lIt.hasNext(); i++) {
+	// 		int row = this._data.getIndex(i);
+	// 		if(lIt.value() == i) {
+	// 			col = 1;
+	// 			lIt.next();
+	// 		}
+	// 		else
+	// 			col = 0;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	for(; i < this._numRows; i++)
+	// 		ag.increment(this._data.getIndex(i) * nCol);
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+
+	// 	while(lIt.hasNext()) {
+	// 		int row = this._data.getIndex(lIt.value());
+	// 		int col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+
+	// 	while(lIt.hasNext()) {
+	// 		int row = this._data.getIndex(lIt.value());
+	// 		lIt.next();
+	// 		ag.increment(row);
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kl = 0; kl < NVL; kl++) {
+	// 		final int bOffL = lhs._ptr[kl];
+	// 		final int bLenL = lhs.len(kl);
+	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
+	// 			sLenL = lhs._data[bOffL + bixL];
+	// 			for(int i = 1; i <= sLenL; i++) {
+	// 				int idx = this._data.getIndex(offL + lhs._data[bOffL + bixL + i]);
+	// 				ag.increment(kl + idx * NVL);
+	// 			}
+	// 		}
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kl = 0; kl < NVL; kl++) {
+	// 		final int boffL = lhs._ptr[kl];
+	// 		final int blenL = lhs.len(kl);
+	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
+	// 			startL += lhs._data[boffL + bixL];
+	// 			lenL = lhs._data[boffL + bixL + 1];
+	// 			final int endL = startL + lenL;
+	// 			for(int i = startL; i < endL; i++) {
+	// 				int kr = _data.getIndex(i) * NVL;
+	// 				ag.increment(kl + kr);
+	// 			}
+	// 		}
+	// 	}
+	// 	return ag;
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -615,7 +625,7 @@ public class ColGroupDDC extends ColGroupValue {
 	@Override
 	public void readFields(DataInput in) throws IOException {
 		super.readFields(in);
-		_data = MapToFactory.readIn(in, getNumValues());
+		_data = MapToFactory.readIn(in);
 	}
 
 	@Override
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 11adb66..e1ade47 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
@@ -57,7 +57,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		// do nothing
 	}
 
@@ -83,7 +83,6 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		return ColGroupType.EMPTY;
 	}
 
-
 	@Override
 	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
 		// do nothing.
@@ -94,25 +93,25 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		// do nothing.
 	}
 
-	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		// do nothing.
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	// do nothing.
+	// }
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		// do nothing.
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	// do nothing.
+	// }
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		// do nothing.
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	// do nothing.
+	// }
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		// do nothing.
-	}
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	// do nothing.
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -130,7 +129,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		if(val0 == 0)
 			return this;
 		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[_colIndexes.length]).applyScalarOp(op, val0, _colIndexes.length));
+			new Dictionary(new double[_colIndexes.length]).apply(op));
 	}
 
 	@Override
@@ -138,7 +137,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		if(sparseSafe)
 			return this;
 		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[_colIndexes.length]).applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left));
+			new Dictionary(new double[_colIndexes.length]).applyBinaryRowOp(op, v, true, _colIndexes, left));
 	}
 
 	@Override
@@ -167,11 +166,6 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	}
 
 	@Override
-	protected int containsAllZeroTuple() {
-		return 0;
-	}
-
-	@Override
 	protected double computeMxx(double c, Builtin builtin) {
 		return builtin.execute(c, 0);
 	}
@@ -200,7 +194,11 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	@Override
 	public void tsmm(double[] result, int numColumns) {
 		// do nothing
+	}
 
+	@Override
+	public void tsmm(double[] result, int numColumns, int idxStart, int idxEnd) {
+		// do nothing.
 	}
 
 	@Override
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 ffcc2fb..32cebf8 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
@@ -37,11 +37,12 @@ import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.PartitionerType;
 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.insertionsort.AInsertionSorter;
+import org.apache.sysds.runtime.compress.colgroup.insertionsort.InsertionSorterFactory;
 import org.apache.sysds.runtime.compress.colgroup.mapping.AMapToData;
 import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
-import org.apache.sysds.runtime.compress.colgroup.tree.AInsertionSorter;
-import org.apache.sysds.runtime.compress.colgroup.tree.InsertionSorterFactory;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorExact;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
@@ -56,7 +57,7 @@ import org.apache.sysds.runtime.util.CommonThreadPool;
 /**
  * Factory pattern for constructing ColGroups.
  */
-public class ColGroupFactory {
+public final class ColGroupFactory {
 	private static final Log LOG = LogFactory.getLog(ColGroupFactory.class.getName());
 
 	/**
@@ -156,7 +157,8 @@ public class ColGroupFactory {
 	private static Collection<AColGroup> compressColGroup(MatrixBlock in, int[] colIndexes,
 		CompressionSettings compSettings) {
 		if(in.isEmpty())
-			return Collections.singletonList(new ColGroupEmpty(colIndexes, compSettings.transposed ? in.getNumColumns(): in.getNumRows()));
+			return Collections.singletonList(
+				new ColGroupEmpty(colIndexes, compSettings.transposed ? in.getNumColumns() : in.getNumRows()));
 		else if(in.isInSparseFormat() && compSettings.transposed) {
 			final SparseBlock sb = in.getSparseBlock();
 			for(int col : colIndexes)
@@ -283,7 +285,7 @@ public class ColGroupFactory {
 	public static AColGroup compress(int[] colIndexes, int rlen, ABitmap ubm, CompressionType compType,
 		CompressionSettings cs, MatrixBlock rawMatrixBlock, double tupleSparsity) {
 
-		if(compType == CompressionType.UNCOMPRESSED && cs.columnPartitioner == PartitionerType.COST_MATRIX_MULT)
+		if(compType == CompressionType.UNCOMPRESSED && PartitionerType.isCostBased(cs.columnPartitioner))
 			compType = CompressionType.DDC;
 
 		final IntArrayList[] of = ubm.getOffsetList();
@@ -355,31 +357,34 @@ public class ColGroupFactory {
 
 	private static AColGroup setupMultiValueZeroColGroup(int[] colIndexes, ABitmap ubm, int numRows, ADictionary dict) {
 		IntArrayList[] offsets = ubm.getOffsetList();
+		try {
+			final int numOffsets = (int) ubm.getNumOffsets();
+			AInsertionSorter s = InsertionSorterFactory.create(numOffsets, numRows, offsets);
+			int[] _indexes = s.getIndexes();
+			AMapToData _data = s.getData();
 
-		final int numOffsets = (int) ubm.getNumOffsets();
-		AInsertionSorter s = InsertionSorterFactory.create(numOffsets, offsets.length, numRows);
-		s.insert(offsets);
-		int[] _indexes = s.getIndexes();
-		AMapToData _data = s.getData();
-
-		return new ColGroupSDCZeros(colIndexes, numRows, dict, _indexes, _data, null);
+			return new ColGroupSDCZeros(colIndexes, numRows, dict, _indexes, _data, null);
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException(
+				"Failed to construct SDC Zero Group with columns :" + Arrays.toString(colIndexes), e);
+		}
 	}
 
 	private static AColGroup setupMultiValueColGroup(int[] colIndexes, int numZeros, int largestOffset, ABitmap ubm,
 		int numRows, int largestIndex, ADictionary dict) {
-		IntArrayList[] offsets = ubm.getOffsetList();
-
-		AInsertionSorter s = InsertionSorterFactory.create(numRows - largestOffset, offsets.length, numRows);
-		s.insert(offsets, largestIndex);
-		int[] _indexes = s.getIndexes();
-		AMapToData _data = s.getData();
 		try {
+			IntArrayList[] offsets = ubm.getOffsetList();
+
+			AInsertionSorter s = InsertionSorterFactory.create(numRows - largestOffset, numRows, offsets, largestIndex);
+			int[] _indexes = s.getIndexes();
+			AMapToData _data = s.getData();
 			AColGroup ret = new ColGroupSDC(colIndexes, numRows, dict, _indexes, _data, null);
 			return ret;
 		}
 		catch(Exception e) {
-			LOG.error(Arrays.toString(_indexes));
-			throw new DMLCompressionException(e);
+			throw new DMLCompressionException(
+				"Failed to construct SDC Group with columns :" + Arrays.toString(colIndexes), e);
 		}
 
 	}
@@ -406,24 +411,10 @@ public class ColGroupFactory {
 
 	private static AColGroup compressDDC(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
 		double tupleSparsity) {
-
-		boolean _zeros = ubm.getNumOffsets() < (long) rlen;
-		ADictionary dict = (_zeros) ? DictionaryFactory.createWithAppendedZeroTuple(ubm,
-			tupleSparsity) : DictionaryFactory.create(ubm, tupleSparsity);
-		int numVals = ubm.getNumValues();
-		AMapToData _data = MapToFactory.create(rlen, numVals + (_zeros ? 1 : 0));
-		if(_zeros)
-			_data.fill(numVals);
-
-		for(int i = 0; i < numVals; i++) {
-			IntArrayList tmpList = ubm.getOffsetsList(i);
-			final int sz = tmpList.size();
-			for(int k = 0; k < sz; k++)
-				_data.set(tmpList.get(k), i);
-		}
-
-		return new ColGroupDDC(colIndexes, rlen, dict, _data, null);
-
+		boolean zeros = ubm.getNumOffsets() < (long) rlen;
+		ADictionary dict = DictionaryFactory.create(ubm, tupleSparsity, zeros);
+		AMapToData data = MapToFactory.create(rlen, zeros, ubm.getOffsetList());
+		return new ColGroupDDC(colIndexes, rlen, dict, data, null);
 	}
 
 	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs,
@@ -469,4 +460,21 @@ public class ColGroupFactory {
 		rle._colIndexes = colIndexes;
 		return rle;
 	}
+
+	public static AColGroup genColGroupConst(int numRows, int numCols, double value) {
+
+		int[] colIndices = new int[numCols];
+		for(int i = 0; i < numCols; i++)
+			colIndices[i] = i;
+
+		if(value == 0)
+			return new ColGroupEmpty(colIndices, numRows);
+
+		double[] values = new double[numCols];
+		for(int i = 0; i < numCols; i++)
+			values[i] = value;
+
+		ADictionary dict = new Dictionary(values);
+		return new ColGroupConst(colIndices, numRows, dict);
+	}
 }
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 712a574..e00422b 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
@@ -26,10 +26,6 @@ import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 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.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
-import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -70,58 +66,14 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		final double[] values = getValues();
-
-		// cache blocking config and position array
-		int[] apos = skipScan(numVals, rl);
-		double[] c = target.getDenseBlockValues();
-		// cache conscious append via horizontal scans
-		for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-
-				if(bix >= blen)
-					continue;
-				int len = _data[boff + bix];
-				int pos = boff + bix + 1;
-				for(int i = pos; i < pos + len; i++) {
-					int row = bi + _data[i];
-					if(row >= rl && row < ru) {
-						int rix = row - (rl - offT);
-						int rc = rix * target.getNumColumns();
-						for(int j = 0; j < numCols; j++) {
-							double v = c[rc + _colIndexes[j]];
-							double nv = c[rc + _colIndexes[j]] + values[off + j];
-							if(v == 0.0 && nv != 0.0) {
-								target.setNonZeros(target.getNonZeros() + 1);
-							}
-							c[rc + _colIndexes[j]] = nv;
-
-						}
-					}
-				}
-				apos[k] += len + 1;
-			}
-		}
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
-
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
 		final int offOut = (rl - offT);
 		final int targetCols = target.getNumColumns();
-		final double[] values = getValues();
-
+		
 		// cache blocking config and position array
 		int[] apos = skipScan(numVals, rl);
 		double[] c = target.getDenseBlockValues();
@@ -153,141 +105,147 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		final double[] values = getValues();
-
-		// cache blocking config and position array
-		int[] apos = new int[numVals];
-		int[] cix = new int[numCols];
-
-		// prepare target col indexes
-		for(int j = 0; j < numCols; j++)
-			cix[j] = colixTargets[_colIndexes[j]];
-
-		// cache conscious append via horizontal scans
-		for(int bi = 0; bi < _numRows; bi += blksz) {
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int len = _data[boff + bix];
-				int pos = boff + bix + 1;
-				for(int i = pos; i < pos + len; i++)
-					for(int j = 0, rix = bi + _data[i]; j < numCols; j++)
-						if(values[off + j] != 0) {
-							double v = target.quickGetValue(rix, _colIndexes[j]);
-							target.setValue(rix, cix[j], values[off + j] + v);
-						}
-				apos[k] += len + 1;
-			}
-		}
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		int numCols = getNumCols();
-		int numVals = getNumValues();
-		double[] c = target.getDenseBlockValues();
-		double[] values = getValues();
-
-		// cache blocking config and position array
-		int[] apos = new int[numVals];
-
-		// cache conscious append via horizontal scans
-		int nnz = 0;
-		for(int bi = 0; bi < _numRows; bi += blksz) {
-			// Arrays.fill(c, bi, Math.min(bi + blksz, _numRows), 0);
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int len = _data[boff + bix];
-				int pos = boff + bix + 1;
-				for(int i = pos; i < pos + len; i++) {
-					c[bi + _data[i]] += values[off + colpos];
-					nnz++;
-				}
-				apos[k] += len + 1;
-			}
-		}
-		target.setNonZeros(nnz);
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		int numCols = getNumCols();
-		int numVals = getNumValues();
-		double[] c = target.getDenseBlockValues();
-		double[] values = getValues();
-
-		// cache blocking config and position array
-		int[] apos = skipScan(numVals, rl);
-
-		// cache conscious append via horizontal scans
-		int nnz = 0;
-		for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int len = _data[boff + bix];
-				int pos = boff + bix + 1;
-				for(int i = pos; i < pos + len; i++) {
-					int index = bi + _data[i];
-					if(index >= rl && index < ru) {
-						c[index - rl] += values[off + colpos];
-						nnz++;
-					}
-				}
-				apos[k] += len + 1;
-			}
-		}
-		target.setNonZeros(nnz);
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock values) {
+		throw new NotImplementedException();
 	}
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		int numCols = getNumCols();
-		int numVals = getNumValues();
-		double[] values = getValues();
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int numCols = getNumCols();
+	// 	final int numVals = getNumValues();
+	// 	final double[] values = getValues();
+
+	// 	// cache blocking config and position array
+	// 	int[] apos = new int[numVals];
+	// 	int[] cix = new int[numCols];
+
+	// 	// prepare target col indexes
+	// 	for(int j = 0; j < numCols; j++)
+	// 		cix[j] = colixTargets[_colIndexes[j]];
+
+	// 	// cache conscious append via horizontal scans
+	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int len = _data[boff + bix];
+	// 			int pos = boff + bix + 1;
+	// 			for(int i = pos; i < pos + len; i++)
+	// 				for(int j = 0, rix = bi + _data[i]; j < numCols; j++)
+	// 					if(values[off + j] != 0) {
+	// 						double v = target.quickGetValue(rix, _colIndexes[j]);
+	// 						target.setValue(rix, cix[j], values[off + j] + v);
+	// 					}
+	// 			apos[k] += len + 1;
+	// 		}
+	// 	}
+	// }
 
-		// cache blocking config and position array
-		int[] apos = skipScan(numVals, rl);
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	int numCols = getNumCols();
+	// 	int numVals = getNumValues();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double[] values = getValues();
+
+	// 	// cache blocking config and position array
+	// 	int[] apos = new int[numVals];
+
+	// 	// cache conscious append via horizontal scans
+	// 	int nnz = 0;
+	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
+	// 		// Arrays.fill(c, bi, Math.min(bi + blksz, _numRows), 0);
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int len = _data[boff + bix];
+	// 			int pos = boff + bix + 1;
+	// 			for(int i = pos; i < pos + len; i++) {
+	// 				c[bi + _data[i]] += values[off + colpos];
+	// 				nnz++;
+	// 			}
+	// 			apos[k] += len + 1;
+	// 		}
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
 
-		// cache conscious append via horizontal scans
-		for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	int numCols = getNumCols();
+	// 	int numVals = getNumValues();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	double[] values = getValues();
+
+	// 	// cache blocking config and position array
+	// 	int[] apos = skipScan(numVals, rl);
+
+	// 	// cache conscious append via horizontal scans
+	// 	int nnz = 0;
+	// 	for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int len = _data[boff + bix];
+	// 			int pos = boff + bix + 1;
+	// 			for(int i = pos; i < pos + len; i++) {
+	// 				int index = bi + _data[i];
+	// 				if(index >= rl && index < ru) {
+	// 					c[index - rl] += values[off + colpos];
+	// 					nnz++;
+	// 				}
+	// 			}
+	// 			apos[k] += len + 1;
+	// 		}
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
 
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int len = _data[boff + bix];
-				int pos = boff + bix + 1;
-				for(int i = pos; i < pos + len; i++) {
-					int index = bi + _data[i];
-					if(index >= rl && index < ru)
-						c[index - rl] += values[off + colpos];
-				}
-				apos[k] += len + 1;
-			}
-		}
-	}
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	int numCols = getNumCols();
+	// 	int numVals = getNumValues();
+	// 	double[] values = getValues();
+
+	// 	// cache blocking config and position array
+	// 	int[] apos = skipScan(numVals, rl);
+
+	// 	// cache conscious append via horizontal scans
+	// 	for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int len = _data[boff + bix];
+	// 			int pos = boff + bix + 1;
+	// 			for(int i = pos; i < pos + len; i++) {
+	// 				int index = bi + _data[i];
+	// 				if(index >= rl && index < ru)
+	// 					c[index - rl] += values[off + colpos];
+	// 			}
+	// 			apos[k] += len + 1;
+	// 		}
+	// 	}
+	// }
 
 	@Override
 	public int[] getCounts(int[] counts) {
@@ -703,7 +661,7 @@ public class ColGroupOLE extends ColGroupOffset {
 	// }
 
 	@Override
-	protected final void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numVals = getNumValues();
@@ -947,46 +905,51 @@ public class ColGroupOLE extends ColGroupOffset {
 		return sb.toString();
 	}
 
-	@Override
-	public double[] preAggregate(double[] a, int row) {
-		final int numVals = getNumValues();
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-
-		int[] apos = allocIVector(numVals, true);
-		double[] cvals = allocDVector(numVals, true);
-		int off = row * _numRows;
-		for(int ai = 0; ai < _numRows; ai += blksz2) {
-			int aimax = Math.min(ai + blksz2, _numRows);
-
-			// horizontal segment scan, incl pos maintenance
-			for(int k = 0; k < numVals; k++) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				double vsum = 0;
-
-				for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
-					// prepare length, start, and end pos
-					int len = _data[boff + bix];
-					int pos = boff + bix + 1;
-
-					// iterate over bitmap blocks and compute partial results (a[i]*1)
-					vsum += LinearAlgebraUtils.vectSum(a, _data, ii + off, pos, len);
-					bix += len + 1;
-				}
-
-				apos[k] = bix;
-				cvals[k] += vsum;
-			}
-		}
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// 	final int numVals = getNumValues();
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
+
+	// 	int[] apos = allocIVector(numVals, true);
+	// 	double[] cvals = allocDVector(numVals, true);
+	// 	int off = row * _numRows;
+	// 	for(int ai = 0; ai < _numRows; ai += blksz2) {
+	// 		int aimax = Math.min(ai + blksz2, _numRows);
+
+	// 		// horizontal segment scan, incl pos maintenance
+	// 		for(int k = 0; k < numVals; k++) {
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			double vsum = 0;
+
+	// 			for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
+	// 				// prepare length, start, and end pos
+	// 				int len = _data[boff + bix];
+	// 				int pos = boff + bix + 1;
+
+	// 				// iterate over bitmap blocks and compute partial results (a[i]*1)
+	// 				vsum += LinearAlgebraUtils.vectSum(a, _data, ii + off, pos, len);
+	// 				bix += len + 1;
+	// 			}
+
+	// 			apos[k] = bix;
+	// 			cvals[k] += vsum;
+	// 		}
+	// 	}
+
+	// 	return cvals;
+	// }
 
-		return cvals;
-	}
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// 	return null;
+	// }
 
 	@Override
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
-		return null;
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
+		throw new NotImplementedException();
 	}
 
 	@Override
@@ -1052,138 +1015,138 @@ public class ColGroupOLE extends ColGroupOffset {
 		return encodedBlocks;
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kr = 0; kr < NVR; kr++) {
-			final int bOffR = this._ptr[kr];
-			final int bLenR = this.len(kr);
-			final int krOff = kr * NVL;
-			for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-				sLenR = this._data[bOffR + bixR];
-				for(int j = 1; j <= sLenR; j++) {
-					int idx = lhs._data.getIndex(offR + this._data[bOffR + bixR + j]);
-					ag.increment(idx + krOff);
-				}
-			}
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		final int defL = NVL - 1;
-
-		for(int kr = 0; kr < NVR; kr++) {
-			AIterator lIt = lhs._indexes.getIterator();
-			final int bOffR = this._ptr[kr];
-			final int bLenR = this.len(kr);
-			final int krOff = kr * NVL;
-			for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-				sLenR = this._data[bOffR + bixR];
-				for(int j = 1; j <= sLenR; j++) {
-					final int row = offR + this._data[bOffR + bixR + j];
-					lIt.skipTo(row);
-					if(lIt.value() == row)
-						ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
-					else
-						ag.increment(defL + krOff);
-				}
-			}
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kr = 0; kr < NVR; kr++) {
+	// 		final int bOffR = this._ptr[kr];
+	// 		final int bLenR = this.len(kr);
+	// 		final int krOff = kr * NVL;
+	// 		for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
+	// 			sLenR = this._data[bOffR + bixR];
+	// 			for(int j = 1; j <= sLenR; j++) {
+	// 				int idx = lhs._data.getIndex(offR + this._data[bOffR + bixR + j]);
+	// 				ag.increment(idx + krOff);
+	// 			}
+	// 		}
+	// 	}
+
+	// 	return ag;
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kr = 0; kr < NVR; kr++) {
-			final AIterator lIt = lhs._indexes.getIterator();
-			final int bOffR = this._ptr[kr];
-			final int bLenR = this.len(kr);
-			final int krOff = kr * NVL;
-			for(int bixR = 0, offR = 0, sLenR = 0; lIt.hasNext() && bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-				sLenR = this._data[bOffR + bixR];
-				for(int j = 1; lIt.hasNext() && j <= sLenR; j++) {
-					final int row = offR + this._data[bOffR + bixR + j];
-					lIt.skipTo(row);
-					if(lIt.value() == row)
-						ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
-				}
-			}
-		}
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	final int defL = NVL - 1;
+
+	// 	for(int kr = 0; kr < NVR; kr++) {
+	// 		AIterator lIt = lhs._indexes.getIterator();
+	// 		final int bOffR = this._ptr[kr];
+	// 		final int bLenR = this.len(kr);
+	// 		final int krOff = kr * NVL;
+	// 		for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
+	// 			sLenR = this._data[bOffR + bixR];
+	// 			for(int j = 1; j <= sLenR; j++) {
+	// 				final int row = offR + this._data[bOffR + bixR + j];
+	// 				lIt.skipTo(row);
+	// 				if(lIt.value() == row)
+	// 					ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
+	// 				else
+	// 					ag.increment(defL + krOff);
+	// 			}
+	// 		}
+	// 	}
+
+	// 	return ag;
+	// }
 
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kr = 0; kr < NVR; kr++) {
+	// 		final AIterator lIt = lhs._indexes.getIterator();
+	// 		final int bOffR = this._ptr[kr];
+	// 		final int bLenR = this.len(kr);
+	// 		final int krOff = kr * NVL;
+	// 		for(int bixR = 0, offR = 0, sLenR = 0; lIt.hasNext() && bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
+	// 			sLenR = this._data[bOffR + bixR];
+	// 			for(int j = 1; lIt.hasNext() && j <= sLenR; j++) {
+	// 				final int row = offR + this._data[bOffR + bixR + j];
+	// 				lIt.skipTo(row);
+	// 				if(lIt.value() == row)
+	// 					ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
+	// 			}
+	// 		}
+	// 	}
+
+	// 	return ag;
+	// }
 
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kl = 0; kl < NVL; kl++) {
-			final int bOffL = lhs._ptr[kl];
-			final int bLenL = lhs.len(kl);
-			for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-				sLenL = lhs._data[bOffL + bixL];
-				for(int i = 1; i <= sLenL; i++) {
-					final int col = offL + lhs._data[bOffL + bixL + i];
-					for(int kr = 0; kr < NVR; kr++) {
-						final int bOffR = this._ptr[kr];
-						final int bLenR = this.len(kr);
-						final int krOff = kr * NVL;
-						for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-							sLenR = this._data[bOffR + bixR];
-							for(int j = 1; j <= sLenR; j++)
-								if(col == offR + this._data[bOffR + bixR + j])
-									ag.increment(kl + krOff);
-						}
-					}
-				}
-			}
-		}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kl = 0; kl < NVL; kl++) {
+	// 		final int bOffL = lhs._ptr[kl];
+	// 		final int bLenL = lhs.len(kl);
+	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
+	// 			sLenL = lhs._data[bOffL + bixL];
+	// 			for(int i = 1; i <= sLenL; i++) {
+	// 				final int col = offL + lhs._data[bOffL + bixL + i];
+	// 				for(int kr = 0; kr < NVR; kr++) {
+	// 					final int bOffR = this._ptr[kr];
+	// 					final int bLenR = this.len(kr);
+	// 					final int krOff = kr * NVL;
+	// 					for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
+	// 						sLenR = this._data[bOffR + bixR];
+	// 						for(int j = 1; j <= sLenR; j++)
+	// 							if(col == offR + this._data[bOffR + bixR + j])
+	// 								ag.increment(kl + krOff);
+	// 					}
+	// 				}
+	// 			}
+	// 		}
+	// 	}
+
+	// 	return ag;
+	// }
 
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -1209,9 +1172,4 @@ public class ColGroupOLE extends ColGroupOffset {
 	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
-
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		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 1c81fca..b7dd658 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
@@ -27,9 +27,6 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.compress.CompressionSettings;
 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.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
-import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -68,55 +65,11 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
-		final double[] values = getValues();
-
-		// 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++) {
-							double v = c[rc + _colIndexes[j]];
-							double nv = c[rc + _colIndexes[j]] + values[off + j];
-							if(v == 0.0 && nv != 0.0) {
-								target.setNonZeros(target.getNonZeros() + 1);
-							}
-							c[rc + _colIndexes[j]] = nv;
-
-						}
-					}
-					start += len;
-				}
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		final double[] values = getValues();
 
 		// position and start offset arrays
 		int[] astart = new int[numVals];
@@ -150,172 +103,178 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-		// if(getNumValues() > 1) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		final double[] values = getValues();
-
-		// position and start offset arrays
-		int[] apos = new int[numVals];
-		int[] astart = new int[numVals];
-		int[] cix = new int[numCols];
-
-		// prepare target col indexes
-		for(int j = 0; j < numCols; j++)
-			cix[j] = colixTargets[_colIndexes[j]];
-
-		// cache conscious append via horizontal scans
-		for(int bi = 0; bi < _numRows; bi += blksz) {
-			int bimax = Math.min(bi + blksz, _numRows);
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int start = astart[k];
-				for(; bix < blen & start < bimax; bix += 2) {
-					start += _data[boff + bix];
-					int len = _data[boff + bix + 1];
-					for(int i = start; i < start + len; i++)
-						for(int j = 0; j < numCols; j++)
-							if(values[off + j] != 0) {
-								double v = target.quickGetValue(i, _colIndexes[j]);
-								target.setValue(i, _colIndexes[j], values[off + j] + v);
-							}
-
-					start += len;
-				}
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-		// }
-		// else {
-		// // call generic decompression with decoder
-		// super.decompressToBlock(target, colixTargets);
-		// }
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-
-		// position and start offset arrays
-		int[] astart = new int[numVals];
-		int[] apos = allocIVector(numVals, true);
-
-		// cache conscious append via horizontal scans
-		int nnz = 0;
-		for(int bi = 0; bi < _numRows; bi += blksz) {
-			int bimax = Math.min(bi + blksz, _numRows);
-			// Arrays.fill(c, bi, bimax, 0);
-			for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				if(bix >= blen)
-					continue;
-				int start = astart[k];
-				for(; bix < blen & start < bimax; bix += 2) {
-					start += _data[boff + bix];
-					int len = _data[boff + bix + 1];
-					for(int i = start; i < start + len; i++)
-						c[i] += values[off + colpos];
-					nnz += len;
-					start += len;
-				}
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-		target.setNonZeros(nnz);
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-
-		// position and start offset arrays
-		int[] astart = new int[numVals];
-		int[] apos = allocIVector(numVals, true);
-
-		// cache conscious append via horizontal scans
-		int nnz = 0;
-		for(int bi = (rl / blksz) * blksz; 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];
-				if(bix >= blen)
-					continue;
-				int start = astart[k];
-				for(; bix < blen & start < bimax; bix += 2) {
-					start += _data[boff + bix];
-					int len = _data[boff + bix + 1];
-					if(start + len >= rl) {
-						int offsetStart = Math.max(start, rl);
-						for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
-							c[i - rl] += values[off + colpos];
-						nnz += len - (offsetStart - start);
-					}
-					start += len;
-				}
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-		target.setNonZeros(nnz);
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock values) {
+		throw new NotImplementedException();
 	}
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final int numCols = getNumCols();
-		final int numVals = getNumValues();
-		final double[] values = getValues();
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
+	// 	// if(getNumValues() > 1) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int numCols = getNumCols();
+	// 	final int numVals = getNumValues();
+	// 	final double[] values = getValues();
+
+	// 	// position and start offset arrays
+	// 	int[] apos = new int[numVals];
+	// 	int[] astart = new int[numVals];
+	// 	int[] cix = new int[numCols];
+
+	// 	// prepare target col indexes
+	// 	for(int j = 0; j < numCols; j++)
+	// 		cix[j] = colixTargets[_colIndexes[j]];
+
+	// 	// cache conscious append via horizontal scans
+	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
+	// 		int bimax = Math.min(bi + blksz, _numRows);
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int start = astart[k];
+	// 			for(; bix < blen & start < bimax; bix += 2) {
+	// 				start += _data[boff + bix];
+	// 				int len = _data[boff + bix + 1];
+	// 				for(int i = start; i < start + len; i++)
+	// 					for(int j = 0; j < numCols; j++)
+	// 						if(values[off + j] != 0) {
+	// 							double v = target.quickGetValue(i, _colIndexes[j]);
+	// 							target.setValue(i, _colIndexes[j], values[off + j] + v);
+	// 						}
+
+	// 				start += len;
+	// 			}
+	// 			apos[k] = bix;
+	// 			astart[k] = start;
+	// 		}
+	// 	}
+	// 	// }
+	// 	// else {
+	// 	// // call generic decompression with decoder
+	// 	// super.decompressToBlock(target, colixTargets);
+	// 	// }
+	// }
 
-		// position and start offset arrays
-		int[] astart = new int[numVals];
-		int[] apos = allocIVector(numVals, true);
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int numCols = getNumCols();
+	// 	final int numVals = getNumValues();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	final double[] values = getValues();
+
+	// 	// position and start offset arrays
+	// 	int[] astart = new int[numVals];
+	// 	int[] apos = allocIVector(numVals, true);
+
+	// 	// cache conscious append via horizontal scans
+	// 	int nnz = 0;
+	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
+	// 		int bimax = Math.min(bi + blksz, _numRows);
+	// 		// Arrays.fill(c, bi, bimax, 0);
+	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int start = astart[k];
+	// 			for(; bix < blen & start < bimax; bix += 2) {
+	// 				start += _data[boff + bix];
+	// 				int len = _data[boff + bix + 1];
+	// 				for(int i = start; i < start + len; i++)
+	// 					c[i] += values[off + colpos];
+	// 				nnz += len;
+	// 				start += len;
+	// 			}
+	// 			apos[k] = bix;
+	// 			astart[k] = start;
+	// 		}
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
 
-		// cache conscious append via horizontal scans
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int numCols = getNumCols();
+	// 	final int numVals = getNumValues();
+	// 	double[] c = target.getDenseBlockValues();
+	// 	final double[] values = getValues();
+
+	// 	// position and start offset arrays
+	// 	int[] astart = new int[numVals];
+	// 	int[] apos = allocIVector(numVals, true);
+
+	// 	// cache conscious append via horizontal scans
+	// 	int nnz = 0;
+	// 	for(int bi = (rl / blksz) * blksz; 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];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int start = astart[k];
+	// 			for(; bix < blen & start < bimax; bix += 2) {
+	// 				start += _data[boff + bix];
+	// 				int len = _data[boff + bix + 1];
+	// 				if(start + len >= rl) {
+	// 					int offsetStart = Math.max(start, rl);
+	// 					for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
+	// 						c[i - rl] += values[off + colpos];
+	// 					nnz += len - (offsetStart - start);
+	// 				}
+	// 				start += len;
+	// 			}
+	// 			apos[k] = bix;
+	// 			astart[k] = start;
+	// 		}
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
 
-		for(int bi = (rl / blksz) * blksz; 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];
-				if(bix >= blen)
-					continue;
-				int start = astart[k];
-				for(; bix < blen & start < bimax; bix += 2) {
-					start += _data[boff + bix];
-					int len = _data[boff + bix + 1];
-					if(start + len >= rl) {
-						int offsetStart = Math.max(start, rl);
-						for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
-							c[i - rl] += values[off + colpos];
-					}
-					start += len;
-				}
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-	}
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	final int numCols = getNumCols();
+	// 	final int numVals = getNumValues();
+	// 	final double[] values = getValues();
+
+	// 	// position and start offset arrays
+	// 	int[] astart = new int[numVals];
+	// 	int[] apos = allocIVector(numVals, true);
+
+	// 	// cache conscious append via horizontal scans
+
+	// 	for(int bi = (rl / blksz) * blksz; 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];
+	// 			if(bix >= blen)
+	// 				continue;
+	// 			int start = astart[k];
+	// 			for(; bix < blen & start < bimax; bix += 2) {
+	// 				start += _data[boff + bix];
+	// 				int len = _data[boff + bix + 1];
+	// 				if(start + len >= rl) {
+	// 					int offsetStart = Math.max(start, rl);
+	// 					for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
+	// 						c[i - rl] += values[off + colpos];
+	// 				}
+	// 				start += len;
+	// 			}
+	// 			apos[k] = bix;
+	// 			astart[k] = start;
+	// 		}
+	// 	}
+	// }
 
 	@Override
 	public int[] getCounts(int[] counts) {
@@ -658,7 +617,7 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	protected final void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 
 		final int numVals = getNumValues();
 
@@ -903,46 +862,51 @@ public class ColGroupRLE extends ColGroupOffset {
 		return new Pair<>(apos, astart);
 	}
 
-	@Override
-	public double[] preAggregate(double[] a, int row) {
-		final int numVals = getNumValues();
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		// current pos per OLs / output values
-		int[] astart = new int[numVals];
-		int[] apos = allocIVector(numVals, true);
-		double[] cvals = allocDVector(numVals, true);
-		int off = row * _numRows;
-
-		// step 2: cache conscious matrix-vector via horizontal scans
-		for(int ai = 0; ai < _numRows; ai += blksz) {
-			int aimax = Math.min(ai + blksz, _numRows);
-
-			// horizontal scan, incl pos maintenance
-			for(int k = 0; k < numVals; k++) {
-				int boff = _ptr[k];
-				int blen = len(k);
-				int bix = apos[k];
-				int start = astart[k];
-
-				// compute partial results, not aligned
-				while(bix < blen & start < aimax) {
-					start += _data[boff + bix];
-					int len = _data[boff + bix + 1];
-					cvals[k] += LinearAlgebraUtils.vectSum(a, start + off, len);
-					start += len;
-					bix += 2;
-				}
-
-				apos[k] = bix;
-				astart[k] = start;
-			}
-		}
-		return cvals;
-	}
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// 	final int numVals = getNumValues();
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	// current pos per OLs / output values
+	// 	int[] astart = new int[numVals];
+	// 	int[] apos = allocIVector(numVals, true);
+	// 	double[] cvals = allocDVector(numVals, true);
+	// 	int off = row * _numRows;
+
+	// 	// step 2: cache conscious matrix-vector via horizontal scans
+	// 	for(int ai = 0; ai < _numRows; ai += blksz) {
+	// 		int aimax = Math.min(ai + blksz, _numRows);
+
+	// 		// horizontal scan, incl pos maintenance
+	// 		for(int k = 0; k < numVals; k++) {
+	// 			int boff = _ptr[k];
+	// 			int blen = len(k);
+	// 			int bix = apos[k];
+	// 			int start = astart[k];
+
+	// 			// compute partial results, not aligned
+	// 			while(bix < blen & start < aimax) {
+	// 				start += _data[boff + bix];
+	// 				int len = _data[boff + bix + 1];
+	// 				cvals[k] += LinearAlgebraUtils.vectSum(a, start + off, len);
+	// 				start += len;
+	// 				bix += 2;
+	// 			}
+
+	// 			apos[k] = bix;
+	// 			astart[k] = start;
+	// 		}
+	// 	}
+	// 	return cvals;
+	// }
 
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// 	return null;
+	// }
+	
 	@Override
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
-		return null;
+	public void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
+		throw new NotImplementedException();
 	}
 
 	@Override
@@ -1053,93 +1017,93 @@ public class ColGroupRLE extends ColGroupOffset {
 		return ret;
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kr = 0; kr < NVR; kr++) {
-			final int boffL = _ptr[kr];
-			final int blenL = len(kr);
-			final int offKr = kr * NVL;
-			for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-				startL += _data[boffL + bixL];
-				lenL = _data[boffL + bixL + 1];
-				final int endL = startL + lenL;
-				for(int i = startL; i < endL; i++)
-					ag.increment(lhs._data.getIndex(i) + offKr);
-
-			}
-		}
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kr = 0; kr < NVR; kr++) {
+	// 		final int boffL = _ptr[kr];
+	// 		final int blenL = len(kr);
+	// 		final int offKr = kr * NVL;
+	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
+	// 			startL += _data[boffL + bixL];
+	// 			lenL = _data[boffL + bixL + 1];
+	// 			final int endL = startL + lenL;
+	// 			for(int i = startL; i < endL; i++)
+	// 				ag.increment(lhs._data.getIndex(i) + offKr);
+
+	// 		}
+	// 	}
+	// 	return ag;
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kl = 0; kl < NVL; kl++) {
-			final int boffL = lhs._ptr[kl];
-			final int blenL = lhs.len(kl);
-			for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-				startL += lhs._data[boffL + bixL];
-				lenL = lhs._data[boffL + bixL + 1];
-				final int endL = startL + lenL;
-				for(int kr = 0; kr < NVR; kr++) {
-					final int boffR = _ptr[kr];
-					final int blenR = len(kr);
-					final int krOff = kr * NVL;
-					for(int bixR = 0, startR = 0, lenR = 0; bixR < blenR & startR < endL; startR += lenR, bixR += 2) {
-						startR += _data[boffR + bixR];
-						lenR = _data[boffR + bixR + 1];
-						final int endR = startR + lenR;
-						if(startL < endR && startR < endL) {
-							final int endOverlap = Math.min(endR, endL);
-							final int startOverlap = Math.max(startL, startR);
-							final int lenOverlap = endOverlap - startOverlap;
-							ag.increment(kl + krOff, lenOverlap);
-						}
-					}
-				}
-			}
-		}
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	for(int kl = 0; kl < NVL; kl++) {
+	// 		final int boffL = lhs._ptr[kl];
+	// 		final int blenL = lhs.len(kl);
+	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
+	// 			startL += lhs._data[boffL + bixL];
+	// 			lenL = lhs._data[boffL + bixL + 1];
+	// 			final int endL = startL + lenL;
+	// 			for(int kr = 0; kr < NVR; kr++) {
+	// 				final int boffR = _ptr[kr];
+	// 				final int blenR = len(kr);
+	// 				final int krOff = kr * NVL;
+	// 				for(int bixR = 0, startR = 0, lenR = 0; bixR < blenR & startR < endL; startR += lenR, bixR += 2) {
+	// 					startR += _data[boffR + bixR];
+	// 					lenR = _data[boffR + bixR + 1];
+	// 					final int endR = startR + lenR;
+	// 					if(startL < endR && startR < endL) {
+	// 						final int endOverlap = Math.min(endR, endL);
+	// 						final int startOverlap = Math.max(startL, startR);
+	// 						final int lenOverlap = endOverlap - startOverlap;
+	// 						ag.increment(kl + krOff, lenOverlap);
+	// 					}
+	// 				}
+	// 			}
+	// 		}
+	// 	}
+	// 	return ag;
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -1162,12 +1126,8 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
 }
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 769b2fe..f56d6db 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
@@ -25,7 +25,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.sysds.runtime.compress.CompressionSettings;
 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;
@@ -33,8 +32,6 @@ import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
 import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -97,24 +94,16 @@ public class ColGroupSDC extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(getNumberNonZeros());
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
-
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
-		final double[] values = getValues();
 		final int offsetToDefault = values.length - nCol;
 
 		double[] c = target.getDenseBlockValues();
 		offT = offT * tCol;
 		int i = rl;
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		AIterator it = _indexes.getIterator(rl);
 		for(; i < ru && it.hasNext(); i++, offT += tCol) {
 			if(it.value() == i) {
 				int offset = _data.getIndex(it.getDataIndexAndIncrement()) * nCol;
@@ -133,57 +122,97 @@ public class ColGroupSDC extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		throw new NotImplementedException("Not Implemented");
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colPos) {
-		final double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-		final double defaultVal = values[values.length - _colIndexes.length + colPos];
-		int i = 0;
-		final AIterator it = _indexes.getIterator();
-		for(; i < _numRows && it.hasNext(); i++) {
-			if(it.value() == i)
-				c[i] += values[_data.getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colPos];
-			else
-				c[i] += defaultVal;
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock sb) {
+		final int tCol = target.getNumColumns();
+		final int offsetToDefault = sb.numRows() - 1;
+		if(sb.isEmpty(offsetToDefault)) {
+			throw new NotImplementedException("Implement a SDCZeros decompress if this is the case");
 		}
-		for(; i < _numRows; i++)
-			c[i] += defaultVal;
-
-		target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	}
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
-
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final int nCol = _colIndexes.length;
-		final double[] values = getValues();
-		final int offsetToDefault = values.length - nCol + colpos;
-		final AIterator it = _indexes.getIterator();
+		final int defApos = sb.pos(offsetToDefault);
+		final int defAlen = sb.size(offsetToDefault) + defApos;
+		final double[] defAvals = sb.values(offsetToDefault);
+		final int[] defAix = sb.indexes(offsetToDefault);
 
-		int offT = 0;
+		double[] c = target.getDenseBlockValues();
+		offT = offT * tCol;
 		int i = rl;
-		it.skipTo(rl);
-
-		for(; i < ru && it.hasNext(); i++, offT++) {
+		AIterator it = _indexes.getIterator(rl);
+		for(; i < ru && it.hasNext(); i++, offT += tCol) {
 			if(it.value() == i) {
-				int offset = _data.getIndex(it.getDataIndexAndIncrement()) * nCol;
-				c[offT] += values[offset + colpos];
+				int dictIndex = _data.getIndex(it.getDataIndexAndIncrement());
+				if(sb.isEmpty(dictIndex))
+					continue;
+				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[offT + _colIndexes[aix[j]]] += avals[j];
 			}
 			else
-				c[offT] += values[offsetToDefault];
+				for(int j = defApos; j < defAlen; j++)
+					c[offT + _colIndexes[defAix[j]]] += defAvals[j];
 		}
 
-		for(; i < ru; i++, offT++)
-			c[offT] += values[offsetToDefault];
-	}
+		for(; i < ru; i++, offT += tCol)
+			for(int j = defApos; j < defAlen; j++)
+				c[offT + _colIndexes[defAix[j]]] += defAvals[j];
+
+	}
+
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colPos) {
+	// 	final double[] c = target.getDenseBlockValues();
+	// 	final double[] values = getValues();
+	// 	final double defaultVal = values[values.length - _colIndexes.length + colPos];
+	// 	int i = 0;
+	// 	final AIterator it = _indexes.getIterator();
+	// 	for(; i < _numRows && it.hasNext(); i++) {
+	// 		if(it.value() == i)
+	// 			c[i] += values[_data.getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colPos];
+	// 		else
+	// 			c[i] += defaultVal;
+	// 	}
+	// 	for(; i < _numRows; i++)
+	// 		c[i] += defaultVal;
+
+	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	final int nCol = _colIndexes.length;
+	// 	final double[] values = getValues();
+	// 	final int offsetToDefault = values.length - nCol + colpos;
+	// 	final AIterator it = _indexes.getIterator(rl);
+
+	// 	int offT = 0;
+	// 	int i = rl;
+
+	// 	for(; i < ru && it.hasNext(); i++, offT++) {
+	// 		if(it.value() == i) {
+	// 			int offset = _data.getIndex(it.getDataIndexAndIncrement()) * nCol;
+	// 			c[offT] += values[offset + colpos];
+	// 		}
+	// 		else
+	// 			c[offT] += values[offsetToDefault];
+	// 	}
+
+	// 	for(; i < ru; i++, offT++)
+	// 		c[offT] += values[offsetToDefault];
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -192,13 +221,13 @@ public class ColGroupSDC extends ColGroupValue {
 		if(ix < 0)
 			throw new RuntimeException("Column index " + c + " not in group.");
 
-		// // get value
-		AIterator it = _indexes.getIterator();
-		it.skipTo(r);
+		// get value
+		AIterator it = _indexes.getIterator(r);
+		final int nCol = _colIndexes.length;
 		if(it.value() == r)
-			return _dict.getValue(_data.getIndex(it.getDataIndex()) * _colIndexes.length + ix);
+			return _dict.getValue(_data.getIndex(it.getDataIndex()) * nCol + ix);
 		else
-			return _dict.getValue(getNumValues() * _colIndexes.length - _colIndexes.length + ix);
+			return _dict.getValue(getNumValues() * nCol - nCol + ix);
 	}
 
 	@Override
@@ -207,14 +236,13 @@ public class ColGroupSDC extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		final int numVals = getNumValues();
 		// // pre-aggregate nnz per value tuple
 		double[] vals = _dict.sumAllRowsToDouble(square, _colIndexes.length);
 
 		int rix = rl;
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		AIterator it = _indexes.getIterator(rl);
 		for(; rix < ru && it.hasNext(); rix++) {
 			if(it.value() != rix)
 				c[rix] += vals[numVals - 1];
@@ -235,8 +263,7 @@ public class ColGroupSDC extends ColGroupValue {
 
 		double[] vals = _dict.aggregateTuples(builtin, _colIndexes.length);
 
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		AIterator it = _indexes.getIterator(rl);
 
 		int rix = rl;
 		for(; rix < ru && it.hasNext(); rix++) {
@@ -262,8 +289,7 @@ public class ColGroupSDC extends ColGroupValue {
 		final int def = counts.length - 1;
 
 		int i = rl;
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		AIterator it = _indexes.getIterator(rl);
 
 		for(; i < ru && it.hasNext(); i++) {
 			if(i == it.value())
@@ -282,52 +308,92 @@ public class ColGroupSDC extends ColGroupValue {
 		return _data.getIndex(r);
 	}
 
-	public double[] preAggregate(double[] a, int row) {
-		final int numVals = getNumValues();
-		final double[] vals = allocDVector(numVals, true);
-		final AIterator it = _indexes.getIterator();
-		final int def = numVals - 1;
-
-		int i = 0;
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// final int numVals = getNumValues();
+	// final double[] vals = allocDVector(numVals, true);
+	// final AIterator it = _indexes.getIterator();
+	// final int def = numVals - 1;
+
+	// int i = 0;
+
+	// if(row > 0) {
+	// int offA = _numRows * row;
+	// for(; i < _numRows && it.hasNext(); i++, offA++)
+	// if(it.value() == i)
+	// vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[offA];
+	// else
+	// vals[def] += a[offA];
+	// for(; i < _numRows; i++, offA++)
+	// vals[def] += a[offA];
+	// }
+	// else {
+	// for(; i < _numRows && it.hasNext(); i++)
+	// if(it.value() == i)
+	// vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[i];
+	// else
+	// vals[def] += a[i];
+	// for(; i < _numRows; i++)
+	// vals[def] += a[i];
+	// }
+	// return vals;
+	// }
+
+	@Override
+	public void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
+		else
+			preAggregateDense(m, preAgg, rl, ru);
+	}
 
-		if(row > 0) {
-			int offA = _numRows * row;
-			for(; i < _numRows && it.hasNext(); i++, offA++)
-				if(it.value() == i)
-					vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[offA];
-				else
-					vals[def] += a[offA];
-			for(; i < _numRows; i++, offA++)
-				vals[def] += a[offA];
-		}
-		else {
-			for(; i < _numRows && it.hasNext(); i++)
-				if(it.value() == i)
-					vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[i];
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			final int def = offOut + numVals - 1;
+			final AIterator it = _indexes.getIterator();
+			int rc = 0;
+			int offLeft = rowLeft * _numRows;
+			for(; rc < _numRows && it.hasNext(); rc++, offLeft++) {
+				if(it.value() == rc)
+					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += mV[offLeft];
 				else
-					vals[def] += a[i];
-			for(; i < _numRows; i++)
-				vals[def] += a[i];
+					preAV[def] += mV[offLeft];
+			}
+
+			for(; rc < _numRows; rc++, offLeft++) {
+				preAV[def] += mV[offLeft];
+			}
 		}
-		return vals;
 	}
 
-	@Override
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
 		final int numVals = getNumValues();
-		final double[] vals = allocDVector(numVals, true);
-		final int[] indexes = sb.indexes(row);
-		final double[] sparseV = sb.values(row);
-		final AIterator it = _indexes.getIterator();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final AIterator it = _indexes.getIterator();
+			final int def = offOut + numVals - 1;
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final int[] aix = sb.indexes(rowLeft);
+			final double[] avals = sb.values(rowLeft);
+			int j = apos;
+			for(; j < alen && it.hasNext(); j++) {
+				it.skipTo(aix[j]);
+				if(it.value() == aix[j])
+					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += avals[j];
+				else
+					preAV[def] += avals[j];
+			}
 
-		for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++) {
-			it.skipTo(indexes[i]);
-			if(it.value() == indexes[i])
-				vals[getIndex(it.getDataIndexAndIncrement())] += sparseV[i];
-			else
-				vals[numVals - 1] += sparseV[i];
+			for(; j < alen; j++) {
+				preAV[def] += avals[j];
+			}
 		}
-		return vals;
 	}
 
 	@Override
@@ -360,7 +426,7 @@ public class ColGroupSDC extends ColGroupValue {
 	public void readFields(DataInput in) throws IOException {
 		super.readFields(in);
 		_indexes = OffsetFactory.readIn(in);
-		_data = MapToFactory.readIn(in, getNumValues());
+		_data = MapToFactory.readIn(in);
 	}
 
 	@Override
@@ -394,214 +460,214 @@ public class ColGroupSDC extends ColGroupValue {
 		return sb.toString();
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int nCol = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator it = _indexes.getIterator();
-		final int offsetToDefault = this.getNumValues() - 1;
-
-		int i = 0;
-
-		int row;
-		for(; i < this._numRows && it.hasNext(); i++) {
-			int col = lhs._data.getIndex(i);
-			if(it.value() == i)
-				row = getIndex(it.getDataIndexAndIncrement());
-			else
-				row = offsetToDefault;
-			ag.increment(col + row * nCol);
-		}
-		row = offsetToDefault;
-		for(; i < this._numRows; i++) {
-			int col = lhs._data.getIndex(i);
-			ag.increment(col + row * nCol);
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		final int lhsNV = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = lhsNV * rhsNV;
-		final int nCol = lhs.getNumValues();
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		final int defL = lhsNV - 1;
-		final int defR = rhsNV - 1;
-
-		AIterator lIt = lhs._indexes.getIterator();
-		AIterator rIt = _indexes.getIterator();
-
-		int i = 0;
-		int col;
-		int row;
-		for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-			if(lIt.value() == i)
-				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-			else
-				col = defL;
-			if(rIt.value() == i)
-				row = this.getIndex(rIt.getDataIndexAndIncrement());
-			else
-				row = defR;
-			ag.increment(col + row * nCol);
-		}
-
-		if(lIt.hasNext()) {
-			row = defR;
-			for(; i < this._numRows && lIt.hasNext(); i++) {
-				if(lIt.value() == i)
-					col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-				else
-					col = defL;
-
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		if(rIt.hasNext()) {
-			col = defL;
-			for(; i < this._numRows && rIt.hasNext(); i++) {
-				if(rIt.value() == i)
-					row = this.getIndex(rIt.getDataIndexAndIncrement());
-				else
-					row = defR;
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		ag.increment(defL + defR * nCol, this._numRows - i);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		final int lhsNV = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = lhsNV * rhsNV;
-		final int nCol = lhs.getNumValues();
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final int defR = rhsNV - 1;
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		int i = 0;
-		int col;
-		int row;
-		for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-			if(lIt.value() == i) {
-				col = 1;
-				lIt.next();
-			}
-			else
-				col = 0;
-			if(rIt.value() == i)
-				row = this.getIndex(rIt.getDataIndexAndIncrement());
-			else
-				row = defR;
-			ag.increment(col + row * nCol);
-		}
-
-		if(lIt.hasNext()) {
-			row = defR;
-			for(; i < this._numRows && lIt.hasNext(); i++) {
-				if(lIt.value() == i) {
-					col = 1;
-					lIt.next();
-				}
-				else
-					col = 0;
-
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		if(rIt.hasNext()) {
-			for(; i < this._numRows && rIt.hasNext(); i++) {
-				if(rIt.value() == i)
-					row = this.getIndex(rIt.getDataIndexAndIncrement());
-				else
-					row = defR;
-				ag.increment(row * nCol);
-			}
-		}
-
-		ag.increment(defR * nCol, this._numRows - i);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int defR = (rhsNV - 1) * nCol;
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value())
-				ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) +
-					this.getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-			else if(lIt.value() > rIt.value())
-				rIt.next();
-			else
-				ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
-
-		while(lIt.hasNext())
-			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		final int defR = (NVR - 1) * NVL;
-
-		for(int kl = 0; kl < NVL; kl++) {
-			AIterator it = _indexes.getIterator();
-			final int bOffL = lhs._ptr[kl];
-			final int bLenL = lhs.len(kl);
-			for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-				sLenL = lhs._data[bOffL + bixL];
-				for(int i = 1; i <= sLenL; i++) {
-					final int col = offL + lhs._data[bOffL + bixL + i];
-					it.skipTo(col);
-					if(it.value() == col)
-						ag.increment(kl + this.getIndex(it.getDataIndexAndIncrement()) * NVL);
-					else
-						ag.increment(kl + defR);
-
-				}
-			}
-		}
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator it = _indexes.getIterator();
+	// 	final int offsetToDefault = this.getNumValues() - 1;
+
+	// 	int i = 0;
+
+	// 	int row;
+	// 	for(; i < this._numRows && it.hasNext(); i++) {
+	// 		int col = lhs._data.getIndex(i);
+	// 		if(it.value() == i)
+	// 			row = getIndex(it.getDataIndexAndIncrement());
+	// 		else
+	// 			row = offsetToDefault;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+	// 	row = offsetToDefault;
+	// 	for(; i < this._numRows; i++) {
+	// 		int col = lhs._data.getIndex(i);
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	final int lhsNV = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = lhsNV * rhsNV;
+	// 	final int nCol = lhs.getNumValues();
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	final int defL = lhsNV - 1;
+	// 	final int defR = rhsNV - 1;
+
+	// 	AIterator lIt = lhs._indexes.getIterator();
+	// 	AIterator rIt = _indexes.getIterator();
+
+	// 	int i = 0;
+	// 	int col;
+	// 	int row;
+	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
+	// 		if(lIt.value() == i)
+	// 			col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+	// 		else
+	// 			col = defL;
+	// 		if(rIt.value() == i)
+	// 			row = this.getIndex(rIt.getDataIndexAndIncrement());
+	// 		else
+	// 			row = defR;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	if(lIt.hasNext()) {
+	// 		row = defR;
+	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
+	// 			if(lIt.value() == i)
+	// 				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+	// 			else
+	// 				col = defL;
+
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	if(rIt.hasNext()) {
+	// 		col = defL;
+	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
+	// 			if(rIt.value() == i)
+	// 				row = this.getIndex(rIt.getDataIndexAndIncrement());
+	// 			else
+	// 				row = defR;
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	ag.increment(defL + defR * nCol, this._numRows - i);
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	final int lhsNV = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = lhsNV * rhsNV;
+	// 	final int nCol = lhs.getNumValues();
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final int defR = rhsNV - 1;
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = _indexes.getIterator();
+
+	// 	int i = 0;
+	// 	int col;
+	// 	int row;
+	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
+	// 		if(lIt.value() == i) {
+	// 			col = 1;
+	// 			lIt.next();
+	// 		}
+	// 		else
+	// 			col = 0;
+	// 		if(rIt.value() == i)
+	// 			row = this.getIndex(rIt.getDataIndexAndIncrement());
+	// 		else
+	// 			row = defR;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	if(lIt.hasNext()) {
+	// 		row = defR;
+	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
+	// 			if(lIt.value() == i) {
+	// 				col = 1;
+	// 				lIt.next();
+	// 			}
+	// 			else
+	// 				col = 0;
+
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	if(rIt.hasNext()) {
+	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
+	// 			if(rIt.value() == i)
+	// 				row = this.getIndex(rIt.getDataIndexAndIncrement());
+	// 			else
+	// 				row = defR;
+	// 			ag.increment(row * nCol);
+	// 		}
+	// 	}
+
+	// 	ag.increment(defR * nCol, this._numRows - i);
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int defR = (rhsNV - 1) * nCol;
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = _indexes.getIterator();
+
+	// 	while(lIt.hasNext() && rIt.hasNext())
+	// 		if(lIt.value() == rIt.value())
+	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) +
+	// 				this.getIndex(rIt.getDataIndexAndIncrement()) * nCol);
+	// 		else if(lIt.value() > rIt.value())
+	// 			rIt.next();
+	// 		else
+	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
+
+	// 	while(lIt.hasNext())
+	// 		ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	final int NVR = this.getNumValues();
+	// 	final int NVL = lhs.getNumValues();
+	// 	final int retSize = NVR * NVL;
+	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// 	final int defR = (NVR - 1) * NVL;
+
+	// 	for(int kl = 0; kl < NVL; kl++) {
+	// 		AIterator it = _indexes.getIterator();
+	// 		final int bOffL = lhs._ptr[kl];
+	// 		final int bLenL = lhs.len(kl);
+	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
+	// 			sLenL = lhs._data[bOffL + bixL];
+	// 			for(int i = 1; i <= sLenL; i++) {
+	// 				final int col = offL + lhs._data[bOffL + bixL + i];
+	// 				it.skipTo(col);
+	// 				if(it.value() == col)
+	// 					ag.increment(kl + this.getIndex(it.getDataIndexAndIncrement()) * NVL);
+	// 				else
+	// 					ag.increment(kl + defR);
+
+	// 			}
+	// 		}
+	// 	}
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -746,9 +812,4 @@ public class ColGroupSDC extends ColGroupValue {
 		return ret;
 	}
 
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
-
 }
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 493afac..d749ea5 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,8 +30,6 @@ 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.OffsetFactory;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -86,17 +84,10 @@ public class ColGroupSDCSingle extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(_numRows * _colIndexes.length + target.getNonZeros());
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
-
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
-		final double[] values = getValues();
 		final int offsetToDefault = values.length - nCol;
 
 		double[] c = target.getDenseBlockValues();
@@ -121,62 +112,68 @@ public class ColGroupSDCSingle extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		throw new NotImplementedException("Not Implemented");
-	}
-
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		final double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-		final int offsetToDefault = _colIndexes.length;
-		final AIterator it = _indexes.getIterator();
-		final double v1 = values[offsetToDefault + colpos];
-		final double v2 = values[colpos];
-
-		int i = 0;
-		for(; i < _numRows && it.hasNext(); i++) {
-			if(it.value() == i) {
-				c[i] += v1;
-				it.next();
-			}
-			else
-				c[i] += v2;
-		}
-		for(; i < _numRows; i++)
-			c[i] += v2;
-
-		target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock values) {
+		throw new NotImplementedException();
 	}
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
-
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final int nCol = _colIndexes.length;
-		final double[] values = getValues();
-		final int offsetToDefault = values.length - nCol;
-		final AIterator it = _indexes.getIterator();
-
-		int offT = 0;
-		int i = rl;
-		it.skipTo(rl);
-
-		for(; i < ru && it.hasNext(); i++, offT++) {
-			if(it.value() == i) {
-				it.next();
-				c[offT] += values[colpos];
-			}
-			else
-				c[offT] += values[offsetToDefault + colpos];
-		}
-
-		for(; i < ru; i++, offT++)
-			c[offT] += values[offsetToDefault + colpos];
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	final double[] c = target.getDenseBlockValues();
+	// 	final double[] values = getValues();
+	// 	final int offsetToDefault = _colIndexes.length;
+	// 	final AIterator it = _indexes.getIterator();
+	// 	final double v1 = values[offsetToDefault + colpos];
+	// 	final double v2 = values[colpos];
+
+	// 	int i = 0;
+	// 	for(; i < _numRows && it.hasNext(); i++) {
+	// 		if(it.value() == i) {
+	// 			c[i] += v1;
+	// 			it.next();
+	// 		}
+	// 		else
+	// 			c[i] += v2;
+	// 	}
+	// 	for(; i < _numRows; i++)
+	// 		c[i] += v2;
+
+	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	final int nCol = _colIndexes.length;
+	// 	final double[] values = getValues();
+	// 	final int offsetToDefault = values.length - nCol;
+	// 	final AIterator it = _indexes.getIterator();
+
+	// 	int offT = 0;
+	// 	int i = rl;
+	// 	it.skipTo(rl);
+
+	// 	for(; i < ru && it.hasNext(); i++, offT++) {
+	// 		if(it.value() == i) {
+	// 			it.next();
+	// 			c[offT] += values[colpos];
+	// 		}
+	// 		else
+	// 			c[offT] += values[offsetToDefault + colpos];
+	// 	}
+
+	// 	for(; i < ru; i++, offT++)
+	// 		c[offT] += values[offsetToDefault + colpos];
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -185,12 +182,11 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		if(ix < 0)
 			throw new RuntimeException("Column index " + c + " not in group.");
 
-		AIterator it = _indexes.getIterator();
-		it.skipTo(r);
+		AIterator it = _indexes.getIterator(r);
 		if(it.value() == r)
-			return _dict.getValue(ix + c);
+			return _dict.getValue(ix);
 		else
-			return _dict.getValue(_colIndexes.length + c);
+			return _dict.getValue(_colIndexes.length + ix);
 
 	}
 
@@ -200,7 +196,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 
 		// // pre-aggregate nnz per value tuple
 		final double[] vals = _dict.sumAllRowsToDouble(square, _colIndexes.length);
@@ -265,57 +261,115 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		return counts;
 	}
 
-	public double[] preAggregate(double[] a, int row) {
-		final int numVals = getNumValues();
-		final double[] vals = allocDVector(numVals, true);
-		final AIterator it = _indexes.getIterator();
-
-		int i = 0;
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// final int numVals = getNumValues();
+	// final double[] vals = allocDVector(numVals, true);
+	// final AIterator it = _indexes.getIterator();
+
+	// int i = 0;
+
+	// if(row > 0) {
+	// int offA = _numRows * row;
+	// for(; i < _numRows && it.hasNext(); i++, offA++)
+	// if(it.value() == i) {
+	// it.next();
+	// vals[0] += a[offA];
+	// }
+	// else
+	// vals[1] += a[offA];
+	// for(; i < _numRows; i++, offA++)
+	// vals[1] += a[offA];
+	// }
+	// else {
+	// for(; i < _numRows && it.hasNext(); i++)
+	// if(it.value() == i) {
+	// it.next();
+	// vals[0] += a[i];
+	// }
+	// else
+	// vals[1] += a[i];
+	// for(; i < _numRows; i++)
+	// vals[1] += a[i];
+	// }
+
+	// return vals;
+	// }
+
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// final int numVals = getNumValues();
+	// final double[] vals = allocDVector(numVals, true);
+	// final int[] indexes = sb.indexes(row);
+	// final double[] sparseV = sb.values(row);
+	// final AIterator it = _indexes.getIterator();
+
+	// for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++) {
+	// it.skipTo(indexes[i]);
+	// if(it.value() == indexes[i]) {
+	// vals[0] += sparseV[i];
+	// it.next();
+	// }
+	// else
+	// vals[1] += sparseV[i];
+	// }
+	// return vals;
+	// }
+
+	@Override
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
+		else
+			preAggregateDense(m, preAgg, rl, ru);
+	}
 
-		if(row > 0) {
-			int offA = _numRows * row;
-			for(; i < _numRows && it.hasNext(); i++, offA++)
-				if(it.value() == i){
-					it.next();
-					vals[0] += a[offA];
-				}
-				else
-					vals[1] += a[offA];
-			for(; i < _numRows; i++, offA++)
-				vals[1] += a[offA];
-		}
-		else {
-			for(; i < _numRows && it.hasNext(); i++)
-				if(it.value() == i){
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			final AIterator it = _indexes.getIterator();
+			final int def = offOut + 1;
+			int rc = 0;
+			int offLeft = rowLeft * _numRows;
+			for(; rc < _numRows; rc++, offLeft++) {
+				if(it.value() == rc) {
+					preAV[offOut] += mV[offLeft];
 					it.next();
-					vals[0] += a[i];
 				}
 				else
-					vals[1] += a[i];
-			for(; i < _numRows; i++)
-				vals[1] += a[i];
+					preAV[def] += mV[offLeft];
+			}
+			for(; rc < _numRows; rc++, offLeft++) {
+				preAV[def] += mV[offLeft];
+			}
 		}
-
-		return vals;
 	}
 
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
 		final int numVals = getNumValues();
-		final double[] vals = allocDVector(numVals, true);
-		final int[] indexes = sb.indexes(row);
-		final double[] sparseV = sb.values(row);
-		final AIterator it = _indexes.getIterator();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final AIterator it = _indexes.getIterator();
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final int[] aix = sb.indexes(rowLeft);
+			final double[] avals = sb.values(rowLeft);
+			final int def = offOut + 1;
+			for(int j = apos; j < alen; j++) {
+				it.skipTo(aix[j]);
+				if(it.value() == aix[j]) {
+					preAV[offOut] += avals[j];
+					it.next();
+				}
+				else
+					preAV[def] += avals[j];
 
-		for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++) {
-			it.skipTo(indexes[i]);
-			if(it.value() == indexes[i]) {
-				vals[0] += sparseV[i];
-				it.next();
 			}
-			else
-				vals[1] += sparseV[i];
 		}
-		return vals;
 	}
 
 	@Override
@@ -374,181 +428,181 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		return sb.toString();
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator it = _indexes.getIterator();
-
-		int i = 0;
-
-		int row;
-		for(; i < this._numRows && it.hasNext(); i++) {
-			int col = lhs._data.getIndex(i);
-			if(it.value() == i) {
-				row = 0;
-				it.next();
-			}
-			else
-				row = 1;
-
-			if(col < lhs.getNumValues())
-				ag.increment(col + row * nCol);
-		}
-		row = 0;
-		for(; i < this._numRows; i++) {
-			int col = lhs._data.getIndex(i);
-			if(col < lhs.getNumValues())
-				ag.increment(col + row * nCol);
-		}
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		final int lhsNV = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = lhsNV * rhsNV;
-		final int nCol = lhs.getNumValues();
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final int defL = lhsNV - 1;
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		int i = 0;
-		int col;
-		int row;
-		for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-			if(lIt.value() == i)
-				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-			else
-				col = defL;
-			if(rIt.value() == i) {
-				row = 0;
-				rIt.next();
-			}
-			else
-				row = 1;
-			ag.increment(col + row * nCol);
-		}
-
-		if(lIt.hasNext()) {
-			row = 0;
-			for(; i < this._numRows && lIt.hasNext(); i++) {
-				if(lIt.value() == i)
-					col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-				else
-					col = defL;
-
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		if(rIt.hasNext()) {
-			col = defL;
-			for(; i < this._numRows && rIt.hasNext(); i++) {
-				if(rIt.value() == i) {
-					row = 0;
-					rIt.next();
-				}
-				else
-					row = 1;
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		ag.increment(defL, this._numRows - i);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		final int lhsNV = lhs.getNumValues();
-		final int rhsNV = this.getNumValues();
-		final int retSize = lhsNV * rhsNV;
-		final int nCol = lhs.getNumValues();
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		;
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		int i = 0;
-		int col;
-		int row;
-		for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-			if(lIt.value() == i) {
-				col = 1;
-				lIt.next();
-			}
-			else
-				col = 0;
-			if(rIt.value() == i) {
-				row = 1;
-				rIt.next();
-			}
-			else
-				row = 0;
-			ag.increment(col + row * nCol);
-		}
-
-		if(lIt.hasNext()) {
-			row = 1;
-			for(; i < _numRows && lIt.hasNext(); i++) {
-				if(lIt.value() == i) {
-					col = 1;
-					lIt.next();
-				}
-				else
-					col = 0;
-
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		if(rIt.hasNext()) {
-			col = 1;
-			for(; i < _numRows && rIt.hasNext(); i++) {
-				if(rIt.value() == i) {
-					row = 1;
-					rIt.next();
-				}
-				else
-					row = 0;
-				ag.increment(col + row * nCol);
-			}
-		}
-
-		ag.increment(0, _numRows - i);
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator it = _indexes.getIterator();
+
+	// 	int i = 0;
+
+	// 	int row;
+	// 	for(; i < this._numRows && it.hasNext(); i++) {
+	// 		int col = lhs._data.getIndex(i);
+	// 		if(it.value() == i) {
+	// 			row = 0;
+	// 			it.next();
+	// 		}
+	// 		else
+	// 			row = 1;
+
+	// 		if(col < lhs.getNumValues())
+	// 			ag.increment(col + row * nCol);
+	// 	}
+	// 	row = 0;
+	// 	for(; i < this._numRows; i++) {
+	// 		int col = lhs._data.getIndex(i);
+	// 		if(col < lhs.getNumValues())
+	// 			ag.increment(col + row * nCol);
+	// 	}
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	final int lhsNV = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = lhsNV * rhsNV;
+	// 	final int nCol = lhs.getNumValues();
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final int defL = lhsNV - 1;
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = _indexes.getIterator();
+
+	// 	int i = 0;
+	// 	int col;
+	// 	int row;
+	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
+	// 		if(lIt.value() == i)
+	// 			col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+	// 		else
+	// 			col = defL;
+	// 		if(rIt.value() == i) {
+	// 			row = 0;
+	// 			rIt.next();
+	// 		}
+	// 		else
+	// 			row = 1;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	if(lIt.hasNext()) {
+	// 		row = 0;
+	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
+	// 			if(lIt.value() == i)
+	// 				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+	// 			else
+	// 				col = defL;
+
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	if(rIt.hasNext()) {
+	// 		col = defL;
+	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
+	// 			if(rIt.value() == i) {
+	// 				row = 0;
+	// 				rIt.next();
+	// 			}
+	// 			else
+	// 				row = 1;
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	ag.increment(defL, this._numRows - i);
+
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	final int lhsNV = lhs.getNumValues();
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int retSize = lhsNV * rhsNV;
+	// 	final int nCol = lhs.getNumValues();
+	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	;
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = _indexes.getIterator();
+
+	// 	int i = 0;
+	// 	int col;
+	// 	int row;
+	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
+	// 		if(lIt.value() == i) {
+	// 			col = 1;
+	// 			lIt.next();
+	// 		}
+	// 		else
+	// 			col = 0;
+	// 		if(rIt.value() == i) {
+	// 			row = 1;
+	// 			rIt.next();
+	// 		}
+	// 		else
+	// 			row = 0;
+	// 		ag.increment(col + row * nCol);
+	// 	}
+
+	// 	if(lIt.hasNext()) {
+	// 		row = 1;
+	// 		for(; i < _numRows && lIt.hasNext(); i++) {
+	// 			if(lIt.value() == i) {
+	// 				col = 1;
+	// 				lIt.next();
+	// 			}
+	// 			else
+	// 				col = 0;
+
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	if(rIt.hasNext()) {
+	// 		col = 1;
+	// 		for(; i < _numRows && rIt.hasNext(); i++) {
+	// 			if(rIt.value() == i) {
+	// 				row = 1;
+	// 				rIt.next();
+	// 			}
+	// 			else
+	// 				row = 0;
+	// 			ag.increment(col + row * nCol);
+	// 		}
+	// 	}
+
+	// 	ag.increment(0, _numRows - i);
+	// 	return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -585,7 +639,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 				}
 				else if(itThat.value() < itThis.value())
 					itThat.next();
-				else{
+				else {
 					itThis.next();
 					// that._dict.addToEntry(ret, defThat, 0, nCol);
 				}
@@ -677,8 +731,4 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	}
 
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
 }
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 7e68e17..ff9da0e 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
@@ -30,8 +30,6 @@ 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.OffsetFactory;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -88,21 +86,14 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(_indexes.getSize() * _colIndexes.length + target.getNonZeros());
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
-		final double[] values = getValues();
 		final int offTCorrected = offT - rl;
 		final double[] c = target.getDenseBlockValues();
 
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		AIterator it = _indexes.getIterator(rl);
 
 		while(it.hasNext() && it.value() < ru) {
 			int rc = (offTCorrected + it.value()) * tCol;
@@ -114,37 +105,42 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		throw new NotImplementedException("Not Implemented");
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock values) {
+		throw new NotImplementedException();
 	}
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		final double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-		final AIterator it = _indexes.getIterator();
-		while(it.hasNext()) {
-			c[it.value()] += values[_colIndexes.length + colpos];
-			it.next();
-		}
-		target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	final double[] c = target.getDenseBlockValues();
+	// 	final double[] values = getValues();
+	// 	final AIterator it = _indexes.getIterator();
+	// 	while(it.hasNext()) {
+	// 		c[it.value()] += values[_colIndexes.length + colpos];
+	// 		it.next();
+	// 	}
+	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+	// }
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final double[] values = getValues();
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
-		while(it.hasNext() && it.value() < ru) {
-			c[it.value() - rl] += values[colpos];
-			it.next();
-		}
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// 	final double[] values = getValues();
+	// 	final AIterator it = _indexes.getIterator(rl);
+	// 	while(it.hasNext() && it.value() < ru) {
+	// 		c[it.value() - rl] += values[colpos];
+	// 		it.next();
+	// 	}
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -152,8 +148,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		if(ix < 0)
 			throw new RuntimeException("Column index " + c + " not in group.");
 
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(r);
+		final AIterator it = _indexes.getIterator(r);
 		if(it.value() == r)
 			return _dict.getValue(ix);
 		else
@@ -164,8 +159,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	@Override
 	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int nCol = _colIndexes.length;
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 		while(it.hasNext() && it.value() < ru) {
 			rnnz[it.value() - rl] += nCol;
 			it.next();
@@ -173,10 +167,9 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		final double vals = _dict.sumAllRowsToDouble(square, _colIndexes.length)[0];
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 		while(it.hasNext() && it.value() < ru) {
 			c[it.value()] += vals;
 			it.next();
@@ -187,8 +180,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	@Override
 	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru) {
 		final double vals = _dict.aggregateTuples(builtin, _colIndexes.length)[0];
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 		int rix = rl;
 		for(; rix < ru && it.hasNext(); rix++) {
 			if(it.value() != rix)
@@ -213,8 +205,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 
 	@Override
 	public int[] getCounts(int rl, int ru, int[] counts) {
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 
 		while(it.hasNext() && it.value() < ru) {
 			it.next();
@@ -226,46 +217,90 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		return counts;
 	}
 
-	public double[] preAggregate(double[] a, int aRows) {
-		final double[] vals = allocDVector(getNumValues(), true);
-		final AIterator it = _indexes.getIterator();
-		if(aRows > 0) {
-			final int offT = _numRows * aRows;
-			while(it.hasNext()) {
-				final int i = it.value();
-				vals[0] += a[i + offT];
-				it.next();
-			}
-		}
+	// @Override
+	// public double[] preAggregate(double[] a, int row) {
+	// 	final double[] vals = allocDVector(getNumValues(), true);
+	// 	final AIterator it = _indexes.getIterator();
+	// 	if(row > 0) {
+	// 		final int offT = _numRows * row;
+	// 		while(it.hasNext()) {
+	// 			final int i = it.value();
+	// 			vals[0] += a[i + offT];
+	// 			it.next();
+	// 		}
+	// 	}
+	// 	else
+	// 		while(it.hasNext()) {
+	// 			final int i = it.value();
+	// 			vals[0] += a[i];
+	// 			it.next();
+	// 		}
+
+	// 	return vals;
+	// }
+
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// 	final double[] vals = allocDVector(getNumValues(), true);
+	// 	final int[] sbIndexes = sb.indexes(row);
+	// 	final double[] sparseV = sb.values(row);
+	// 	final AIterator it = _indexes.getIterator();
+	// 	final int sbEnd = sb.size(row) + sb.pos(row);
+
+	// 	int sbP = sb.pos(row);
+
+	// 	while(it.hasNext() && sbP < sbEnd) {
+	// 		if(it.value() == sbIndexes[sbP])
+	// 			vals[0] += sparseV[sbP++];
+	// 		if(sbP < sbEnd)
+	// 			it.skipTo(sbIndexes[sbP]);
+	// 		while(sbP < sbEnd && sbIndexes[sbP] < it.value())
+	// 			sbP++;
+	// 	}
+
+	// 	return vals;
+	// }
+
+	@Override
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
 		else
+			preAggregateDense(m, preAgg, rl, ru);
+	}
+
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			final AIterator it = _indexes.getIterator();
+			final int offLeft = rowLeft * _numRows;
 			while(it.hasNext()) {
 				final int i = it.value();
-				vals[0] += a[i];
+				preAV[offOut] += mV[offLeft + i];
 				it.next();
 			}
-
-		return vals;
+		}
 	}
 
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
-		final double[] vals = allocDVector(getNumValues(), true);
-		final int[] sbIndexes = sb.indexes(row);
-		final double[] sparseV = sb.values(row);
-		final AIterator it = _indexes.getIterator();
-		final int sbEnd = sb.size(row) + sb.pos(row);
-
-		int sbP = sb.pos(row);
-
-		while(it.hasNext() && sbP < sbEnd) {
-			if(it.value() == sbIndexes[sbP])
-				vals[0] += sparseV[sbP++];
-			if(sbP < sbEnd)
-				it.skipTo(sbIndexes[sbP]);
-			while(sbP < sbEnd && sbIndexes[sbP] < it.value())
-				sbP++;
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final AIterator it = _indexes.getIterator();
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final int[] aix = sb.indexes(rowLeft);
+			final double[] avals = sb.values(rowLeft);
+			for(int j = apos; j < alen; j++) {
+				it.skipTo(aix[j]);
+				if(it.value() == aix[j])
+					preAV[offOut] += avals[j];
+			}
 		}
-
-		return vals;
 	}
 
 	@Override
@@ -300,12 +335,12 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	// private ADictionary swapEntries(ADictionary aDictionary) {
-	// 	double[] values = aDictionary.getValues().clone();
-	// 	double[] swap = new double[_colIndexes.length];
-	// 	System.arraycopy(values, 0, swap, 0, _colIndexes.length);
-	// 	System.arraycopy(values, _colIndexes.length, values, 0, _colIndexes.length);
-	// 	System.arraycopy(swap, 0, values, _colIndexes.length, _colIndexes.length);
-	// 	return new Dictionary(values);
+	// double[] values = aDictionary.getValues().clone();
+	// double[] swap = new double[_colIndexes.length];
+	// System.arraycopy(values, 0, swap, 0, _colIndexes.length);
+	// System.arraycopy(values, _colIndexes.length, values, 0, _colIndexes.length);
+	// System.arraycopy(swap, 0, values, _colIndexes.length, _colIndexes.length);
+	// return new Dictionary(values);
 	// }
 
 	@Override
@@ -346,87 +381,87 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		return sb.toString();
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator it = _indexes.getIterator();
-
-		while(it.hasNext()) {
-			final int col = lhs._data.getIndex(it.value());
-			ag.increment(col);
-		}
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator it = _indexes.getIterator();
 
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// 	while(it.hasNext()) {
+	// 		final int col = lhs._data.getIndex(it.value());
+	// 		ag.increment(col);
+	// 	}
+	// 	return ag;
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = this._indexes.getIterator();
-
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value()) {
-				ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()));
-				rIt.next();
-			}
-			else if(lIt.value() < rIt.value())
-				lIt.next();
-			else
-				rIt.next();
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-		return ag;
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		// we always know that there is only one value in each column group.
-		int[] ret = new int[1];
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = this._indexes.getIterator();
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value()) {
-				ret[0]++;
-				lIt.next();
-				rIt.next();
-			}
-			else if(lIt.value() < rIt.value())
-				lIt.next();
-			else
-				rIt.next();
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// 	final int rhsNV = this.getNumValues();
+	// 	final int nCol = lhs.getNumValues();
+	// 	final int retSize = nCol * rhsNV;
+
+	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = this._indexes.getIterator();
+
+	// 	while(lIt.hasNext() && rIt.hasNext())
+	// 		if(lIt.value() == rIt.value()) {
+	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()));
+	// 			rIt.next();
+	// 		}
+	// 		else if(lIt.value() < rIt.value())
+	// 			lIt.next();
+	// 		else
+	// 			rIt.next();
+
+	// 	return ag;
+	// }
 
-		return PreAggregateFactory.ag(ret);
-	}
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// 	// we always know that there is only one value in each column group.
+	// 	int[] ret = new int[1];
+	// 	final AIterator lIt = lhs._indexes.getIterator();
+	// 	final AIterator rIt = this._indexes.getIterator();
+	// 	while(lIt.hasNext() && rIt.hasNext())
+	// 		if(lIt.value() == rIt.value()) {
+	// 			ret[0]++;
+	// 			lIt.next();
+	// 			rIt.next();
+	// 		}
+	// 		else if(lIt.value() < rIt.value())
+	// 			lIt.next();
+	// 		else
+	// 			rIt.next();
+
+	// 	return PreAggregateFactory.ag(ret);
+	// }
 
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// 		+ this.getClass().getSimpleName());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -440,7 +475,19 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 
 	@Override
 	public Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
-		throw new NotImplementedException();
+		final AIterator itThat = that._indexes.getIterator();
+		final AIterator itThis = that._indexes.getIterator();
+		final int nCol = that._colIndexes.length;
+
+		while(itThat.hasNext() && itThis.hasNext()) {
+			final int v = itThat.value();
+			if(v == itThis.skipTo(v))
+				that._dict.addToEntry(ret, that.getIndex(itThat.getDataIndex()), 0, nCol);
+
+			itThat.next();
+		}
+
+		return ret;
 	}
 
 	@Override
@@ -464,8 +511,4 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		throw new NotImplementedException();
 	}
 
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
 }
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 f04f1a3..036d40e 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
@@ -25,8 +25,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.sysds.runtime.DMLCompressionException;
-import org.apache.sysds.runtime.compress.CompressionSettings;
 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;
@@ -34,8 +32,6 @@ import org.apache.sysds.runtime.compress.colgroup.mapping.MapToFactory;
 import org.apache.sysds.runtime.compress.colgroup.offset.AIterator;
 import org.apache.sysds.runtime.compress.colgroup.offset.AOffset;
 import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -76,19 +72,14 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	protected ColGroupSDCZeros(int[] colIndices, int numRows, ADictionary dict, int[] indexes, AMapToData data,
 		int[] cachedCounts) {
 		super(colIndices, numRows, dict, cachedCounts);
-		if(data == null)
-			throw new DMLCompressionException("data null input In SDC Construction");
 		_indexes = OffsetFactory.create(indexes, numRows);
 		_data = data;
 		_zeros = true;
-
 	}
 
 	protected ColGroupSDCZeros(int[] colIndices, int numRows, ADictionary dict, AOffset offsets, AMapToData data,
 		int[] cachedCounts) {
 		super(colIndices, numRows, dict, cachedCounts);
-		if(data == null)
-			throw new DMLCompressionException("data null input In SDC Construction");
 		_indexes = offsets;
 		_data = data;
 		_zeros = true;
@@ -105,22 +96,14 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		decompressToBlockUnSafe(target, rl, ru, offT);
-		target.setNonZeros(getNumberNonZeros());
-	}
-
-	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
+	protected void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
-		final double[] values = getValues();
 		final int offTCorrected = offT - rl;
 		final double[] c = target.getDenseBlockValues();
-
-		AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
-
+		AIterator it = _indexes.getIterator(rl);
+		offT = offT * tCol;
 		while(it.hasNext() && it.value() < ru) {
 			int rc = (offTCorrected + it.value()) * tCol;
 			int offC = getIndex(it.getDataIndexAndIncrement()) * nCol;
@@ -131,33 +114,55 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	protected void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock sb) {
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+		final int tCol = target.getNumColumns();
+		final int offTCorrected = offT - rl;
 		final double[] c = target.getDenseBlockValues();
-		final double[] values = getValues();
-		final AIterator it = _indexes.getIterator();
-		while(it.hasNext())
-			c[it.value()] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
-		target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+		AIterator it = _indexes.getIterator(rl);
+		while(it.hasNext() && it.value() < ru) {
+			final int rc = (offTCorrected + it.value()) * tCol;
+			final int dictIndex = getIndex(it.getDataIndexAndIncrement());
+			if(sb.isEmpty(dictIndex))
+				continue;
+
+			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[rc + _colIndexes[aix[j]]] += avals[j];
+		}
 	}
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// throw new NotImplementedException();
+	// }
 
-	@Override
-	public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-		final double[] values = getValues();
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
-		while(it.hasNext() && it.value() < ru)
-			c[it.value() - rl] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// final double[] c = target.getDenseBlockValues();
+	// final double[] values = getValues();
+	// final AIterator it = _indexes.getIterator();
+	// while(it.hasNext())
+	// c[it.value()] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
+	// target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// throw new NotImplementedException();
+	// }
+
+	// @Override
+	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
+	// final double[] values = getValues();
+	// final AIterator it = _indexes.getIterator(rl);
+	// while(it.hasNext() && it.value() < ru)
+	// c[it.value() - rl] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -165,8 +170,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		if(ix < 0)
 			throw new RuntimeException("Column index " + c + " not in group.");
 
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(r);
+		final AIterator it = _indexes.getIterator(r);
 		if(it.value() == r)
 			return _dict.getValue(getIndex(it.getDataIndex()) * _colIndexes.length + ix);
 		else
@@ -177,8 +181,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	@Override
 	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int nCol = _colIndexes.length;
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 		while(it.hasNext() && it.value() < ru) {
 			rnnz[it.value() - rl] += nCol;
 			it.next();
@@ -186,10 +189,9 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	}
 
 	@Override
-	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
+	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 		final double[] vals = _dict.sumAllRowsToDouble(square, _colIndexes.length);
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 		while(it.hasNext() && it.value() < ru)
 			c[it.value()] += vals[getIndex(it.getDataIndexAndIncrement())];
 	}
@@ -197,8 +199,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	@Override
 	protected void computeRowMxx(double[] c, Builtin builtin, int rl, int ru) {
 		final double[] vals = _dict.aggregateTuples(builtin, _colIndexes.length);
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 
 		int rix = rl;
 		for(; rix < ru && it.hasNext(); rix++) {
@@ -222,8 +223,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	public int[] getCounts(int rl, int ru, int[] counts) {
 
 		int i = rl;
-		final AIterator it = _indexes.getIterator();
-		it.skipTo(rl);
+		final AIterator it = _indexes.getIterator(rl);
 
 		int zeros = 0;
 		while(it.hasNext() && it.value() < ru) {
@@ -242,46 +242,87 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		return _data.getIndex(r);
 	}
 
-	@Override
-	public double[] preAggregate(double[] a, int aRows) {
-		final double[] vals = allocDVector(getNumValues(), true);
-		final AIterator it = _indexes.getIterator();
-		if(aRows > 0) {
-			final int offT = _numRows * aRows;
-			while(it.hasNext()) {
-				final int i = it.value();
-				vals[getIndex(it.getDataIndexAndIncrement())] += a[i + offT];
-			}
-		}
+	// @Override
+	// public double[] preAggregate(double[] a, int aRows) {
+	// final double[] vals = allocDVector(getNumValues(), true);
+	// final AIterator it = _indexes.getIterator();
+	// if(aRows > 0) {
+	// final int offT = _numRows * aRows;
+	// while(it.hasNext()) {
+	// final int i = it.value();
+	// vals[getIndex(it.getDataIndexAndIncrement())] += a[i + offT];
+	// }
+	// }
+	// else
+	// while(it.hasNext()) {
+	// final int i = it.value();
+	// vals[getIndex(it.getDataIndexAndIncrement())] += a[i];
+	// }
+
+	// return vals;
+	// }
+
+	// @Override
+	// public double[] preAggregateSparse(SparseBlock sb, int row) {
+	// final double[] vals = allocDVector(getNumValues(), true);
+	// final int[] sbIndexes = sb.indexes(row);
+	// final double[] sparseV = sb.values(row);
+	// final AIterator it = _indexes.getIterator();
+	// final int sbEnd = sb.size(row) + sb.pos(row);
+
+	// int sbP = sb.pos(row);
+
+	// while(it.hasNext() && sbP < sbEnd) {
+	// if(it.value() == sbIndexes[sbP])
+	// vals[getIndex(it.getDataIndexAndIncrement())] += sparseV[sbP++];
+	// if(sbP < sbEnd)
+	// it.skipTo(sbIndexes[sbP]);
+	// while(sbP < sbEnd && sbIndexes[sbP] < it.value())
+	// sbP++;
+	// }
+
+	// return vals;
+	// }
+
+	@Override
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		if(m.isInSparseFormat())
+			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
 		else
+			preAggregateDense(m, preAgg, rl, ru);
+	}
+
+	private void preAggregateDense(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final double[] mV = m.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			final AIterator it = _indexes.getIterator();
+			final int offLeft = rowLeft * _numRows;
 			while(it.hasNext()) {
 				final int i = it.value();
-				vals[getIndex(it.getDataIndexAndIncrement())] += a[i];
+				preAV[offOut + getIndex(it.getDataIndexAndIncrement())] += mV[offLeft + i];
 			}
-
-		return vals;
+		}
 	}
 
-	@Override
-	public double[] preAggregateSparse(SparseBlock sb, int row) {
-		final double[] vals = allocDVector(getNumValues(), true);
-		final int[] sbIndexes = sb.indexes(row);
-		final double[] sparseV = sb.values(row);
-		final AIterator it = _indexes.getIterator();
-		final int sbEnd = sb.size(row) + sb.pos(row);
-
-		int sbP = sb.pos(row);
-
-		while(it.hasNext() && sbP < sbEnd) {
-			if(it.value() == sbIndexes[sbP])
-				vals[getIndex(it.getDataIndexAndIncrement())] += sparseV[sbP++];
-			if(sbP < sbEnd)
-				it.skipTo(sbIndexes[sbP]);
-			while(sbP < sbEnd && sbIndexes[sbP] < it.value())
-				sbP++;
+	private void preAggregateSparse(SparseBlock sb, MatrixBlock preAgg, int rl, int ru) {
+		final double[] preAV = preAgg.getDenseBlockValues();
+		final int numVals = getNumValues();
+		for(int rowLeft = rl, offOut = 0; rowLeft < ru; rowLeft++, offOut += numVals) {
+			if(sb.isEmpty(rowLeft))
+				continue;
+			final AIterator it = _indexes.getIterator();
+			final int apos = sb.pos(rowLeft);
+			final int alen = sb.size(rowLeft) + apos;
+			final int[] aix = sb.indexes(rowLeft);
+			final double[] avals = sb.values(rowLeft);
+			for(int j = apos; j < alen; j++) {
+				it.skipTo(aix[j]);
+				if(it.value() == aix[j])
+					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += avals[j];
+			}
 		}
-
-		return vals;
 	}
 
 	@Override
@@ -325,7 +366,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	public void readFields(DataInput in) throws IOException {
 		super.readFields(in);
 		_indexes = OffsetFactory.readIn(in);
-		_data = MapToFactory.readIn(in, getNumValues());
+		_data = MapToFactory.readIn(in);
 	}
 
 	@Override
@@ -358,132 +399,132 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		return sb.toString();
 	}
 
-	@Override
-	public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator it = _indexes.getIterator();
-
-		while(it.hasNext()) {
-			final int col = lhs._data.getIndex(it.value());
-			final int row = getIndex(it.getDataIndexAndIncrement());
-			ag.increment(col + row * nCol);
-		}
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-
-		final int defL = nCol - 1;
-		final int retSize = nCol * rhsNV;
-
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		AIterator lIt = lhs._indexes.getIterator();
-		AIterator rIt = this._indexes.getIterator();
-
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value())
-				ag.increment(
-					lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-			else if(lIt.value() > rIt.value())
-				ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-			else
-				lIt.next();
-
-		while(rIt.hasNext())
-			ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value())
-				ag.increment(
-					lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-			else if(lIt.value() < rIt.value())
-				lIt.next();
-			else
-				rIt.next();
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-		final int rhsNV = this.getNumValues();
-		final int nCol = lhs.getNumValues();
-		final int retSize = nCol * rhsNV;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-		final AIterator lIt = lhs._indexes.getIterator();
-		final AIterator rIt = _indexes.getIterator();
-
-		while(lIt.hasNext() && rIt.hasNext())
-			if(lIt.value() == rIt.value()) {
-				ag.increment(getIndex(rIt.getDataIndexAndIncrement()));
-				lIt.next();
-			}
-			else if(lIt.value() < rIt.value())
-				lIt.next();
-			else
-				rIt.next();
-
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-		final int NVR = this.getNumValues();
-		final int NVL = lhs.getNumValues();
-		final int retSize = NVR * NVL;
-		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-		final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-		for(int kl = 0; kl < NVL; kl++) {
-			final AIterator rIt = _indexes.getIterator();
-			final int bOffL = lhs._ptr[kl];
-			final int bLenL = lhs.len(kl);
-			for(int bixL = 0, offL = 0, sLenL = 0; rIt.hasNext() && bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-				sLenL = lhs._data[bOffL + bixL];
-				for(int i = 1; rIt.hasNext() && i <= sLenL; i++) {
-					final int col = offL + lhs._data[bOffL + bixL + i];
-					rIt.skipTo(col);
-					if(rIt.value() == col)
-						ag.increment(kl + getIndex(rIt.getDataIndexAndIncrement()) * NVL);
-
-				}
-			}
-		}
-		return ag;
-	}
-
-	@Override
-	public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// @Override
+	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
+	// final int rhsNV = this.getNumValues();
+	// final int nCol = lhs.getNumValues();
+	// final int retSize = nCol * rhsNV;
+	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// final AIterator it = _indexes.getIterator();
+
+	// while(it.hasNext()) {
+	// final int col = lhs._data.getIndex(it.value());
+	// final int row = getIndex(it.getDataIndexAndIncrement());
+	// ag.increment(col + row * nCol);
+	// }
+	// return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
+	// final int rhsNV = this.getNumValues();
+	// final int nCol = lhs.getNumValues();
+
+	// final int defL = nCol - 1;
+	// final int retSize = nCol * rhsNV;
+
+	// IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// AIterator lIt = lhs._indexes.getIterator();
+	// AIterator rIt = this._indexes.getIterator();
+
+	// while(lIt.hasNext() && rIt.hasNext())
+	// if(lIt.value() == rIt.value())
+	// ag.increment(
+	// lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
+	// else if(lIt.value() > rIt.value())
+	// ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
+	// else
+	// lIt.next();
+
+	// while(rIt.hasNext())
+	// ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
+
+	// return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
+	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// + this.getClass().getSimpleName());
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
+	// final int rhsNV = this.getNumValues();
+	// final int nCol = lhs.getNumValues();
+	// final int retSize = nCol * rhsNV;
+
+	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// final AIterator lIt = lhs._indexes.getIterator();
+	// final AIterator rIt = _indexes.getIterator();
+
+	// while(lIt.hasNext() && rIt.hasNext())
+	// if(lIt.value() == rIt.value())
+	// ag.increment(
+	// lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
+	// else if(lIt.value() < rIt.value())
+	// lIt.next();
+	// else
+	// rIt.next();
+
+	// return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
+	// final int rhsNV = this.getNumValues();
+	// final int nCol = lhs.getNumValues();
+	// final int retSize = nCol * rhsNV;
+	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+	// final AIterator lIt = lhs._indexes.getIterator();
+	// final AIterator rIt = _indexes.getIterator();
+
+	// while(lIt.hasNext() && rIt.hasNext())
+	// if(lIt.value() == rIt.value()) {
+	// ag.increment(getIndex(rIt.getDataIndexAndIncrement()));
+	// lIt.next();
+	// }
+	// else if(lIt.value() < rIt.value())
+	// lIt.next();
+	// else
+	// rIt.next();
+
+	// return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
+	// final int NVR = this.getNumValues();
+	// final int NVL = lhs.getNumValues();
+	// final int retSize = NVR * NVL;
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
+
+	// for(int kl = 0; kl < NVL; kl++) {
+	// final AIterator rIt = _indexes.getIterator();
+	// final int bOffL = lhs._ptr[kl];
+	// final int bLenL = lhs.len(kl);
+	// for(int bixL = 0, offL = 0, sLenL = 0; rIt.hasNext() && bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
+	// sLenL = lhs._data[bOffL + bixL];
+	// for(int i = 1; rIt.hasNext() && i <= sLenL; i++) {
+	// final int col = offL + lhs._data[bOffL + bixL + i];
+	// rIt.skipTo(col);
+	// if(rIt.value() == col)
+	// ag.increment(kl + getIndex(rIt.getDataIndexAndIncrement()) * NVL);
+
+	// }
+	// }
+	// }
+	// return ag;
+	// }
+
+	// @Override
+	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
+	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// + this.getClass().getSimpleName());
+	// }
 
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
@@ -549,8 +590,4 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		throw new NotImplementedException();
 	}
 
-	@Override
-	public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-		throw new NotImplementedException();
-	}
 }
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 faa3bb3..8f5d59d 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,7 +28,7 @@ import org.apache.sysds.runtime.compress.colgroup.offset.OffsetFactory;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.utils.MemoryEstimates;
 
-public class ColGroupSizes {
+public final class ColGroupSizes {
 	protected static final Log LOG = LogFactory.getLog(ColGroupSizes.class.getName());
 
 	public static long estimateInMemorySizeGroup(int nrColumns) {
@@ -44,7 +44,6 @@ public class ColGroupSizes {
 	public static long estimateInMemorySizeGroupValue(int nrColumns, int nrValues, double tupleSparsity,
 		boolean lossy) {
 		long size = estimateInMoemorySizeCompressedColumn(nrColumns);
-		// LOG.error("MemorySize Group Value: " + nrColumns + " " + nrValues + " " + lossy);
 		size += 8; // Dictionary Reference.
 		size += 8; // Counts reference
 		size += 1; // _zeros boolean reference
@@ -56,7 +55,6 @@ public class ColGroupSizes {
 
 	public static long estimateInMemorySizeDDC(int nrCols, int numTuples, int dataLength, double tupleSparsity,
 		boolean lossy) {
-		// LOG.error("Arguments for DDC memory Estimate " + nrCols + " " + numTuples + " " + dataLength + " " + lossy);
 		long size = estimateInMemorySizeGroupValue(nrCols, numTuples, tupleSparsity, lossy);
 		size += MapToFactory.estimateInMemorySize(dataLength, numTuples);
 		return size;
@@ -64,7 +62,6 @@ public class ColGroupSizes {
 
 	public static long estimateInMemorySizeOffset(int nrColumns, int nrValues, int pointers, int offsetLength,
 		double tupleSparsity, boolean lossy) {
-		// LOG.error("Offset Size: " + nrColumns + " " + nrValues + " " + pointers + " " + offsetLength);
 		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues, tupleSparsity, lossy);
 		size += MemoryEstimates.intArrayCost(pointers);
 		size += MemoryEstimates.charArrayCost(offsetLength);
@@ -73,17 +70,14 @@ public class ColGroupSizes {
 
 	public static long estimateInMemorySizeOLE(int nrColumns, int nrValues, int offsetLength, int nrRows,
 		double tupleSparsity, boolean lossy) {
-		// LOG.error(nrColumns + " " + nrValues + " " + offsetLength + " " + nrRows + " " + lossy);
 		nrColumns = nrColumns > 0 ? nrColumns : 1;
 		offsetLength += (nrRows / CompressionSettings.BITMAP_BLOCK_SZ) * 2;
-		long size = estimateInMemorySizeOffset(nrColumns, nrValues, nrValues  + 1, offsetLength,
-			tupleSparsity, lossy);
+		long size = estimateInMemorySizeOffset(nrColumns, nrValues, nrValues + 1, offsetLength, tupleSparsity, lossy);
 		return size;
 	}
 
 	public static long estimateInMemorySizeRLE(int nrColumns, int nrValues, int nrRuns, int nrRows,
 		double tupleSparsity, boolean lossy) {
-		// LOG.error("RLE Size: " + nrColumns + " " + nrValues + " " + nrRuns + " " + nrRows);
 		int offsetLength = (nrRuns) * 2;
 		long size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues) + 1, offsetLength, tupleSparsity, lossy);
 
@@ -94,10 +88,9 @@ public class ColGroupSizes {
 		boolean largestOffIsZero, boolean containNoZeroValues, double tupleSparsity, boolean lossy) {
 		long size = estimateInMemorySizeGroupValue(nrColumns,
 			nrValues + (largestOffIsZero || containNoZeroValues ? 0 : 1), tupleSparsity, lossy);
-		// LOG.error("SDC Estimation values: " + nrColumns + " " + nrValues + " " + nrRows + " " + largestOff);
-		size += OffsetFactory.estimateInMemorySize(nrRows - largestOff - 1, nrRows);
+		size += OffsetFactory.estimateInMemorySize(nrRows - largestOff, nrRows);
 		if(nrValues > 1)
-			size += MapToFactory.estimateInMemorySize(nrRows - largestOff, nrValues);
+			size += MapToFactory.estimateInMemorySize(nrRows - largestOff, nrValues - 1);
 		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 562e4d8..6e8524e 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,6 +25,7 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
@@ -77,7 +78,7 @@ public class ColGroupUncompressed extends AColGroup {
 			final int col = colIndicesList[0];
 			if(transposed) {
 				_data = rawBlock.slice(col, col, 0, rawBlock.getNumColumns() - 1);
-				_data = LibMatrixReorg.transposeInPlace(_data, 1);
+				_data = LibMatrixReorg.transposeInPlace(_data, InfrastructureAnalyzer.getLocalParallelism());
 			}
 			else
 				_data = rawBlock.slice(0, rawBlock.getNumRows() - 1, col, col);
@@ -158,42 +159,8 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
-		double[] c = target.getDenseBlockValues();
-		final int nCol = _colIndexes.length;
-		final int tCol = target.getNumColumns();
-		long nnz = 0;
-		if(_data.isEmpty())
-			return;
-		else if(_data.isInSparseFormat()) {
-			SparseBlock sb = _data.getSparseBlock();
-			for(int row = rl; row < ru; row++, offT += tCol) {
-				if(!sb.isEmpty(row)) {
-					int apos = sb.pos(row);
-					int alen = sb.size(row) + apos;
-					int[] aix = sb.indexes(row);
-					double[] avals = sb.values(row);
-					nnz += alen;
-					for(int col = apos; col < alen; col++) {
-						c[_colIndexes[aix[col]] + offT] += avals[col];
-					}
-				}
-			}
-		}
-		else {
-			double[] values = _data.getDenseBlockValues();
-			offT = offT * tCol;
-			int offS = rl * nCol;
-			for(int row = rl; row < ru; row++, offT += tCol, offS += nCol) {
-				for(int j = 0; j < nCol; j++) {
-					final double v = values[offS + j];
-					if(v != 0) {
-						c[offT + _colIndexes[j]] += v;
-						nnz++;
-					}
-				}
-			}
-		}
-		target.setNonZeros(nnz + target.getNonZeros());
+		decompressToBlockUnSafe(target, rl, ru, offT);
+		target.setNonZeros(_data.getNonZeros() + target.getNonZeros());
 	}
 
 	@Override
@@ -228,43 +195,43 @@ public class ColGroupUncompressed extends AColGroup {
 		}
 	}
 
-	@Override
-	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	// @Override
+	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-		double[] c = target.getDenseBlockValues();
-		int nnz = 0;
-		int off = colpos;
-		if(_data.isInSparseFormat()) {
-			for(int i = 0; i < _data.getNumRows(); i++) {
-				c[i] += _data.quickGetValue(i, colpos);
-				if(c[i] != 0)
-					nnz++;
-			}
-		}
-		else {
-			double[] denseValues = _data.getDenseBlockValues();
-			for(int i = 0; i < _data.getNumRows(); i++, off += _colIndexes.length) {
-				c[i] += denseValues[off];
-				if(c[i] != 0)
-					nnz++;
-			}
-		}
-		target.setNonZeros(nnz);
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
+	// 	double[] c = target.getDenseBlockValues();
+	// 	int nnz = 0;
+	// 	int off = colpos;
+	// 	if(_data.isInSparseFormat()) {
+	// 		for(int i = 0; i < _data.getNumRows(); i++) {
+	// 			c[i] += _data.quickGetValue(i, colpos);
+	// 			if(c[i] != 0)
+	// 				nnz++;
+	// 		}
+	// 	}
+	// 	else {
+	// 		double[] denseValues = _data.getDenseBlockValues();
+	// 		for(int i = 0; i < _data.getNumRows(); i++, off += _colIndexes.length) {
+	// 			c[i] += denseValues[off];
+	// 			if(c[i] != 0)
+	// 				nnz++;
+	// 		}
+	// 	}
+	// 	target.setNonZeros(nnz);
+	// }
 
-	@Override
-	public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	// @Override
+	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
 
-	@Override
-	public void decompressColumnToBlock(double[] target, int colpos, int rl, int ru) {
-		throw new NotImplementedException("Not Implemented");
-	}
+	// @Override
+	// public void decompressColumnToBlock(double[] target, int colpos, int rl, int ru) {
+	// 	throw new NotImplementedException("Not Implemented");
+	// }
 
 	@Override
 	public double get(int r, int c) {
@@ -275,46 +242,7 @@ public class ColGroupUncompressed extends AColGroup {
 			return _data.quickGetValue(r, ix);
 	}
 
-	// @Override
-	// public void rightMultByVector(double[] b, double[] c, int rl, int ru, double[] dictVals) {
-	// throw new NotImplementedException("Should not be called use other matrix function");
-	// }
-
-	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		// Pull out the relevant rows of the vector
-		int clen = _colIndexes.length;
-
-		MatrixBlock shortVector = new MatrixBlock(clen, 1, false);
-		shortVector.allocateDenseBlock();
-		double[] b = shortVector.getDenseBlockValues();
-		for(int colIx = 0; colIx < clen; colIx++)
-			b[colIx] = vector.quickGetValue(_colIndexes[colIx], 0);
-		shortVector.recomputeNonZeros();
-
-		// Multiply the selected columns by the appropriate parts of the vector
-		LibMatrixMult.matrixMult(_data, shortVector, result, rl, ru);
-	}
-
-	public void rightMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
-		// Pull out the relevant rows of the vector
-
-		int clen = _colIndexes.length;
-		MatrixBlock subMatrix = new MatrixBlock(clen, matrix.getNumColumns(), false);
-		subMatrix.allocateDenseBlock();
-		double[] b = subMatrix.getDenseBlockValues();
-
-		for(int colIx = 0; colIx < clen; colIx++) {
-			int row = _colIndexes[colIx];
-			for(int col = 0; col < matrix.getNumColumns(); col++)
-				b[colIx * matrix.getNumColumns() + col] = matrix.quickGetValue(row, col);
-		}
-
-		subMatrix.setNonZeros(clen * matrix.getNumColumns());
-
-		// // Multiply the selected columns by the appropriate parts of the vector
-		LibMatrixMult.matrixMult(_data, subMatrix, result);
-	}
-
+	@Override
 	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
 
 		final MatrixBlock tmpRet = new MatrixBlock(ru - rl, _data.getNumColumns(), false);
@@ -328,14 +256,16 @@ public class ColGroupUncompressed extends AColGroup {
 		else if(tmpRet.isInSparseFormat()) {
 			final SparseBlock sb = tmpRet.getSparseBlock();
 			for(int rowIdx = 0; rowIdx < ru - rl; rowIdx++, offT += result.getNumColumns()) {
-				if(!sb.isEmpty(rowIdx)) {
-					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];
-				}
+				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 {
@@ -372,7 +302,7 @@ public class ColGroupUncompressed extends AColGroup {
 		else
 			_data.binaryOperations(op, that, resultBlock);
 
-		return new ColGroupUncompressed(_colIndexes, resultBlock, false);
+		return new ColGroupUncompressed(_colIndexes, resultBlock);
 	}
 
 	public void unaryAggregateOperations(AggregateUnaryOperator op, double[] ret) {
@@ -531,11 +461,17 @@ public class ColGroupUncompressed extends AColGroup {
 		}
 		else {
 			double[] tmpV = tmp.getDenseBlockValues();
-			for(int row = 0, offRet = 0, offTmp = 0; row < tCol; row++, offRet += numColumns, offTmp += tCol)
+			for(int row = 0, offTmp = 0; row < tCol; row++, offTmp += tCol) {
+				final int offRet = _colIndexes[row] * numColumns;
 				for(int col = row; col < tCol; col++)
 					result[offRet + _colIndexes[col]] += tmpV[offTmp + col];
+			}
 		}
+	}
 
+	@Override
+	public void tsmm(double[] result, int numColumns, int idxStart, int idxEnd) {
+		throw new NotImplementedException();
 	}
 
 	@Override
@@ -543,7 +479,7 @@ public class ColGroupUncompressed extends AColGroup {
 		MatrixBlock newData = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), _data.isInSparseFormat());
 		// _data.copy(newData);
 		newData.copy(_data);
-		return new ColGroupUncompressed(_colIndexes, newData, false);
+		return new ColGroupUncompressed(_colIndexes, newData);
 	}
 
 	@Override
@@ -572,7 +508,7 @@ public class ColGroupUncompressed extends AColGroup {
 			return;
 		if(lhs instanceof ColGroupUncompressed) {
 			ColGroupUncompressed lhsUC = (ColGroupUncompressed) lhs;
-			MatrixBlock tmpRet = new MatrixBlock(_colIndexes.length, _colIndexes.length, 0);
+			MatrixBlock tmpRet = new MatrixBlock(lhs.getNumCols(), _colIndexes.length, 0);
 
 			if(lhsUC._data == this._data) {
 
@@ -583,7 +519,7 @@ public class ColGroupUncompressed extends AColGroup {
 				LOG.warn("Inefficient Left Matrix Multiplication with transpose of left hand side : t(l) %*% r");
 				MatrixBlock lhData = lhsUC._data;
 				MatrixBlock transposed = new MatrixBlock(lhData.getNumColumns(), lhData.getNumRows(), false);
-				LibMatrixReorg.transpose(lhData, transposed);
+				LibMatrixReorg.transpose(lhData, transposed, InfrastructureAnalyzer.getLocalParallelism());
 				transposed.setNonZeros(lhData.getNonZeros());
 				// do transposed left hand side, matrix multiplication.
 				LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
@@ -594,46 +530,64 @@ public class ColGroupUncompressed extends AColGroup {
 				return;
 			else if(tmpRet.isInSparseFormat()) {
 				SparseBlock sb = tmpRet.getSparseBlock();
-				for(int rowIdx = 0, offT = 0; rowIdx < tmpRet.getNumRows(); rowIdx++, offT += result.getNumColumns()) {
-					if(!sb.isEmpty(rowIdx)) {
-						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];
-					}
+				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);
+					final int offRes = lhs._colIndexes[row] * result.getNumColumns();
+					for(int col = apos; col < alen; col++)
+						resV[offRes + _colIndexes[aix[col]]] += avals[col];
 				}
 			}
 			else {
 				double[] tmpRetV = tmpRet.getDenseBlockValues();
-				for(int j = 0, offTemp = 0, offT = 0;
-					j < tmpRet.getNumRows();
-					j++, offTemp += _colIndexes.length, offT += result.getNumColumns()) {
-					for(int i = 0; i < _colIndexes.length; i++)
-						resV[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
+				for(int row = 0; row < lhs._colIndexes.length; row++) {
+					final int offRes = lhs._colIndexes[row] * result.getNumColumns();
+					final int offTmp = lhs._colIndexes.length * row;
+					for(int col = 0; col < _colIndexes.length; col++) {
+						resV[offRes + _colIndexes[col]] += tmpRetV[offTmp + col];
+					}
 				}
 			}
-
 		}
 		else {
-
-			LOG.warn("Inefficient transpose of uncompressed to fit to"
+			LOG.warn("\nInefficient transpose of uncompressed to fit to"
 				+ " t(AColGroup) %*% UncompressedColGroup mult by colGroup uncompressed column"
-				+ " Currently solved by t(t(Uncompressed) %*% AColGroup");
-			MatrixBlock tmpTransposedResult = new MatrixBlock(result.getNumColumns(), result.getNumRows(), false);
+				+ "\nCurrently solved by t(t(Uncompressed) %*% AColGroup)");
+			MatrixBlock ucCG = getData();
+			// make a function that allows the result of the mult to be directly output to a temporary matrix.
+			MatrixBlock tmpTransposedResult = new MatrixBlock(ucCG.getNumColumns(), result.getNumColumns(), false);
 			tmpTransposedResult.allocateDenseBlock();
 
-			MatrixBlock ucCG = getData();
 			MatrixBlock tmp = new MatrixBlock(ucCG.getNumColumns(), ucCG.getNumRows(), ucCG.isInSparseFormat());
 			LibMatrixReorg.transpose(ucCG, tmp, InfrastructureAnalyzer.getLocalParallelism());
 			lhs.leftMultByMatrix(tmp, tmpTransposedResult);
+			tmpTransposedResult.setNonZeros(ucCG.getNumColumns() * result.getNumColumns());
 
 			final double[] resV = result.getDenseBlockValues();
-			final double[] tmpV = tmpTransposedResult.getDenseBlockValues();
-			for(int row = 0; row < result.getNumRows(); row++) {
-				for(int col = 0; col < result.getNumColumns(); col++) {
-					resV[row * result.getNumColumns() + col] += tmpV[col * result.getNumRows() + row];
+			final int[] lhsC = lhs._colIndexes;
+			final int[] rhsC = _colIndexes;
+
+			// allocate the resulting matrix into the correct result indexes.
+			// Note that the intermediate matrix is transposed, therefore the indexes are different than a normal
+			// allocation.
+
+			if(tmpTransposedResult.isEmpty())
+				return;
+			else if(tmpTransposedResult.isInSparseFormat())
+				throw new NotImplementedException();
+			else {
+				final double[] tmpV = tmpTransposedResult.getDenseBlockValues();
+				final int nCol = result.getNumColumns();
+
+				for(int row = 0; row < rhsC.length; row++) {
+					final int offR = rhsC[row];
+					final int offT = row * nCol;
+					for(int col = 0; col < lhsC.length; col++)
+						resV[offR + lhsC[col] * nCol] += tmpV[offT + lhsC[col]];
 				}
 			}
 		}
@@ -646,20 +600,52 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
-		MatrixBlock newData = _data.slice(0, _data.getNumRows() - 1, idStart, idEnd - 1, true);
-		if(newData.isEmpty())
-			return new ColGroupEmpty(outputCols, newData.getNumRows());
-		return new ColGroupUncompressed(outputCols, newData, false);
+		try {
+			MatrixBlock newData = _data.slice(0, _data.getNumRows() - 1, idStart, idEnd - 1, true);
+			if(newData.isEmpty())
+				return new ColGroupEmpty(outputCols, newData.getNumRows());
+			return new ColGroupUncompressed(outputCols, newData);
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException("Error in slicing of uncompressed column group", e);
+		}
 	}
 
 	@Override
 	public AColGroup rightMultByMatrix(MatrixBlock right) {
-		int[] outputCols = new int[right.getNumColumns()];
+		final int nColR = right.getNumColumns();
+		int[] outputCols = new int[nColR];
 		for(int i = 0; i < outputCols.length; i++)
 			outputCols[i] = i;
-		MatrixBlock out = new MatrixBlock(_data.getNumRows(), right.getNumColumns(), true);
-		LibMatrixMult.matrixMult(_data, right, out, InfrastructureAnalyzer.getLocalParallelism());
-		return new ColGroupUncompressed(outputCols, out, false);
+		if(_data.isEmpty() || right.isEmpty())
+			return new ColGroupEmpty(outputCols, _data.getNumRows());
+		MatrixBlock subBlockRight;
+
+		if(right.isInSparseFormat()) {
+			subBlockRight = new MatrixBlock(_data.getNumColumns(), nColR, true);
+			subBlockRight.allocateSparseRowsBlock();
+			final SparseBlock sbR = right.getSparseBlock();
+			final SparseBlock subR = subBlockRight.getSparseBlock();
+			for(int i = 0; i < _colIndexes.length; i++)
+				subR.set(i, sbR.get(_colIndexes[i]), false);
+		}
+		else {
+			subBlockRight = new MatrixBlock(_data.getNumColumns(), nColR, false);
+			subBlockRight.allocateDenseBlock();
+			final double[] sbr = subBlockRight.getDenseBlockValues();
+			final double[] rightV = right.getDenseBlockValues();
+			for(int i = 0; i < _colIndexes.length; i++) {
+				final int offSubBlock = i * nColR;
+				final int offRight = _colIndexes[i] * nColR;
+				System.arraycopy(rightV, offRight, sbr, offSubBlock, nColR);
+			}
+		}
+		// Hack to force computation without having to count all non zeros.
+		subBlockRight.setNonZeros(_data.getNumColumns() * nColR);
+		MatrixBlock out = new MatrixBlock(_data.getNumRows(), nColR, false);
+		LibMatrixMult.matrixMult(_data, subBlockRight, out, InfrastructureAnalyzer.getLocalParallelism());
+		return new ColGroupUncompressed(outputCols, out);
+
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index 42bf407..af9ab37 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -34,16 +34,16 @@ 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.pre.ArrPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
+import org.apache.sysds.runtime.instructions.InstructionUtils;
 import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 
@@ -54,7 +54,7 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 public abstract class ColGroupValue extends ColGroupCompressed implements Cloneable {
 	private static final long serialVersionUID = 3786247536054353658L;
 
-	/** thread-local pairs of reusable temporary vectors for positions and values */
+	/** Thread-local pairs of reusable temporary vectors for positions and values */
 	private static ThreadLocal<Pair<int[], double[]>> memPool = new ThreadLocal<Pair<int[], double[]>>() {
 		@Override
 		protected Pair<int[], double[]> initialValue() {
@@ -85,11 +85,34 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public void decompressToBlock(MatrixBlock target, int rl, int ru) {
-		decompressToBlock(target, rl, ru, rl);
+	public final void decompressToBlockSafe(MatrixBlock target, int rl, int ru, int offT) {
+		decompressToBlockUnSafe(target, rl, ru, offT);
+		target.setNonZeros(getNumberNonZeros() + target.getNonZeros());
 	}
 
 	@Override
+	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
+		if(_dict instanceof MatrixBlockDictionary) {
+			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
+			final MatrixBlock mb = md.getMatrixBlock();
+			if(mb.isEmpty())
+				return;
+			else if(mb.isInSparseFormat())
+				decompressToBlockUnSafeSparseDictionary(target, rl, ru, offT, mb.getSparseBlock());
+			else
+				decompressToBlockUnSafeDenseDictionary(target, rl, ru, offT, mb.getDenseBlockValues());
+		}
+		else
+			decompressToBlockUnSafeDenseDictionary(target, rl, ru, offT, _dict.getValues());
+	}
+
+	protected abstract void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		SparseBlock sb);
+
+	protected abstract void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
+		double[] values);
+
+	@Override
 	public final int getNumValues() {
 		return _dict.getNumberOfValues(_colIndexes.length);
 	}
@@ -108,12 +131,12 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		_dict.addMaxAndMin(ret, _colIndexes);
 	}
 
-	protected void setDictionary(ADictionary dict) {
+	protected final void setDictionary(ADictionary dict) {
 		_dict = dict;
 	}
 
 	@Override
-	public MatrixBlock getValuesAsBlock() {
+	public final MatrixBlock getValuesAsBlock() {
 		_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
 		MatrixBlock ret = ((MatrixBlockDictionary) _dict).getMatrixBlock();
 		if(_zeros) {
@@ -175,11 +198,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return true;
 	}
 
-	@Override
-	protected int containsAllZeroTuple() {
-		return _dict.hasZeroTuple(_colIndexes.length);
-	}
-
 	protected final double sumValues(int valIx, double[] b, double[] dictVals) {
 		final int numCols = getNumCols();
 		final int valOff = valIx * numCols;
@@ -198,31 +216,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return val;
 	}
 
-	protected double[] preaggValues(int numVals, double[] b, double[] dictVals) {
-		return preaggValues(numVals, b, false, dictVals, 0);
-	}
-
-	protected double[] preaggValues(int numVals, double[] b, double[] dictVals, int off) {
-		return preaggValues(numVals, b, false, dictVals, off);
-	}
-
-	protected double[] preaggValues(int numVals, double[] b, boolean allocNew, double[] dictVals, int off) {
-		// + 1 to enable containing a zero value. which we have added at the length of
-		// the arrays index.
-		double[] ret = allocNew ? new double[numVals + 1] : allocDVector(numVals + 1, true);
-
-		if(_colIndexes.length == 1) {
-			for(int k = 0; k < numVals; k++)
-				ret[k] = dictVals[k] * b[_colIndexes[0] + off];
-		}
-		else {
-			for(int k = 0; k < numVals; k++)
-				ret[k] = sumValues(k, b, dictVals, off);
-		}
-
-		return ret;
-	}
-
 	private int[] getAggregateColumnsSetDense(double[] b, int cl, int cu, int cut) {
 		Set<Integer> aggregateColumnsSet = new HashSet<>();
 		final int retCols = (cu - cl);
@@ -243,28 +236,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return aggregateColumns;
 	}
 
-	private Pair<int[], double[]> preaggValuesFromDense(final int numVals, final double[] b, final int cl, final int cu,
-		final int cut) {
-
-		final int[] aggregateColumns = getAggregateColumnsSetDense(b, cl, cu, cut);
-		final double[] ret = new double[numVals * aggregateColumns.length];
-
-		for(int k = 0, off = 0;
-			k < numVals * _colIndexes.length;
-			k += _colIndexes.length, off += aggregateColumns.length) {
-			for(int h = 0; h < _colIndexes.length; h++) {
-				int idb = _colIndexes[h] * cut;
-				double v = _dict.getValue(k + h);
-				if(v != 0)
-					for(int i = 0; i < aggregateColumns.length; i++)
-						ret[off + i] += v * b[idb + aggregateColumns[i]];
-			}
-		}
-
-		return new ImmutablePair<>(aggregateColumns, ret);
-	}
-
-	private int[] getAggregateColumnsSetSparse(SparseBlock b) {
+	private int[] getAggregateColumnsSetSparse(SparseBlock b, int retCols) {
 		Set<Integer> aggregateColumnsSet = new HashSet<>();
 
 		for(int h = 0; h < _colIndexes.length; h++) {
@@ -275,6 +247,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 					aggregateColumnsSet.add(sIndexes[i]);
 				}
 			}
+			if(aggregateColumnsSet.size() == retCols)
+				break;
 		}
 
 		int[] aggregateColumns = aggregateColumnsSet.stream().mapToInt(x -> x).toArray();
@@ -282,12 +256,9 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return aggregateColumns;
 	}
 
-	private Pair<int[], double[]> preaggValuesFromSparse(int numVals, SparseBlock b, int cl, int cu, int cut) {
-
-		int[] aggregateColumns = getAggregateColumnsSetSparse(b);
-
-		double[] ret = new double[numVals * aggregateColumns.length];
-
+	private double[] preaggValuesFromSparse(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)) {
@@ -306,23 +277,10 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				}
 			}
 		}
-		return new ImmutablePair<>(aggregateColumns, ret);
-	}
-
-	public Pair<int[], double[]> preaggForRightMultiplyValues(int numVals, MatrixBlock b, int cl, int cu, int cut) {
-		return b.isInSparseFormat() ? preaggValuesFromSparse(numVals, b.getSparseBlock(), cl, cu,
-			cut) : preaggValuesFromDense(numVals, b.getDenseBlockValues(), cl, cu, cut);
+		return ret;
 	}
 
-	// protected static double[] sparsePreaggValues(int numVals, double v, boolean allocNew, ADictionary dict) {
-	// double[] ret = allocNew ? new double[numVals + 1] : allocDVector(numVals + 1, true);
-
-	// for(int k = 0; k < numVals; k++)
-	// ret[k] = dictVals[k] * v;
-	// return ret;
-	// }
-
-	protected double computeMxx(double c, Builtin builtin) {
+	protected final double computeMxx(double c, Builtin builtin) {
 		if(_zeros)
 			c = builtin.execute(c, 0);
 		if(_dict != null)
@@ -331,7 +289,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			return c;
 	}
 
-	protected void computeColMxx(double[] c, Builtin builtin) {
+	protected final 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);
@@ -346,7 +304,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param op scalar operation to perform
 	 * @return transformed copy of value metadata for this column group
 	 */
-	protected ADictionary applyScalarOp(ScalarOperator op) {
+	protected final ADictionary applyScalarOp(ScalarOperator op) {
 		return _dict.clone().apply(op);
 	}
 
@@ -362,7 +320,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 *                appended.
 	 * @return The new Dictionary containing the values.
 	 */
-	protected ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
+	protected final ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
 		return _dict.applyScalarOp(op, newVal, numCols);
 	}
 
@@ -375,18 +333,11 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param left       Specify which side the operation is executed on.
 	 * @return The new Dictionary with values.
 	 */
-	public ADictionary applyBinaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
+	protected final ADictionary applyBinaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
 		return sparseSafe ? _dict.clone().applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left) : _dict
 			.applyBinaryRowOp(op, v, sparseSafe, _colIndexes, left);
 	}
 
-	protected void setandExecute(double[] c, boolean square, double val, int rix) {
-		if(square)
-			c[rix] += val * val;
-		else
-			c[rix] += val;
-	}
-
 	public static void setupThreadLocalMemory(int len) {
 		if(memPool.get() == null || memPool.get().getLeft().length < len) {
 			Pair<int[], double[]> p = new ImmutablePair<>(new int[len], new double[len]);
@@ -400,7 +351,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	protected static double[] allocDVector(int len, boolean reset) {
 		Pair<int[], double[]> p = memPool.get();
-
 		// sanity check for missing setup
 		if(p == null) {
 			return new double[len];
@@ -449,14 +399,13 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public boolean isLossy() {
+	public final boolean isLossy() {
 		return _dict.isLossy();
 	}
 
 	@Override
 	public void readFields(DataInput in) throws IOException {
 		super.readFields(in);
-
 		_zeros = in.readBoolean();
 		_dict = DictionaryFactory.read(in);
 	}
@@ -465,9 +414,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
 		out.writeBoolean(_zeros);
-
 		_dict.write(out);
-
 	}
 
 	@Override
@@ -487,7 +434,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	public abstract int[] getCounts(int rl, int ru, int[] out);
 
-	protected void computeSum(double[] c, boolean square) {
+	protected final void computeSum(double[] c, boolean square) {
 		if(_dict != null)
 			if(square)
 				c[0] += _dict.sumsq(getCounts(), _colIndexes.length);
@@ -495,82 +442,47 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				c[0] += _dict.sum(getCounts(), _colIndexes.length);
 	}
 
-	protected abstract void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean);
-
-	protected void computeColSums(double[] c, boolean square) {
+	protected final void computeColSums(double[] c, boolean square) {
 		_dict.colSum(c, getCounts(), _colIndexes, square);
 	}
 
-	protected abstract void computeRowMxx(double[] c, Builtin builtin, int rl, int ru);
-
-	protected Object clone() throws CloneNotSupportedException {
-		return super.clone();
-	}
-
-	public AColGroup copyAndSet(double[] newDictionary) {
+	protected Object clone() {
 		try {
-			ColGroupValue clone = (ColGroupValue) this.clone();
-			clone.setDictionary(new Dictionary(newDictionary));
-			return clone;
+			return super.clone();
 		}
 		catch(CloneNotSupportedException e) {
-			e.printStackTrace();
+			throw new DMLCompressionException("Error while cloning: " + getClass().getSimpleName(), e);
 		}
-		return null;
+	}
+
+	public AColGroup copyAndSet(double[] newDictionary) {
+		return copyAndSet(new Dictionary(newDictionary));
 	}
 
 	public AColGroup copyAndSet(ADictionary newDictionary) {
-		try {
-			ColGroupValue clone = (ColGroupValue) this.clone();
-			clone.setDictionary(newDictionary);
-			return clone;
-		}
-		catch(CloneNotSupportedException e) {
-			e.printStackTrace();
-		}
-		return null;
+		ColGroupValue clone = (ColGroupValue) this.clone();
+		clone.setDictionary(newDictionary);
+		return clone;
 	}
 
 	public AColGroup copyAndSet(int[] colIndexes, double[] newDictionary) {
-		try {
-			ColGroupValue clone = (ColGroupValue) this.clone();
-			clone.setDictionary(new Dictionary(newDictionary));
-			clone.setColIndices(colIndexes);
-			return clone;
-		}
-		catch(CloneNotSupportedException e) {
-			e.printStackTrace();
-		}
-		return null;
+		return copyAndSet(colIndexes, new Dictionary(newDictionary));
 	}
 
 	public AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary) {
-		try {
-			ColGroupValue clone = (ColGroupValue) this.clone();
-			clone.setDictionary(newDictionary);
-			clone.setColIndices(colIndexes);
-			return clone;
-		}
-		catch(CloneNotSupportedException e) {
-			e.printStackTrace();
-		}
-		return null;
+		ColGroupValue clone = (ColGroupValue) this.clone();
+		clone.setDictionary(newDictionary);
+		clone.setColIndices(colIndexes);
+		return clone;
 	}
 
 	@Override
 	public ColGroupValue copy() {
-		try {
-			ColGroupValue clone = (ColGroupValue) this.clone();
-			return clone;
-		}
-		catch(CloneNotSupportedException e) {
-			e.printStackTrace();
-		}
-		return null;
+		return (ColGroupValue) this.clone();
 	}
 
 	@Override
-	protected AColGroup sliceSingleColumn(int idx) {
+	protected final AColGroup sliceSingleColumn(int idx) {
 		ColGroupValue ret = (ColGroupValue) copy();
 		ret._colIndexes = new int[] {0};
 		if(ret._dict != null)
@@ -583,7 +495,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	protected AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
+	protected final AColGroup sliceMultiColumns(int idStart, int idEnd, int[] outputCols) {
 
 		ColGroupValue ret = (ColGroupValue) copy();
 		ret._dict = ret._dict != null ? ret._dict.sliceOutColumnRange(idStart, idEnd, _colIndexes.length) : null;
@@ -593,139 +505,83 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	/**
-	 * Post scale for left Multiplication
-	 * 
-	 * @param dictValues The dictionary values materialized as double array.
-	 * @param vals       The values aggregated from the left side row vector.
-	 * @param c          The output matrix
-	 * @param numVals    The number of values contained in the dictionary.
-	 */
-	protected void postScaling(double[] dictValues, double[] vals, MatrixBlock c, int numVals) {
-		postScaling(dictValues, vals, c, numVals, 0);
-	}
-
-	/**
-	 * Post scale for left Multiplication
-	 * 
-	 * @param dictValues The dictionary values materialized as double array.
-	 * @param vals       The values aggregated from the left side row vector.
-	 * @param c          The output matrix
-	 * @param numVals    The number of values contained in the dictionary.
-	 * @param row        The row index in the output c to assign the result to.
-	 * @param totalCols  The total number of columns in c.
-	 */
-	protected void postScaling(double[] dictValues, double[] vals, MatrixBlock c, int numVals, int row) {
-		final int ncol = getNumCols();
-		int valOff = 0;
-		final double[] cv = c.getDenseBlockValues();
-		final int totalCols = c.getNumColumns();
-
-		for(int k = 0; k < numVals; k++) {
-			double aval = vals[k];
-			for(int j = 0; j < ncol; j++) {
-				int colIx = _colIndexes[j] + row * totalCols;
-				cv[colIx] += aval * dictValues[valOff++];
-			}
-		}
-	}
-
-	/**
-	 * Pre aggregate a vector
+	 * Pre aggregate for left Multiplication
 	 * 
-	 * @param a The vector to aggregate
-	 * @return The pre-aggregated values.
+	 * @param m  The matrixBlock to pre aggregate
+	 * @param rl Start row
+	 * @param ru End row
+	 * @return The Pre aggregated values contained in a MatrixBlock
 	 */
-	public double[] preAggregate(double[] a) {
-		return preAggregate(a, 0);
-	}
-
-	public abstract MatrixBlock preAggregate(MatrixBlock m, int rl, int ru);
-
-	/**
-	 * Pre aggregates for left multiplication
-	 * 
-	 * @param a   The input dense vector or matrix to aggregate
-	 * @param row The row index to aggregate
-	 * @return The pre-aggregated values.
-	 */
-	public abstract double[] preAggregate(double[] a, int row);
-
-	/**
-	 * Pre aggregate for left multiplication
-	 * 
-	 * @param sb The vector to aggregate
-	 * @return The pre-aggregated values.
-	 */
-	public double[] preAggregate(SparseBlock sb) {
-		return preAggregateSparseWithCheck(sb, 0);
-	}
-
-	private double[] preAggregateSparseWithCheck(SparseBlock sb, int row) {
-		if(sb != null && !sb.isEmpty(row))
-			return preAggregateSparse(sb, row);
-		else
-			return null;
+	protected final MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
+		final int numVals = getNumValues();
+		final int lhsRows = ru - rl;
+		final double[] vals = allocDVector(lhsRows * numVals, true);
+		final DenseBlock retB = new DenseBlockFP64(new int[] {lhsRows, numVals}, vals);
+		MatrixBlock preAgg = new MatrixBlock(lhsRows, numVals, retB);
+		preAggregate(m, preAgg, rl, ru);
+		preAgg.recomputeNonZeros();
+		return preAgg;
 	}
 
 	/**
-	 * Pre aggregate for left multiplication of sparse vector or matrix.
+	 * Pre aggregate for left Multiplication.
 	 * 
-	 * @param sb  The input sparse vector or matrix to aggregate
-	 * @param row The row index to aggregate
-	 * @return The pre-aggregated values.
+	 * @param m      Matrix to preAggregate
+	 * @param preAgg Matrix to preAggregate into
+	 * @param rl     Start row
+	 * @param ru     End row
 	 */
-	public abstract double[] preAggregateSparse(SparseBlock sb, int row);
+	protected abstract void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru);
 
 	public abstract int getIndexStructureHash();
 
-	public IPreAggregate preAggregate(ColGroupValue lhs) {
-		IPreAggregate r = preCallAggregate(lhs);
-		return r;
-	}
-
-	public IPreAggregate preCallAggregate(ColGroupValue lhs) {
-		// LOG.error(lhs.getClass().getSimpleName() + " in " + this.getClass().getSimpleName() + " "
-		// + Arrays.toString(lhs.getColIndices()) + " " + Arrays.toString(this.getColIndices()));
-
-		if(lhs instanceof ColGroupDDC)
-			return preAggregateDDC((ColGroupDDC) lhs);
-		else if(lhs instanceof ColGroupSDC)
-			return preAggregateSDC((ColGroupSDC) lhs);
-		else if(lhs instanceof ColGroupSDCSingle)
-			return preAggregateSDCSingle((ColGroupSDCSingle) lhs);
-		else if(lhs instanceof ColGroupSDCZeros)
-			return preAggregateSDCZeros((ColGroupSDCZeros) lhs);
-		else if(lhs instanceof ColGroupSDCSingleZeros)
-			return preAggregateSDCSingleZeros((ColGroupSDCSingleZeros) lhs);
-		else if(lhs instanceof ColGroupOLE)
-			return preAggregateOLE((ColGroupOLE) lhs);
-		else if(lhs instanceof ColGroupRLE)
-			return preAggregateRLE((ColGroupRLE) lhs);
-		else if(lhs instanceof ColGroupConst)
-			return preAggregateCONST((ColGroupConst) lhs);
+	// private IPreAggregate preAggregate(ColGroupValue lhs) {
+	// IPreAggregate r = preCallAggregate(lhs);
+	// return r;
+	// }
 
-		throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-			+ this.getClass().getSimpleName());
-	}
+	// private IPreAggregate preCallAggregate(ColGroupValue lhs) {
+	// // (lhs.getClass().getSimpleName() + " in " + this.getClass().getSimpleName() + " "
+	// // + Arrays.toString(lhs.getColIndices()) + " " + Arrays.toString(this.getColIndices()));
+
+	// if(lhs instanceof ColGroupDDC)
+	// return preAggregateDDC((ColGroupDDC) lhs);
+	// else if(lhs instanceof ColGroupSDC)
+	// return preAggregateSDC((ColGroupSDC) lhs);
+	// else if(lhs instanceof ColGroupSDCSingle)
+	// return preAggregateSDCSingle((ColGroupSDCSingle) lhs);
+	// else if(lhs instanceof ColGroupSDCZeros)
+	// return preAggregateSDCZeros((ColGroupSDCZeros) lhs);
+	// else if(lhs instanceof ColGroupSDCSingleZeros)
+	// return preAggregateSDCSingleZeros((ColGroupSDCSingleZeros) lhs);
+	// else if(lhs instanceof ColGroupOLE)
+	// return preAggregateOLE((ColGroupOLE) lhs);
+	// else if(lhs instanceof ColGroupRLE)
+	// return preAggregateRLE((ColGroupRLE) lhs);
+	// else if(lhs instanceof ColGroupConst)
+	// return preAggregateCONST((ColGroupConst) lhs);
+
+	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
+	// + this.getClass().getSimpleName());
+	// }
 
-	public IPreAggregate preAggregateCONST(ColGroupConst lhs) {
-		// LOG.error(Arrays.toString(getCounts()));
-		return new ArrPreAggregate(getCounts());
-	}
+	// public IPreAggregate preAggregateCONST(ColGroupConst lhs) {
+	// 	return new ArrPreAggregate(getCounts());
+	// }
 
-	public abstract IPreAggregate preAggregateDDC(ColGroupDDC lhs);
+	// public abstract IPreAggregate preAggregateDDC(ColGroupDDC lhs);
 
-	public abstract IPreAggregate preAggregateSDC(ColGroupSDC lhs);
+	// public abstract IPreAggregate preAggregateSDC(ColGroupSDC lhs);
 
-	public abstract IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs);
+	// public abstract IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs);
 
-	public abstract IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs);
+	// public abstract IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs);
 
-	public abstract IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs);
+	// public abstract IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs);
 
-	public abstract IPreAggregate preAggregateOLE(ColGroupOLE lhs);
+	// public abstract IPreAggregate preAggregateOLE(ColGroupOLE lhs);
 
-	public abstract IPreAggregate preAggregateRLE(ColGroupRLE lhs);
+	// public abstract IPreAggregate preAggregateRLE(ColGroupRLE lhs);
 
 	/**
 	 * Pre aggregate into a dictionary. It is assumed that "that" have more distinct values than, "this".
@@ -734,13 +590,10 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param preModify specifies if the matrix in this
 	 * @return A aggregate dictionary
 	 */
-	public Dictionary preAggregateThatIndexStructure(ColGroupValue that, boolean preModify) {
+	public final Dictionary preAggregateThatIndexStructure(ColGroupValue that, boolean preModify) {
 		int outputLength = that._colIndexes.length * this.getNumValues();
 		Dictionary ret = new Dictionary(new double[outputLength]);
 
-		// if(preModify)
-		// LOG.error(preModify + " " + that.getClass().getSimpleName() + " in " + this.getClass().getSimpleName());
-
 		if(that instanceof ColGroupDDC)
 			return preAggregateThatDDCStructure((ColGroupDDC) that, ret);
 		else if(that instanceof ColGroupSDC)
@@ -775,20 +628,13 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
+	public final void leftMultByAColGroup(AColGroup lhs, MatrixBlock result) {
 		if(lhs instanceof ColGroupEmpty)
 			return;
 		else if(lhs instanceof ColGroupValue)
 			leftMultByColGroupValue((ColGroupValue) lhs, result);
-		else if(lhs instanceof ColGroupUncompressed) {
-			LOG.warn("Inefficient transpose of uncompressed to fit to "
-				+ "template need t(UnCompressedColGroup) %*% AColGroup support");
-			MatrixBlock ucCG = ((ColGroupUncompressed) lhs).getData();
-			MatrixBlock tmp = new MatrixBlock(ucCG.getNumColumns(), ucCG.getNumRows(), ucCG.isInSparseFormat());
-			LibMatrixReorg.transpose(ucCG, tmp, InfrastructureAnalyzer.getLocalParallelism());
-			leftMultByMatrix(tmp, result);
-
-		}
+		else if(lhs instanceof ColGroupUncompressed)
+			leftMultByUncompressedColGroup((ColGroupUncompressed) lhs, result);
 		else
 			throw new DMLCompressionException(
 				"Not supported left multiplication with A ColGroup of type: " + lhs.getClass().getSimpleName());
@@ -812,15 +658,14 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			}
 			else
 				matrixMultDictionariesAndOutputToColIndexesWithScaling(lhs._dict, this._dict, lhs._colIndexes,
-					this._colIndexes, resV, numCols, getCounts());
-
+					this._colIndexes, result, getCounts());
 		}
 		else if(lhs instanceof ColGroupConst || this instanceof ColGroupConst) {
 			ADictionary r = this instanceof ColGroupConst ? this._dict : new Dictionary(
 				this._dict.colSum(getCounts(), rCol));
 			ADictionary l = lhs instanceof ColGroupConst ? lhs._dict : new Dictionary(
 				lhs._dict.colSum(lhs.getCounts(), lCol));
-			matrixMultDictionariesAndOutputToColIndexes(l, r, lhs._colIndexes, this._colIndexes, resV, numCols);
+			matrixMultDictionariesAndOutputToColIndexes(l, r, lhs._colIndexes, this._colIndexes, result);
 		}
 		else {
 			int[] countsRight = getCounts();
@@ -841,8 +686,8 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				ColGroupValue thisM = (mct != null) ? (ColGroupValue) this
 					.copyAndSet(this._dict.subtractTuple(mct)) : this;
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(thisM, true);
-				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes, resV,
-					numCols);
+				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes,
+					result);
 			}
 			else if(skipLeft > threshold && !(lhs instanceof ColGroupDDC)) {
 				double[] mct = lhs._dict.getMostCommonTuple(lhs.getCounts(), lCol);
@@ -852,43 +697,67 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 				ColGroupValue lhsM = (mct != null) ? (ColGroupValue) lhs.copyAndSet(lhs._dict.subtractTuple(mct)) : lhs;
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhsM, true);
-				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes, resV,
-					numCols);
+				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes,
+					result);
 			}
 			else if(nvR * rCol < nvL * lCol) {
 				Dictionary preAgg = lhs.preAggregateThatIndexStructure(this, false);
-				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes, resV,
-					numCols);
+				matrixMultDictionariesAndOutputToColIndexes(lhs._dict, preAgg, lhs._colIndexes, this._colIndexes,
+					result);
 			}
 			else {
 				Dictionary preAgg = this.preAggregateThatIndexStructure(lhs, false);
-				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes, resV,
-					numCols);
+				matrixMultDictionariesAndOutputToColIndexes(preAgg, this._dict, lhs._colIndexes, this._colIndexes,
+					result);
 			}
 		}
 	}
 
+	private void leftMultByUncompressedColGroup(ColGroupUncompressed lhs, MatrixBlock result) {
+		MatrixBlock ucCG = lhs.getData();
+		if(this instanceof ColGroupConst) {
+			AggregateUnaryOperator auop = InstructionUtils.parseBasicAggregateUnaryOperator("uac+", 1);
+			MatrixBlock tmp = ucCG.aggregateUnaryOperations(auop, new MatrixBlock(),
+				Math.max(ucCG.getNumRows(), ucCG.getNumColumns()), null, true);
+			ADictionary l = new MatrixBlockDictionary(tmp);
+			matrixMultDictionariesAndOutputToColIndexes(l, _dict, lhs._colIndexes, _colIndexes, result);
+		}
+		else {
+			LOG.warn("Inefficient transpose of uncompressed to fit to "
+				+ "template need t(UnCompressedColGroup) %*% AColGroup support");
+			MatrixBlock tmp = new MatrixBlock(ucCG.getNumColumns(), ucCG.getNumRows(), ucCG.isInSparseFormat());
+			LibMatrixReorg.transpose(ucCG, tmp, InfrastructureAnalyzer.getLocalParallelism());
+
+			leftMultByMatrix(tmp, result, lhs._colIndexes);
+		}
+	}
+
 	@Override
-	public void tsmm(double[] result, int numColumns) {
+	public final void tsmm(double[] result, int numColumns) {
 
-		// final int[] counts = getCounts();
+		final int[] counts = getCounts();
 
-		// _dict = _dict.getAsMatrixBlockDictionary(_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);
-		// 	else
-		// 		tsmmDense(result, numColumns, mb.getDenseBlockValues(), counts);
-		// }
-		// else
-		// 	tsmmDense(result, numColumns, getValues(), counts);
+		_dict = _dict.getAsMatrixBlockDictionary(_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);
+			else
+				tsmmDense(result, numColumns, mb.getDenseBlockValues(), counts);
+		}
+		else
+			tsmmDense(result, numColumns, getValues(), counts);
 
 	}
 
+	@Override
+	public final void tsmm(double[] result, int numColumns, int idxStart, int idxEnd) {
+		throw new NotImplementedException();
+	}
+
 	private void tsmmDense(double[] result, int numColumns, double[] values, int[] counts) {
 		if(values == null)
 			return;
@@ -926,24 +795,24 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public boolean containsValue(double pattern) {
+	public final boolean containsValue(double pattern) {
 		return _dict.containsValue(pattern);
 	}
 
 	@Override
-	public long getNumberNonZeros() {
+	public final long getNumberNonZeros() {
 		int[] counts = getCounts();
 		return _dict.getNumberNonZeros(counts, _colIndexes.length);
 	}
 
 	private static void matrixMultDictionariesAndOutputToColIndexesWithScaling(final ADictionary left,
-		final ADictionary right, final int[] leftRows, final int[] rightColumns, final double[] result,
-		final int outCols, final int[] counts) {
+		final ADictionary right, final int[] leftRows, final int[] rightColumns, final MatrixBlock result,
+		final int[] counts) {
 		final boolean modifyRight = right.getInMemorySize() > left.getInMemorySize();
 		ADictionary rightM = modifyRight ? right.scaleTuples(counts, rightColumns.length) : right;
 		ADictionary leftM = modifyRight ? left : left.scaleTuples(counts, leftRows.length);
 
-		matrixMultDictionariesAndOutputToColIndexes(leftM, rightM, leftRows, rightColumns, result, outCols);
+		matrixMultDictionariesAndOutputToColIndexes(leftM, rightM, leftRows, rightColumns, result);
 
 	}
 
@@ -1020,11 +889,10 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * @param right     The right side dictionary
 	 * @param rowsLeft  The number of rows and the row indexes on the left hand side
 	 * @param colsRight The number of columns and the column indexes on the right hand side
-	 * @param result    The result matrix to put the results into, linearized row major
-	 * @param outCols   The output columns count, to know how much to offset into with results.
+	 * @param result    The result matrix to put the results into.
 	 */
 	private static void matrixMultDictionariesAndOutputToColIndexes(ADictionary left, ADictionary right, int[] rowsLeft,
-		int[] colsRight, double[] result, int outCols) {
+		int[] colsRight, MatrixBlock result) {
 
 		try {
 			double[] leftV = null;
@@ -1033,22 +901,36 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			if(left instanceof MatrixBlockDictionary) {
 				MatrixBlockDictionary leftD = left.getAsMatrixBlockDictionary(rowsLeft.length);
 				MatrixBlock leftMB = leftD.getMatrixBlock();
-				if(leftMB.isEmpty())
+				if(leftMB.isEmpty()) {
+					LOG.error("Left is empty: " + leftMB);
 					return;
+				}
 				else if(right instanceof MatrixBlockDictionary) {
 					MatrixBlockDictionary rightD = right.getAsMatrixBlockDictionary(colsRight.length);
 					MatrixBlock rightMB = rightD.getMatrixBlock();
 					if(rightMB.isEmpty())
 						return;
-					else if(rightMB.isInSparseFormat() && leftMB.isInSparseFormat()) {
+					else if(rightMB.isInSparseFormat() && leftMB.isInSparseFormat())
 						throw new NotImplementedException("Not Supported sparse sparse dictionary multiplication");
-					}
+					else if(rightMB.isInSparseFormat())
+						matrixMultDictionariesAndOutputToColIndecesDenseSparse(leftMB.getDenseBlockValues(),
+							rightMB.getSparseBlock(), rowsLeft, colsRight, result);
+					else if(leftMB.isInSparseFormat())
+						matrixMultDictionariesAndOutputToColIndecesSparseDense(leftMB.getSparseBlock(),
+							rightMB.getDenseBlockValues(), rowsLeft, colsRight, result);
+					else
+						matrixMultDictionariesAndOutputToColIndexesDenseDense(leftMB.getDenseBlockValues(),
+							rightMB.getDenseBlockValues(), rowsLeft, colsRight, result);
+					return;
 				}
 				else if(leftMB.isInSparseFormat()) {
 					matrixMultDictionariesAndOutputToColIndecesSparseDense(leftMB.getSparseBlock(), right.getValues(),
-						rowsLeft, colsRight, result, outCols);
+						rowsLeft, colsRight, result);
 					return;
 				}
+				else {
+					leftV = leftMB.getDenseBlockValues();
+				}
 			}
 			else {
 				leftV = left.getValues();
@@ -1058,24 +940,25 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 				MatrixBlockDictionary rightD = right.getAsMatrixBlockDictionary(colsRight.length);
 				MatrixBlock rightMB = rightD.getMatrixBlock();
 
-				if(rightMB.isEmpty())
+				if(rightMB.isEmpty()) {
+					LOG.error("Right is empty: " + rightMB);
 					return;
+				}
 				else if(rightMB.isInSparseFormat()) {
 					matrixMultDictionariesAndOutputToColIndecesDenseSparse(leftV, rightMB.getSparseBlock(), rowsLeft,
-						colsRight, result, outCols);
+						colsRight, result);
 					return;
 				}
+				else {
+					rightV = rightMB.getDenseBlockValues();
+				}
 			}
 			else {
 				rightV = right.getValues();
 			}
 
-			if(leftV != null && rightV != null) {
-				// default if there was not sparse found;
-				LOG.warn("Inefficient forced dense values");
-				matrixMultDictionariesAndOutputToColIndexesDenseDense(leftV, rightV, rowsLeft, colsRight, result,
-					outCols);
-			}
+			if(leftV != null && rightV != null)
+				matrixMultDictionariesAndOutputToColIndexesDenseDense(leftV, rightV, rowsLeft, colsRight, result);
 
 		}
 		catch(Exception e) {
@@ -1088,25 +971,29 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	private static void matrixMultDictionariesAndOutputToColIndexesDenseDense(double[] left, double[] right,
-		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		int[] rowsLeft, int[] colsRight, MatrixBlock result) {
 		final int commonDim = Math.min(left.length / rowsLeft.length, right.length / colsRight.length);
+
+		final double[] resV = result.getDenseBlockValues();
 		for(int k = 0; k < commonDim; k++) {
 			final int offL = k * rowsLeft.length;
 			final int offR = k * colsRight.length;
 			for(int i = 0; i < rowsLeft.length; i++) {
-				final int offOut = rowsLeft[i] * outCols;
+				final int offOut = rowsLeft[i] * result.getNumColumns();
 				final double vl = left[offL + i];
 				if(vl != 0)
 					for(int j = 0; j < colsRight.length; j++) {
 						final double vr = right[offR + j];
-						result[offOut + colsRight[j]] += vl * vr;
+						resV[offOut + colsRight[j]] += vl * vr;
 					}
 			}
 		}
 	}
 
 	private static void matrixMultDictionariesAndOutputToColIndecesSparseDense(SparseBlock left, double[] right,
-		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		int[] rowsLeft, int[] colsRight, MatrixBlock result) {
+
+		final double[] resV = result.getDenseBlockValues();
 		final int commonDim = Math.min(left.numRows(), right.length / colsRight.length);
 		for(int i = 0; i < commonDim; i++) {
 			if(left.isEmpty(i))
@@ -1117,16 +1004,17 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			final double[] leftVals = left.values(i);
 			final int offRight = i * colsRight.length;
 			for(int k = apos; k < alen; k++) {
-				final int offOut = rowsLeft[aix[k]] * outCols;
+				final int offOut = rowsLeft[aix[k]] * result.getNumColumns();
 				final double v = leftVals[k];
 				for(int j = 0; j < colsRight.length; j++)
-					result[offOut + colsRight[j]] += v * right[offRight + j];
+					resV[offOut + colsRight[j]] += v * right[offRight + j];
 			}
 		}
 	}
 
 	private static void matrixMultDictionariesAndOutputToColIndecesDenseSparse(double[] left, SparseBlock right,
-		int[] rowsLeft, int[] colsRight, double[] result, int outCols) {
+		int[] rowsLeft, int[] colsRight, MatrixBlock result) {
+		final double[] resV = result.getDenseBlockValues();
 		final int commonDim = Math.min(left.length / rowsLeft.length, right.numRows());
 		for(int i = 0; i < commonDim; i++) {
 			if(right.isEmpty(i))
@@ -1137,18 +1025,18 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			final double[] rightVals = right.values(i);
 			final int offLeft = i * rowsLeft.length;
 			for(int j = 0; j < rowsLeft.length; j++) {
-				final int offOut = rowsLeft[j] * outCols;
+				final int offOut = rowsLeft[j] * result.getNumColumns();
 				final double v = left[offLeft + j];
 				if(v != 0)
 					for(int k = apos; k < alen; k++) {
-						result[offOut + colsRight[aix[k]]] += v * rightVals[k];
+						resV[offOut + colsRight[aix[k]]] += v * rightVals[k];
 					}
 			}
 		}
 	}
 
 	@Override
-	public boolean isDense() {
+	public final boolean isDense() {
 		return !_zeros;
 	}
 
@@ -1157,69 +1045,145 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * 
 	 * @param matrix matrix to left multiply
 	 * @param result matrix block result
-	 * @param values The materialized values contained in the ColGroupValue
 	 * @param rl     The row to start the matrix multiplication from
 	 * @param ru     The row to stop the matrix multiplication at.
 	 */
 	@Override
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
-		final int numVals = getNumValues();
+	public final void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int rl, int ru) {
+		try {
+			if(matrix.isEmpty())
+				return;
+
+			MatrixBlock tmpRes = leftMultByMatrixIntermediateMatrix(matrix, rl, ru);
+
+			addMatrixToResult(tmpRes, result, rl, ru);
+
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException(this.getClass().getSimpleName() + " Failed to Left Matrix Multiply", e);
+		}
+	}
+
+	private MatrixBlock leftMultByMatrixIntermediateMatrix(MatrixBlock matrix, int rl, int ru) {
+		// Get dictionary.
+		MatrixBlock dictM = forceMatrixBlockDictionary().getMatrixBlock();
+		// Allocate temporary matrix to multiply into.
+		MatrixBlock tmpRes = new MatrixBlock(matrix.getNumRows(), _colIndexes.length, false);
+		// Pre aggregate the matrix into same size as dictionary
+		MatrixBlock preAgg = preAggregate(matrix, rl, ru);
+
+		LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
+		return tmpRes;
+	}
+
+	private void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result, int[] outputRows) {
+		try {
+			if(matrix.isEmpty())
+				return;
+			MatrixBlock tmpRes = leftMultByMatrixIntermediateMatrix(matrix, 0, matrix.getNumRows());
+			addMatrixToResult(tmpRes, result, outputRows);
+
+		}
+		catch(Exception e) {
+			throw new DMLCompressionException(
+				this.getClass().getSimpleName() + " Failed to multiply with an uncompressed column group", e);
+		}
+	}
+
+	private MatrixBlockDictionary forceMatrixBlockDictionary() {
 		if(!(_dict instanceof MatrixBlockDictionary))
 			_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
+		return((MatrixBlockDictionary) _dict);
+	}
 
-		MatrixBlock dictM = ((MatrixBlockDictionary) _dict).getMatrixBlock();
-		dictM.examSparsity();
-		MatrixBlock tmpRes = new MatrixBlock(1, _colIndexes.length, false);
-		for(int i = rl; i < ru; i++) {
-			double[] vals = matrix.isInSparseFormat() ? preAggregateSparseWithCheck(matrix.getSparseBlock(),
-				i) : preAggregate(matrix.getDenseBlockValues(), i);
-			if(vals != null) {
-				DenseBlock preAggV = new DenseBlockFP64(new int[] {1, numVals}, vals);
-				MatrixBlock preAgg = new MatrixBlock(1, numVals, preAggV);
-				preAgg.setNonZeros(numVals);
-				// LOG.error("PreAgg Sparsity " + preAgg.getSparsity() + " nnz " + preAgg.getNonZeros());
-				LibMatrixMult.matrixMult(preAgg, dictM, tmpRes);
-				addVectorToResult(tmpRes, result, i);
-				tmpRes.reset();
+	private void addMatrixToResult(MatrixBlock tmp, MatrixBlock result, int rl, int ru) {
+		if(tmp.isEmpty())
+			return;
+		final double[] retV = result.getDenseBlockValues();
+		final int nColRet = result.getNumColumns();
+		if(tmp.isInSparseFormat()) {
+			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];
+				}
 			}
 		}
 	}
 
-	private void addVectorToResult(MatrixBlock tmp, MatrixBlock result, int row) {
+	private void addMatrixToResult(MatrixBlock tmp, MatrixBlock result, int[] rowIndexes) {
 		if(tmp.isEmpty())
 			return;
 		final double[] retV = result.getDenseBlockValues();
 		final int nColRet = result.getNumColumns();
-		final int offR = row * nColRet;
 		if(tmp.isInSparseFormat()) {
-			final SparseBlock sb = tmp.getSparseBlock();
-			if(sb.isEmpty(0))
-				return;
-			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 = apos; i < apos + alen; i++)
-				retV[offR + _colIndexes[aix[i]]] += avals[i];
-
+			SparseBlock sb = tmp.getSparseBlock();
+			for(int row = 0; row < rowIndexes.length; row++) {
+				final int apos = sb.pos(row);
+				final int alen = sb.size(row);
+				final int[] aix = sb.indexes(row);
+				final double[] avals = sb.values(row);
+				final int offR = rowIndexes[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 nColTmp = tmp.getNumColumns();
-			// for(int row = rl, offT = 0, offR = rl * nColRet; row < ru; row++, offT += nColTmp, offR += nColRet) {
-			for(int col = 0; col < _colIndexes.length; col++)
-				retV[offR + _colIndexes[col]] += tmpV[col];
-
-			// }
+			final int nCol = _colIndexes.length;
+			for(int row = 0, offT = 0; row < rowIndexes.length; row++, offT += nCol) {
+				final int offR = rowIndexes[row] * nColRet;
+				for(int col = 0; col < nCol; col++) {
+					retV[offR + _colIndexes[col]] += tmpV[offT + col];
+				}
+			}
 		}
 	}
 
-	public AColGroup rightMultByMatrix(MatrixBlock right) {
-		Pair<int[], double[]> pre = preaggForRightMultiplyValues(getNumValues(), right, 0, right.getNumColumns(),
-			right.getNumColumns());
-		if(pre.getLeft().length > 0)
-			return copyAndSet(pre.getLeft(), pre.getRight());
-		return null;
+	public final AColGroup rightMultByMatrix(MatrixBlock right) {
+
+		if(right.isEmpty())
+			return null;
+		final int cl = 0;
+		final int cu = right.getNumColumns();
+		final int cut = right.getNumColumns();
+		final int nCol = right.getNumColumns();
+		final int numVals = getNumValues();
+		int[] agCols;
+		double[] ret;
+		if(right.isInSparseFormat()) {
+			final SparseBlock sb = right.getSparseBlock();
+			agCols = getAggregateColumnsSetSparse(sb, nCol);
+			if(agCols.length == 0)
+				return null;
+			ret = preaggValuesFromSparse(numVals, sb, agCols, cl, cu, cut);
+		}
+		else {
+			double[] rightV = right.getDenseBlockValues();
+			agCols = getAggregateColumnsSetDense(rightV, cl, cu, cut);
+			if(agCols.length == 0)
+				return null;
+			ret = new double[numVals * agCols.length];
+			_dict.preaggValuesFromDense(numVals, _colIndexes, agCols, rightV, ret, cut);
+		}
+
+		return copyAndSet(agCols, ret);
 	}
 
 	@Override
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 e1d4152..df6f648 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
@@ -29,8 +29,7 @@ import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 
 /**
- * This dictionary class aims to encapsulate the storage and operations over unique floating point values of a column
- * group.
+ * This dictionary class aims to encapsulate the storage and operations over unique tuple values of a column group.
  */
 public abstract class ADictionary {
 
@@ -52,15 +51,6 @@ public abstract class ADictionary {
 	public abstract double getValue(int i);
 
 	/**
-	 * Determines if the content has a zero tuple. meaning all values at a specific row are zero value. This is useful
-	 * information to find out if the dictionary is used in a dense context. To improve some specific operations.
-	 * 
-	 * @param nCol The number of columns in the dictionary.
-	 * @return The index at which the zero tuple is located.
-	 */
-	public abstract int hasZeroTuple(int nCol);
-
-	/**
 	 * Returns the memory usage of the dictionary.
 	 * 
 	 * @return a long value in number of bytes for the dictionary.
@@ -87,13 +77,6 @@ public abstract class ADictionary {
 	public abstract double[] aggregateTuples(Builtin fn, int nCol);
 
 	/**
-	 * returns the count of values contained in the dictionary.
-	 * 
-	 * @return an integer of count of values.
-	 */
-	public abstract int size();
-
-	/**
 	 * 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.
 	 * 
@@ -113,15 +96,46 @@ public abstract class ADictionary {
 	 */
 	public abstract ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols);
 
+	/**
+	 * Apply binary row operation on this dictionary.
+	 * 
+	 * @param op         The operation to this dictionary
+	 * @param v          The values to use on the left hand side.
+	 * @param sparseSafe boolean specifying if the operation is safe, and therefore dont need to allocate an extended
+	 *                   dictionary
+	 * @param colIndexes The column indexes to consider inside v.
+	 * @param left       A Boolean specifying if the operation is done on the left or right side of the dictionary.
+	 * @return A new dictionary containing the updated values.
+	 */
 	public ADictionary applyBinaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes,
 		boolean left) {
 		return (left) ? applyBinaryRowOpLeft(op, v, sparseSafe, colIndexes) : applyBinaryRowOpRight(op, v, sparseSafe,
 			colIndexes);
 	}
 
+	/**
+	 * Apply binary row operation on this dictionary on the left side.
+	 * 
+	 * @param op         The operation to this dictionary
+	 * @param v          The values to use on the left hand side.
+	 * @param sparseSafe boolean specifying if the operation is safe, and therefore dont need to allocate an extended
+	 *                   dictionary
+	 * @param colIndexes The column indexes to consider inside v.
+	 * @return A new dictionary containing the updated values.
+	 */
 	public abstract ADictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe,
 		int[] colIndexes);
 
+	/**
+	 * Apply binary row operation on this dictionary on the right side.
+	 * 
+	 * @param op         The operation to this dictionary
+	 * @param v          The values to use on the right hand side.
+	 * @param sparseSafe boolean specifying if the operation is safe, and therefore dont need to allocate an extended
+	 *                   dictionary
+	 * @param colIndexes The column indexes to consider inside v.
+	 * @return A new dictionary containing the updated values.
+	 */
 	public abstract ADictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe,
 		int[] colIndexes);
 
@@ -130,6 +144,12 @@ public abstract class ADictionary {
 	 */
 	public abstract ADictionary clone();
 
+	/**
+	 * Clone the dictionary, and extend size of the dictionary by a given length
+	 * 
+	 * @param len The length to extend the dictionary, it is assumed this value is positive.
+	 * @return a clone of the dictionary, extended by len.
+	 */
 	public abstract ADictionary cloneAndExtend(int len);
 
 	/**
@@ -173,14 +193,6 @@ public abstract class ADictionary {
 	public abstract int getNumberOfValues(int ncol);
 
 	/**
-	 * Materializes a Zero tuple at the last index of the dictionary.
-	 * 
-	 * @param numCols The number of columns in the dictionary
-	 * @return the new Dictionary with materialized zero tuple.
-	 */
-	// public abstract IDictionary materializeZeroValue(int numCols);
-
-	/**
 	 * Method used as a pre-aggregate of each tuple in the dictionary, to single double values.
 	 * 
 	 * Note if the number of columns is one the actual dictionaries values are simply returned.
@@ -201,14 +213,50 @@ public abstract class ADictionary {
 	 */
 	public abstract double sumRow(int k, boolean square, int nrColumns);
 
+	/**
+	 * get the column sum of this dictionary only.
+	 * 
+	 * @param counts the counts of the values contained
+	 * @param nCol   The number of columns contained in each tuple.
+	 * @return the colSums of this column group.
+	 */
 	public abstract double[] colSum(int[] counts, int nCol);
 
+	/**
+	 * Get the column sum of the values contained in the dictionary
+	 * 
+	 * @param c          The output array allocated to contain all column groups output.
+	 * @param counts     The counts of the individual tuples.
+	 * @param colIndexes The columns indexes of the parent column group, this indicate where to put the column sum into
+	 *                   the c output.
+	 * @param square     Specify if the values should be squared
+	 */
 	public abstract void colSum(double[] c, int[] counts, int[] colIndexes, boolean square);
 
-	public abstract double sum(int[] counts, int ncol);
+	/**
+	 * Get the sum of the values contained in the dictionary
+	 * 
+	 * @param counts The counts of the individual tuples
+	 * @param nCol   The number of columns contained
+	 * @return The sum scaled by the counts provided.
+	 */
+	public abstract double sum(int[] counts, int nCol);
 
-	public abstract double sumsq(int[] counts, int ncol);
+	/**
+	 * Get the square sum of the values contained in the dictionary
+	 * 
+	 * @param counts The counts of the individual tuples
+	 * @param nCol   The number of columns contained
+	 * @return The square sum scaled by the counts provided.
+	 */
+	public abstract double sumsq(int[] counts, int nCol);
 
+	/**
+	 * Get a string representation of the dictionary, that considers the layout of the data.
+	 * 
+	 * @param colIndexes The number of columns in the dictionary.
+	 * @return A string that is nicer to print.
+	 */
 	public abstract String getString(int colIndexes);
 
 	/**
@@ -239,11 +287,17 @@ public abstract class ADictionary {
 	 */
 	public abstract ADictionary reExpandColumns(int max);
 
+	/**
+	 * Detect if the dictionary contains a specific value.
+	 * 
+	 * @param pattern The value to search for
+	 * @return true if the value is contained else false.
+	 */
 	public abstract boolean containsValue(double pattern);
 
 	/**
 	 * Calculate the number of non zeros in the dictionary. The number of non zeros should be scaled with the counts
-	 * given
+	 * given. This gives the exact number of non zero values in the parent column group.
 	 * 
 	 * @param counts The counts of each dictionary entry
 	 * @param nCol   The number of columns in this dictionary
@@ -251,8 +305,6 @@ public abstract class ADictionary {
 	 */
 	public abstract long getNumberNonZeros(int[] counts, int nCol);
 
-	public abstract long getNumberNonZerosContained();
-
 	/**
 	 * Copies and adds the dictionary entry from this dictionary to the d dictionary
 	 * 
@@ -269,6 +321,7 @@ public abstract class ADictionary {
 	 * returns null if that tuple is all zero values.
 	 * 
 	 * @param counts The counts of the individual tuples contained, managed by the column group.
+	 * @param nCol   The number of columns contained in this dictionary
 	 * @return a new double array containing the most common value
 	 */
 	public double[] getMostCommonTuple(int[] counts, int nCol) {
@@ -283,6 +336,13 @@ public abstract class ADictionary {
 		return getTuple(maxIndex, nCol);
 	}
 
+	/**
+	 * Get the values contained in a specific tuple of the dictionary.
+	 * 
+	 * @param index The index where the values are located
+	 * @param nCol  The number of columns contained in this dictionary
+	 * @return a materialized double array containing the tuple.
+	 */
 	public abstract double[] getTuple(int index, int nCol);
 
 	/**
@@ -298,6 +358,7 @@ public abstract class ADictionary {
 	 * Get this dictionary as a matrixBlock dictionary. This allows us to use optimized kernels coded elsewhere in the
 	 * system, such as matrix multiplication.
 	 * 
+	 * @param nCol The number of columns contained in this column group.
 	 * @return A Dictionary containing a MatrixBlock.
 	 */
 	public abstract MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol);
@@ -310,4 +371,18 @@ public abstract class ADictionary {
 	 * @return A New dictionary (since we don't want to modify the underlying dictionary)
 	 */
 	public abstract ADictionary scaleTuples(int[] scaling, int nCol);
+
+	/**
+	 * Pre Aggregate values for right Matrix Multiplication.
+	 * 
+	 * @param numVals          The number of values contained in this dictionary
+	 * @param colIndexes       The column indexes that is associated with the parent column group
+	 * @param aggregateColumns The column to aggregate, this is preprocessed, to find remove consideration for empty
+	 *                         columns
+	 * @param b                The values in the right hand side matrix
+	 * @param ret              The double array to put in the aggregate.
+	 * @param cut              The number of columns in b.
+	 */
+	public abstract void preaggValuesFromDense(final int numVals, final int[] colIndexes, final int[] aggregateColumns,
+		final double[] b, final double[] ret, final int cut);
 }
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 770557b..80872d5 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
@@ -71,19 +71,6 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public int hasZeroTuple(int nCol) {
-		int len = getNumberOfValues(nCol);
-		for(int i = 0, off = 0; i < len; i++, off += nCol) {
-			boolean allZeros = true;
-			for(int j = 0; j < nCol; j++)
-				allZeros &= (_values[off + j] == 0);
-			if(allZeros)
-				return i;
-		}
-		return -1;
-	}
-
-	@Override
 	public double aggregate(double init, Builtin fn) {
 		// full aggregate can disregard tuple boundaries
 		double ret = init;
@@ -122,7 +109,7 @@ public class Dictionary extends ADictionary {
 		double[] values = new double[_values.length + numCols];
 		for(int i = 0; i < _values.length; i++)
 			values[i] = op.executeScalar(_values[i]);
-		
+
 		Arrays.fill(values, _values.length, _values.length + numCols, newVal);
 		return new Dictionary(values);
 	}
@@ -208,13 +195,13 @@ public class Dictionary extends ADictionary {
 		return 1 + 4 + 8 * size();
 	}
 
-	public int size() {
-		return (_values == null) ? 0 : _values.length;
+	private int size() {
+		return _values.length;
 	}
 
 	@Override
 	public int getNumberOfValues(int nCol) {
-		return (_values == null) ? 0 : _values.length / nCol;
+		return _values.length / nCol;
 	}
 
 	@Override
@@ -234,8 +221,7 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public double sumRow(int k, boolean square, int nrColumns) {
-		if(_values == null)
-			return 0;
+
 		int valOff = k * nrColumns;
 		double res = 0.0;
 		if(!square) {
@@ -265,8 +251,6 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
-		if(_values == null)
-			return;
 		for(int k = 0; k < _values.length / colIndexes.length; k++) {
 			final int cntk = counts[k];
 			for(int j = 0; j < colIndexes.length; j++) {
@@ -282,8 +266,6 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public double sum(int[] counts, int ncol) {
-		if(_values == null)
-			return 0;
 		double out = 0;
 		int valOff = 0;
 		for(int k = 0; k < _values.length / ncol; k++) {
@@ -297,8 +279,6 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public double sumsq(int[] counts, int ncol) {
-		if(_values == null)
-			return 0;
 		double out = 0;
 		int valOff = 0;
 		for(int k = 0; k < _values.length / ncol; k++) {
@@ -322,8 +302,7 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public void addMaxAndMin(double[] ret, int[] colIndexes) {
-		if(_values == null || _values.length == 0)
-			return;
+
 		double[] mins = new double[colIndexes.length];
 		double[] maxs = new double[colIndexes.length];
 		for(int i = 0; i < colIndexes.length; i++) {
@@ -347,15 +326,14 @@ public class Dictionary extends ADictionary {
 		if(colIndexes == 1)
 			sb.append(Arrays.toString(_values));
 		else {
-			sb.append("[");
+			sb.append("[\n");
 			for(int i = 0; i < _values.length - 1; i++) {
 				sb.append(_values[i]);
-				sb.append((i) % (colIndexes) == colIndexes - 1 ? "\n: " : ", ");
-			}
-			if(_values != null && _values.length > 0) {
-				sb.append(_values[_values.length - 1]);
+				sb.append((i) % (colIndexes) == colIndexes - 1 ? "\nt" + i + ": " : ", ");
 			}
-			sb.append("]");
+			sb.append(_values[_values.length - 1]);
+
+			sb.append("\n]");
 		}
 		return sb.toString();
 	}
@@ -388,10 +366,6 @@ public class Dictionary extends ADictionary {
 
 	@Override
 	public boolean containsValue(double pattern) {
-
-		if(_values == null)
-			return false;
-
 		boolean NaNpattern = Double.isNaN(pattern);
 
 		if(NaNpattern) {
@@ -440,16 +414,6 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
-	public long getNumberNonZerosContained() {
-		long count = 0;
-		for(double v : _values) {
-			if(v != 0.0)
-				count++;
-		}
-		return count;
-	}
-
-	@Override
 	public double[] getTuple(int index, int nCol) {
 
 		final double[] tuple = new double[nCol];
@@ -457,7 +421,7 @@ public class Dictionary extends ADictionary {
 		for(int i = index * nCol, off = 0; i < (index + 1) * nCol && i < _values.length; i++, off++) {
 			final double v = _values[i];
 			if(v != 0) {
-				tuple[off] = _values[i];
+				tuple[off] = v;
 				allZero = false;
 			}
 		}
@@ -479,6 +443,7 @@ public class Dictionary extends ADictionary {
 		final int nRow = _values.length / nCol;
 		DenseBlock dictV = new DenseBlockFP64(new int[] {nRow, nCol}, _values);
 		MatrixBlock dictM = new MatrixBlock(nRow, nCol, dictV);
+		dictM.getNonZeros();
 		dictM.examSparsity();
 		return new MatrixBlockDictionary(dictM);
 	}
@@ -506,4 +471,20 @@ public class Dictionary extends ADictionary {
 		}
 		return new Dictionary(scaledValues);
 	}
+
+	@Override
+	public void preaggValuesFromDense(int numVals, int[] colIndexes, int[] aggregateColumns, double[] b, double[] ret,
+		int cut) {
+		for(int k = 0, off = 0;
+			k < numVals * colIndexes.length;
+			k += colIndexes.length, off += aggregateColumns.length) {
+			for(int h = 0; h < colIndexes.length; h++) {
+				int idb = colIndexes[h] * cut;
+				double v = _values[k + h];
+				if(v != 0)
+					for(int i = 0; i < aggregateColumns.length; i++)
+						ret[off + i] += v * b[idb + aggregateColumns[i]];
+			}
+		}
+	}
 }
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 1fd47d8..c8bc14c 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
@@ -31,6 +31,7 @@ import org.apache.sysds.runtime.compress.utils.Bitmap;
 import org.apache.sysds.runtime.compress.utils.BitmapLossy;
 import org.apache.sysds.runtime.compress.utils.MultiColBitmap;
 import org.apache.sysds.runtime.data.SparseBlock;
+import org.apache.sysds.runtime.data.SparseRow;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 public class DictionaryFactory {
@@ -69,6 +70,10 @@ public class DictionaryFactory {
 		return create(ubm, 1.0);
 	}
 
+	public static ADictionary create(ABitmap ubm, double sparsity, boolean withZeroTuple) {
+		return (withZeroTuple) ? createWithAppendedZeroTuple(ubm, sparsity) : create(ubm, sparsity);
+	}
+
 	public static ADictionary create(ABitmap ubm, double sparsity) {
 		if(ubm instanceof BitmapLossy)
 			return new QDictionary((BitmapLossy) ubm);
@@ -111,7 +116,6 @@ public class DictionaryFactory {
 	}
 
 	public static ADictionary createWithAppendedZeroTuple(ABitmap ubm, double sparsity) {
-		// Log.warn("Inefficient creation of dictionary, to then allocate again.");
 		final int nRows = ubm.getNumValues() + 1;
 		final int nCols = ubm.getNumColumns();
 		if(ubm instanceof Bitmap) {
@@ -149,37 +153,94 @@ public class DictionaryFactory {
 			throw new NotImplementedException(
 				"Not implemented creation of bitmap type : " + ubm.getClass().getSimpleName());
 		}
-
 	}
 
-	public static ADictionary moveFrequentToLastDictionaryEntry(ADictionary dict, ABitmap ubm, int numRows,
+	public static ADictionary moveFrequentToLastDictionaryEntry(ADictionary dict, ABitmap ubm, int nRow,
 		int largestIndex) {
-		LOG.warn("Inefficient creation of dictionary, to then allocate again to move one entry to end.");
-		final double[] dictValues = dict.getValues();
-		final int zeros = numRows - (int) ubm.getNumOffsets();
+		final int zeros = nRow - (int) ubm.getNumOffsets();
 		final int nCol = ubm.getNumColumns();
-		final int offsetToLargest = largestIndex * nCol;
+		final int largestIndexSize = ubm.getOffsetsList(largestIndex).size();
+		if(dict instanceof MatrixBlockDictionary) {
+			MatrixBlockDictionary mbd = (MatrixBlockDictionary) dict;
+			MatrixBlock mb = mbd.getMatrixBlock();
+			if(mb.isEmpty()) {
+				if(zeros == 0)
+					return dict;
+				else
+					return new MatrixBlockDictionary(new MatrixBlock(mb.getNumRows() + 1, mb.getNumColumns(), true));
+			}
+			else if(mb.isInSparseFormat()) {
+				MatrixBlockDictionary mbdn = moveToLastDictionaryEntrySparse(mb.getSparseBlock(), largestIndex, zeros,
+					nCol, largestIndexSize);
+				MatrixBlock mbn = mbdn.getMatrixBlock();
+				mbn.setNonZeros(mb.getNonZeros());
+				if(mbn.getNonZeros() == 0)
+					mbn.recomputeNonZeros();
+				return mbdn;
+			}
+			else
+				return moveToLastDictionaryEntryDense(mb.getDenseBlockValues(), largestIndex, zeros, nCol,
+					largestIndexSize);
+		}
+		else
+			return moveToLastDictionaryEntryDense(dict.getValues(), largestIndex, zeros, nCol, largestIndexSize);
+
+	}
+
+	private static MatrixBlockDictionary moveToLastDictionaryEntrySparse(SparseBlock sb, int indexToMove, int zeros,
+		int nCol, int largestIndexSize) {
+
+		if(zeros == 0) {
+			MatrixBlock ret = new MatrixBlock(sb.numRows(), nCol, true);
+			ret.setSparseBlock(sb);
+			final SparseRow swap = sb.get(indexToMove);
+			for(int i = indexToMove + 1; i < sb.numRows(); i++)
+				sb.set(i - 1, sb.get(i), false);
+			sb.set(sb.numRows() - 1, swap, false);
+			return new MatrixBlockDictionary(ret);
+		}
+
+		MatrixBlock ret = new MatrixBlock(sb.numRows() + 1, nCol, true);
+		ret.allocateSparseRowsBlock();
+		final SparseBlock retB = ret.getSparseBlock();
+		if(zeros > largestIndexSize) {
+			for(int i = 0; i < sb.numRows(); i++)
+				retB.set(i, sb.get(i), false);
+		}
+		else {
+			for(int i = 0; i < indexToMove; i++)
+				retB.set(i, sb.get(i), false);
+
+			retB.set(sb.numRows(), sb.get(indexToMove), false);
+			for(int i = indexToMove + 1; i < sb.numRows(); i++)
+				retB.set(i - 1, sb.get(i), false);
+		}
+		return new MatrixBlockDictionary(ret);
+	}
+
+	private static ADictionary moveToLastDictionaryEntryDense(double[] values, int indexToMove, int zeros, int nCol,
+		int largestIndexSize) {
+		final int offsetToLargest = indexToMove * nCol;
 
 		if(zeros == 0) {
 			final double[] swap = new double[nCol];
-			System.arraycopy(dictValues, offsetToLargest, swap, 0, nCol);
-			for(int i = offsetToLargest; i < dictValues.length - nCol; i++) {
-				dictValues[i] = dictValues[i + nCol];
-			}
-			System.arraycopy(swap, 0, dictValues, dictValues.length - nCol, nCol);
-			return dict;
+			System.arraycopy(values, offsetToLargest, swap, 0, nCol);
+			for(int i = offsetToLargest; i < values.length - nCol; i++)
+				values[i] = values[i + nCol];
+
+			System.arraycopy(swap, 0, values, values.length - nCol, nCol);
+			return new Dictionary(values);
 		}
 
-		final int largestIndexSize = ubm.getOffsetsList(largestIndex).size();
-		final double[] newDict = new double[dictValues.length + nCol];
+		final double[] newDict = new double[values.length + nCol];
 
 		if(zeros > largestIndexSize)
-			System.arraycopy(dictValues, 0, newDict, 0, dictValues.length);
+			System.arraycopy(values, 0, newDict, 0, values.length);
 		else {
-			System.arraycopy(dictValues, 0, newDict, 0, offsetToLargest);
-			System.arraycopy(dictValues, offsetToLargest + nCol, newDict, offsetToLargest,
-				dictValues.length - offsetToLargest - nCol);
-			System.arraycopy(dictValues, offsetToLargest, newDict, newDict.length - nCol, nCol);
+			System.arraycopy(values, 0, newDict, 0, offsetToLargest);
+			System.arraycopy(values, offsetToLargest + nCol, newDict, offsetToLargest,
+				values.length - offsetToLargest - nCol);
+			System.arraycopy(values, offsetToLargest, newDict, newDict.length - nCol, nCol);
 		}
 		return new Dictionary(newDict);
 	}
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 aae66e6..4c1ab04 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
@@ -1,3 +1,22 @@
+/*
+ * 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.dictionary;
 
 import java.io.DataInput;
@@ -5,6 +24,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -17,595 +37,606 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 
 public class MatrixBlockDictionary extends ADictionary {
 
-    private MatrixBlock _data;
-
-    public MatrixBlockDictionary(MatrixBlock data) {
-        _data = data;
-    }
-
-    public MatrixBlock getMatrixBlock() {
-        return _data;
-    }
-
-    @Override
-    public double[] getValues() {
-        LOG.warn("Inefficient force dense format.");
-        if(_data.isInSparseFormat())
-            _data.sparseToDense();
-        return _data.getDenseBlockValues();
-    }
-
-    @Override
-    public double getValue(int i) {
-        final int nCol = _data.getNumColumns();
-        LOG.warn("inefficient get value at index");
-        return _data.quickGetValue(i / nCol, i % nCol);
-    }
-
-    @Override
-    public int hasZeroTuple(int nCol) {
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++)
-                if(sb.isEmpty(i))
-                    return i;
-        }
-        else
-            throw new NotImplementedException();
-
-        return -1;
-    }
-
-    @Override
-    public long getInMemorySize() {
-        return 8 + _data.estimateSizeInMemory();
-    }
-
-    public static long getInMemorySize(int numberValues, int numberColumns, double sparsity) {
-        return 8 + MatrixBlock.estimateSizeInMemory(numberValues, numberColumns, sparsity);
-    }
-
-    @Override
-    public double aggregate(double init, Builtin fn) {
-        if(fn.getBuiltinCode() == BuiltinCode.MAX)
-            return fn.execute(init, _data.max());
-        else if(fn.getBuiltinCode() == BuiltinCode.MIN)
-            return fn.execute(init, _data.min());
-        else
-            throw new NotImplementedException();
-    }
-
-    @Override
-    public double[] aggregateTuples(Builtin fn, int nCol) {
-        double[] ret = new double[_data.getNumRows()];
-        if(_data.isEmpty())
-            return ret;
-        else if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final double[] avals = sb.values(i);
-                    ret[i] = avals[apos];
-                    for(int j = apos + 1; j < alen; j++)
-                        ret[i] = fn.execute(ret[i], avals[j]);
-
-                    if(sb.size(i) < _data.getNumColumns())
-                        ret[i] = fn.execute(ret[i], 0);
-                }
-                else
-                    ret[i] = fn.execute(ret[i], 0);
-            }
-        }
-        else if(nCol == 1)
-            return _data.getDenseBlockValues();
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                ret[k] = values[off++];
-                for(int j = 1; j < _data.getNumColumns(); j++)
-                    ret[k] = fn.execute(ret[k], values[off++]);
-            }
-        }
-        return ret;
-    }
-
-    @Override
-    public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
-        if(_data.isEmpty()) {
-            for(int j = 0; j < colIndexes.length; j++) {
-                final int idx = colIndexes[j];
-                c[idx] = fn.execute(c[idx], 0);
-            }
-        }
-        else if(_data.isInSparseFormat()) {
-            MatrixBlock t = LibMatrixReorg.transposeInPlace(_data, 1);
-            if(!t.isInSparseFormat()) {
-                throw new NotImplementedException();
-            }
-            SparseBlock sbt = t.getSparseBlock();
-
-            for(int i = 0; i < _data.getNumColumns(); i++) {
-                final int idx = colIndexes[i];
-                if(!sbt.isEmpty(i)) {
-                    final int apos = sbt.pos(i);
-                    final int alen = sbt.size(i) + apos;
-                    final double[] avals = sbt.values(i);
-                    for(int j = apos; j < alen; j++)
-                        c[idx] = fn.execute(c[idx], avals[j]);
-                    if(avals.length != _data.getNumRows())
-                        c[idx] = fn.execute(c[idx], 0);
-                }
-                else
-                    c[idx] = fn.execute(c[idx], 0);
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final int idx = colIndexes[j];
-                    c[idx] = fn.execute(c[idx], values[off++]);
-                }
-            }
-        }
-    }
-
-    @Override
-    public int size() {
-        return (int) _data.getNonZeros();
-    }
-
-    @Override
-    public ADictionary apply(ScalarOperator op) {
-        MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
-        return new MatrixBlockDictionary(res);
-    }
-
-    @Override
-    public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
-        MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
-        final int lastRow = res.getNumRows();
-        MatrixBlock res2 = new MatrixBlock(lastRow + 1, res.getNumColumns(), true);
-        if(res.isEmpty()) {
-            for(int i = 0; i < numCols; i++)
-                res2.appendValue(lastRow, i, newVal);
-            return new MatrixBlockDictionary(res2);
-        }
-        else {
-            res.append(new MatrixBlock(1, numCols, newVal), res2, false);
-            return new MatrixBlockDictionary(res2);
-        }
-    }
-
-    @Override
-    public ADictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
-        MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
-        for(int i = 0; i < colIndexes.length; i++)
-            rowVector.quickSetValue(0, i, v[colIndexes[i]]);
-        MatrixBlock res = new MatrixBlock();
-        if(sparseSafe) {
-            rowVector.binaryOperations(op, _data, res);
-        }
-        else {
-            if(!_data.isInSparseFormat())
-                LOG.warn("Inefficient binary row op allocating Matrix multiple times");
-            MatrixBlock tmp = new MatrixBlock();
-            tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
-            rowVector.binaryOperations(op, tmp, res);
-
-        }
-        return new MatrixBlockDictionary(res);
-    }
-
-    @Override
-    public ADictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
-        MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
-        for(int i = 0; i < colIndexes.length; i++)
-            rowVector.quickSetValue(0, i, v[colIndexes[i]]);
-        MatrixBlock res = new MatrixBlock();
-        if(sparseSafe) {
-            _data.binaryOperations(op, rowVector, res);
-        }
-        else {
-            if(!_data.isInSparseFormat())
-                LOG.warn("Inefficient binary row op allocating Matrix multiple times");
-            MatrixBlock tmp = new MatrixBlock();
-            tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
-            tmp.binaryOperations(op, rowVector, res);
-        }
-        return new MatrixBlockDictionary(res);
-    }
-
-    @Override
-    public ADictionary clone() {
-        MatrixBlock ret = new MatrixBlock();
-        ret.copy(_data);
-        return new MatrixBlockDictionary(ret);
-    }
-
-    @Override
-    public ADictionary cloneAndExtend(int len) {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public boolean isLossy() {
-        return false;
-    }
-
-    @Override
-    public int getNumberOfValues(int ncol) {
-        return _data.getNumRows();
-    }
-
-    @Override
-    public double[] sumAllRowsToDouble(boolean square, int nrColumns) {
-        double[] ret = new double[_data.getNumRows()];
-
-        if(_data.isEmpty())
-            return ret;
-        else if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final double[] avals = sb.values(i);
-                    for(int j = apos; j < alen; j++) {
-                        ret[i] += (square) ? avals[j] * avals[j] : avals[j];
-                    }
-                }
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final double v = values[off++];
-                    ret[k] += (square) ? v * v : v;
-                }
-            }
-        }
-        return ret;
-    }
-
-    @Override
-    public double sumRow(int k, boolean square, int nrColumns) {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public double[] colSum(int[] counts, int nCol) {
-        if(_data.isEmpty())
-            return null;
-        double[] ret = new double[nCol];
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    // double tmpSum = 0;
-                    final int count = counts[i];
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final int[] aix = sb.indexes(i);
-                    final double[] avals = sb.values(i);
-                    for(int j = apos; j < alen; j++) {
-                        ret[aix[j]] += count * avals[j];
-                    }
-                }
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                final int countK = counts[k];
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final double v = values[off++];
-                    ret[j] += v * countK;
-                }
-            }
-        }
-        return ret;
-    }
-
-    @Override
-    public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
-        if(_data.isEmpty())
-            return;
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    // double tmpSum = 0;
-                    final int count = counts[i];
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final int[] aix = sb.indexes(i);
-                    final double[] avals = sb.values(i);
-                    for(int j = apos; j < alen; j++) {
-                        c[colIndexes[aix[j]]] += square ? count * avals[j] * avals[j] : count * avals[j];
-                    }
-                }
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                final int countK = counts[k];
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final double v = values[off++];
-                    c[colIndexes[j]] += square ? v * v * countK : v * countK;
-                }
-            }
-        }
-    }
-
-    @Override
-    public double sum(int[] counts, int ncol) {
-        double tmpSum = 0;
-        if(_data.isEmpty())
-            return tmpSum;
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    final int count = counts[i];
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final double[] avals = sb.values(i);
-                    for(int j = apos; j < alen; j++) {
-                        tmpSum += count * avals[j];
-                    }
-                }
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                final int countK = counts[k];
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final double v = values[off++];
-                    tmpSum += v * countK;
-                }
-            }
-        }
-        return tmpSum;
-    }
-
-    @Override
-    public double sumsq(int[] counts, int ncol) {
-        double tmpSum = 0;
-        if(_data.isEmpty())
-            return tmpSum;
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sb.isEmpty(i)) {
-                    final int count = counts[i];
-                    final int apos = sb.pos(i);
-                    final int alen = sb.size(i) + apos;
-                    final double[] avals = sb.values(i);
-                    for(int j = apos; j < alen; j++) {
-                        tmpSum += count * avals[j] * avals[j];
-                    }
-                }
-            }
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int k = 0; k < _data.getNumRows(); k++) {
-                final int countK = counts[k];
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    final double v = values[off++];
-                    tmpSum += v * v * countK;
-                }
-            }
-        }
-        return tmpSum;
-    }
-
-    @Override
-    public String getString(int colIndexes) {
-        return _data.toString();
-    }
-
-    @Override
-    public void addMaxAndMin(double[] ret, int[] colIndexes) {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
-        MatrixBlock retBlock = _data.slice(0, _data.getNumRows() - 1, idxStart, idxEnd - 1);
-        return new MatrixBlockDictionary(retBlock);
-    }
-
-    @Override
-    public ADictionary reExpandColumns(int max) {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public boolean containsValue(double pattern) {
-        return _data.containsValue(pattern);
-    }
-
-    @Override
-    public long getNumberNonZeros(int[] counts, int nCol) {
-        if(_data.isEmpty())
-            return 0;
-
-        long nnz = 0;
-        if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++)
-                if(!sb.isEmpty(i))
-                    nnz += sb.size(i) * counts[i];
-
-        }
-        else {
-            double[] values = _data.getDenseBlockValues();
-            int off = 0;
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                int countThisTuple = 0;
-                for(int j = 0; j < _data.getNumColumns(); j++) {
-                    double v = values[off++];
-                    if(v != 0)
-                        countThisTuple++;
-                }
-                nnz += countThisTuple * counts[i];
-            }
-        }
-        return nnz;
-    }
-
-    @Override
-    public long getNumberNonZerosContained() {
-        throw new NotImplementedException();
-    }
-
-    @Override
-    public void addToEntry(Dictionary d, int fr, int to, int nCol) {
-        double[] v = d.getValues();
-        if(_data.isEmpty())
-            return;
-        else if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            if(sb.isEmpty(fr))
-                return;
-            final int apos = sb.pos(fr);
-            final int alen = sb.size(fr) + apos;
-            final int[] aix = sb.indexes(fr);
-            final double[] avals = sb.values(fr);
-            final int offsetTo = nCol * to;
-            for(int j = apos; j < alen; j++) {
-                v[offsetTo + aix[j]] += avals[j];
-            }
-        }
-        else {
-            final int sf = nCol * fr; // start from
-            final int ef = sf + nCol; // end from
-            final double[] thisV = _data.getDenseBlockValues();
-            for(int i = sf, j = nCol * to; i < ef; i++, j++) {
-                v[j] += thisV[i];
-            }
-        }
-    }
-
-    @Override
-    public double[] getTuple(int index, int nCol) {
-        if(_data.isEmpty() || index >= _data.getNumRows())
-            return null;
-        else if(_data.isInSparseFormat()) {
-            SparseBlock sb = _data.getSparseBlock();
-            if(sb.isEmpty(index))
-                return null;
-            double[] tuple = new double[nCol];
-            final int apos = sb.pos(index);
-            final int alen = sb.size(index) + apos;
-            final int[] aix = sb.indexes(index);
-            final double[] avals = sb.values(index);
-            for(int j = apos; j < alen; j++) {
-                tuple[aix[j]] = avals[j];
-            }
-            return tuple;
-        }
-        else {
-            double[] tuple = new double[nCol];
-            double[] values = _data.getDenseBlockValues();
-            int offset = index * nCol;
-            for(int i = 0; i < nCol; i++, offset++)
-                tuple[i] = values[offset];
-            return tuple;
-        }
-    }
-
-    @Override
-    public ADictionary subtractTuple(double[] tuple) {
-        DenseBlockFP64 b = new DenseBlockFP64(new int[] {1, tuple.length}, tuple);
-        MatrixBlock rowVector = new MatrixBlock(1, tuple.length, b);
-        MatrixBlock res = new MatrixBlock(_data.getNumColumns(), _data.getNumRows(), _data.isInSparseFormat());
-        _data.binaryOperations(new BinaryOperator(Minus.getMinusFnObject()), rowVector, res);
-        return new MatrixBlockDictionary(res);
-    }
-
-    @Override
-    public MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol) {
-        // Simply return this.
-        return this;
-    }
-
-    @Override
-    public String toString() {
-        return "MatrixBlock Dictionary :" + _data.toString();
-    }
-
-    @Override
-    public ADictionary scaleTuples(int[] scaling, int nCol) {
-        if(_data.isEmpty()) {
-            throw new NotImplementedException("could return null here? or empty DictionaryMatrixBlock...");
-        }
-        else if(_data.isInSparseFormat()) {
-            MatrixBlock retBlock = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), true);
-            retBlock.allocateSparseRowsBlock(true);
-            SparseBlock sbRet = retBlock.getSparseBlock();
-            SparseBlock sbThis = _data.getSparseBlock();
-            for(int i = 0; i < _data.getNumRows(); i++) {
-                if(!sbThis.isEmpty(i)) {
-                    sbRet.set(i, sbThis.get(i), true);
-
-                    final int count = scaling[i];
-                    final int apos = sbRet.pos(i);
-                    final int alen = sbRet.size(i) + apos;
-                    final double[] avals = sbRet.values(i);
-                    for(int j = apos; j < alen; j++)
-                        avals[j] = count * avals[j];
-                }
-            }
-            retBlock.setNonZeros(_data.getNonZeros());
-            return new MatrixBlockDictionary(retBlock);
-        }
-        else {
-            final double[] _values = _data.getDenseBlockValues();
-            final double[] scaledValues = new double[_values.length];
-            int off = 0;
-            for(int tuple = 0; tuple < _values.length / nCol; tuple++) {
-                final int scale = scaling[tuple];
-                for(int v = 0; v < nCol; v++) {
-                    scaledValues[off] = _values[off] * scale;
-                    off++;
-                }
-            }
-            DenseBlockFP64 db = new DenseBlockFP64(new int[] {_data.getNumRows(), _data.getNumColumns()}, scaledValues);
-            MatrixBlock retBlock = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), db);
-            retBlock.setNonZeros(_data.getNonZeros());
-            return new MatrixBlockDictionary(retBlock);
-        }
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-        out.writeByte(DictionaryFactory.Type.MATRIX_BLOCK_DICT.ordinal());
-        _data.write(out);
-    }
-
-    public static MatrixBlockDictionary read(DataInput in) throws IOException {
-        MatrixBlock ret = new MatrixBlock();
-        ret.readFields(in);
-        return new MatrixBlockDictionary(ret);
-    }
-
-    @Override
-    public long getExactSizeOnDisk() {
-        return 1 + _data.getExactSizeOnDisk();
-    }
+	private MatrixBlock _data;
+
+	public MatrixBlockDictionary(MatrixBlock data) {
+		_data = data;
+	}
+
+	public MatrixBlock getMatrixBlock() {
+		return _data;
+	}
+
+	@Override
+	public double[] getValues() {
+		throw new DMLCompressionException("Get Values should not be called when you have a MatrixBlockDictionary");
+	}
+
+	@Override
+	public double getValue(int i) {
+		final int nCol = _data.getNumColumns();
+		final int row = i / nCol;
+		if(row > _data.getNumRows())
+			return 0;
+		final int col = i % nCol;
+		return _data.quickGetValue(row, col);
+	}
+
+	@Override
+	public long getInMemorySize() {
+		return 8 + _data.estimateSizeInMemory();
+	}
+
+	public static long getInMemorySize(int numberValues, int numberColumns, double sparsity) {
+		return 8 + MatrixBlock.estimateSizeInMemory(numberValues, numberColumns, sparsity);
+	}
+
+	@Override
+	public double aggregate(double init, Builtin fn) {
+		if(fn.getBuiltinCode() == BuiltinCode.MAX)
+			return fn.execute(init, _data.max());
+		else if(fn.getBuiltinCode() == BuiltinCode.MIN)
+			return fn.execute(init, _data.min());
+		else
+			throw new NotImplementedException();
+	}
+
+	@Override
+	public double[] aggregateTuples(Builtin fn, int nCol) {
+		double[] ret = new double[_data.getNumRows()];
+		if(_data.isEmpty())
+			return ret;
+		else if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final double[] avals = sb.values(i);
+					ret[i] = avals[apos];
+					for(int j = apos + 1; j < alen; j++)
+						ret[i] = fn.execute(ret[i], avals[j]);
+
+					if(sb.size(i) < _data.getNumColumns())
+						ret[i] = fn.execute(ret[i], 0);
+				}
+				else
+					ret[i] = fn.execute(ret[i], 0);
+			}
+		}
+		else if(nCol == 1)
+			return _data.getDenseBlockValues();
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				ret[k] = values[off++];
+				for(int j = 1; j < _data.getNumColumns(); j++)
+					ret[k] = fn.execute(ret[k], values[off++]);
+			}
+		}
+		return ret;
+	}
+
+	@Override
+	public void aggregateCols(double[] c, Builtin fn, int[] colIndexes) {
+		if(_data.isEmpty()) {
+			for(int j = 0; j < colIndexes.length; j++) {
+				final int idx = colIndexes[j];
+				c[idx] = fn.execute(c[idx], 0);
+			}
+		}
+		else if(_data.isInSparseFormat()) {
+			MatrixBlock t = LibMatrixReorg.transposeInPlace(_data, 1);
+			if(!t.isInSparseFormat()) {
+				throw new NotImplementedException();
+			}
+			SparseBlock sbt = t.getSparseBlock();
+
+			for(int i = 0; i < _data.getNumColumns(); i++) {
+				final int idx = colIndexes[i];
+				if(!sbt.isEmpty(i)) {
+					final int apos = sbt.pos(i);
+					final int alen = sbt.size(i) + apos;
+					final double[] avals = sbt.values(i);
+					for(int j = apos; j < alen; j++)
+						c[idx] = fn.execute(c[idx], avals[j]);
+					if(alen != _data.getNumRows())
+						c[idx] = fn.execute(c[idx], 0);
+				}
+				else
+					c[idx] = fn.execute(c[idx], 0);
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final int idx = colIndexes[j];
+					c[idx] = fn.execute(c[idx], values[off++]);
+				}
+			}
+		}
+	}
+
+	@Override
+	public ADictionary apply(ScalarOperator op) {
+		MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
+		return new MatrixBlockDictionary(res);
+	}
+
+	@Override
+	public ADictionary applyScalarOp(ScalarOperator op, double newVal, int numCols) {
+		MatrixBlock res = _data.scalarOperations(op, new MatrixBlock());
+		final int lastRow = res.getNumRows();
+		MatrixBlock res2 = new MatrixBlock(lastRow + 1, res.getNumColumns(), true);
+		if(res.isEmpty()) {
+			for(int i = 0; i < numCols; i++)
+				res2.appendValue(lastRow, i, newVal);
+			return new MatrixBlockDictionary(res2);
+		}
+		else {
+			res.append(new MatrixBlock(1, numCols, newVal), res2, false);
+			return new MatrixBlockDictionary(res2);
+		}
+	}
+
+	@Override
+	public ADictionary applyBinaryRowOpLeft(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+		MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
+		for(int i = 0; i < colIndexes.length; i++)
+			rowVector.quickSetValue(0, i, v[colIndexes[i]]);
+		MatrixBlock res = new MatrixBlock();
+		if(sparseSafe)
+			rowVector.binaryOperations(op, _data, res);
+		else {
+			MatrixBlock tmp = new MatrixBlock();
+			tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
+			rowVector.binaryOperations(op, tmp, res);
+		}
+		return new MatrixBlockDictionary(res);
+	}
+
+	@Override
+	public ADictionary applyBinaryRowOpRight(BinaryOperator op, double[] v, boolean sparseSafe, int[] colIndexes) {
+		MatrixBlock rowVector = new MatrixBlock(1, colIndexes.length, false);
+		for(int i = 0; i < colIndexes.length; i++)
+			rowVector.quickSetValue(0, i, v[colIndexes[i]]);
+		MatrixBlock res = new MatrixBlock();
+		if(sparseSafe) {
+			_data.binaryOperations(op, rowVector, res);
+		}
+		else {
+			if(!_data.isInSparseFormat())
+				LOG.warn("Inefficient binary row op allocating Matrix multiple times");
+			MatrixBlock tmp = new MatrixBlock();
+			tmp = _data.append(new MatrixBlock(1, _data.getNumColumns(), 0), tmp, false);
+			tmp.binaryOperations(op, rowVector, res);
+		}
+		return new MatrixBlockDictionary(res);
+	}
+
+	@Override
+	public ADictionary clone() {
+		MatrixBlock ret = new MatrixBlock();
+		ret.copy(_data);
+		return new MatrixBlockDictionary(ret);
+	}
+
+	@Override
+	public ADictionary cloneAndExtend(int len) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public boolean isLossy() {
+		return false;
+	}
+
+	@Override
+	public int getNumberOfValues(int ncol) {
+		return _data.getNumRows();
+	}
+
+	@Override
+	public double[] sumAllRowsToDouble(boolean square, int nrColumns) {
+		double[] ret = new double[_data.getNumRows()];
+
+		if(_data.isEmpty())
+			return ret;
+		else if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final double[] avals = sb.values(i);
+					for(int j = apos; j < alen; j++) {
+						ret[i] += (square) ? avals[j] * avals[j] : avals[j];
+					}
+				}
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final double v = values[off++];
+					ret[k] += (square) ? v * v : v;
+				}
+			}
+		}
+		return ret;
+	}
+
+	@Override
+	public double sumRow(int k, boolean square, int nrColumns) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public double[] colSum(int[] counts, int nCol) {
+		if(_data.isEmpty())
+			return null;
+		double[] ret = new double[nCol];
+		if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					// double tmpSum = 0;
+					final int count = counts[i];
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final int[] aix = sb.indexes(i);
+					final double[] avals = sb.values(i);
+					for(int j = apos; j < alen; j++) {
+						ret[aix[j]] += count * avals[j];
+					}
+				}
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				final int countK = counts[k];
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final double v = values[off++];
+					ret[j] += v * countK;
+				}
+			}
+		}
+		return ret;
+	}
+
+	@Override
+	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
+		if(_data.isEmpty())
+			return;
+		if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					// double tmpSum = 0;
+					final int count = counts[i];
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final int[] aix = sb.indexes(i);
+					final double[] avals = sb.values(i);
+					for(int j = apos; j < alen; j++) {
+						c[colIndexes[aix[j]]] += square ? count * avals[j] * avals[j] : count * avals[j];
+					}
+				}
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				final int countK = counts[k];
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final double v = values[off++];
+					c[colIndexes[j]] += square ? v * v * countK : v * countK;
+				}
+			}
+		}
+	}
+
+	@Override
+	public double sum(int[] counts, int ncol) {
+		double tmpSum = 0;
+		if(_data.isEmpty())
+			return tmpSum;
+		if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					final int count = counts[i];
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final double[] avals = sb.values(i);
+					for(int j = apos; j < alen; j++) {
+						tmpSum += count * avals[j];
+					}
+				}
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				final int countK = counts[k];
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final double v = values[off++];
+					tmpSum += v * countK;
+				}
+			}
+		}
+		return tmpSum;
+	}
+
+	@Override
+	public double sumsq(int[] counts, int ncol) {
+		double tmpSum = 0;
+		if(_data.isEmpty())
+			return tmpSum;
+		if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sb.isEmpty(i)) {
+					final int count = counts[i];
+					final int apos = sb.pos(i);
+					final int alen = sb.size(i) + apos;
+					final double[] avals = sb.values(i);
+					for(int j = apos; j < alen; j++) {
+						tmpSum += count * avals[j] * avals[j];
+					}
+				}
+			}
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int k = 0; k < _data.getNumRows(); k++) {
+				final int countK = counts[k];
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					final double v = values[off++];
+					tmpSum += v * v * countK;
+				}
+			}
+		}
+		return tmpSum;
+	}
+
+	@Override
+	public String getString(int colIndexes) {
+		return _data.toString();
+	}
+
+	@Override
+	public void addMaxAndMin(double[] ret, int[] colIndexes) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
+		MatrixBlock retBlock = _data.slice(0, _data.getNumRows() - 1, idxStart, idxEnd - 1);
+		return new MatrixBlockDictionary(retBlock);
+	}
+
+	@Override
+	public ADictionary reExpandColumns(int max) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public boolean containsValue(double pattern) {
+		return _data.containsValue(pattern);
+	}
+
+	@Override
+	public long getNumberNonZeros(int[] counts, int nCol) {
+		if(_data.isEmpty())
+			return 0;
+
+		long nnz = 0;
+		if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++)
+				if(!sb.isEmpty(i))
+					nnz += sb.size(i) * counts[i];
+
+		}
+		else {
+			double[] values = _data.getDenseBlockValues();
+			int off = 0;
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				int countThisTuple = 0;
+				for(int j = 0; j < _data.getNumColumns(); j++) {
+					double v = values[off++];
+					if(v != 0)
+						countThisTuple++;
+				}
+				nnz += countThisTuple * counts[i];
+			}
+		}
+		return nnz;
+	}
+
+	@Override
+	public void addToEntry(Dictionary d, int fr, int to, int nCol) {
+		double[] v = d.getValues();
+		if(_data.isEmpty())
+			return;
+		else if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			if(sb.isEmpty(fr))
+				return;
+			final int apos = sb.pos(fr);
+			final int alen = sb.size(fr) + apos;
+			final int[] aix = sb.indexes(fr);
+			final double[] avals = sb.values(fr);
+			final int offsetTo = nCol * to;
+			for(int j = apos; j < alen; j++) {
+				v[offsetTo + aix[j]] += avals[j];
+			}
+		}
+		else {
+			final int sf = nCol * fr; // start from
+			final int ef = sf + nCol; // end from
+			final double[] thisV = _data.getDenseBlockValues();
+			for(int i = sf, j = nCol * to; i < ef; i++, j++) {
+				v[j] += thisV[i];
+			}
+		}
+	}
+
+	@Override
+	public double[] getTuple(int index, int nCol) {
+		if(_data.isEmpty() || index >= _data.getNumRows())
+			return null;
+		else if(_data.isInSparseFormat()) {
+			SparseBlock sb = _data.getSparseBlock();
+			if(sb.isEmpty(index))
+				return null;
+			double[] tuple = new double[nCol];
+			final int apos = sb.pos(index);
+			final int alen = sb.size(index) + apos;
+			final int[] aix = sb.indexes(index);
+			final double[] avals = sb.values(index);
+			for(int j = apos; j < alen; j++) {
+				tuple[aix[j]] = avals[j];
+			}
+			return tuple;
+		}
+		else {
+			double[] tuple = new double[nCol];
+			double[] values = _data.getDenseBlockValues();
+			int offset = index * nCol;
+			for(int i = 0; i < nCol; i++, offset++)
+				tuple[i] = values[offset];
+			return tuple;
+		}
+	}
+
+	@Override
+	public ADictionary subtractTuple(double[] tuple) {
+		DenseBlockFP64 b = new DenseBlockFP64(new int[] {1, tuple.length}, tuple);
+		MatrixBlock rowVector = new MatrixBlock(1, tuple.length, b);
+		MatrixBlock res = new MatrixBlock(_data.getNumColumns(), _data.getNumRows(), _data.isInSparseFormat());
+		_data.binaryOperations(new BinaryOperator(Minus.getMinusFnObject()), rowVector, res);
+		return new MatrixBlockDictionary(res);
+	}
+
+	@Override
+	public MatrixBlockDictionary getAsMatrixBlockDictionary(int nCol) {
+		// Simply return this.
+		return this;
+	}
+
+	@Override
+	public String toString() {
+		return "MatrixBlock Dictionary :" + _data.toString();
+	}
+
+	@Override
+	public ADictionary scaleTuples(int[] scaling, int nCol) {
+		if(_data.isEmpty()) {
+			throw new NotImplementedException("could return null here? or empty DictionaryMatrixBlock...");
+		}
+		else if(_data.isInSparseFormat()) {
+			MatrixBlock retBlock = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), true);
+			retBlock.allocateSparseRowsBlock(true);
+			SparseBlock sbRet = retBlock.getSparseBlock();
+			SparseBlock sbThis = _data.getSparseBlock();
+			for(int i = 0; i < _data.getNumRows(); i++) {
+				if(!sbThis.isEmpty(i)) {
+					sbRet.set(i, sbThis.get(i), true);
+
+					final int count = scaling[i];
+					final int apos = sbRet.pos(i);
+					final int alen = sbRet.size(i) + apos;
+					final double[] avals = sbRet.values(i);
+					for(int j = apos; j < alen; j++)
+						avals[j] = count * avals[j];
+				}
+			}
+			retBlock.setNonZeros(_data.getNonZeros());
+			return new MatrixBlockDictionary(retBlock);
+		}
... 4520 lines suppressed ...

[systemds] 07/07: [SYSTEMDS-2746] CLA ReplaceOperation

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

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

commit d430902eb0a55f5369f8a0f237de340710f3de48
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Mon May 31 17:27:09 2021 +0200

    [SYSTEMDS-2746] CLA ReplaceOperation
    
    Closes #1276
---
 .../runtime/compress/CompressedMatrixBlock.java    |  50 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |  21 +-
 .../runtime/compress/colgroup/ColGroupConst.java   | 173 ------
 .../runtime/compress/colgroup/ColGroupDDC.java     | 299 ----------
 .../runtime/compress/colgroup/ColGroupEmpty.java   |  31 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |   8 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     | 619 ---------------------
 .../runtime/compress/colgroup/ColGroupRLE.java     | 525 -----------------
 .../runtime/compress/colgroup/ColGroupSDC.java     | 297 +---------
 .../compress/colgroup/ColGroupSDCSingle.java       | 315 +----------
 .../compress/colgroup/ColGroupSDCSingleZeros.java  | 185 +-----
 .../compress/colgroup/ColGroupSDCZeros.java        | 211 +------
 .../compress/colgroup/ColGroupUncompressed.java    |  44 +-
 .../runtime/compress/colgroup/ColGroupValue.java   | 118 ++--
 .../compress/colgroup/dictionary/ADictionary.java  |  14 +
 .../compress/colgroup/dictionary/Dictionary.java   |  18 +
 .../colgroup/dictionary/MatrixBlockDictionary.java |  10 +
 .../compress/colgroup/dictionary/QDictionary.java  |   5 +
 .../runtime/compress/lib/CLALibLeftMultBy.java     |   3 +-
 .../sysds/runtime/matrix/data/MatrixBlock.java     |  29 +-
 .../component/compress/CompressedMatrixTest.java   |  26 +-
 21 files changed, 234 insertions(+), 2767 deletions(-)

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 aff61b8..6d1d02b 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -78,7 +78,6 @@ import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysds.runtime.instructions.cp.ScalarObject;
 import org.apache.sysds.runtime.instructions.spark.data.IndexedMatrixValue;
 import org.apache.sysds.runtime.matrix.data.CTableMap;
-import org.apache.sysds.runtime.matrix.data.LibMatrixBincell;
 import org.apache.sysds.runtime.matrix.data.LibMatrixDatagen;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -476,22 +475,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	public MatrixBlock chainMatrixMultOperations(MatrixBlock v, MatrixBlock w, MatrixBlock out, ChainType ctype,
 		int k) {
 
-		if(this.getNumColumns() != v.getNumRows())
-			throw new DMLRuntimeException(
-				"Dimensions mismatch on mmchain operation (" + this.getNumColumns() + " != " + v.getNumRows() + ")");
-		if(v.getNumColumns() != 1)
-			throw new DMLRuntimeException(
-				"Invalid input vector (column vector expected, but ncol=" + v.getNumColumns() + ")");
-		if(w != null && w.getNumColumns() != 1)
-			throw new DMLRuntimeException(
-				"Invalid weight vector (column vector expected, but ncol=" + w.getNumColumns() + ")");
+		checkMMChain(ctype, v, w);
 
 		// multi-threaded MMChain of single uncompressed ColGroup
-		if(isSingleUncompressedGroup()) {
+		if(isSingleUncompressedGroup())
 			return ((ColGroupUncompressed) _colGroups.get(0)).getData().chainMatrixMultOperations(v, w, out, ctype, k);
-		}
-
-		// Timing time = LOG.isDebugEnabled() ? new Timing(true) : null;
 
 		// prepare result
 		if(out != null)
@@ -505,16 +493,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 		BinaryOperator bop = new BinaryOperator(Multiply.getMultiplyFnObject());
 
-		// compute matrix mult
-
-		// boolean tryOverlapOutput = v.getNumColumns() > _colGroups.size();
 		MatrixBlock tmp = CLALibRightMultBy.rightMultByMatrix(this, v, null, k, true);
 
 		if(ctype == ChainType.XtwXv) {
-			if(tmp instanceof CompressedMatrixBlock)
-				tmp = CLALibBinaryCellOp.binaryOperations(bop, (CompressedMatrixBlock) tmp, w, null);
-			else
-				LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
+			// if(tmp instanceof CompressedMatrixBlock)
+			tmp = CLALibBinaryCellOp.binaryOperations(bop, (CompressedMatrixBlock) tmp, w, null);
+			// else
+			// LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
 		}
 
 		if(tmp instanceof CompressedMatrixBlock)
@@ -687,10 +672,25 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 	@Override
 	public MatrixBlock replaceOperations(MatrixValue result, double pattern, double replacement) {
-		printDecompressWarning("replaceOperations " + pattern + "  -> " + replacement);
-		LOG.error("Overlapping? : " + isOverlapping() + " If not then wite a proper replace command");
-		MatrixBlock tmp = getUncompressed(this);
-		return tmp.replaceOperations(result, pattern, replacement);
+		if(isOverlapping()) {
+			printDecompressWarning("replaceOperations " + pattern + "  -> " + replacement);
+			MatrixBlock tmp = getUncompressed(this);
+			return tmp.replaceOperations(result, pattern, replacement);
+		}
+		else {
+
+			CompressedMatrixBlock ret = new CompressedMatrixBlock(getNumRows(), getNumColumns());
+			final List<AColGroup> prev = getColGroups();
+			final int colGroupsLength = prev.size();
+			final List<AColGroup> retList = new ArrayList<>(colGroupsLength);
+			for(int i = 0; i < colGroupsLength; i++) {
+				retList.add(prev.get(i).replace(pattern, replacement));
+			}
+			ret.allocateColGroupList(retList);
+			ret.recomputeNonZeros();
+			ret.setOverlapping(false); // since the other if checks it
+			return ret;
+		}
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
index 0e68d0a..6b3ffd7 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
@@ -188,12 +188,12 @@ public abstract class AColGroup implements Serializable {
 	public abstract void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT);
 
 	// /**
-	//  * Decompress the contents of this column group into uncompressed packed columns
-	//  * 
-	//  * @param target          a dense matrix block. The block must have enough space to hold the contents of this column
-	//  *                        group.
-	//  * @param colIndexTargets array that maps column indices in the original matrix block to columns of target.
-	//  */
+	// * Decompress the contents of this column group into uncompressed packed columns
+	// *
+	// * @param target a dense matrix block. The block must have enough space to hold the contents of this column
+	// * group.
+	// * @param colIndexTargets array that maps column indices in the original matrix block to columns of target.
+	// */
 	// public abstract void decompressToBlock(MatrixBlock target, int[] colIndexTargets);
 
 	/**
@@ -524,6 +524,15 @@ public abstract class AColGroup implements Serializable {
 	 */
 	public abstract long getNumberNonZeros();
 
+	/**
+	 * Make a copy of the column group values, and replace all values that match pattern with replacement value.
+	 * 
+	 * @param pattern The value to look for
+	 * @param replace The value to replace the other value with
+	 * @return A new Column Group, reusing the index structure but with new values.
+	 */
+	public abstract AColGroup replace(double pattern, double replace);
+
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
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 019c6e1..ab01e27 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
@@ -22,9 +22,7 @@ package org.apache.sysds.runtime.compress.colgroup;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
-import org.apache.sysds.runtime.compress.colgroup.dictionary.Dictionary;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -107,69 +105,11 @@ public class ColGroupConst extends ColGroupValue {
 		throw new NotImplementedException();
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	int ncol = getNumCols();
-	// 	double[] values = getValues();
-	// 	for(int i = 0; i < _numRows; i++)
-	// 		for(int colIx = 0; colIx < ncol; colIx++) {
-	// 			int origMatrixColIx = _colIndexes[colIx];
-	// 			int col = colIndexTargets[origMatrixColIx];
-	// 			double cellVal = values[colIx];
-	// 			target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
-	// 		}
-
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colPos) {
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double v = _dict.getValue(colPos);
-	// 	if(v != 0)
-	// 		for(int i = 0; i < c.length; i++)
-	// 			c[i] += v;
-
-	// 	target.setNonZeros(_numRows);
-
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colPos, int rl, int ru) {
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double v = _dict.getValue(colPos);
-	// 	final int length = ru - rl;
-	// 	if(v != 0)
-	// 		for(int i = 0; i < length; i++)
-	// 			c[i] += v;
-
-	// 	target.setNonZeros(_numRows);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colPos, int rl, int ru) {
-	// 	double v = _dict.getValue(colPos);
-	// 	final int length = ru - rl;
-	// 	if(v != 0)
-	// 		for(int i = 0; i < length; i++)
-	// 			c[i] += v;
-
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		return _dict.getValue(Arrays.binarySearch(_colIndexes, c));
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// return new double[] {preAggregateSingle(a, row)};
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// return new double[] {preAggregateSparseSingle(sb, row)};
-	// }
-
 	@Override
 	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
@@ -204,54 +144,6 @@ public class ColGroupConst extends ColGroupValue {
 		}
 	}
 
-
-	// public double preAggregateSparseSingle(SparseBlock sb, int row) {
-	// double v = 0;
-	// double[] sparseV = sb.values(row);
-	// for(int i = sb.pos(row); i < sb.pos(row) + sb.size(row); i++) {
-	// v += sparseV[i];
-	// }
-	// return v;
-	// }
-
-	// private double preAggregateSingle(double[] a, int row) {
-	// double vals = 0;
-	// for(int off = _numRows * row; off < _numRows * row + _numRows; off++)
-	// vals += a[off];
-	// return vals;
-	// }
-
-	// @Override
-	// public void leftMultByMatrix(MatrixBlock a, MatrixBlock c, int rl, int ru) {
-	// final double[] cV = c.getDenseBlockValues();
-	// final double[] values = getValues();
-	// if(values == null || a.isEmpty())
-	// return;
-	// else if(a.isInSparseFormat()) {
-	// SparseBlock sb = a.getSparseBlock();
-	// for(int i = rl; i < ru; i++) {
-
-	// if(!sb.isEmpty(i)) {
-	// double v = preAggregateSparseSingle(sb, i);
-	// int offC = i * c.getNumColumns();
-	// for(int j = 0; j < _colIndexes.length; j++)
-	// cV[offC + _colIndexes[j]] += v * values[j];
-
-	// }
-	// }
-	// }
-	// else {
-	// double[] aV = a.getDenseBlockValues();
-	// for(int i = rl; i < ru; i++) {
-	// double preAggVals = preAggregateSingle(aV, i);
-	// int offC = i * c.getNumColumns();
-	// for(int j = 0; j < _colIndexes.length; j++)
-	// cV[offC + _colIndexes[j]] += preAggVals * values[j];
-
-	// }
-	// }
-	// }
-
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		return new ColGroupConst(_colIndexes, _numRows, applyScalarOp(op));
@@ -276,71 +168,6 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public int getIndexStructureHash() {
-		throw new NotImplementedException("This function should not be called");
-	}
-
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	return new ArrPreAggregate(lhs.getCounts());
-	// }
-
-	@Override
-	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
-		throw new DMLCompressionException("Does not make sense to call this");
-	}
-
-	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
-		throw new DMLCompressionException("Does not make sense to call this");
-	}
-
-	@Override
-	public Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
-		throw new DMLCompressionException("Does not make sense to call this");
-	}
-
-	@Override
-	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
-		throw new DMLCompressionException("Does not make sense to call this");
-	}
-
-	@Override
-	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
-		throw new DMLCompressionException("Does not make sense to call this");
-	}
-
-	@Override
 	protected boolean sameIndexStructure(ColGroupCompressed that) {
 		return that instanceof ColGroupEmpty || that instanceof ColGroupConst;
 	}
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 6cdbe4e..0caa4b7 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
@@ -93,68 +93,6 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	int ncol = getNumCols();
-	// 	double[] dictionary = getValues();
-	// 	for(int i = 0; i < _numRows; i++) {
-	// 		int rowIndex = _data.getIndex(i) * ncol;
-	// 		for(int colIx = 0; colIx < ncol; colIx++) {
-	// 			int origMatrixColIx = _colIndexes[colIx];
-	// 			int col = colIndexTargets[origMatrixColIx];
-	// 			double cellVal = dictionary[rowIndex + colIx];
-	// 			target.quickSetValue(i, col, target.quickGetValue(i, col) + cellVal);
-	// 		}
-
-	// 	}
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	int ncol = getNumCols();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double[] values = getValues();
-	// 	int nnz = 0;
-	// 	for(int i = 0; i < _numRows; i++) {
-	// 		int index = _data.getIndex(i);
-	// 		if(index < getNumValues())
-	// 			nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
-	// 		else
-	// 			nnz++;
-
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	int ncol = getNumCols();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double[] values = getValues();
-	// 	final int numValues = getNumValues();
-	// 	int nnz = 0;
-	// 	for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-	// 		int index = _data.getIndex(r);
-	// 		if(index < numValues)
-	// 			nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
-	// 		else
-	// 			nnz++;
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	int ncol = getNumCols();
-	// 	double[] values = getValues();
-	// 	final int numValues = getNumValues();
-	// 	for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-	// 		int index = _data.getIndex(r);
-	// 		if(index < numValues)
-	// 			c[i] += values[(index) * ncol + colpos];
-	// 	}
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		// find local column index
@@ -217,31 +155,6 @@ public class ColGroupDDC extends ColGroupValue {
 		return counts;
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// double[] vals = allocDVector(getNumValues(), true);
-	// if(row > 0)
-	// for(int i = 0, off = _numRows * row; i < _numRows; i++, off++)
-	// vals[_data.getIndex(i)] += a[off];
-	// else
-	// for(int i = 0; i < _numRows; i++)
-	// vals[_data.getIndex(i)] += a[i];
-
-	// return vals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-
-	// double[] vals = allocDVector(getNumValues(), true);
-	// int[] indexes = sb.indexes(row);
-	// double[] sparseV = sb.values(row);
-	// for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++)
-	// vals[_data.getIndex(indexes[i])] += sparseV[i];
-	// return vals;
-
-	// }
-
 	@Override
 	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
@@ -277,217 +190,6 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 	}
 
-	// @Override
-	// public MatrixBlock preAggregate(MatrixBlock m, int rl, int ru) {
-
-	// final int retCols = getNumValues();
-	// final int retRows = ru - rl;
-	// final double[] vals = allocDVector(retRows * retCols, true);
-	// final DenseBlock retB = new DenseBlockFP64(new int[] {retRows, retCols}, vals);
-	// final MatrixBlock ret = new MatrixBlock(retRows, retCols, retB);
-
-	// final double[] mV = m.getDenseBlockValues();
-
-	// ret.setNonZeros(retRows * retCols);
-	// for(int k = rl; k < ru; k++) {
-	// final int offT = ret.getNumColumns() * k;
-	// final int offM = m.getNumColumns() * k;
-	// for(int i = 0; i < _numRows; i++) {
-	// int index = _data.getIndex(i);
-	// vals[offT + index] += mV[offM + i];
-	// }
-	// }
-	// return ret;
-	// }
-
-	/**
-	 * Generic get value for byte-length-agnostic access to first column.
-	 * 
-	 * @param r      Global row index
-	 * @param values The values contained in the column groups dictionary
-	 * @return value
-	 */
-	protected double getData(int r, double[] values) {
-		int index = _data.getIndex(r);
-		return (index < values.length) ? values[index] : 0.0;
-	}
-
-	/**
-	 * Generic get value for byte-length-agnostic access.
-	 * 
-	 * @param r      Global row index
-	 * @param colIx  Local column index
-	 * @param values The values contained in the column groups dictionary
-	 * @return value
-	 */
-	protected double getData(int r, int colIx, double[] values) {
-		int index = _data.getIndex(r) * _colIndexes.length + colIx;
-		return (index < values.length) ? values[index] : 0.0;
-	}
-
-	/**
-	 * Generic set value for byte-length-agnostic write of encoded value.
-	 * 
-	 * @param r    global row index
-	 * @param code encoded value
-	 */
-	protected void setData(int r, int code) {
-		_data.set(r, code);
-	}
-
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	// int[] m = _data.materializeMultiplied(nCol);
-	// 	for(int i = 0; i < this._numRows; i++)
-	// 		ag.increment(lhs._data.getIndex(i) + this._data.getIndex(i) * nCol);
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	AIterator lIt = lhs._indexes.getIterator();
-	// 	final int offsetToDefault = nCol - 1;
-
-	// 	int i = 0;
-
-	// 	int col;
-	// 	for(; i < this._numRows && lIt.hasNext(); i++) {
-	// 		int row = this._data.getIndex(i);
-	// 		if(lIt.value() == i)
-	// 			col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
-
-	// 		else
-	// 			col = offsetToDefault;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-	// 	col = offsetToDefault;
-	// 	for(; i < this._numRows; i++) {
-	// 		int row = this._data.getIndex(i);
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-
-	// 	int i = 0;
-
-	// 	int col;
-	// 	for(; i < this._numRows && lIt.hasNext(); i++) {
-	// 		int row = this._data.getIndex(i);
-	// 		if(lIt.value() == i) {
-	// 			col = 1;
-	// 			lIt.next();
-	// 		}
-	// 		else
-	// 			col = 0;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	for(; i < this._numRows; i++)
-	// 		ag.increment(this._data.getIndex(i) * nCol);
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-
-	// 	while(lIt.hasNext()) {
-	// 		int row = this._data.getIndex(lIt.value());
-	// 		int col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-
-	// 	while(lIt.hasNext()) {
-	// 		int row = this._data.getIndex(lIt.value());
-	// 		lIt.next();
-	// 		ag.increment(row);
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kl = 0; kl < NVL; kl++) {
-	// 		final int bOffL = lhs._ptr[kl];
-	// 		final int bLenL = lhs.len(kl);
-	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-	// 			sLenL = lhs._data[bOffL + bixL];
-	// 			for(int i = 1; i <= sLenL; i++) {
-	// 				int idx = this._data.getIndex(offL + lhs._data[bOffL + bixL + i]);
-	// 				ag.increment(kl + idx * NVL);
-	// 			}
-	// 		}
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kl = 0; kl < NVL; kl++) {
-	// 		final int boffL = lhs._ptr[kl];
-	// 		final int blenL = lhs.len(kl);
-	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-	// 			startL += lhs._data[boffL + bixL];
-	// 			lenL = lhs._data[boffL + bixL + 1];
-	// 			final int endL = startL + lenL;
-	// 			for(int i = startL; i < endL; i++) {
-	// 				int kr = _data.getIndex(i) * NVL;
-	// 				ag.increment(kl + kr);
-	// 			}
-	// 		}
-	// 	}
-	// 	return ag;
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		final int nCol = that._colIndexes.length;
@@ -618,7 +320,6 @@ public class ColGroupDDC extends ColGroupValue {
 	@Override
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
-		// write data
 		_data.write(out);
 	}
 
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 e1ade47..fc6c1f3 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
@@ -93,26 +93,6 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		// do nothing.
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	// do nothing.
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	// do nothing.
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	// do nothing.
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	// do nothing.
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		return 0;
@@ -128,8 +108,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		double val0 = op.executeScalar(0);
 		if(val0 == 0)
 			return this;
-		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[_colIndexes.length]).apply(op));
+		return new ColGroupConst(_colIndexes, _numRows, new Dictionary(new double[_colIndexes.length]).apply(op));
 	}
 
 	@Override
@@ -240,4 +219,12 @@ public class ColGroupEmpty extends ColGroupCompressed {
 	public AColGroup rightMultByMatrix(MatrixBlock right) {
 		return null;
 	}
+
+	@Override
+	public AColGroup replace(double pattern, double replace) {
+		if(pattern == 0)
+			return ColGroupFactory.getColGroupConst(getNumRows(), _colIndexes, replace);
+		else
+			return new ColGroupEmpty(_colIndexes, getNumRows());
+	}
 }
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 32cebf8..1cb92c7 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
@@ -469,12 +469,16 @@ public final class ColGroupFactory {
 
 		if(value == 0)
 			return new ColGroupEmpty(colIndices, numRows);
+		else
+			return getColGroupConst(numRows, colIndices, value);
+	}
 
+	public static AColGroup getColGroupConst(int numRows, int[] cols, double value ){
+		final int numCols = cols.length;
 		double[] values = new double[numCols];
 		for(int i = 0; i < numCols; i++)
 			values[i] = value;
-
 		ADictionary dict = new Dictionary(values);
-		return new ColGroupConst(colIndices, numRows, dict);
+		return new ColGroupConst(cols, numRows, dict);
 	}
 }
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 e00422b..d04f6cd 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
@@ -110,143 +110,6 @@ public class ColGroupOLE extends ColGroupOffset {
 		throw new NotImplementedException();
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	final int numVals = getNumValues();
-	// 	final double[] values = getValues();
-
-	// 	// cache blocking config and position array
-	// 	int[] apos = new int[numVals];
-	// 	int[] cix = new int[numCols];
-
-	// 	// prepare target col indexes
-	// 	for(int j = 0; j < numCols; j++)
-	// 		cix[j] = colixTargets[_colIndexes[j]];
-
-	// 	// cache conscious append via horizontal scans
-	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int len = _data[boff + bix];
-	// 			int pos = boff + bix + 1;
-	// 			for(int i = pos; i < pos + len; i++)
-	// 				for(int j = 0, rix = bi + _data[i]; j < numCols; j++)
-	// 					if(values[off + j] != 0) {
-	// 						double v = target.quickGetValue(rix, _colIndexes[j]);
-	// 						target.setValue(rix, cix[j], values[off + j] + v);
-	// 					}
-	// 			apos[k] += len + 1;
-	// 		}
-	// 	}
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	int numCols = getNumCols();
-	// 	int numVals = getNumValues();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double[] values = getValues();
-
-	// 	// cache blocking config and position array
-	// 	int[] apos = new int[numVals];
-
-	// 	// cache conscious append via horizontal scans
-	// 	int nnz = 0;
-	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
-	// 		// Arrays.fill(c, bi, Math.min(bi + blksz, _numRows), 0);
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int len = _data[boff + bix];
-	// 			int pos = boff + bix + 1;
-	// 			for(int i = pos; i < pos + len; i++) {
-	// 				c[bi + _data[i]] += values[off + colpos];
-	// 				nnz++;
-	// 			}
-	// 			apos[k] += len + 1;
-	// 		}
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	int numCols = getNumCols();
-	// 	int numVals = getNumValues();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	double[] values = getValues();
-
-	// 	// cache blocking config and position array
-	// 	int[] apos = skipScan(numVals, rl);
-
-	// 	// cache conscious append via horizontal scans
-	// 	int nnz = 0;
-	// 	for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int len = _data[boff + bix];
-	// 			int pos = boff + bix + 1;
-	// 			for(int i = pos; i < pos + len; i++) {
-	// 				int index = bi + _data[i];
-	// 				if(index >= rl && index < ru) {
-	// 					c[index - rl] += values[off + colpos];
-	// 					nnz++;
-	// 				}
-	// 			}
-	// 			apos[k] += len + 1;
-	// 		}
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	int numCols = getNumCols();
-	// 	int numVals = getNumValues();
-	// 	double[] values = getValues();
-
-	// 	// cache blocking config and position array
-	// 	int[] apos = skipScan(numVals, rl);
-
-	// 	// cache conscious append via horizontal scans
-	// 	for(int bi = (rl / blksz) * blksz; bi < ru; bi += blksz) {
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int len = _data[boff + bix];
-	// 			int pos = boff + bix + 1;
-	// 			for(int i = pos; i < pos + len; i++) {
-	// 				int index = bi + _data[i];
-	// 				if(index >= rl && index < ru)
-	// 					c[index - rl] += values[off + colpos];
-	// 			}
-	// 			apos[k] += len + 1;
-	// 		}
-	// 	}
-	// }
-
 	@Override
 	public int[] getCounts(int[] counts) {
 		final int numVals = getNumValues();
@@ -353,313 +216,6 @@ public class ColGroupOLE extends ColGroupOffset {
 		return new ColGroupOLE(_colIndexes, _numRows, false, rvalues, rbitmaps, rbitmapOffs, getCachedCounts());
 	}
 
-	// @Override
-	// public void rightMultByVector(double[] b, double[] c, int rl, int ru, double[] dictVals) {
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// final int numVals = getNumValues();
-
-	// if(rl % blksz != 0)
-	// throw new DMLCompressionException("All blocks should be starting at block segments for OLE");
-
-	// if(numVals > 1 && _numRows > blksz * 2) {
-	// // since single segment scans already exceed typical L2 cache sizes
-	// // and because there is some overhead associated with blocking, the
-	// // best configuration aligns with L3 cache size (x*vcores*64K*8B < L3)
-	// // x=4 leads to a good yet slightly conservative compromise for single-/
-	// // multi-threaded and typical number of cores and L3 cache sizes
-	// final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-	// int[] apos = skipScan(numVals, rl);
-	// double[] aval = preaggValues(numVals, b, dictVals);
-
-	// // step 2: cache conscious matrix-vector via horizontal scans
-	// for(int bi = rl; bi < ru; bi += blksz2) {
-	// int bimax = Math.min(bi + blksz2, 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];
-
-	// for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
-	// // prepare length, start, and end pos
-	// int len = _data[boff + bix];
-	// int pos = boff + bix + 1;
-
-	// // compute partial results
-	// LinearAlgebraUtils.vectAdd(val, c, _data, pos, ii, len);
-	// bix += len + 1;
-	// }
-
-	// apos[k] = bix;
-	// }
-	// }
-	// }
-	// else {
-	// // 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 = sumValues(k, b, dictVals);
-
-	// // iterate over bitmap blocks and add values
-	// if(val != 0) {
-	// int bix = 0;
-	// int off = 0;
-	// int slen = -1;
-
-	// // scan to beginning offset if necessary
-	// if(rl > 0) {
-	// for(; bix < blen & off < rl; bix += slen + 1, off += blksz) {
-	// slen = _data[boff + bix];
-	// }
-	// }
-
-	// // compute partial results
-	// for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
-	// slen = _data[boff + bix];
-	// for(int blckIx = 1; blckIx <= slen; blckIx++) {
-	// c[off + _data[boff + bix + blckIx]] += val;
-	// }
-	// }
-	// }
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int
-	// rl,
-	// int ru) {
-
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// final int numVals = getNumValues();
-
-	// if(numVals > 1 && _numRows > blksz * 2) {
-	// final int blksz2 = blksz * 2;
-	// int[] apos = skipScan(numVals, rl);
-	// int blockStart = rl - rl % blksz;
-	// for(int bi = blockStart; bi < ru; bi += blksz2) {
-	// int bimax = Math.min(bi + blksz2, ru);
-	// for(int k = 0; k < numVals; k++) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// int bix = apos[k];
-	// for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
-	// int len = _data[boff + bix];
-	// int pos = _data[boff + bix + 1];
-	// if(pos >= rl)
-	// addV(c, preAggregatedB, outputColumns, (bi + pos) * thatNrColumns, k);
-	// bix += len + 1;
-	// }
-	// apos[k] = bix;
-	// }
-	// }
-	// }
-	// else {
-	// for(int k = 0; k < numVals; k++) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// int bix = skipScanVal(k, rl);
-	// int off = rl;
-	// int slen = 0;
-	// // compute partial results
-	// for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
-	// slen = _data[boff + bix];
-	// for(int blckIx = 1; blckIx <= slen; blckIx++) {
-	// int rowIdx = (_data[boff + bix + blckIx] + off) * thatNrColumns;
-	// addV(c, preAggregatedB, outputColumns, rowIdx, k);
-	// }
-	// }
-	// }
-	// }
-	// }
-
-	// private static void addV(double[] c, double[] preAggregatedB, int[] outputColumns, int rowIdx, int k) {
-	// int n = k * outputColumns.length;
-	// for(int i = 0; i < outputColumns.length; i++) {
-	// c[rowIdx + outputColumns[i]] += preAggregatedB[n + i];
-	// }
-	// }
-
-	// @Override
-	// public void leftMultByRowVector(double[] a, double[] c, int numVals, double[] values) {
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-
-	// if(numVals >= 1 && _numRows > blksz)
-	// leftMultByRowVectorBlocking(a, c, numVals, values);
-	// else
-	// leftMultByRowVectorNonBlocking(a, c, numVals, values);
-
-	// }
-
-	// private void leftMultByRowVectorBlocking(double[] a, double[] c, int numVals, double[] values) {
-	// double[] cvals = preAggregate(a);
-	// postScaling(values, cvals, c, numVals);
-	// }
-
-	// private void leftMultByRowVectorNonBlocking(double[] a, double[] c, int numVals, double[] values) {
-	// // iterate over all values and their bitmaps
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// final int numCols = getNumCols();
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-
-	// // iterate over bitmap blocks and add partial results
-	// double vsum = 0;
-	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
-	// vsum += LinearAlgebraUtils.vectSum(a, _data, off, boff + bix + 1, _data[boff + bix]);
-
-	// // scale partial results by values and write results
-	// for(int j = 0; j < numCols; j++)
-	// c[_colIndexes[j]] += vsum * values[valOff + j];
-	// }
-	// }
-
-	// @Override
-	// public void leftMultByMatrix(double[] a, double[] c, double[] values, int numRows, int numCols, int rl, int ru,
-	// int vOff) {
-	// final int numVals = getNumValues();
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// if(numVals >= 1 && _numRows > blksz)
-	// leftMultByMatrixBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
-	// else
-	// leftMultByMatrixNonBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
-
-	// }
-
-	// private void leftMultByMatrixBlocking(double[] a, double[] c, double[] values, int numRows, int numCols, int rl,
-	// int ru, int vOff, int numVals) {
-	// for(int i = rl; i < ru; i++) {
-	// double[] cvals = preAggregate(a, i);
-	// postScaling(values, cvals, c, numVals, i, numCols);
-	// }
-	// }
-
-	// private void leftMultByMatrixNonBlocking(double[] a, double[] c, double[] values, int numRows, int numCols, int
-	// rl,
-	// int ru, int vOff, int numVals) {
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// for(int i = rl, offR = vOff * _numRows; i < ru; i++, offR += _numRows) {
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-
-	// // iterate over bitmap blocks and add partial results
-	// double vsum = 0;
-	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
-	// vsum += LinearAlgebraUtils.vectSum(a, _data, off + offR, boff + bix + 1, _data[boff + bix]);
-
-	// // scale partial results by values and write results
-
-	// int offC = i * numCols;
-	// for(int j = 0; j < _colIndexes.length; j++) {
-	// int colIx = _colIndexes[j] + offC;
-	// c[colIx] += vsum * values[valOff + j];
-	// }
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int
-	// row) {
-	// // final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// // final int numVals = getNumValues();
-	// throw new NotImplementedException("Not implemented Sparse multiplication OLE");
-	// // if(numVals > 1 && _numRows > blksz)
-	// // leftMultBySparseMatrixBlocking(sb, c, values, numRows, numCols, row, tmpA, numVals);
-	// // else
-	// // leftMultBySparseMatrixNonBlock(sb, c, values, numRows, numCols, row, tmpA, numVals);
-
-	// }
-
-	// private void leftMultBySparseMatrixBlocking(SparseBlock sb, double[] c, double[] values, int numRows, int
-	// numCols,
-	// int row, double[] tmpA, int numVals) {
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// int[] indexes = sb.indexes(row);
-	// double[] sparseV = sb.values(row);
-
-	// // cache blocking config (see matrix-vector mult for explanation)
-	// final int blksz2 = 2 * CompressionSettings.BITMAP_BLOCK_SZ;
-
-	// // step 1: prepare position and value arrays
-	// int[] apos = allocIVector(numVals, true);
-	// double[] cvals = allocDVector(numVals, true);
-	// // step 2: cache conscious matrix-vector via horizontal scans
-	// int pI = sb.pos(row);
-	// for(int ai = 0; ai < _numRows; ai += blksz2) {
-	// int aimax = Math.min(ai + blksz2, _numRows);
-	// Arrays.fill(tmpA, 0);
-	// for(; pI < sparseEndIndex && indexes[pI] < aimax; pI++) {
-	// if(indexes[pI] >= ai)
-	// tmpA[indexes[pI] - ai] = sparseV[pI];
-	// }
-
-	// // horizontal segment scan, incl pos maintenance
-	// for(int k = 0; k < numVals; k++) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// int bix = apos[k];
-	// double vsum = 0;
-	// for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
-	// int len = _data[boff + bix];
-	// int pos = boff + bix + 1;
-	// int blockId = (ii / blksz) % 2;
-	// vsum += LinearAlgebraUtils.vectSum(tmpA, _data, blockId * blksz, pos, len);
-	// bix += len + 1;
-	// }
-
-	// apos[k] = bix;
-	// cvals[k] += vsum;
-	// }
-	// }
-
-	// int offC = row * numCols;
-	// // step 3: scale partial results by values and write to global output
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length)
-	// for(int j = 0; j < _colIndexes.length; j++) {
-	// int colIx = _colIndexes[j] + offC;
-	// c[colIx] += cvals[k] * values[valOff + j];
-	// }
-
-	// }
-
-	// private void leftMultBySparseMatrixNonBlock(SparseBlock sb, double[] c, double[] values, int numRows, int
-	// numCols,
-	// int row, double[] tmpA, int numVals) {
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// int[] indexes = sb.indexes(row);
-	// double[] sparseV = sb.values(row);
-
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// double vsum = 0;
-	// int pI = sb.pos(row);
-	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz) {
-	// // blockId = off / blksz;
-	// Arrays.fill(tmpA, 0);
-	// for(; pI < sparseEndIndex && indexes[pI] < off + blksz; pI++) {
-	// if(indexes[pI] >= off)
-	// tmpA[indexes[pI] - off] = sparseV[pI];
-	// }
-	// vsum += LinearAlgebraUtils.vectSum(tmpA, _data, 0, boff + bix + 1, _data[boff + bix]);
-	// }
-
-	// for(int j = 0; j < _colIndexes.length; j++) {
-	// int Voff = _colIndexes[j] + row * numCols;
-	// c[Voff] += vsum * values[valOff + j];
-	// }
-	// }
-	// }
-
 	@Override
 	protected void computeRowSums(double[] c, boolean square, int rl, int ru) {
 
@@ -905,48 +461,6 @@ public class ColGroupOLE extends ColGroupOffset {
 		return sb.toString();
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// 	final int numVals = getNumValues();
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-
-	// 	int[] apos = allocIVector(numVals, true);
-	// 	double[] cvals = allocDVector(numVals, true);
-	// 	int off = row * _numRows;
-	// 	for(int ai = 0; ai < _numRows; ai += blksz2) {
-	// 		int aimax = Math.min(ai + blksz2, _numRows);
-
-	// 		// horizontal segment scan, incl pos maintenance
-	// 		for(int k = 0; k < numVals; k++) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			double vsum = 0;
-
-	// 			for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
-	// 				// prepare length, start, and end pos
-	// 				int len = _data[boff + bix];
-	// 				int pos = boff + bix + 1;
-
-	// 				// iterate over bitmap blocks and compute partial results (a[i]*1)
-	// 				vsum += LinearAlgebraUtils.vectSum(a, _data, ii + off, pos, len);
-	// 				bix += len + 1;
-	// 			}
-
-	// 			apos[k] = bix;
-	// 			cvals[k] += vsum;
-	// 		}
-	// 	}
-
-	// 	return cvals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// 	return null;
-	// }
-
 	@Override
 	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
 		throw new NotImplementedException();
@@ -1015,139 +529,6 @@ public class ColGroupOLE extends ColGroupOffset {
 		return encodedBlocks;
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kr = 0; kr < NVR; kr++) {
-	// 		final int bOffR = this._ptr[kr];
-	// 		final int bLenR = this.len(kr);
-	// 		final int krOff = kr * NVL;
-	// 		for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-	// 			sLenR = this._data[bOffR + bixR];
-	// 			for(int j = 1; j <= sLenR; j++) {
-	// 				int idx = lhs._data.getIndex(offR + this._data[bOffR + bixR + j]);
-	// 				ag.increment(idx + krOff);
-	// 			}
-	// 		}
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	final int defL = NVL - 1;
-
-	// 	for(int kr = 0; kr < NVR; kr++) {
-	// 		AIterator lIt = lhs._indexes.getIterator();
-	// 		final int bOffR = this._ptr[kr];
-	// 		final int bLenR = this.len(kr);
-	// 		final int krOff = kr * NVL;
-	// 		for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-	// 			sLenR = this._data[bOffR + bixR];
-	// 			for(int j = 1; j <= sLenR; j++) {
-	// 				final int row = offR + this._data[bOffR + bixR + j];
-	// 				lIt.skipTo(row);
-	// 				if(lIt.value() == row)
-	// 					ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
-	// 				else
-	// 					ag.increment(defL + krOff);
-	// 			}
-	// 		}
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kr = 0; kr < NVR; kr++) {
-	// 		final AIterator lIt = lhs._indexes.getIterator();
-	// 		final int bOffR = this._ptr[kr];
-	// 		final int bLenR = this.len(kr);
-	// 		final int krOff = kr * NVL;
-	// 		for(int bixR = 0, offR = 0, sLenR = 0; lIt.hasNext() && bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-	// 			sLenR = this._data[bOffR + bixR];
-	// 			for(int j = 1; lIt.hasNext() && j <= sLenR; j++) {
-	// 				final int row = offR + this._data[bOffR + bixR + j];
-	// 				lIt.skipTo(row);
-	// 				if(lIt.value() == row)
-	// 					ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + krOff);
-	// 			}
-	// 		}
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kl = 0; kl < NVL; kl++) {
-	// 		final int bOffL = lhs._ptr[kl];
-	// 		final int bLenL = lhs.len(kl);
-	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-	// 			sLenL = lhs._data[bOffL + bixL];
-	// 			for(int i = 1; i <= sLenL; i++) {
-	// 				final int col = offL + lhs._data[bOffL + bixL + i];
-	// 				for(int kr = 0; kr < NVR; kr++) {
-	// 					final int bOffR = this._ptr[kr];
-	// 					final int bLenR = this.len(kr);
-	// 					final int krOff = kr * NVL;
-	// 					for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
-	// 						sLenR = this._data[bOffR + bixR];
-	// 						for(int j = 1; j <= sLenR; j++)
-	// 							if(col == offR + this._data[bOffR + bixR + j])
-	// 								ag.increment(kl + krOff);
-	// 					}
-	// 				}
-	// 			}
-	// 		}
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		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 b7dd658..80f17dd 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
@@ -108,174 +108,6 @@ public class ColGroupRLE extends ColGroupOffset {
 		throw new NotImplementedException();
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-	// 	// if(getNumValues() > 1) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	final int numVals = getNumValues();
-	// 	final double[] values = getValues();
-
-	// 	// position and start offset arrays
-	// 	int[] apos = new int[numVals];
-	// 	int[] astart = new int[numVals];
-	// 	int[] cix = new int[numCols];
-
-	// 	// prepare target col indexes
-	// 	for(int j = 0; j < numCols; j++)
-	// 		cix[j] = colixTargets[_colIndexes[j]];
-
-	// 	// cache conscious append via horizontal scans
-	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
-	// 		int bimax = Math.min(bi + blksz, _numRows);
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int start = astart[k];
-	// 			for(; bix < blen & start < bimax; bix += 2) {
-	// 				start += _data[boff + bix];
-	// 				int len = _data[boff + bix + 1];
-	// 				for(int i = start; i < start + len; i++)
-	// 					for(int j = 0; j < numCols; j++)
-	// 						if(values[off + j] != 0) {
-	// 							double v = target.quickGetValue(i, _colIndexes[j]);
-	// 							target.setValue(i, _colIndexes[j], values[off + j] + v);
-	// 						}
-
-	// 				start += len;
-	// 			}
-	// 			apos[k] = bix;
-	// 			astart[k] = start;
-	// 		}
-	// 	}
-	// 	// }
-	// 	// else {
-	// 	// // call generic decompression with decoder
-	// 	// super.decompressToBlock(target, colixTargets);
-	// 	// }
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	final int numVals = getNumValues();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	final double[] values = getValues();
-
-	// 	// position and start offset arrays
-	// 	int[] astart = new int[numVals];
-	// 	int[] apos = allocIVector(numVals, true);
-
-	// 	// cache conscious append via horizontal scans
-	// 	int nnz = 0;
-	// 	for(int bi = 0; bi < _numRows; bi += blksz) {
-	// 		int bimax = Math.min(bi + blksz, _numRows);
-	// 		// Arrays.fill(c, bi, bimax, 0);
-	// 		for(int k = 0, off = 0; k < numVals; k++, off += numCols) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int start = astart[k];
-	// 			for(; bix < blen & start < bimax; bix += 2) {
-	// 				start += _data[boff + bix];
-	// 				int len = _data[boff + bix + 1];
-	// 				for(int i = start; i < start + len; i++)
-	// 					c[i] += values[off + colpos];
-	// 				nnz += len;
-	// 				start += len;
-	// 			}
-	// 			apos[k] = bix;
-	// 			astart[k] = start;
-	// 		}
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	final int numVals = getNumValues();
-	// 	double[] c = target.getDenseBlockValues();
-	// 	final double[] values = getValues();
-
-	// 	// position and start offset arrays
-	// 	int[] astart = new int[numVals];
-	// 	int[] apos = allocIVector(numVals, true);
-
-	// 	// cache conscious append via horizontal scans
-	// 	int nnz = 0;
-	// 	for(int bi = (rl / blksz) * blksz; 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];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int start = astart[k];
-	// 			for(; bix < blen & start < bimax; bix += 2) {
-	// 				start += _data[boff + bix];
-	// 				int len = _data[boff + bix + 1];
-	// 				if(start + len >= rl) {
-	// 					int offsetStart = Math.max(start, rl);
-	// 					for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
-	// 						c[i - rl] += values[off + colpos];
-	// 					nnz += len - (offsetStart - start);
-	// 				}
-	// 				start += len;
-	// 			}
-	// 			apos[k] = bix;
-	// 			astart[k] = start;
-	// 		}
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	final int numVals = getNumValues();
-	// 	final double[] values = getValues();
-
-	// 	// position and start offset arrays
-	// 	int[] astart = new int[numVals];
-	// 	int[] apos = allocIVector(numVals, true);
-
-	// 	// cache conscious append via horizontal scans
-
-	// 	for(int bi = (rl / blksz) * blksz; 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];
-	// 			if(bix >= blen)
-	// 				continue;
-	// 			int start = astart[k];
-	// 			for(; bix < blen & start < bimax; bix += 2) {
-	// 				start += _data[boff + bix];
-	// 				int len = _data[boff + bix + 1];
-	// 				if(start + len >= rl) {
-	// 					int offsetStart = Math.max(start, rl);
-	// 					for(int i = offsetStart; i < Math.min(start + len, bimax); i++)
-	// 						c[i - rl] += values[off + colpos];
-	// 				}
-	// 				start += len;
-	// 			}
-	// 			apos[k] = bix;
-	// 			astart[k] = start;
-	// 		}
-	// 	}
-	// }
-
 	@Override
 	public int[] getCounts(int[] counts) {
 		final int numVals = getNumValues();
@@ -283,12 +115,8 @@ public class ColGroupRLE extends ColGroupOffset {
 		for(int k = 0; k < numVals; k++) {
 			int boff = _ptr[k];
 			int blen = len(k);
-			// int curRunEnd = 0;
 			int count = 0;
 			for(int bix = 0; bix < blen; bix += 2) {
-				// int curRunStartOff = curRunEnd + _data[boff + bix];
-				// curRunEnd = curRunStartOff + _data[boff + bix + 1];
-				// count += curRunEnd - curRunStartOff;
 				count += _data[boff + bix + 1];
 			}
 			sum += count;
@@ -326,230 +154,6 @@ public class ColGroupRLE extends ColGroupOffset {
 		return counts;
 	}
 
-	// @Override
-	// public void rightMultByVector(double[] b, double[] c, int rl, int ru, double[] dictVals) {
-	// final int numVals = getNumValues();
-	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// // L3 cache alignment, see comment rightMultByVector OLE column group
-	// // core difference of RLE to OLE is that runs are not segment alignment,
-	// // which requires care of handling runs crossing cache-buckets
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-
-	// // step 1: prepare position and value arrays
-
-	// // current pos / values per RLE list
-
-	// // step 2: cache conscious matrix-vector via horizontal scans
-	// for(int bi = rl; bi < ru; bi += blksz) {
-	// int[] astart = new int[numVals];
-	// int[] apos = skipScan(numVals, rl, astart);
-	// double[] aval = preaggValues(numVals, b, dictVals);
-	// 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 & bix < bimax) {
-	// int lstart = _data[boff + bix];
-	// int llen = _data[boff + bix + 1];
-	// int len = Math.min(start + lstart + llen, bimax) - Math.max(bi, start + lstart);
-	// if(len > 0) {
-	// LinearAlgebraUtils.vectAdd(val, c, Math.max(bi, start + lstart), len);
-	// }
-	// 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 = sumValues(k, b, dictVals);
-	// int bix = 0;
-	// int start = 0;
-
-	// // scan to beginning offset if necessary
-	// if(rl > 0) { // rl aligned with blksz
-	// 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;
-	// }
-	// }
-
-	// // compute partial results, not aligned
-	// while(bix < blen) {
-	// int lstart = _data[boff + bix];
-	// int llen = _data[boff + bix + 1];
-	// LinearAlgebraUtils.vectAdd(val, c, Math.max(rl, start + lstart),
-	// Math.min(start + lstart + llen, ru) - Math.max(rl, start + lstart));
-	// if(start + lstart + llen >= ru)
-	// break;
-	// start += lstart + llen;
-	// bix += 2;
-	// }
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int
-	// rl,
-	// int ru) {
-	// final int nrVals = getNumValues();
-	// for(int k = 0; k < nrVals; k++) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-	// int bix = 0;
-	// int start = 0;
-
-	// // scan to beginning offset if necessary
-	// if(rl > 0) { // rl aligned with blksz
-	// 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;
-	// }
-	// }
-	// // compute partial results, not aligned
-	// while(bix < blen) {
-	// int lstart = _data[boff + bix];
-	// int llen = _data[boff + bix + 1];
-	// LinearAlgebraUtils.vectListAdd(preAggregatedB, c, Math.max(rl, start + lstart),
-	// Math.min(start + lstart + llen, ru), outputColumns, thatNrColumns, k);
-	// if(start + lstart + llen >= ru)
-	// break;
-	// start += lstart + llen;
-	// bix += 2;
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void leftMultByRowVector(double[] a, double[] c, int numVals, double[] values) {
-	// final int numCols = getNumCols();
-
-	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// double[] cvals = preAggregate(a, 0);
-	// postScaling(values, cvals, c, numVals);
-	// }
-	// else {
-	// // iterate over all values and their bitmaps
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-
-	// double vsum = 0;
-	// int curRunEnd = 0;
-	// for(int bix = 0; bix < blen; bix += 2) {
-	// int curRunStartOff = curRunEnd + _data[boff + bix];
-	// int curRunLen = _data[boff + bix + 1];
-	// vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff, curRunLen);
-	// curRunEnd = curRunStartOff + curRunLen;
-	// }
-
-	// // scale partial results by values and write results
-	// for(int j = 0; j < numCols; j++)
-	// c[_colIndexes[j]] += vsum * values[valOff + j];
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void leftMultByMatrix(final double[] a, final double[] c, final double[] values, final int numRows,
-	// final int numCols, int rl, final int ru, final int vOff) {
-
-	// final int numVals = getNumValues();
-	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// for(int i = rl; i < ru; i++) {
-	// double[] cvals = preAggregate(a, i);
-	// postScaling(values, cvals, c, numVals, i, numCols);
-	// }
-	// }
-	// else {
-	// // iterate over all values and their bitmaps
-	// for(int i = rl, off = vOff * _numRows; i < ru; i++, off += _numRows) {
-	// int offC = i * numCols;
-	// int valOff = 0;
-	// for(int k = 0; k < numVals; k++) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-
-	// double vsum = 0;
-	// int curRunEnd = 0;
-	// for(int bix = 0; bix < blen; bix += 2) {
-	// int curRunStartOff = curRunEnd + _data[boff + bix];
-	// int curRunLen = _data[boff + bix + 1];
-	// vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff + off, curRunLen);
-	// curRunEnd = curRunStartOff + curRunLen;
-	// }
-
-	// for(int j = 0; j < _colIndexes.length; j++) {
-	// int colIx = _colIndexes[j] + offC;
-	// // scale partial results by values and write results
-	// c[colIx] += vsum * values[valOff++];
-	// }
-	// }
-	// }
-	// }
-	// }
-
-	// @Override
-	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int
-	// row) {
-
-	// final int numVals = getNumValues();
-	// int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// int[] indexes = sb.indexes(row);
-	// double[] sparseV = sb.values(row);
-	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// int boff = _ptr[k];
-	// int blen = len(k);
-
-	// double vsum = 0;
-	// int pointSparse = sb.pos(row);
-	// int curRunEnd = 0;
-	// for(int bix = 0; bix < blen; bix += 2) {
-	// int curRunStartOff = curRunEnd + _data[boff + bix];
-	// int curRunLen = _data[boff + bix + 1];
-	// curRunEnd = curRunStartOff + curRunLen;
-	// while(pointSparse < sparseEndIndex && indexes[pointSparse] < curRunStartOff) {
-	// pointSparse++;
-	// }
-	// while(pointSparse != sparseEndIndex && indexes[pointSparse] >= curRunStartOff &&
-	// indexes[pointSparse] < curRunEnd) {
-	// vsum += sparseV[pointSparse++];
-	// }
-	// if(pointSparse == sparseEndIndex) {
-	// break;
-	// }
-	// }
-
-	// for(int j = 0; j < _colIndexes.length; j++) {
-	// int Voff = _colIndexes[j] + row * numCols;
-	// c[Voff] += vsum * values[valOff + j];
-	// }
-	// }
-
-	// }
-
 	@Override
 	public AColGroup scalarOperation(ScalarOperator op) {
 		double val0 = op.executeScalar(0);
@@ -862,47 +466,6 @@ public class ColGroupRLE extends ColGroupOffset {
 		return new Pair<>(apos, astart);
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// 	final int numVals = getNumValues();
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	// current pos per OLs / output values
-	// 	int[] astart = new int[numVals];
-	// 	int[] apos = allocIVector(numVals, true);
-	// 	double[] cvals = allocDVector(numVals, true);
-	// 	int off = row * _numRows;
-
-	// 	// step 2: cache conscious matrix-vector via horizontal scans
-	// 	for(int ai = 0; ai < _numRows; ai += blksz) {
-	// 		int aimax = Math.min(ai + blksz, _numRows);
-
-	// 		// horizontal scan, incl pos maintenance
-	// 		for(int k = 0; k < numVals; k++) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			int start = astart[k];
-
-	// 			// compute partial results, not aligned
-	// 			while(bix < blen & start < aimax) {
-	// 				start += _data[boff + bix];
-	// 				int len = _data[boff + bix + 1];
-	// 				cvals[k] += LinearAlgebraUtils.vectSum(a, start + off, len);
-	// 				start += len;
-	// 				bix += 2;
-	// 			}
-
-	// 			apos[k] = bix;
-	// 			astart[k] = start;
-	// 		}
-	// 	}
-	// 	return cvals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// 	return null;
-	// }
 	
 	@Override
 	public void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
@@ -1017,94 +580,6 @@ public class ColGroupRLE extends ColGroupOffset {
 		return ret;
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kr = 0; kr < NVR; kr++) {
-	// 		final int boffL = _ptr[kr];
-	// 		final int blenL = len(kr);
-	// 		final int offKr = kr * NVL;
-	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-	// 			startL += _data[boffL + bixL];
-	// 			lenL = _data[boffL + bixL + 1];
-	// 			final int endL = startL + lenL;
-	// 			for(int i = startL; i < endL; i++)
-	// 				ag.increment(lhs._data.getIndex(i) + offKr);
-
-	// 		}
-	// 	}
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	for(int kl = 0; kl < NVL; kl++) {
-	// 		final int boffL = lhs._ptr[kl];
-	// 		final int blenL = lhs.len(kl);
-	// 		for(int bixL = 0, startL = 0, lenL = 0; bixL < blenL && startL < _numRows; startL += lenL, bixL += 2) {
-	// 			startL += lhs._data[boffL + bixL];
-	// 			lenL = lhs._data[boffL + bixL + 1];
-	// 			final int endL = startL + lenL;
-	// 			for(int kr = 0; kr < NVR; kr++) {
-	// 				final int boffR = _ptr[kr];
-	// 				final int blenR = len(kr);
-	// 				final int krOff = kr * NVL;
-	// 				for(int bixR = 0, startR = 0, lenR = 0; bixR < blenR & startR < endL; startR += lenR, bixR += 2) {
-	// 					startR += _data[boffR + bixR];
-	// 					lenR = _data[boffR + bixR + 1];
-	// 					final int endR = startR + lenR;
-	// 					if(startL < endR && startR < endL) {
-	// 						final int endOverlap = Math.min(endR, endL);
-	// 						final int startOverlap = Math.max(startL, startR);
-	// 						final int lenOverlap = endOverlap - startOverlap;
-	// 						ag.increment(kl + krOff, lenOverlap);
-	// 					}
-	// 				}
-	// 			}
-	// 		}
-	// 	}
-	// 	return ag;
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		throw new NotImplementedException();
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 f56d6db..052bf66 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
@@ -162,58 +162,6 @@ public class ColGroupSDC extends ColGroupValue {
 
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colPos) {
-	// 	final double[] c = target.getDenseBlockValues();
-	// 	final double[] values = getValues();
-	// 	final double defaultVal = values[values.length - _colIndexes.length + colPos];
-	// 	int i = 0;
-	// 	final AIterator it = _indexes.getIterator();
-	// 	for(; i < _numRows && it.hasNext(); i++) {
-	// 		if(it.value() == i)
-	// 			c[i] += values[_data.getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colPos];
-	// 		else
-	// 			c[i] += defaultVal;
-	// 	}
-	// 	for(; i < _numRows; i++)
-	// 		c[i] += defaultVal;
-
-	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	final int nCol = _colIndexes.length;
-	// 	final double[] values = getValues();
-	// 	final int offsetToDefault = values.length - nCol + colpos;
-	// 	final AIterator it = _indexes.getIterator(rl);
-
-	// 	int offT = 0;
-	// 	int i = rl;
-
-	// 	for(; i < ru && it.hasNext(); i++, offT++) {
-	// 		if(it.value() == i) {
-	// 			int offset = _data.getIndex(it.getDataIndexAndIncrement()) * nCol;
-	// 			c[offT] += values[offset + colpos];
-	// 		}
-	// 		else
-	// 			c[offT] += values[offsetToDefault];
-	// 	}
-
-	// 	for(; i < ru; i++, offT++)
-	// 		c[offT] += values[offsetToDefault];
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		// find local column index
@@ -308,37 +256,6 @@ public class ColGroupSDC extends ColGroupValue {
 		return _data.getIndex(r);
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// final int numVals = getNumValues();
-	// final double[] vals = allocDVector(numVals, true);
-	// final AIterator it = _indexes.getIterator();
-	// final int def = numVals - 1;
-
-	// int i = 0;
-
-	// if(row > 0) {
-	// int offA = _numRows * row;
-	// for(; i < _numRows && it.hasNext(); i++, offA++)
-	// if(it.value() == i)
-	// vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[offA];
-	// else
-	// vals[def] += a[offA];
-	// for(; i < _numRows; i++, offA++)
-	// vals[def] += a[offA];
-	// }
-	// else {
-	// for(; i < _numRows && it.hasNext(); i++)
-	// if(it.value() == i)
-	// vals[_data.getIndex(it.getDataIndexAndIncrement())] += a[i];
-	// else
-	// vals[def] += a[i];
-	// for(; i < _numRows; i++)
-	// vals[def] += a[i];
-	// }
-	// return vals;
-	// }
-
 	@Override
 	public void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
@@ -383,8 +300,9 @@ public class ColGroupSDC extends ColGroupValue {
 			final double[] avals = sb.values(rowLeft);
 			int j = apos;
 			for(; j < alen && it.hasNext(); j++) {
-				it.skipTo(aix[j]);
-				if(it.value() == aix[j])
+				final int index = aix[j];
+				it.skipTo(index);
+				if(it.value() == index)
 					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += avals[j];
 				else
 					preAV[def] += avals[j];
@@ -460,215 +378,6 @@ public class ColGroupSDC extends ColGroupValue {
 		return sb.toString();
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator it = _indexes.getIterator();
-	// 	final int offsetToDefault = this.getNumValues() - 1;
-
-	// 	int i = 0;
-
-	// 	int row;
-	// 	for(; i < this._numRows && it.hasNext(); i++) {
-	// 		int col = lhs._data.getIndex(i);
-	// 		if(it.value() == i)
-	// 			row = getIndex(it.getDataIndexAndIncrement());
-	// 		else
-	// 			row = offsetToDefault;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-	// 	row = offsetToDefault;
-	// 	for(; i < this._numRows; i++) {
-	// 		int col = lhs._data.getIndex(i);
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	final int lhsNV = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = lhsNV * rhsNV;
-	// 	final int nCol = lhs.getNumValues();
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	final int defL = lhsNV - 1;
-	// 	final int defR = rhsNV - 1;
-
-	// 	AIterator lIt = lhs._indexes.getIterator();
-	// 	AIterator rIt = _indexes.getIterator();
-
-	// 	int i = 0;
-	// 	int col;
-	// 	int row;
-	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-	// 		if(lIt.value() == i)
-	// 			col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-	// 		else
-	// 			col = defL;
-	// 		if(rIt.value() == i)
-	// 			row = this.getIndex(rIt.getDataIndexAndIncrement());
-	// 		else
-	// 			row = defR;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	if(lIt.hasNext()) {
-	// 		row = defR;
-	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
-	// 			if(lIt.value() == i)
-	// 				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-	// 			else
-	// 				col = defL;
-
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	if(rIt.hasNext()) {
-	// 		col = defL;
-	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
-	// 			if(rIt.value() == i)
-	// 				row = this.getIndex(rIt.getDataIndexAndIncrement());
-	// 			else
-	// 				row = defR;
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	ag.increment(defL + defR * nCol, this._numRows - i);
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	final int lhsNV = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = lhsNV * rhsNV;
-	// 	final int nCol = lhs.getNumValues();
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final int defR = rhsNV - 1;
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = _indexes.getIterator();
-
-	// 	int i = 0;
-	// 	int col;
-	// 	int row;
-	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-	// 		if(lIt.value() == i) {
-	// 			col = 1;
-	// 			lIt.next();
-	// 		}
-	// 		else
-	// 			col = 0;
-	// 		if(rIt.value() == i)
-	// 			row = this.getIndex(rIt.getDataIndexAndIncrement());
-	// 		else
-	// 			row = defR;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	if(lIt.hasNext()) {
-	// 		row = defR;
-	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
-	// 			if(lIt.value() == i) {
-	// 				col = 1;
-	// 				lIt.next();
-	// 			}
-	// 			else
-	// 				col = 0;
-
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	if(rIt.hasNext()) {
-	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
-	// 			if(rIt.value() == i)
-	// 				row = this.getIndex(rIt.getDataIndexAndIncrement());
-	// 			else
-	// 				row = defR;
-	// 			ag.increment(row * nCol);
-	// 		}
-	// 	}
-
-	// 	ag.increment(defR * nCol, this._numRows - i);
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int defR = (rhsNV - 1) * nCol;
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = _indexes.getIterator();
-
-	// 	while(lIt.hasNext() && rIt.hasNext())
-	// 		if(lIt.value() == rIt.value())
-	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) +
-	// 				this.getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-	// 		else if(lIt.value() > rIt.value())
-	// 			rIt.next();
-	// 		else
-	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
-
-	// 	while(lIt.hasNext())
-	// 		ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()) + defR);
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	final int NVR = this.getNumValues();
-	// 	final int NVL = lhs.getNumValues();
-	// 	final int retSize = NVR * NVL;
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// 	final int defR = (NVR - 1) * NVL;
-
-	// 	for(int kl = 0; kl < NVL; kl++) {
-	// 		AIterator it = _indexes.getIterator();
-	// 		final int bOffL = lhs._ptr[kl];
-	// 		final int bLenL = lhs.len(kl);
-	// 		for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-	// 			sLenL = lhs._data[bOffL + bixL];
-	// 			for(int i = 1; i <= sLenL; i++) {
-	// 				final int col = offL + lhs._data[bOffL + bixL + i];
-	// 				it.skipTo(col);
-	// 				if(it.value() == col)
-	// 					ag.increment(kl + this.getIndex(it.getDataIndexAndIncrement()) * NVL);
-	// 				else
-	// 					ag.increment(kl + defR);
-
-	// 			}
-	// 		}
-	// 	}
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 
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 d749ea5..1d0280c 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
@@ -117,64 +117,6 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		throw new NotImplementedException();
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	final double[] c = target.getDenseBlockValues();
-	// 	final double[] values = getValues();
-	// 	final int offsetToDefault = _colIndexes.length;
-	// 	final AIterator it = _indexes.getIterator();
-	// 	final double v1 = values[offsetToDefault + colpos];
-	// 	final double v2 = values[colpos];
-
-	// 	int i = 0;
-	// 	for(; i < _numRows && it.hasNext(); i++) {
-	// 		if(it.value() == i) {
-	// 			c[i] += v1;
-	// 			it.next();
-	// 		}
-	// 		else
-	// 			c[i] += v2;
-	// 	}
-	// 	for(; i < _numRows; i++)
-	// 		c[i] += v2;
-
-	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	final int nCol = _colIndexes.length;
-	// 	final double[] values = getValues();
-	// 	final int offsetToDefault = values.length - nCol;
-	// 	final AIterator it = _indexes.getIterator();
-
-	// 	int offT = 0;
-	// 	int i = rl;
-	// 	it.skipTo(rl);
-
-	// 	for(; i < ru && it.hasNext(); i++, offT++) {
-	// 		if(it.value() == i) {
-	// 			it.next();
-	// 			c[offT] += values[colpos];
-	// 		}
-	// 		else
-	// 			c[offT] += values[offsetToDefault + colpos];
-	// 	}
-
-	// 	for(; i < ru; i++, offT++)
-	// 		c[offT] += values[offsetToDefault + colpos];
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		// find local column index
@@ -261,61 +203,6 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		return counts;
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// final int numVals = getNumValues();
-	// final double[] vals = allocDVector(numVals, true);
-	// final AIterator it = _indexes.getIterator();
-
-	// int i = 0;
-
-	// if(row > 0) {
-	// int offA = _numRows * row;
-	// for(; i < _numRows && it.hasNext(); i++, offA++)
-	// if(it.value() == i) {
-	// it.next();
-	// vals[0] += a[offA];
-	// }
-	// else
-	// vals[1] += a[offA];
-	// for(; i < _numRows; i++, offA++)
-	// vals[1] += a[offA];
-	// }
-	// else {
-	// for(; i < _numRows && it.hasNext(); i++)
-	// if(it.value() == i) {
-	// it.next();
-	// vals[0] += a[i];
-	// }
-	// else
-	// vals[1] += a[i];
-	// for(; i < _numRows; i++)
-	// vals[1] += a[i];
-	// }
-
-	// return vals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// final int numVals = getNumValues();
-	// final double[] vals = allocDVector(numVals, true);
-	// final int[] indexes = sb.indexes(row);
-	// final double[] sparseV = sb.values(row);
-	// final AIterator it = _indexes.getIterator();
-
-	// for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++) {
-	// it.skipTo(indexes[i]);
-	// if(it.value() == indexes[i]) {
-	// vals[0] += sparseV[i];
-	// it.next();
-	// }
-	// else
-	// vals[1] += sparseV[i];
-	// }
-	// return vals;
-	// }
-
 	@Override
 	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
@@ -359,15 +246,20 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			final int[] aix = sb.indexes(rowLeft);
 			final double[] avals = sb.values(rowLeft);
 			final int def = offOut + 1;
-			for(int j = apos; j < alen; j++) {
-				it.skipTo(aix[j]);
-				if(it.value() == aix[j]) {
+			int j = apos;
+			for(;it.hasNext() && j < alen; j++) {
+				final int index = aix[j];
+				it.skipTo(index);
+				if(index == it.value()) {
 					preAV[offOut] += avals[j];
 					it.next();
 				}
 				else
 					preAV[def] += avals[j];
+			}
 
+			for(; j < alen; j++) {
+				preAV[def] += avals[j];
 			}
 		}
 	}
@@ -428,200 +320,9 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		return sb.toString();
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator it = _indexes.getIterator();
-
-	// 	int i = 0;
-
-	// 	int row;
-	// 	for(; i < this._numRows && it.hasNext(); i++) {
-	// 		int col = lhs._data.getIndex(i);
-	// 		if(it.value() == i) {
-	// 			row = 0;
-	// 			it.next();
-	// 		}
-	// 		else
-	// 			row = 1;
-
-	// 		if(col < lhs.getNumValues())
-	// 			ag.increment(col + row * nCol);
-	// 	}
-	// 	row = 0;
-	// 	for(; i < this._numRows; i++) {
-	// 		int col = lhs._data.getIndex(i);
-	// 		if(col < lhs.getNumValues())
-	// 			ag.increment(col + row * nCol);
-	// 	}
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	final int lhsNV = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = lhsNV * rhsNV;
-	// 	final int nCol = lhs.getNumValues();
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final int defL = lhsNV - 1;
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = _indexes.getIterator();
-
-	// 	int i = 0;
-	// 	int col;
-	// 	int row;
-	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-	// 		if(lIt.value() == i)
-	// 			col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-	// 		else
-	// 			col = defL;
-	// 		if(rIt.value() == i) {
-	// 			row = 0;
-	// 			rIt.next();
-	// 		}
-	// 		else
-	// 			row = 1;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	if(lIt.hasNext()) {
-	// 		row = 0;
-	// 		for(; i < this._numRows && lIt.hasNext(); i++) {
-	// 			if(lIt.value() == i)
-	// 				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
-	// 			else
-	// 				col = defL;
-
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	if(rIt.hasNext()) {
-	// 		col = defL;
-	// 		for(; i < this._numRows && rIt.hasNext(); i++) {
-	// 			if(rIt.value() == i) {
-	// 				row = 0;
-	// 				rIt.next();
-	// 			}
-	// 			else
-	// 				row = 1;
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	ag.increment(defL, this._numRows - i);
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	final int lhsNV = lhs.getNumValues();
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int retSize = lhsNV * rhsNV;
-	// 	final int nCol = lhs.getNumValues();
-	// 	IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	;
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = _indexes.getIterator();
-
-	// 	int i = 0;
-	// 	int col;
-	// 	int row;
-	// 	for(; i < this._numRows && lIt.hasNext() && rIt.hasNext(); i++) {
-	// 		if(lIt.value() == i) {
-	// 			col = 1;
-	// 			lIt.next();
-	// 		}
-	// 		else
-	// 			col = 0;
-	// 		if(rIt.value() == i) {
-	// 			row = 1;
-	// 			rIt.next();
-	// 		}
-	// 		else
-	// 			row = 0;
-	// 		ag.increment(col + row * nCol);
-	// 	}
-
-	// 	if(lIt.hasNext()) {
-	// 		row = 1;
-	// 		for(; i < _numRows && lIt.hasNext(); i++) {
-	// 			if(lIt.value() == i) {
-	// 				col = 1;
-	// 				lIt.next();
-	// 			}
-	// 			else
-	// 				col = 0;
-
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	if(rIt.hasNext()) {
-	// 		col = 1;
-	// 		for(; i < _numRows && rIt.hasNext(); i++) {
-	// 			if(rIt.value() == i) {
-	// 				row = 1;
-	// 				rIt.next();
-	// 			}
-	// 			else
-	// 				row = 0;
-	// 			ag.increment(col + row * nCol);
-	// 		}
-	// 	}
-
-	// 	ag.increment(0, _numRows - i);
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		throw new NotImplementedException();
-		// final AIterator it = _indexes.getIterator();
-		// final int offsetToDefault = this.getNumValues() - 1;
-		// final int nCol = that._colIndexes.length;
-
-		// int i = 0;
-
-		// for(; i < _numRows && it.hasNext(); i++) {
-		// int to = (it.value() == i) ? 1 : 0;
-		// that._dict.addToEntry(ret, that.getIndex(i), to, nCol);
-		// }
-
-		// for(; i < _numRows; i++)
-		// that._dict.addToEntry(ret, that.getIndex(i), 0, nCol);
-
-		// return ret;
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSDCSingleZeros.java
index ff9da0e..743bfff 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
@@ -110,38 +110,6 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		throw new NotImplementedException();
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	final double[] c = target.getDenseBlockValues();
-	// 	final double[] values = getValues();
-	// 	final AIterator it = _indexes.getIterator();
-	// 	while(it.hasNext()) {
-	// 		c[it.value()] += values[_colIndexes.length + colpos];
-	// 		it.next();
-	// 	}
-	// 	target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// 	final double[] values = getValues();
-	// 	final AIterator it = _indexes.getIterator(rl);
-	// 	while(it.hasNext() && it.value() < ru) {
-	// 		c[it.value() - rl] += values[colpos];
-	// 		it.next();
-	// 	}
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		int ix = Arrays.binarySearch(_colIndexes, c);
@@ -217,52 +185,8 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		return counts;
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int row) {
-	// 	final double[] vals = allocDVector(getNumValues(), true);
-	// 	final AIterator it = _indexes.getIterator();
-	// 	if(row > 0) {
-	// 		final int offT = _numRows * row;
-	// 		while(it.hasNext()) {
-	// 			final int i = it.value();
-	// 			vals[0] += a[i + offT];
-	// 			it.next();
-	// 		}
-	// 	}
-	// 	else
-	// 		while(it.hasNext()) {
-	// 			final int i = it.value();
-	// 			vals[0] += a[i];
-	// 			it.next();
-	// 		}
-
-	// 	return vals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// 	final double[] vals = allocDVector(getNumValues(), true);
-	// 	final int[] sbIndexes = sb.indexes(row);
-	// 	final double[] sparseV = sb.values(row);
-	// 	final AIterator it = _indexes.getIterator();
-	// 	final int sbEnd = sb.size(row) + sb.pos(row);
-
-	// 	int sbP = sb.pos(row);
-
-	// 	while(it.hasNext() && sbP < sbEnd) {
-	// 		if(it.value() == sbIndexes[sbP])
-	// 			vals[0] += sparseV[sbP++];
-	// 		if(sbP < sbEnd)
-	// 			it.skipTo(sbIndexes[sbP]);
-	// 		while(sbP < sbEnd && sbIndexes[sbP] < it.value())
-	// 			sbP++;
-	// 	}
-
-	// 	return vals;
-	// }
-
 	@Override
-	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru){
+	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
 			preAggregateSparse(m.getSparseBlock(), preAgg, rl, ru);
 		else
@@ -295,10 +219,18 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 			final int alen = sb.size(rowLeft) + apos;
 			final int[] aix = sb.indexes(rowLeft);
 			final double[] avals = sb.values(rowLeft);
-			for(int j = apos; j < alen; j++) {
-				it.skipTo(aix[j]);
-				if(it.value() == aix[j])
-					preAV[offOut] += avals[j];
+			int j = apos;
+			while(it.hasNext() && j < alen) {
+				final int index = aix[j];
+				final int v = it.value();
+				if(index < v)
+					j++;
+				else if(index == v){
+					preAV[offOut] += avals[j++];
+					it.next();
+				}
+				else
+					it.next();
 			}
 		}
 	}
@@ -334,15 +266,6 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		}
 	}
 
-	// private ADictionary swapEntries(ADictionary aDictionary) {
-	// double[] values = aDictionary.getValues().clone();
-	// double[] swap = new double[_colIndexes.length];
-	// System.arraycopy(values, 0, swap, 0, _colIndexes.length);
-	// System.arraycopy(values, _colIndexes.length, values, 0, _colIndexes.length);
-	// System.arraycopy(swap, 0, values, _colIndexes.length, _colIndexes.length);
-	// return new Dictionary(values);
-	// }
-
 	@Override
 	public void write(DataOutput out) throws IOException {
 		super.write(out);
@@ -381,88 +304,6 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		return sb.toString();
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator it = _indexes.getIterator();
-
-	// 	while(it.hasNext()) {
-	// 		final int col = lhs._data.getIndex(it.value());
-	// 		ag.increment(col);
-	// 	}
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// 	final int rhsNV = this.getNumValues();
-	// 	final int nCol = lhs.getNumValues();
-	// 	final int retSize = nCol * rhsNV;
-
-	// 	final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = this._indexes.getIterator();
-
-	// 	while(lIt.hasNext() && rIt.hasNext())
-	// 		if(lIt.value() == rIt.value()) {
-	// 			ag.increment(lhs.getIndex(lIt.getDataIndexAndIncrement()));
-	// 			rIt.next();
-	// 		}
-	// 		else if(lIt.value() < rIt.value())
-	// 			lIt.next();
-	// 		else
-	// 			rIt.next();
-
-	// 	return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// 	// we always know that there is only one value in each column group.
-	// 	int[] ret = new int[1];
-	// 	final AIterator lIt = lhs._indexes.getIterator();
-	// 	final AIterator rIt = this._indexes.getIterator();
-	// 	while(lIt.hasNext() && rIt.hasNext())
-	// 		if(lIt.value() == rIt.value()) {
-	// 			ret[0]++;
-	// 			lIt.next();
-	// 			rIt.next();
-	// 		}
-	// 		else if(lIt.value() < rIt.value())
-	// 			lIt.next();
-	// 		else
-	// 			rIt.next();
-
-	// 	return PreAggregateFactory.ag(ret);
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// 	throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// 		+ this.getClass().getSimpleName());
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		throw new NotImplementedException();
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 036d40e..9b3d704 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
@@ -136,34 +136,6 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		}
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// throw new NotImplementedException();
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// final double[] c = target.getDenseBlockValues();
-	// final double[] values = getValues();
-	// final AIterator it = _indexes.getIterator();
-	// while(it.hasNext())
-	// c[it.value()] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
-	// target.setNonZeros(getNumberNonZeros() / _colIndexes.length);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// throw new NotImplementedException();
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] c, int colpos, int rl, int ru) {
-	// final double[] values = getValues();
-	// final AIterator it = _indexes.getIterator(rl);
-	// while(it.hasNext() && it.value() < ru)
-	// c[it.value() - rl] += values[getIndex(it.getDataIndexAndIncrement()) * _colIndexes.length + colpos];
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		int ix = Arrays.binarySearch(_colIndexes, c);
@@ -242,48 +214,6 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		return _data.getIndex(r);
 	}
 
-	// @Override
-	// public double[] preAggregate(double[] a, int aRows) {
-	// final double[] vals = allocDVector(getNumValues(), true);
-	// final AIterator it = _indexes.getIterator();
-	// if(aRows > 0) {
-	// final int offT = _numRows * aRows;
-	// while(it.hasNext()) {
-	// final int i = it.value();
-	// vals[getIndex(it.getDataIndexAndIncrement())] += a[i + offT];
-	// }
-	// }
-	// else
-	// while(it.hasNext()) {
-	// final int i = it.value();
-	// vals[getIndex(it.getDataIndexAndIncrement())] += a[i];
-	// }
-
-	// return vals;
-	// }
-
-	// @Override
-	// public double[] preAggregateSparse(SparseBlock sb, int row) {
-	// final double[] vals = allocDVector(getNumValues(), true);
-	// final int[] sbIndexes = sb.indexes(row);
-	// final double[] sparseV = sb.values(row);
-	// final AIterator it = _indexes.getIterator();
-	// final int sbEnd = sb.size(row) + sb.pos(row);
-
-	// int sbP = sb.pos(row);
-
-	// while(it.hasNext() && sbP < sbEnd) {
-	// if(it.value() == sbIndexes[sbP])
-	// vals[getIndex(it.getDataIndexAndIncrement())] += sparseV[sbP++];
-	// if(sbP < sbEnd)
-	// it.skipTo(sbIndexes[sbP]);
-	// while(sbP < sbEnd && sbIndexes[sbP] < it.value())
-	// sbP++;
-	// }
-
-	// return vals;
-	// }
-
 	@Override
 	protected void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru) {
 		if(m.isInSparseFormat())
@@ -317,10 +247,16 @@ public class ColGroupSDCZeros extends ColGroupValue {
 			final int alen = sb.size(rowLeft) + apos;
 			final int[] aix = sb.indexes(rowLeft);
 			final double[] avals = sb.values(rowLeft);
-			for(int j = apos; j < alen; j++) {
-				it.skipTo(aix[j]);
-				if(it.value() == aix[j])
-					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += avals[j];
+			int j = apos;
+			while(it.hasNext() && j < alen) {
+				final int index = aix[j];
+				final int val = it.value();
+				if(index < val)
+					j++;
+				else if(index == val)
+					preAV[offOut + _data.getIndex(it.getDataIndexAndIncrement())] += avals[j++];
+				else
+					it.next();
 			}
 		}
 	}
@@ -399,133 +335,6 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		return sb.toString();
 	}
 
-	// @Override
-	// public IPreAggregate preAggregateDDC(ColGroupDDC lhs) {
-	// final int rhsNV = this.getNumValues();
-	// final int nCol = lhs.getNumValues();
-	// final int retSize = nCol * rhsNV;
-	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// final AIterator it = _indexes.getIterator();
-
-	// while(it.hasNext()) {
-	// final int col = lhs._data.getIndex(it.value());
-	// final int row = getIndex(it.getDataIndexAndIncrement());
-	// ag.increment(col + row * nCol);
-	// }
-	// return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDC(ColGroupSDC lhs) {
-	// final int rhsNV = this.getNumValues();
-	// final int nCol = lhs.getNumValues();
-
-	// final int defL = nCol - 1;
-	// final int retSize = nCol * rhsNV;
-
-	// IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// AIterator lIt = lhs._indexes.getIterator();
-	// AIterator rIt = this._indexes.getIterator();
-
-	// while(lIt.hasNext() && rIt.hasNext())
-	// if(lIt.value() == rIt.value())
-	// ag.increment(
-	// lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-	// else if(lIt.value() > rIt.value())
-	// ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-	// else
-	// lIt.next();
-
-	// while(rIt.hasNext())
-	// ag.increment(defL + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-
-	// return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs) {
-	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// + this.getClass().getSimpleName());
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs) {
-	// final int rhsNV = this.getNumValues();
-	// final int nCol = lhs.getNumValues();
-	// final int retSize = nCol * rhsNV;
-
-	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// final AIterator lIt = lhs._indexes.getIterator();
-	// final AIterator rIt = _indexes.getIterator();
-
-	// while(lIt.hasNext() && rIt.hasNext())
-	// if(lIt.value() == rIt.value())
-	// ag.increment(
-	// lhs.getIndex(lIt.getDataIndexAndIncrement()) + getIndex(rIt.getDataIndexAndIncrement()) * nCol);
-	// else if(lIt.value() < rIt.value())
-	// lIt.next();
-	// else
-	// rIt.next();
-
-	// return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs) {
-	// final int rhsNV = this.getNumValues();
-	// final int nCol = lhs.getNumValues();
-	// final int retSize = nCol * rhsNV;
-	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-	// final AIterator lIt = lhs._indexes.getIterator();
-	// final AIterator rIt = _indexes.getIterator();
-
-	// while(lIt.hasNext() && rIt.hasNext())
-	// if(lIt.value() == rIt.value()) {
-	// ag.increment(getIndex(rIt.getDataIndexAndIncrement()));
-	// lIt.next();
-	// }
-	// else if(lIt.value() < rIt.value())
-	// lIt.next();
-	// else
-	// rIt.next();
-
-	// return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateOLE(ColGroupOLE lhs) {
-	// final int NVR = this.getNumValues();
-	// final int NVL = lhs.getNumValues();
-	// final int retSize = NVR * NVL;
-	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// final IPreAggregate ag = PreAggregateFactory.ag(retSize);
-
-	// for(int kl = 0; kl < NVL; kl++) {
-	// final AIterator rIt = _indexes.getIterator();
-	// final int bOffL = lhs._ptr[kl];
-	// final int bLenL = lhs.len(kl);
-	// for(int bixL = 0, offL = 0, sLenL = 0; rIt.hasNext() && bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
-	// sLenL = lhs._data[bOffL + bixL];
-	// for(int i = 1; rIt.hasNext() && i <= sLenL; i++) {
-	// final int col = offL + lhs._data[bOffL + bixL + i];
-	// rIt.skipTo(col);
-	// if(rIt.value() == col)
-	// ag.increment(kl + getIndex(rIt.getDataIndexAndIncrement()) * NVL);
-
-	// }
-	// }
-	// }
-	// return ag;
-	// }
-
-	// @Override
-	// public IPreAggregate preAggregateRLE(ColGroupRLE lhs) {
-	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// + this.getClass().getSimpleName());
-	// }
-
 	@Override
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		final AIterator itThis = _indexes.getIterator();
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 6e8524e..02c4e82 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
@@ -195,44 +195,6 @@ public class ColGroupUncompressed extends AColGroup {
 		}
 	}
 
-	// @Override
-	// public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos) {
-	// 	double[] c = target.getDenseBlockValues();
-	// 	int nnz = 0;
-	// 	int off = colpos;
-	// 	if(_data.isInSparseFormat()) {
-	// 		for(int i = 0; i < _data.getNumRows(); i++) {
-	// 			c[i] += _data.quickGetValue(i, colpos);
-	// 			if(c[i] != 0)
-	// 				nnz++;
-	// 		}
-	// 	}
-	// 	else {
-	// 		double[] denseValues = _data.getDenseBlockValues();
-	// 		for(int i = 0; i < _data.getNumRows(); i++, off += _colIndexes.length) {
-	// 			c[i] += denseValues[off];
-	// 			if(c[i] != 0)
-	// 				nnz++;
-	// 		}
-	// 	}
-	// 	target.setNonZeros(nnz);
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(MatrixBlock target, int colpos, int rl, int ru) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
-	// @Override
-	// public void decompressColumnToBlock(double[] target, int colpos, int rl, int ru) {
-	// 	throw new NotImplementedException("Not Implemented");
-	// }
-
 	@Override
 	public double get(int r, int c) {
 		final int ix = Arrays.binarySearch(_colIndexes, c);
@@ -652,4 +614,10 @@ public class ColGroupUncompressed extends AColGroup {
 	public int getNumValues() {
 		return _data.getNumRows();
 	}
+
+	@Override
+	public AColGroup replace(double pattern, double replace) {
+		MatrixBlock replaced = _data.replaceOperations(new MatrixBlock(), pattern, replace);
+		return new ColGroupUncompressed(_colIndexes, replaced);
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index af9ab37..9477989 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -91,7 +91,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	}
 
 	@Override
-	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
+	public final void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT) {
 		if(_dict instanceof MatrixBlockDictionary) {
 			final MatrixBlockDictionary md = (MatrixBlockDictionary) _dict;
 			final MatrixBlock mb = md.getMatrixBlock();
@@ -106,9 +106,27 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			decompressToBlockUnSafeDenseDictionary(target, rl, ru, offT, _dict.getValues());
 	}
 
+	/**
+	 * Decompress to block using a sparse dictionary to lookup into.
+	 * 
+	 * @param target The dense target block to decompress into
+	 * @param rl     The row to start decompression from
+	 * @param ru     The row to end decompression at
+	 * @param offT   The offset into target block to decompress to (use full if the target it a multi block matrix)
+	 * @param sb     the sparse dictionary block to take value tuples from
+	 */
 	protected abstract void decompressToBlockUnSafeSparseDictionary(MatrixBlock target, int rl, int ru, int offT,
 		SparseBlock sb);
 
+	/**
+	 * Decompress to block using a dense dictionary to lookup into.
+	 * 
+	 * @param target The dense target block to decompress into
+	 * @param rl     The row to start decompression from
+	 * @param ru     The row to end decompression at
+	 * @param offT   The offset into target block to decompress to (use full if the target it a multi block matrix)
+	 * @param values The dense dictionary values, linearized row major.
+	 */
 	protected abstract void decompressToBlockUnSafeDenseDictionary(MatrixBlock target, int rl, int ru, int offT,
 		double[] values);
 
@@ -131,10 +149,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		_dict.addMaxAndMin(ret, _colIndexes);
 	}
 
-	protected final void setDictionary(ADictionary dict) {
-		_dict = dict;
-	}
-
 	@Override
 	public final MatrixBlock getValuesAsBlock() {
 		_dict = _dict.getAsMatrixBlockDictionary(_colIndexes.length);
@@ -461,7 +475,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	public AColGroup copyAndSet(ADictionary newDictionary) {
 		ColGroupValue clone = (ColGroupValue) this.clone();
-		clone.setDictionary(newDictionary);
+		clone._dict = newDictionary;
 		return clone;
 	}
 
@@ -471,7 +485,7 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	public AColGroup copyAndSet(int[] colIndexes, ADictionary newDictionary) {
 		ColGroupValue clone = (ColGroupValue) this.clone();
-		clone.setDictionary(newDictionary);
+		clone._dict = newDictionary;
 		clone.setColIndices(colIndexes);
 		return clone;
 	}
@@ -533,56 +547,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 */
 	protected abstract void preAggregate(MatrixBlock m, MatrixBlock preAgg, int rl, int ru);
 
-	public abstract int getIndexStructureHash();
-
-	// private IPreAggregate preAggregate(ColGroupValue lhs) {
-	// IPreAggregate r = preCallAggregate(lhs);
-	// return r;
-	// }
-
-	// private IPreAggregate preCallAggregate(ColGroupValue lhs) {
-	// // (lhs.getClass().getSimpleName() + " in " + this.getClass().getSimpleName() + " "
-	// // + Arrays.toString(lhs.getColIndices()) + " " + Arrays.toString(this.getColIndices()));
-
-	// if(lhs instanceof ColGroupDDC)
-	// return preAggregateDDC((ColGroupDDC) lhs);
-	// else if(lhs instanceof ColGroupSDC)
-	// return preAggregateSDC((ColGroupSDC) lhs);
-	// else if(lhs instanceof ColGroupSDCSingle)
-	// return preAggregateSDCSingle((ColGroupSDCSingle) lhs);
-	// else if(lhs instanceof ColGroupSDCZeros)
-	// return preAggregateSDCZeros((ColGroupSDCZeros) lhs);
-	// else if(lhs instanceof ColGroupSDCSingleZeros)
-	// return preAggregateSDCSingleZeros((ColGroupSDCSingleZeros) lhs);
-	// else if(lhs instanceof ColGroupOLE)
-	// return preAggregateOLE((ColGroupOLE) lhs);
-	// else if(lhs instanceof ColGroupRLE)
-	// return preAggregateRLE((ColGroupRLE) lhs);
-	// else if(lhs instanceof ColGroupConst)
-	// return preAggregateCONST((ColGroupConst) lhs);
-
-	// throw new NotImplementedException("Not supported pre aggregate of :" + lhs.getClass().getSimpleName() + " in "
-	// + this.getClass().getSimpleName());
-	// }
-
-	// public IPreAggregate preAggregateCONST(ColGroupConst lhs) {
-	// 	return new ArrPreAggregate(getCounts());
-	// }
-
-	// public abstract IPreAggregate preAggregateDDC(ColGroupDDC lhs);
-
-	// public abstract IPreAggregate preAggregateSDC(ColGroupSDC lhs);
-
-	// public abstract IPreAggregate preAggregateSDCSingle(ColGroupSDCSingle lhs);
-
-	// public abstract IPreAggregate preAggregateSDCZeros(ColGroupSDCZeros lhs);
-
-	// public abstract IPreAggregate preAggregateSDCSingleZeros(ColGroupSDCSingleZeros lhs);
-
-	// public abstract IPreAggregate preAggregateOLE(ColGroupOLE lhs);
-
-	// public abstract IPreAggregate preAggregateRLE(ColGroupRLE lhs);
-
 	/**
 	 * Pre aggregate into a dictionary. It is assumed that "that" have more distinct values than, "this".
 	 * 
@@ -611,18 +575,32 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 			+ that.getClass().getSimpleName() + " in " + this.getClass().getSimpleName());
 	}
 
-	public abstract Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret);
+	protected int getIndexStructureHash() {
+		throw new NotImplementedException("This base function should not be called");
+	}
+
+	protected Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
+		throw new DMLCompressionException("Does not make sense to call this, implement function for sub class");
+	}
 
-	public abstract Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified);
+	protected Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
+		throw new DMLCompressionException("Does not make sense to call this, implement function for sub class");
+	}
 
-	public abstract Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret);
+	protected Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
+		throw new DMLCompressionException("Does not make sense to call this, implement function for sub class");
+	}
 
-	public abstract Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret);
+	protected Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
+		throw new DMLCompressionException("Does not make sense to call this, implement function for sub class");
+	}
 
-	public abstract Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret,
-		boolean preModified);
+	protected Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret,
+		boolean preModified) {
+		throw new DMLCompressionException("Does not make sense to call this, implement function for sub class");
+	}
 
-	public Dictionary preAggregateThatConstStructure(ColGroupConst that, Dictionary ret) {
+	protected Dictionary preAggregateThatConstStructure(ColGroupConst that, Dictionary ret) {
 		computeColSums(ret.getValues(), false);
 		return ret;
 	}
@@ -1067,8 +1045,12 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	private MatrixBlock leftMultByMatrixIntermediateMatrix(MatrixBlock matrix, int rl, int ru) {
 		// Get dictionary.
 		MatrixBlock dictM = forceMatrixBlockDictionary().getMatrixBlock();
+
 		// Allocate temporary matrix to multiply into.
-		MatrixBlock tmpRes = new MatrixBlock(matrix.getNumRows(), _colIndexes.length, false);
+		final int tmpCol = _colIndexes.length;
+		final int tmpRow = matrix.getNumRows();
+		MatrixBlock tmpRes = new MatrixBlock(tmpRow, tmpCol, false);
+		
 		// Pre aggregate the matrix into same size as dictionary
 		MatrixBlock preAgg = preAggregate(matrix, rl, ru);
 
@@ -1197,4 +1179,10 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		size += _dict.getInMemorySize();
 		return size;
 	}
+
+	@Override
+	public AColGroup replace(double pattern, double replace) {
+		ADictionary replaced = _dict.replace(pattern, replace, _colIndexes.length, _zeros);
+		return copyAndSet(replaced);
+	}
 }
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 df6f648..5b6b28c 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
@@ -385,4 +385,18 @@ public abstract class ADictionary {
 	 */
 	public abstract void preaggValuesFromDense(final int numVals, final int[] colIndexes, final int[] aggregateColumns,
 		final double[] b, final double[] ret, final int cut);
+
+	/**
+	 * Make a copy of the values, and replace all values that match pattern with replacement value. If needed add a new
+	 * column index.
+	 * 
+	 * @param pattern The value to look for
+	 * @param replace The value to replace the other value with
+	 * @param nCol    The number of columns contained in the dictionary.
+	 * @param safe    Specify if the operation require consideration of adding a new tuple. This happens if the
+	 *                dictionary have allocated the last zero tuple or not.
+	 * @return A new Column Group, reusing the index structure but with new values.
+	 */
+	public abstract ADictionary replace(double pattern, double replace, int nCol, boolean safe);
+
 }
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 80872d5..7c975d6 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
@@ -24,6 +24,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.commons.lang.NotImplementedException;
 import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
@@ -487,4 +488,21 @@ public class Dictionary extends ADictionary {
 			}
 		}
 	}
+
+	@Override
+	public ADictionary replace(double pattern, double replace, int nCol, boolean safe) {
+		if(!safe && replace == 0)
+			throw new NotImplementedException("Not implemented Replacement of 0");
+		else {
+			double[] retV = new double[_values.length];
+			for(int i = 0; i < _values.length; i++) {
+				final double v = _values[i];
+				if(v == pattern)
+					retV[i] = replace;
+				else
+					retV[i] = v;
+			}
+			return new Dictionary(retV);
+		}
+	}
 }
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 4c1ab04..69440a8 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
@@ -639,4 +639,14 @@ public class MatrixBlockDictionary extends ADictionary {
 			}
 		}
 	}
+
+	@Override
+	public ADictionary replace(double pattern, double replace, int nCol, boolean safe) {
+		if(!safe && replace == 0)
+			throw new NotImplementedException("Not implemented Replacement of 0");
+		else {
+			MatrixBlock ret = _data.replaceOperations(new MatrixBlock(), pattern, replace);
+			return new MatrixBlockDictionary(ret);
+		}
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index 6055f69..a0e1b29 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -506,4 +506,9 @@ public class QDictionary extends ADictionary {
 		int cut) {
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public ADictionary replace(double pattern, double replace, int nCol, boolean safe) {
+		throw new NotImplementedException();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index 143cfb6..8aa1e98 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -240,6 +240,7 @@ public class CLALibLeftMultBy {
 				ExecutorService pool = CommonThreadPool.get(k);
 				// compute remaining compressed column groups in parallel
 				ArrayList<Callable<Object>> tasks = new ArrayList<>();
+				// int rowBlockSize = Math.min(that.getNumRows(), 16);
 				int rowBlockSize = 1;
 				if(overlapping) {
 					for(int blo = 0; blo < that.getNumRows(); blo += rowBlockSize) {
@@ -330,7 +331,7 @@ public class CLALibLeftMultBy {
 		public Object call() {
 
 			try {
-				ColGroupValue.setupThreadLocalMemory(_v.getLeft());
+				ColGroupValue.setupThreadLocalMemory(_v.getLeft() * (_ru - _rl));
 				_group.leftMultByMatrix(_that, _ret, _rl, _ru);
 			}
 			catch(Exception e) {
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
index fe25916..ce33b22 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixBlock.java
@@ -3781,18 +3781,8 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	}
 
 	public MatrixBlock chainMatrixMultOperations( MatrixBlock v, MatrixBlock w, MatrixBlock out, ChainType ctype, int k ) {
-		//check for transpose type
-		if( !(ctype == ChainType.XtXv || ctype == ChainType.XtwXv || ctype == ChainType.XtXvy) )
-			throw new DMLRuntimeException("Invalid mmchain type '"+ctype.toString()+"'.");
-		
-		//check for matching dimensions
-		if( this.getNumColumns() != v.getNumRows() )
-			throw new DMLRuntimeException("Dimensions mismatch on mmchain operation ("+this.getNumColumns()+" != "+v.getNumRows()+")");
-		if( v.getNumColumns() != 1 )
-			throw new DMLRuntimeException("Invalid input vector (column vector expected, but ncol="+v.getNumColumns()+")");
-		if( w!=null && w.getNumColumns() != 1 )
-			throw new DMLRuntimeException("Invalid weight vector (column vector expected, but ncol="+w.getNumColumns()+")");
-		
+		checkMMChain(ctype, v, w);
+
 		//prepare result
 		if( out != null )
 			out.reset(clen, 1, false);
@@ -3808,6 +3798,21 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 		return out;
 	}
 
+	protected void checkMMChain(ChainType ctype, MatrixBlock v, MatrixBlock w){
+		//check for transpose type
+		if( !(ctype == ChainType.XtXv || ctype == ChainType.XtwXv || ctype == ChainType.XtXvy) )
+			throw new DMLRuntimeException("Invalid mmchain type '"+ctype.toString()+"'.");
+
+		//check for matching dimensions
+		if( this.getNumColumns() != v.getNumRows() )
+			throw new DMLRuntimeException("Dimensions mismatch on mmchain operation ("+this.getNumColumns()+" != "+v.getNumRows()+")");
+		if( v.getNumColumns() != 1 )
+			throw new DMLRuntimeException("Invalid input vector (column vector expected, but ncol="+v.getNumColumns()+")");
+		if( w!=null && w.getNumColumns() != 1 )
+			throw new DMLRuntimeException("Invalid weight vector (column vector expected, but ncol="+w.getNumColumns()+")");
+			
+	}
+
 	public void permutationMatrixMultOperations( MatrixValue m2Val, MatrixValue out1Val, MatrixValue out2Val ) {
 		permutationMatrixMultOperations(m2Val, out1Val, out2Val, 1);
 	}
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
index 1c7e6e9..68e37f7 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedMatrixTest.java
@@ -431,7 +431,6 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 	@Test
 	public void testCompressionStatisticsToString() {
 		try {
-
 			if(cmbStats != null) {
 				String st = cmbStats.toString();
 				assertTrue(st.contains("CompressionStatistics"));
@@ -439,7 +438,7 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 		catch(Exception e) {
 			e.printStackTrace();
-			throw new DMLRuntimeException("Error in printing Compression Statistics");
+			throw new DMLRuntimeException(e);
 		}
 	}
 
@@ -454,7 +453,7 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 		catch(Exception e) {
 			e.printStackTrace();
-			throw new DMLRuntimeException("Error in printing Compression Statistics");
+			throw new DMLRuntimeException(e);
 		}
 	}
 
@@ -469,7 +468,22 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 		catch(Exception e) {
 			e.printStackTrace();
-			throw new DMLRuntimeException("Error in printing Compression Statistics");
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	@Test
+	public void testReplace() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock) || rows * cols > 10000)
+				return;
+			MatrixBlock ret1 = cmb.replaceOperations(new MatrixBlock(), min - 1, 1425);
+			MatrixBlock ret2 = mb.replaceOperations(new MatrixBlock(), min - 1, 1425);
+			compareResultMatrices(ret2, ret1, 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new DMLRuntimeException(e);
 		}
 	}
 
@@ -485,7 +499,7 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 		catch(Exception e) {
 			e.printStackTrace();
-			throw new DMLRuntimeException("Error in printing Compression Statistics");
+			throw new DMLRuntimeException(e);
 		}
 	}
 
@@ -501,7 +515,7 @@ public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
 		}
 		catch(Exception e) {
 			e.printStackTrace();
-			throw new DMLRuntimeException("Error in printing Compression Statistics");
+			throw new DMLRuntimeException(e);
 		}
 	}
 

[systemds] 01/07: [SYSTEMDS-2999] CLA Decompression Unification

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

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

commit bc2602f1a24de39328424aaa90dce40509152031
Author: baunsgaard <ba...@tugraz.at>
AuthorDate: Tue May 4 20:54:23 2021 +0200

    [SYSTEMDS-2999] CLA Decompression Unification
    
    This commit change the blocking of the decompression, to no longer align
    perfectly with 64k blocks, since if a column group contain many columns
    this is sub optimal.
    
    [SYSTEMDS-3000] CLA MM Most Common Element Addition
    
    This commit adds an exploitation of the compressed representation
    that allows add the most common element when multiplying on the
    left side with a compressed transposed matrix.
    This is a common occurrence in MMChain and TSMM and allows sparsity
    exploitation of dense compressed column groups.
---
 .../runtime/compress/CompressedMatrixBlock.java    |  73 ++-
 .../compress/CompressionSettingsBuilder.java       |   2 +-
 .../sysds/runtime/compress/cocode/CoCodeCost.java  |   8 +-
 .../sysds/runtime/compress/colgroup/AColGroup.java |   3 +-
 .../compress/colgroup/ColGroupCompressed.java      |  33 +-
 .../runtime/compress/colgroup/ColGroupConst.java   |   7 +-
 .../runtime/compress/colgroup/ColGroupDDC.java     | 138 ++---
 .../runtime/compress/colgroup/ColGroupEmpty.java   |   6 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |  48 +-
 .../runtime/compress/colgroup/ColGroupOLE.java     | 563 ++++++++++-----------
 .../runtime/compress/colgroup/ColGroupOffset.java  |  14 -
 .../runtime/compress/colgroup/ColGroupRLE.java     | 443 ++++++++--------
 .../runtime/compress/colgroup/ColGroupSDC.java     |  84 ++-
 .../compress/colgroup/ColGroupSDCSingle.java       | 129 ++++-
 .../compress/colgroup/ColGroupSDCSingleZeros.java  |  26 +-
 .../compress/colgroup/ColGroupSDCZeros.java        |  11 +-
 .../compress/colgroup/ColGroupUncompressed.java    |  98 ++--
 .../runtime/compress/colgroup/ColGroupValue.java   | 289 +++++++----
 .../compress/colgroup/dictionary/ADictionary.java  |  27 +-
 .../compress/colgroup/dictionary/Dictionary.java   |  85 +++-
 .../compress/colgroup/dictionary/QDictionary.java  |  39 +-
 .../sysds/runtime/compress/lib/CLALibCompAgg.java  |   1 -
 .../runtime/compress/lib/CLALibLeftMultBy.java     |  63 +--
 .../runtime/compress/lib/CLALibRightMultBy.java    |  82 +--
 .../sysds/runtime/compress/lib/CLALibScalar.java   |   3 +-
 .../sysds/runtime/matrix/data/MatrixValue.java     |   4 +-
 .../component/compress/CompressedTestBase.java     |  23 +-
 .../test/component/compress/TestConstants.java     |   2 +-
 28 files changed, 1355 insertions(+), 949 deletions(-)

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 473a1ec..bb7781e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -199,7 +199,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		Timing time = new Timing(true);
 
 		// preallocation sparse rows to avoid repeated reallocations
-		MatrixBlock ret = new MatrixBlock(rlen, clen, false, -1);
+		MatrixBlock ret = getUncompressedColGroupAndRemoveFromListOfColGroups();
+		if(ret != null && getColGroups().size() == 0)
+			return ret;
+		else if(ret == null)
+			ret = new MatrixBlock(rlen, clen, false, -1);
 
 		ret.allocateDenseBlock();
 		decompress(ret);
@@ -244,7 +248,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return decompress();
 
 		Timing time = new Timing(true);
-		MatrixBlock ret = new MatrixBlock(rlen, clen, false, -1).allocateBlock();
+		MatrixBlock ret = getUncompressedColGroupAndRemoveFromListOfColGroups();
+		if(ret != null && getColGroups().size() == 0)
+			return ret;
+		else if(ret == null)
+			ret = new MatrixBlock(rlen, clen, false, -1);
 		ret.allocateDenseBlock();
 		decompress(ret, k);
 
@@ -258,19 +266,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	public MatrixBlock decompress(MatrixBlock ret, int k) {
-
-		if(nonZeros == -1)
-			ret.setNonZeros(this.recomputeNonZeros());
-		else
-			ret.setNonZeros(nonZeros);
 		try {
 			ExecutorService pool = CommonThreadPool.get(k);
 			int rlen = getNumRows();
 			final int blkz = CompressionSettings.BITMAP_BLOCK_SZ;
-			int blklen = (int) Math.ceil((double) rlen / k);
-			blklen += (blklen % blkz != 0) ? blkz - blklen % blkz : 0;
+			int blklen = (int) Math.max(64, Math.ceil((double) (blkz) / getNumColumns()));
 			ArrayList<DecompressTask> tasks = new ArrayList<>();
-			for(int i = 0; i < k & i * blklen < getNumRows(); i++)
+			for(int i = 0; i * blklen < getNumRows(); i++)
 				tasks.add(new DecompressTask(_colGroups, ret, i * blklen, Math.min((i + 1) * blklen, rlen),
 					overlappingColGroups));
 			List<Future<Long>> rtasks = pool.invokeAll(tasks);
@@ -285,6 +287,34 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			ret.recomputeNonZeros();
 			ret.examSparsity();
 		}
+		else if(nonZeros == -1)
+			ret.setNonZeros(this.recomputeNonZeros());
+		else
+			ret.setNonZeros(nonZeros);
+		return ret;
+	}
+
+	private MatrixBlock getUncompressedColGroupAndRemoveFromListOfColGroups() {
+		// If we have a uncompressed column group that covers all of the matrix,
+		// it makes sense to use as the decompression target.
+		MatrixBlock ret = null;
+		// It is only relevant if we are in overlapping state, or we only have a Uncompressed ColumnGroup left.
+		if(isOverlapping() || _colGroups.size() == 1) {
+			for(int i = 0; i < _colGroups.size(); i++) {
+				AColGroup g = _colGroups.get(i);
+				if(g instanceof ColGroupUncompressed) {
+					// Find an Uncompressed ColumnGroup
+					ColGroupUncompressed guc = (ColGroupUncompressed) g;
+					MatrixBlock gMB = guc.getData();
+					// Make sure that it is the correct dimensions
+					if(gMB.getNumColumns() == this.getNumColumns() && gMB.getNumRows() == this.getNumRows()) {
+						_colGroups.remove(i);
+						return gMB;
+					}
+				}
+			}
+		}
+
 		return ret;
 	}
 
@@ -480,21 +510,22 @@ public class CompressedMatrixBlock extends MatrixBlock {
 
 		// compute matrix mult
 
-		boolean tryOverlapOutput = v.getNumColumns() > _colGroups.size() && w != null && w.getNumRows() > 1;
-		MatrixBlock tmp = CLALibRightMultBy.rightMultByMatrix(this, v, null, k, tryOverlapOutput);
+		// boolean tryOverlapOutput = v.getNumColumns() > _colGroups.size();
+		MatrixBlock tmp = CLALibRightMultBy.rightMultByMatrix(this, v, null, k, true);
 
-		if(tmp instanceof CompressedMatrixBlock) {
-			CompressedMatrixBlock tmpC = (CompressedMatrixBlock) tmp;
-			if(ctype == ChainType.XtwXv)
-				tmpC = (CompressedMatrixBlock) CLALibBinaryCellOp.binaryOperations(bop, tmpC, w, null);
-			tmp = tmpC.decompress(k);
+		if(ctype == ChainType.XtwXv) {
+			if(tmp instanceof CompressedMatrixBlock)
+				tmp = CLALibBinaryCellOp.binaryOperations(bop, (CompressedMatrixBlock) tmp, w, null);
+			else
+				LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
 		}
-		else if(ctype == ChainType.XtwXv)
-			LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
 
-		CLALibLeftMultBy.leftMultByMatrixTransposed(this, tmp, out, k);
-		out = LibMatrixReorg.transposeInPlace(out, k);
+		if(tmp instanceof CompressedMatrixBlock)
+			CLALibLeftMultBy.leftMultByMatrixTransposed(this, (CompressedMatrixBlock) tmp, out, k);
+		else
+			CLALibLeftMultBy.leftMultByMatrixTransposed(this, tmp, out, k);
 
+		out = LibMatrixReorg.transposeInPlace(out, k);
 		out.recomputeNonZeros();
 		return out;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
index 388836f..a53bcfa 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -39,7 +39,7 @@ public class CompressionSettingsBuilder {
 	private boolean investigateEstimate = true;
 	private boolean lossy = false;
 	private EnumSet<CompressionType> validCompressions;
-	private boolean sortValuesByLength = false;
+	private boolean sortValuesByLength = true;
 	private PartitionerType columnPartitioner;
 	private int maxStaticColGroupCoCode = 10;
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
index 2cbc42f..1cce5b1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/CoCodeCost.java
@@ -48,11 +48,15 @@ public class CoCodeCost extends AColumnCoCoder {
 	 */
 	private final int largestDistinct;
 
-	private final static int toSmallForAnalysis = 64;
+	private final int toSmallForAnalysis;
+
+	private final double percentMaxCardinality = 0.08;
 
 	protected CoCodeCost(CompressedSizeEstimator sizeEstimator, CompressionSettings cs, int numRows) {
 		super(sizeEstimator, cs, numRows);
-		largestDistinct = Math.min(4096, Math.max(256, (int) (_numRows * 0.01)));
+		largestDistinct = Math.max(256, (int) (_numRows * percentMaxCardinality));
+		toSmallForAnalysis = largestDistinct / 4;
+		LOG.error("CocodeCost largest Distinct: "+ largestDistinct + " toSmallForAnalysis: " + toSmallForAnalysis);
 	}
 
 	@Override
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/AColGroup.java
index 2f3bdff..8de274f 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
@@ -60,6 +60,7 @@ public abstract class AColGroup implements Serializable {
 
 		/**
 		 * Get the super type of the specific ColGroup Type used.
+		 * 
 		 * @param c The concrete ColGroupType
 		 * @return The super CompressionType.
 		 */
@@ -751,8 +752,8 @@ public abstract class AColGroup implements Serializable {
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
+		sb.append(" ColGroupType: ");
 		sb.append(this.getClass().getSimpleName());
-		sb.append("\n");
 		sb.append(String.format("\n%15s%5d ", "Columns:", _colIndexes.length));
 		sb.append(Arrays.toString(_colIndexes));
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
index a231283..c8f9a41 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupCompressed.java
@@ -89,16 +89,6 @@ public abstract class ColGroupCompressed extends AColGroup {
 
 	protected abstract boolean sameIndexStructure(ColGroupCompressed that);
 
-	public void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru) {
-		if(matrix.isEmpty())
-			return;
-		else if(matrix.isInSparseFormat())
-			leftMultBySparseMatrix(matrix.getSparseBlock(), result, matrix.getNumRows(), numCols, rl, ru);
-		else {
-			leftMultByMatrix(matrix.getDenseBlockValues(), result, matrix.getNumRows(), numCols, rl, ru);
-		}
-	}
-
 	/**
 	 * Multiply with a matrix on the left.
 	 * 
@@ -125,22 +115,33 @@ public abstract class ColGroupCompressed extends AColGroup {
 		int ru);
 
 	@Override
-	public double getMin() {
+	public final void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru) {
+		if(matrix.isEmpty())
+			return;
+		else if(matrix.isInSparseFormat())
+			leftMultBySparseMatrix(matrix.getSparseBlock(), result, matrix.getNumRows(), numCols, rl, ru);
+		else {
+			leftMultByMatrix(matrix.getDenseBlockValues(), result, matrix.getNumRows(), numCols, rl, ru);
+		}
+	}
+
+	@Override
+	public final double getMin() {
 		return computeMxx(Double.POSITIVE_INFINITY, Builtin.getBuiltinFnObject(BuiltinCode.MIN));
 	}
 
 	@Override
-	public double getMax() {
+	public final double getMax() {
 		return computeMxx(Double.NEGATIVE_INFINITY, Builtin.getBuiltinFnObject(BuiltinCode.MAX));
 	}
 
 	@Override
-	public void unaryAggregateOperations(AggregateUnaryOperator op, double[] c) {
+	public final void unaryAggregateOperations(AggregateUnaryOperator op, double[] c) {
 		unaryAggregateOperations(op, c, 0, _numRows);
 	}
 
 	@Override
-	public void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int rl, int ru) {
+	public final void unaryAggregateOperations(AggregateUnaryOperator op, double[] c, int rl, int ru) {
 		// sum and sumsq (reduceall/reducerow over tuples and counts)
 		if(op.aggOp.increOp.fn instanceof Plus || op.aggOp.increOp.fn instanceof KahanPlus ||
 			op.aggOp.increOp.fn instanceof KahanPlusSq) {
@@ -174,13 +175,13 @@ public abstract class ColGroupCompressed extends AColGroup {
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
+		sb.append(" num Rows: " + getNumRows());
 		sb.append(super.toString());
-		sb.append("num Rows: " + getNumRows());
 		return sb.toString();
 	}
 
 	@Override
-	public int getNumRows() {
+	public final int getNumRows() {
 		return _numRows;
 	}
 
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 75ed773..eb38ee0 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
@@ -300,7 +300,7 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new DMLCompressionException("Does not make sense to call this");
 	}
 
@@ -315,6 +315,11 @@ public class ColGroupConst extends ColGroupValue {
 	}
 
 	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+		throw new DMLCompressionException("Does not make sense to call this");
+	}
+
+	@Override
 	protected int containsAllZeroTuple() {
 		return -1;
 	}
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 e9e98d7..aa8a55f 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
@@ -71,11 +71,11 @@ public class ColGroupDDC extends ColGroupValue {
 	public void decompressToBlockUnSafe(MatrixBlock target, int rl, int ru, int offT, double[] values) {
 		final int nCol = _colIndexes.length;
 		final int tCol = target.getNumColumns();
-		double[] c = target.getDenseBlockValues();
+		final double[] c = target.getDenseBlockValues();
 		offT = offT * tCol;
 
 		for(int i = rl; i < ru; i++, offT += tCol) {
-			int rowIndex = getIndex(i) * nCol;
+			final int rowIndex = _data.getIndex(i) * nCol;
 			for(int j = 0; j < nCol; j++)
 				c[offT + _colIndexes[j]] += values[rowIndex + j];
 		}
@@ -87,7 +87,7 @@ public class ColGroupDDC extends ColGroupValue {
 		int ncol = getNumCols();
 		double[] dictionary = getValues();
 		for(int i = 0; i < _numRows; i++) {
-			int rowIndex = getIndex(i) * ncol;
+			int rowIndex = _data.getIndex(i) * ncol;
 			for(int colIx = 0; colIx < ncol; colIx++) {
 				int origMatrixColIx = getColIndex(colIx);
 				int col = colIndexTargets[origMatrixColIx];
@@ -105,7 +105,7 @@ public class ColGroupDDC extends ColGroupValue {
 		double[] values = getValues();
 		int nnz = 0;
 		for(int i = 0; i < _numRows; i++) {
-			int index = getIndex(i);
+			int index = _data.getIndex(i);
 			if(index < getNumValues())
 				nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
 			else
@@ -123,7 +123,7 @@ public class ColGroupDDC extends ColGroupValue {
 		final int numValues = getNumValues();
 		int nnz = 0;
 		for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-			int index = getIndex(r);
+			int index = _data.getIndex(r);
 			if(index < numValues)
 				nnz += ((c[i] += values[(index) * ncol + colpos]) != 0) ? 1 : 0;
 			else
@@ -138,7 +138,7 @@ public class ColGroupDDC extends ColGroupValue {
 		double[] values = getValues();
 		final int numValues = getNumValues();
 		for(int i = 0, r = rl; i < ru - rl; i++, r++) {
-			int index = getIndex(r);
+			int index = _data.getIndex(r);
 			if(index < numValues)
 				c[i] += values[(index) * ncol + colpos];
 		}
@@ -152,7 +152,7 @@ public class ColGroupDDC extends ColGroupValue {
 			throw new RuntimeException("Column index " + c + " not in DDC group.");
 
 		// get value
-		int index = getIndex(r);
+		int index = _data.getIndex(r);
 		if(index < getNumValues())
 			return _dict.getValue(index * _colIndexes.length + ix);
 		else
@@ -167,7 +167,7 @@ public class ColGroupDDC extends ColGroupValue {
 		double[] values = _dict.getValues();
 		for(int i = rl; i < ru; i++) {
 			int lnnz = 0;
-			int index = getIndex(i);
+			int index = _data.getIndex(i);
 			if(index < numVals) {
 				for(int colIx = index; colIx < ncol + index; colIx++) {
 					lnnz += (values[colIx]) != 0 ? 1 : 0;
@@ -181,7 +181,7 @@ public class ColGroupDDC extends ColGroupValue {
 	protected void computeRowSums(double[] c, boolean square, int rl, int ru, boolean mean) {
 		double[] vals = _dict.sumAllRowsToDouble(square, _colIndexes.length);
 		for(int rix = rl; rix < ru; rix++)
-			c[rix] += vals[getIndex(rix)];
+			c[rix] += vals[_data.getIndex(rix)];
 	}
 
 	@Override
@@ -189,7 +189,7 @@ public class ColGroupDDC extends ColGroupValue {
 		final int nCol = getNumCols();
 		double[] preAggregatedRows = _dict.aggregateTuples(builtin, nCol);
 		for(int i = rl; i < ru; i++)
-			c[i] = builtin.execute(c[i], preAggregatedRows[getIndex(i)]);
+			c[i] = builtin.execute(c[i], preAggregatedRows[_data.getIndex(i)]);
 	}
 
 	@Override
@@ -200,7 +200,7 @@ public class ColGroupDDC extends ColGroupValue {
 	@Override
 	public int[] getCounts(int rl, int ru, int[] counts) {
 		for(int i = rl; i < ru; i++) {
-			int index = getIndex(i);
+			int index = _data.getIndex(i);
 			counts[index]++;
 		}
 		return counts;
@@ -211,10 +211,10 @@ public class ColGroupDDC extends ColGroupValue {
 		double[] vals = allocDVector(getNumValues(), true);
 		if(row > 0)
 			for(int i = 0, off = _numRows * row; i < _numRows; i++, off++)
-				vals[getIndex(i)] += a[off];
+				vals[_data.getIndex(i)] += a[off];
 		else
 			for(int i = 0; i < _numRows; i++)
-				vals[getIndex(i)] += a[i];
+				vals[_data.getIndex(i)] += a[i];
 
 		return vals;
 	}
@@ -226,34 +226,12 @@ public class ColGroupDDC extends ColGroupValue {
 		int[] indexes = sb.indexes(row);
 		double[] sparseV = sb.values(row);
 		for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++)
-			vals[getIndex(indexes[i])] += sparseV[i];
+			vals[_data.getIndex(indexes[i])] += sparseV[i];
 		return vals;
 
 	}
 
 	/**
-	 * Generic get index in dictionary for value at row position.
-	 * 
-	 * @param r row position to get dictionary index for.
-	 * @return The dictionary index
-	 */
-	protected int getIndex(int r) {
-		return _data.getIndex(r);
-	}
-
-	/**
-	 * Generic get index in dictionary for value at row, col position. If used consider changing to getIndex and
-	 * precalculate offset to row
-	 * 
-	 * @param r     The row to find
-	 * @param colIx the col index to find
-	 * @return the index in the dictionary containing the specified value
-	 */
-	protected int getIndex(int r, int colIx) {
-		return _data.getIndex(r) * getNumCols() + colIx;
-	}
-
-	/**
 	 * Generic get value for byte-length-agnostic access to first column.
 	 * 
 	 * @param r      Global row index
@@ -261,7 +239,7 @@ public class ColGroupDDC extends ColGroupValue {
 	 * @return value
 	 */
 	protected double getData(int r, double[] values) {
-		int index = getIndex(r);
+		int index = _data.getIndex(r);
 		return (index < values.length) ? values[index] : 0.0;
 	}
 
@@ -274,7 +252,7 @@ public class ColGroupDDC extends ColGroupValue {
 	 * @return value
 	 */
 	protected double getData(int r, int colIx, double[] values) {
-		int index = getIndex(r, colIx);
+		int index = _data.getIndex(r) * _colIndexes.length + colIx;
 		return (index < values.length) ? values[index] : 0.0;
 	}
 
@@ -296,7 +274,7 @@ public class ColGroupDDC extends ColGroupValue {
 		IPreAggregate ag = PreAggregateFactory.ag(retSize);
 		// int[] m = _data.materializeMultiplied(nCol);
 		for(int i = 0; i < this._numRows; i++)
-			ag.increment(lhs.getIndex(i) + this.getIndex(i) * nCol);
+			ag.increment(lhs._data.getIndex(i) + this._data.getIndex(i) * nCol);
 
 		return ag;
 	}
@@ -315,9 +293,9 @@ public class ColGroupDDC extends ColGroupValue {
 
 		int col;
 		for(; i < this._numRows && lIt.hasNext(); i++) {
-			int row = this.getIndex(i);
+			int row = this._data.getIndex(i);
 			if(lIt.value() == i)
-				col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+				col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
 
 			else
 				col = offsetToDefault;
@@ -325,7 +303,7 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 		col = offsetToDefault;
 		for(; i < this._numRows; i++) {
-			int row = this.getIndex(i);
+			int row = this._data.getIndex(i);
 			ag.increment(col + row * nCol);
 		}
 
@@ -344,7 +322,7 @@ public class ColGroupDDC extends ColGroupValue {
 
 		int col;
 		for(; i < this._numRows && lIt.hasNext(); i++) {
-			int row = this.getIndex(i);
+			int row = this._data.getIndex(i);
 			if(lIt.value() == i) {
 				col = 1;
 				lIt.next();
@@ -355,7 +333,7 @@ public class ColGroupDDC extends ColGroupValue {
 		}
 
 		for(; i < this._numRows; i++)
-			ag.increment(this.getIndex(i) * nCol);
+			ag.increment(this._data.getIndex(i) * nCol);
 
 		return ag;
 	}
@@ -369,8 +347,8 @@ public class ColGroupDDC extends ColGroupValue {
 		final AIterator lIt = lhs._indexes.getIterator();
 
 		while(lIt.hasNext()) {
-			int row = this.getIndex(lIt.value());
-			int col = lhs.getIndex(lIt.getDataIndexAndIncrement());
+			int row = this._data.getIndex(lIt.value());
+			int col = lhs._data.getIndex(lIt.getDataIndexAndIncrement());
 			ag.increment(col + row * nCol);
 		}
 
@@ -387,7 +365,7 @@ public class ColGroupDDC extends ColGroupValue {
 		final AIterator lIt = lhs._indexes.getIterator();
 
 		while(lIt.hasNext()) {
-			int row = this.getIndex(lIt.value());
+			int row = this._data.getIndex(lIt.value());
 			lIt.next();
 			ag.increment(row);
 		}
@@ -409,7 +387,7 @@ public class ColGroupDDC extends ColGroupValue {
 			for(int bixL = 0, offL = 0, sLenL = 0; bixL < bLenL; bixL += sLenL + 1, offL += blksz) {
 				sLenL = lhs._data[bOffL + bixL];
 				for(int i = 1; i <= sLenL; i++) {
-					int idx = this.getIndex(offL + lhs._data[bOffL + bixL + i]);
+					int idx = this._data.getIndex(offL + lhs._data[bOffL + bixL + i]);
 					ag.increment(kl + idx * NVL);
 				}
 			}
@@ -433,7 +411,7 @@ public class ColGroupDDC extends ColGroupValue {
 				lenL = lhs._data[boffL + bixL + 1];
 				final int endL = startL + lenL;
 				for(int i = startL; i < endL; i++) {
-					int kr = getIndex(i) * NVL;
+					int kr = _data.getIndex(i) * NVL;
 					ag.increment(kl + kr);
 				}
 			}
@@ -445,27 +423,36 @@ public class ColGroupDDC extends ColGroupValue {
 	public Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret) {
 		final int nCol = that._colIndexes.length;
 		for(int r = 0; r < _numRows; r++)
-			that._dict.addToEntry(ret, that.getIndex(r), this.getIndex(r), nCol);
+			that._dict.addToEntry(ret, that._data.getIndex(r), this._data.getIndex(r), nCol);
 
 		return ret;
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		final AIterator itThat = that._indexes.getIterator();
 		final int offsetToDefault = that.getNumValues() - 1;
 		final int nCol = that._colIndexes.length;
+		if(preModified) {
+			while(itThat.hasNext()) {
+				final int to = _data.getIndex(itThat.value());
+				final int fr = that._data.getIndex(itThat.getDataIndexAndIncrement());
+				that._dict.addToEntry(ret, fr, to, nCol);
+			}
+		}
+		else {
+			int i = 0;
 
-		int i = 0;
+			for(; i < _numRows && itThat.hasNext(); i++) {
+				int fr = (itThat.value() == i) ? that._data
+					.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefault;
+				that._dict.addToEntry(ret, fr, this._data.getIndex(i), nCol);
+			}
 
-		for(; i < _numRows && itThat.hasNext(); i++) {
-			int fr = (itThat.value() == i) ? that.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefault;
-			that._dict.addToEntry(ret, fr, this.getIndex(i), nCol);
+			for(; i < _numRows; i++)
+				that._dict.addToEntry(ret, offsetToDefault, this._data.getIndex(i), nCol);
 		}
 
-		for(; i < _numRows; i++)
-			that._dict.addToEntry(ret, offsetToDefault, this.getIndex(i), nCol);
-
 		return ret;
 	}
 
@@ -475,8 +462,8 @@ public class ColGroupDDC extends ColGroupValue {
 		final int nCol = that._colIndexes.length;
 
 		while(itThat.hasNext()) {
-			final int to = getIndex(itThat.value());
-			final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
+			final int to = _data.getIndex(itThat.value());
+			final int fr = that._data.getIndex(itThat.getDataIndexAndIncrement());
 			that._dict.addToEntry(ret, fr, to, nCol);
 		}
 
@@ -489,7 +476,7 @@ public class ColGroupDDC extends ColGroupValue {
 		final int nCol = that._colIndexes.length;
 
 		while(itThat.hasNext()) {
-			final int to = getIndex(itThat.value());
+			final int to = _data.getIndex(itThat.value());
 			itThat.next();
 			that._dict.addToEntry(ret, 0, to, nCol);
 		}
@@ -498,6 +485,35 @@ public class ColGroupDDC extends ColGroupValue {
 	}
 
 	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
+		final AIterator itThat = that._indexes.getIterator();
+		final int nCol = that._colIndexes.length;
+		if(preModified) {
+			while(itThat.hasNext()) {
+				final int to = _data.getIndex(itThat.value());
+				itThat.next();
+				that._dict.addToEntry(ret, 0, to, nCol);
+			}
+		}
+		else {
+			int i = 0;
+			for(; i < _numRows && itThat.hasNext(); i++) {
+				if(itThat.value() == i) {
+					that._dict.addToEntry(ret, 0, this._data.getIndex(i), nCol);
+					itThat.next();
+				}
+				else
+					that._dict.addToEntry(ret, 1, this._data.getIndex(i), nCol);
+			}
+
+			for(; i < _numRows; i++)
+				that._dict.addToEntry(ret, 1, this._data.getIndex(i), nCol);
+		}
+
+		return ret;
+	}
+
+	@Override
 	public boolean sameIndexStructure(ColGroupCompressed that) {
 		return that instanceof ColGroupDDC && ((ColGroupDDC) that)._data == _data;
 	}
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 5192b26..2046d18 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
@@ -140,7 +140,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		if(val0 == 0)
 			return this;
 		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[0]).applyScalarOp(op, val0, _colIndexes.length));
+			new Dictionary(new double[_colIndexes.length]).applyScalarOp(op, val0, _colIndexes.length));
 	}
 
 	@Override
@@ -148,7 +148,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 		if(sparseSafe)
 			return this;
 		return new ColGroupConst(_colIndexes, _numRows,
-			new Dictionary(new double[0]).applyBinaryRowOp(op.fn, v, sparseSafe, _colIndexes, left));
+			new Dictionary(new double[_colIndexes.length]).applyBinaryRowOp(op.fn, v, sparseSafe, _colIndexes, left));
 	}
 
 	@Override
@@ -183,7 +183,7 @@ public class ColGroupEmpty extends ColGroupCompressed {
 
 	@Override
 	protected double computeMxx(double c, Builtin builtin) {
-		return 0;
+		return builtin.execute(c, 0);
 	}
 
 	@Override
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 49ef877..f16204e 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
@@ -289,14 +289,16 @@ public class ColGroupFactory {
 		if(LOG.isTraceEnabled())
 			LOG.trace("compressing to: " + compType);
 		try {
+			if(cs.sortValuesByLength)
+				ubm.sortValuesByFrequency();
 
 			switch(compType) {
 				case DDC:
 					return compressDDC(colIndexes, rlen, ubm, cs);
 				case RLE:
-					return new ColGroupRLE(colIndexes, rlen, ubm, cs);
+					return compressRLE(colIndexes, rlen, ubm, cs);
 				case OLE:
-					return new ColGroupOLE(colIndexes, rlen, ubm, cs);
+					return compressOLE(colIndexes, rlen, ubm, cs);
 				case SDC:
 					return compressSDC(colIndexes, rlen, ubm, cs);
 				case UNCOMPRESSED:
@@ -460,4 +462,46 @@ public class ColGroupFactory {
 		return new ColGroupDDC(colIndexes, rlen, dict, _data, null);
 
 	}
+
+	private static AColGroup compressOLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+
+		ADictionary dict = ADictionary.getDictionary(ubm);
+		ColGroupOLE ole = new ColGroupOLE(rlen);
+
+		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;
+		}
+
+		// compact bitmaps to linearized representation
+		ole.createCompressedBitmaps(numVals, totalLen, lbitmaps);
+		ole._dict = dict;
+		ole._zeros = ubm.getNumOffsets() < (long) rlen;
+		ole._colIndexes = colIndexes;
+		return ole;
+	}
+
+	private static AColGroup compressRLE(int[] colIndexes, int rlen, ABitmap ubm, CompressionSettings cs) {
+
+		ADictionary dict = ADictionary.getDictionary(ubm);
+		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;
+	}
 }
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 cc3d9f1..8101f40 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
@@ -29,7 +29,6 @@ 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.pre.IPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
-import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -44,7 +43,7 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 public class ColGroupOLE extends ColGroupOffset {
 	private static final long serialVersionUID = -9157676271360528008L;
 
-		/**
+	/**
 	 * Constructor for serialization
 	 * 
 	 * @param numRows Number of rows contained
@@ -53,30 +52,6 @@ public class ColGroupOLE extends ColGroupOffset {
 		super(numRows);
 	}
 
-	/**
-	 * Main constructor. Constructs and stores the necessary bitmaps.
-	 * 
-	 * @param colIndices indices (within the block) of the columns included in this column
-	 * @param numRows    total number of rows in the parent block
-	 * @param ubm        Uncompressed bitmap representation of the block
-	 * @param cs         The Compression settings used for compression
-	 */
-	protected ColGroupOLE(int[] colIndices, int numRows, ABitmap ubm, CompressionSettings cs) {
-		super(colIndices, numRows, ubm, cs);
-		// compress the bitmaps
-		final int numVals = ubm.getNumValues();
-		char[][] lbitmaps = new char[numVals][];
-		int totalLen = 0;
-		for(int i = 0; i < numVals; i++) {
-			lbitmaps[i] = genOffsetBitmap(ubm.getOffsetsList(i).extractValues(), ubm.getNumOffsets(i));
-			totalLen += lbitmaps[i].length;
-		}
-
-		// compact bitmaps to linearized representation
-		createCompressedBitmaps(numVals, totalLen, lbitmaps);
-
-	}
-
 	protected ColGroupOLE(int[] colIndices, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
 		int[] bitmapOffs, int[] counts) {
 		super(colIndices, numRows, zeros, dict, counts);
@@ -425,304 +400,309 @@ public class ColGroupOLE extends ColGroupOffset {
 
 	// @Override
 	// public void rightMultByVector(double[] b, double[] c, int rl, int ru, double[] dictVals) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numVals = getNumValues();
-
-	// 	if(rl % blksz != 0)
-	// 		throw new DMLCompressionException("All blocks should be starting at block segments for OLE");
-
-	// 	if(numVals > 1 && _numRows > blksz * 2) {
-	// 		// since single segment scans already exceed typical L2 cache sizes
-	// 		// and because there is some overhead associated with blocking, the
-	// 		// best configuration aligns with L3 cache size (x*vcores*64K*8B < L3)
-	// 		// x=4 leads to a good yet slightly conservative compromise for single-/
-	// 		// multi-threaded and typical number of cores and L3 cache sizes
-	// 		final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-	// 		int[] apos = skipScan(numVals, rl);
-	// 		double[] aval = preaggValues(numVals, b, dictVals);
-
-	// 		// step 2: cache conscious matrix-vector via horizontal scans
-	// 		for(int bi = rl; bi < ru; bi += blksz2) {
-	// 			int bimax = Math.min(bi + blksz2, 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];
-
-	// 				for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
-	// 					// prepare length, start, and end pos
-	// 					int len = _data[boff + bix];
-	// 					int pos = boff + bix + 1;
-
-	// 					// compute partial results
-	// 					LinearAlgebraUtils.vectAdd(val, c, _data, pos, ii, len);
-	// 					bix += len + 1;
-	// 				}
-
-	// 				apos[k] = bix;
-	// 			}
-	// 		}
-	// 	}
-	// 	else {
-	// 		// 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 = sumValues(k, b, dictVals);
-
-	// 			// iterate over bitmap blocks and add values
-	// 			if(val != 0) {
-	// 				int bix = 0;
-	// 				int off = 0;
-	// 				int slen = -1;
-
-	// 				// scan to beginning offset if necessary
-	// 				if(rl > 0) {
-	// 					for(; bix < blen & off < rl; bix += slen + 1, off += blksz) {
-	// 						slen = _data[boff + bix];
-	// 					}
-	// 				}
-
-	// 				// compute partial results
-	// 				for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
-	// 					slen = _data[boff + bix];
-	// 					for(int blckIx = 1; blckIx <= slen; blckIx++) {
-	// 						c[off + _data[boff + bix + blckIx]] += val;
-	// 					}
-	// 				}
-	// 			}
-	// 		}
-	// 	}
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final int numVals = getNumValues();
+
+	// if(rl % blksz != 0)
+	// throw new DMLCompressionException("All blocks should be starting at block segments for OLE");
+
+	// if(numVals > 1 && _numRows > blksz * 2) {
+	// // since single segment scans already exceed typical L2 cache sizes
+	// // and because there is some overhead associated with blocking, the
+	// // best configuration aligns with L3 cache size (x*vcores*64K*8B < L3)
+	// // x=4 leads to a good yet slightly conservative compromise for single-/
+	// // multi-threaded and typical number of cores and L3 cache sizes
+	// final int blksz2 = CompressionSettings.BITMAP_BLOCK_SZ * 2;
+	// int[] apos = skipScan(numVals, rl);
+	// double[] aval = preaggValues(numVals, b, dictVals);
+
+	// // step 2: cache conscious matrix-vector via horizontal scans
+	// for(int bi = rl; bi < ru; bi += blksz2) {
+	// int bimax = Math.min(bi + blksz2, 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];
+
+	// for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
+	// // prepare length, start, and end pos
+	// int len = _data[boff + bix];
+	// int pos = boff + bix + 1;
+
+	// // compute partial results
+	// LinearAlgebraUtils.vectAdd(val, c, _data, pos, ii, len);
+	// bix += len + 1;
+	// }
+
+	// apos[k] = bix;
+	// }
+	// }
+	// }
+	// else {
+	// // 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 = sumValues(k, b, dictVals);
+
+	// // iterate over bitmap blocks and add values
+	// if(val != 0) {
+	// int bix = 0;
+	// int off = 0;
+	// int slen = -1;
+
+	// // scan to beginning offset if necessary
+	// if(rl > 0) {
+	// for(; bix < blen & off < rl; bix += slen + 1, off += blksz) {
+	// slen = _data[boff + bix];
+	// }
+	// }
+
+	// // compute partial results
+	// for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
+	// slen = _data[boff + bix];
+	// for(int blckIx = 1; blckIx <= slen; blckIx++) {
+	// c[off + _data[boff + bix + blckIx]] += val;
+	// }
+	// }
+	// }
+	// }
+	// }
 	// }
 
 	// @Override
-	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int rl,
-	// 	int ru) {
-
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numVals = getNumValues();
-
-	// 	if(numVals > 1 && _numRows > blksz * 2) {
-	// 		final int blksz2 = blksz * 2;
-	// 		int[] apos = skipScan(numVals, rl);
-	// 		int blockStart = rl - rl % blksz;
-	// 		for(int bi = blockStart; bi < ru; bi += blksz2) {
-	// 			int bimax = Math.min(bi + blksz2, ru);
-	// 			for(int k = 0; k < numVals; k++) {
-	// 				int boff = _ptr[k];
-	// 				int blen = len(k);
-	// 				int bix = apos[k];
-	// 				for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
-	// 					int len = _data[boff + bix];
-	// 					int pos = _data[boff + bix + 1];
-	// 					if(pos >= rl)
-	// 						addV(c, preAggregatedB, outputColumns, (bi + pos) * thatNrColumns, k);
-	// 					bix += len + 1;
-	// 				}
-	// 				apos[k] = bix;
-	// 			}
-	// 		}
-	// 	}
-	// 	else {
-	// 		for(int k = 0; k < numVals; k++) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = skipScanVal(k, rl);
-	// 			int off = rl;
-	// 			int slen = 0;
-	// 			// compute partial results
-	// 			for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
-	// 				slen = _data[boff + bix];
-	// 				for(int blckIx = 1; blckIx <= slen; blckIx++) {
-	// 					int rowIdx = (_data[boff + bix + blckIx] + off) * thatNrColumns;
-	// 					addV(c, preAggregatedB, outputColumns, rowIdx, k);
-	// 				}
-	// 			}
-	// 		}
-	// 	}
+	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int
+	// rl,
+	// int ru) {
+
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final int numVals = getNumValues();
+
+	// if(numVals > 1 && _numRows > blksz * 2) {
+	// final int blksz2 = blksz * 2;
+	// int[] apos = skipScan(numVals, rl);
+	// int blockStart = rl - rl % blksz;
+	// for(int bi = blockStart; bi < ru; bi += blksz2) {
+	// int bimax = Math.min(bi + blksz2, ru);
+	// for(int k = 0; k < numVals; k++) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int bix = apos[k];
+	// for(int ii = bi; ii < bimax && bix < blen; ii += blksz) {
+	// int len = _data[boff + bix];
+	// int pos = _data[boff + bix + 1];
+	// if(pos >= rl)
+	// addV(c, preAggregatedB, outputColumns, (bi + pos) * thatNrColumns, k);
+	// bix += len + 1;
+	// }
+	// apos[k] = bix;
+	// }
+	// }
+	// }
+	// else {
+	// for(int k = 0; k < numVals; k++) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int bix = skipScanVal(k, rl);
+	// int off = rl;
+	// int slen = 0;
+	// // compute partial results
+	// for(; bix < blen & off < ru; bix += slen + 1, off += blksz) {
+	// slen = _data[boff + bix];
+	// for(int blckIx = 1; blckIx <= slen; blckIx++) {
+	// int rowIdx = (_data[boff + bix + blckIx] + off) * thatNrColumns;
+	// addV(c, preAggregatedB, outputColumns, rowIdx, k);
+	// }
+	// }
+	// }
+	// }
 	// }
 
 	// private static void addV(double[] c, double[] preAggregatedB, int[] outputColumns, int rowIdx, int k) {
-	// 	int n = k * outputColumns.length;
-	// 	for(int i = 0; i < outputColumns.length; i++) {
-	// 		c[rowIdx + outputColumns[i]] += preAggregatedB[n + i];
-	// 	}
+	// int n = k * outputColumns.length;
+	// for(int i = 0; i < outputColumns.length; i++) {
+	// c[rowIdx + outputColumns[i]] += preAggregatedB[n + i];
+	// }
 	// }
 
 	// @Override
 	// public void leftMultByRowVector(double[] a, double[] c, int numVals, double[] values) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
 
-	// 	if(numVals >= 1 && _numRows > blksz)
-	// 		leftMultByRowVectorBlocking(a, c, numVals, values);
-	// 	else
-	// 		leftMultByRowVectorNonBlocking(a, c, numVals, values);
+	// if(numVals >= 1 && _numRows > blksz)
+	// leftMultByRowVectorBlocking(a, c, numVals, values);
+	// else
+	// leftMultByRowVectorNonBlocking(a, c, numVals, values);
 
 	// }
 
 	// private void leftMultByRowVectorBlocking(double[] a, double[] c, int numVals, double[] values) {
-	// 	double[] cvals = preAggregate(a);
-	// 	postScaling(values, cvals, c, numVals);
+	// double[] cvals = preAggregate(a);
+	// postScaling(values, cvals, c, numVals);
 	// }
 
 	// private void leftMultByRowVectorNonBlocking(double[] a, double[] c, int numVals, double[] values) {
-	// 	// iterate over all values and their bitmaps
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	final int numCols = getNumCols();
-	// 	for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
-	// 		int boff = _ptr[k];
-	// 		int blen = len(k);
-
-	// 		// iterate over bitmap blocks and add partial results
-	// 		double vsum = 0;
-	// 		for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
-	// 			vsum += LinearAlgebraUtils.vectSum(a, _data, off, boff + bix + 1, _data[boff + bix]);
-
-	// 		// scale partial results by values and write results
-	// 		for(int j = 0; j < numCols; j++)
-	// 			c[_colIndexes[j]] += vsum * values[valOff + j];
-	// 	}
+	// // iterate over all values and their bitmaps
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// final int numCols = getNumCols();
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+
+	// // iterate over bitmap blocks and add partial results
+	// double vsum = 0;
+	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
+	// vsum += LinearAlgebraUtils.vectSum(a, _data, off, boff + bix + 1, _data[boff + bix]);
+
+	// // scale partial results by values and write results
+	// for(int j = 0; j < numCols; j++)
+	// c[_colIndexes[j]] += vsum * values[valOff + j];
+	// }
 	// }
 
 	// @Override
 	// public void leftMultByMatrix(double[] a, double[] c, double[] values, int numRows, int numCols, int rl, int ru,
-	// 	int vOff) {
-	// 	final int numVals = getNumValues();
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	if(numVals >= 1 && _numRows > blksz)
-	// 		leftMultByMatrixBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
-	// 	else
-	// 		leftMultByMatrixNonBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
+	// int vOff) {
+	// final int numVals = getNumValues();
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// if(numVals >= 1 && _numRows > blksz)
+	// leftMultByMatrixBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
+	// else
+	// leftMultByMatrixNonBlocking(a, c, values, numRows, numCols, rl, ru, vOff, numVals);
 
 	// }
 
 	// private void leftMultByMatrixBlocking(double[] a, double[] c, double[] values, int numRows, int numCols, int rl,
-	// 	int ru, int vOff, int numVals) {
-	// 	for(int i = rl; i < ru; i++) {
-	// 		double[] cvals = preAggregate(a, i);
-	// 		postScaling(values, cvals, c, numVals, i, numCols);
-	// 	}
+	// int ru, int vOff, int numVals) {
+	// for(int i = rl; i < ru; i++) {
+	// double[] cvals = preAggregate(a, i);
+	// postScaling(values, cvals, c, numVals, i, numCols);
+	// }
 	// }
 
-	// private void leftMultByMatrixNonBlocking(double[] a, double[] c, double[] values, int numRows, int numCols, int rl,
-	// 	int ru, int vOff, int numVals) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	for(int i = rl, offR = vOff * _numRows; i < ru; i++, offR += _numRows) {
-	// 		for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-
-	// 			// iterate over bitmap blocks and add partial results
-	// 			double vsum = 0;
-	// 			for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
-	// 				vsum += LinearAlgebraUtils.vectSum(a, _data, off + offR, boff + bix + 1, _data[boff + bix]);
-
-	// 			// scale partial results by values and write results
-
-	// 			int offC = i * numCols;
-	// 			for(int j = 0; j < _colIndexes.length; j++) {
-	// 				int colIx = _colIndexes[j] + offC;
-	// 				c[colIx] += vsum * values[valOff + j];
-	// 			}
-	// 		}
-	// 	}
+	// private void leftMultByMatrixNonBlocking(double[] a, double[] c, double[] values, int numRows, int numCols, int
+	// rl,
+	// int ru, int vOff, int numVals) {
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// for(int i = rl, offR = vOff * _numRows; i < ru; i++, offR += _numRows) {
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+
+	// // iterate over bitmap blocks and add partial results
+	// double vsum = 0;
+	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz)
+	// vsum += LinearAlgebraUtils.vectSum(a, _data, off + offR, boff + bix + 1, _data[boff + bix]);
+
+	// // scale partial results by values and write results
+
+	// int offC = i * numCols;
+	// for(int j = 0; j < _colIndexes.length; j++) {
+	// int colIx = _colIndexes[j] + offC;
+	// c[colIx] += vsum * values[valOff + j];
+	// }
+	// }
+	// }
 	// }
 
 	// @Override
-	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int row) {
-	// 	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	// final int numVals = getNumValues();
-	// 	throw new NotImplementedException("Not implemented Sparse multiplication OLE");
-	// 	// if(numVals > 1 && _numRows > blksz)
-	// 	// 	leftMultBySparseMatrixBlocking(sb, c, values, numRows, numCols, row, tmpA, numVals);
-	// 	// else
-	// 	// 	leftMultBySparseMatrixNonBlock(sb, c, values, numRows, numCols, row, tmpA, numVals);
+	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int
+	// row) {
+	// // final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// // final int numVals = getNumValues();
+	// throw new NotImplementedException("Not implemented Sparse multiplication OLE");
+	// // if(numVals > 1 && _numRows > blksz)
+	// // leftMultBySparseMatrixBlocking(sb, c, values, numRows, numCols, row, tmpA, numVals);
+	// // else
+	// // leftMultBySparseMatrixNonBlock(sb, c, values, numRows, numCols, row, tmpA, numVals);
+
+	// }
+
+	// private void leftMultBySparseMatrixBlocking(SparseBlock sb, double[] c, double[] values, int numRows, int
+	// numCols,
+	// int row, double[] tmpA, int numVals) {
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// int sparseEndIndex = sb.size(row) + sb.pos(row);
+	// int[] indexes = sb.indexes(row);
+	// double[] sparseV = sb.values(row);
+
+	// // cache blocking config (see matrix-vector mult for explanation)
+	// final int blksz2 = 2 * CompressionSettings.BITMAP_BLOCK_SZ;
+
+	// // step 1: prepare position and value arrays
+	// int[] apos = allocIVector(numVals, true);
+	// double[] cvals = allocDVector(numVals, true);
+	// // step 2: cache conscious matrix-vector via horizontal scans
+	// int pI = sb.pos(row);
+	// for(int ai = 0; ai < _numRows; ai += blksz2) {
+	// int aimax = Math.min(ai + blksz2, _numRows);
+	// Arrays.fill(tmpA, 0);
+	// for(; pI < sparseEndIndex && indexes[pI] < aimax; pI++) {
+	// if(indexes[pI] >= ai)
+	// tmpA[indexes[pI] - ai] = sparseV[pI];
+	// }
 
+	// // horizontal segment scan, incl pos maintenance
+	// for(int k = 0; k < numVals; k++) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int bix = apos[k];
+	// double vsum = 0;
+	// for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
+	// int len = _data[boff + bix];
+	// int pos = boff + bix + 1;
+	// int blockId = (ii / blksz) % 2;
+	// vsum += LinearAlgebraUtils.vectSum(tmpA, _data, blockId * blksz, pos, len);
+	// bix += len + 1;
 	// }
 
-	// private void leftMultBySparseMatrixBlocking(SparseBlock sb, double[] c, double[] values, int numRows, int numCols,
-	// 	int row, double[] tmpA, int numVals) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// 	int[] indexes = sb.indexes(row);
-	// 	double[] sparseV = sb.values(row);
-
-	// 	// cache blocking config (see matrix-vector mult for explanation)
-	// 	final int blksz2 = 2 * CompressionSettings.BITMAP_BLOCK_SZ;
-
-	// 	// step 1: prepare position and value arrays
-	// 	int[] apos = allocIVector(numVals, true);
-	// 	double[] cvals = allocDVector(numVals, true);
-	// 	// step 2: cache conscious matrix-vector via horizontal scans
-	// 	int pI = sb.pos(row);
-	// 	for(int ai = 0; ai < _numRows; ai += blksz2) {
-	// 		int aimax = Math.min(ai + blksz2, _numRows);
-	// 		Arrays.fill(tmpA, 0);
-	// 		for(; pI < sparseEndIndex && indexes[pI] < aimax; pI++) {
-	// 			if(indexes[pI] >= ai)
-	// 				tmpA[indexes[pI] - ai] = sparseV[pI];
-	// 		}
-
-	// 		// horizontal segment scan, incl pos maintenance
-	// 		for(int k = 0; k < numVals; k++) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-	// 			int bix = apos[k];
-	// 			double vsum = 0;
-	// 			for(int ii = ai; ii < aimax && bix < blen; ii += blksz) {
-	// 				int len = _data[boff + bix];
-	// 				int pos = boff + bix + 1;
-	// 				int blockId = (ii / blksz) % 2;
-	// 				vsum += LinearAlgebraUtils.vectSum(tmpA, _data, blockId * blksz, pos, len);
-	// 				bix += len + 1;
-	// 			}
-
-	// 			apos[k] = bix;
-	// 			cvals[k] += vsum;
-	// 		}
-	// 	}
-
-	// 	int offC = row * numCols;
-	// 	// step 3: scale partial results by values and write to global output
-	// 	for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length)
-	// 		for(int j = 0; j < _colIndexes.length; j++) {
-	// 			int colIx = _colIndexes[j] + offC;
-	// 			c[colIx] += cvals[k] * values[valOff + j];
-	// 		}
+	// apos[k] = bix;
+	// cvals[k] += vsum;
+	// }
+	// }
 
+	// int offC = row * numCols;
+	// // step 3: scale partial results by values and write to global output
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length)
+	// for(int j = 0; j < _colIndexes.length; j++) {
+	// int colIx = _colIndexes[j] + offC;
+	// c[colIx] += cvals[k] * values[valOff + j];
 	// }
 
-	// private void leftMultBySparseMatrixNonBlock(SparseBlock sb, double[] c, double[] values, int numRows, int numCols,
-	// 	int row, double[] tmpA, int numVals) {
-	// 	final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
-	// 	int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// 	int[] indexes = sb.indexes(row);
-	// 	double[] sparseV = sb.values(row);
-
-	// 	for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// 		int boff = _ptr[k];
-	// 		int blen = len(k);
-	// 		double vsum = 0;
-	// 		int pI = sb.pos(row);
-	// 		for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz) {
-	// 			// blockId = off / blksz;
-	// 			Arrays.fill(tmpA, 0);
-	// 			for(; pI < sparseEndIndex && indexes[pI] < off + blksz; pI++) {
-	// 				if(indexes[pI] >= off)
-	// 					tmpA[indexes[pI] - off] = sparseV[pI];
-	// 			}
-	// 			vsum += LinearAlgebraUtils.vectSum(tmpA, _data, 0, boff + bix + 1, _data[boff + bix]);
-	// 		}
-
-	// 		for(int j = 0; j < _colIndexes.length; j++) {
-	// 			int Voff = _colIndexes[j] + row * numCols;
-	// 			c[Voff] += vsum * values[valOff + j];
-	// 		}
-	// 	}
+	// }
+
+	// private void leftMultBySparseMatrixNonBlock(SparseBlock sb, double[] c, double[] values, int numRows, int
+	// numCols,
+	// int row, double[] tmpA, int numVals) {
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ;
+	// int sparseEndIndex = sb.size(row) + sb.pos(row);
+	// int[] indexes = sb.indexes(row);
+	// double[] sparseV = sb.values(row);
+
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// double vsum = 0;
+	// int pI = sb.pos(row);
+	// for(int bix = 0, off = 0; bix < blen; bix += _data[boff + bix] + 1, off += blksz) {
+	// // blockId = off / blksz;
+	// Arrays.fill(tmpA, 0);
+	// for(; pI < sparseEndIndex && indexes[pI] < off + blksz; pI++) {
+	// if(indexes[pI] >= off)
+	// tmpA[indexes[pI] - off] = sparseV[pI];
+	// }
+	// vsum += LinearAlgebraUtils.vectSum(tmpA, _data, 0, boff + bix + 1, _data[boff + bix]);
+	// }
+
+	// for(int j = 0; j < _colIndexes.length; j++) {
+	// int Voff = _colIndexes[j] + row * numCols;
+	// c[Voff] += vsum * values[valOff + j];
+	// }
+	// }
 	// }
 
 	@Override
@@ -1090,7 +1070,7 @@ public class ColGroupOLE extends ColGroupOffset {
 			for(int bixR = 0, offR = 0, sLenR = 0; bixR < bLenR; bixR += sLenR + 1, offR += blksz) {
 				sLenR = this._data[bOffR + bixR];
 				for(int j = 1; j <= sLenR; j++) {
-					int idx = lhs.getIndex(offR + this._data[bOffR + bixR + j]);
+					int idx = lhs._data.getIndex(offR + this._data[bOffR + bixR + j]);
 					ag.increment(idx + krOff);
 				}
 			}
@@ -1214,7 +1194,7 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
@@ -1227,4 +1207,9 @@ public class ColGroupOLE extends ColGroupOffset {
 	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+		throw new NotImplementedException();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
index 176258a..d33e816 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
@@ -24,9 +24,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.colgroup.dictionary.ADictionary;
-import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 
@@ -53,18 +51,6 @@ public abstract class ColGroupOffset extends ColGroupValue {
 		super(numRows);
 	}
 
-	/**
-	 * Main constructor. Stores the headers for the individual bitmaps.
-	 * 
-	 * @param colIndices indices (within the block) of the columns included in this column
-	 * @param numRows    total number of rows in the parent block
-	 * @param ubm        Uncompressed bitmap representation of the block
-	 * @param cs         The Compression settings used for compression
-	 */
-	protected ColGroupOffset(int[] colIndices, int numRows, ABitmap ubm, CompressionSettings cs) {
-		super(colIndices, numRows, ubm, cs);
-	}
-
 	protected ColGroupOffset(int[] colIndices, int numRows, boolean zeros, ADictionary dict, int[] cachedCounts) {
 		super(colIndices, numRows, dict, cachedCounts);
 		_zeros = zeros;
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 d27913b..1e79abc 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
@@ -29,7 +29,6 @@ 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.pre.IPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.PreAggregateFactory;
-import org.apache.sysds.runtime.compress.utils.ABitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -51,31 +50,6 @@ public class ColGroupRLE extends ColGroupOffset {
 		super(numRows);
 	}
 
-	/**
-	 * Main constructor. Constructs and stores the necessary bitmaps.
-	 * 
-	 * @param colIndices indices (within the block) of the columns included in this column
-	 * @param numRows    total number of rows in the parent block
-	 * @param ubm        Uncompressed bitmap representation of the block
-	 * @param cs         The Compression settings used for compression
-	 */
-	protected ColGroupRLE(int[] colIndices, int numRows, ABitmap ubm, CompressionSettings cs) {
-		super(colIndices, numRows, ubm, cs);
-
-		// 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] = genRLEBitmap(ubm.getOffsetsList(k).extractValues(), ubm.getNumOffsets(k));
-			totalLen += lbitmaps[k].length;
-		}
-		// compact bitmaps to linearized representation
-		createCompressedBitmaps(numVals, totalLen, lbitmaps);
-
-	}
-
 	protected ColGroupRLE(int[] colIndices, int numRows, boolean zeros, ADictionary dict, char[] bitmaps,
 		int[] bitmapOffs, int[] cachedCounts) {
 		super(colIndices, numRows, zeros, dict, cachedCounts);
@@ -393,223 +367,225 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	// @Override
 	// public void rightMultByVector(double[] b, double[] c, int rl, int ru, double[] dictVals) {
-	// 	final int numVals = getNumValues();
-	// 	if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// 		// L3 cache alignment, see comment rightMultByVector OLE column group
-	// 		// core difference of RLE to OLE is that runs are not segment alignment,
-	// 		// which requires care of handling runs crossing cache-buckets
-	// 		final int blksz = CompressionSettings.BITMAP_BLOCK_SZ * 2;
-
-	// 		// step 1: prepare position and value arrays
-
-	// 		// current pos / values per RLE list
-
-	// 		// step 2: cache conscious matrix-vector via horizontal scans
-	// 		for(int bi = rl; bi < ru; bi += blksz) {
-	// 			int[] astart = new int[numVals];
-	// 			int[] apos = skipScan(numVals, rl, astart);
-	// 			double[] aval = preaggValues(numVals, b, dictVals);
-	// 			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 & bix < bimax) {
-	// 					int lstart = _data[boff + bix];
-	// 					int llen = _data[boff + bix + 1];
-	// 					int len = Math.min(start + lstart + llen, bimax) - Math.max(bi, start + lstart);
-	// 					if(len > 0) {
-	// 						LinearAlgebraUtils.vectAdd(val, c, Math.max(bi, start + lstart), len);
-	// 					}
-	// 					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 = sumValues(k, b, dictVals);
-	// 			int bix = 0;
-	// 			int start = 0;
-
-	// 			// scan to beginning offset if necessary
-	// 			if(rl > 0) { // rl aligned with blksz
-	// 				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;
-	// 				}
-	// 			}
-
-	// 			// compute partial results, not aligned
-	// 			while(bix < blen) {
-	// 				int lstart = _data[boff + bix];
-	// 				int llen = _data[boff + bix + 1];
-	// 				LinearAlgebraUtils.vectAdd(val, c, Math.max(rl, start + lstart),
-	// 					Math.min(start + lstart + llen, ru) - Math.max(rl, start + lstart));
-	// 				if(start + lstart + llen >= ru)
-	// 					break;
-	// 				start += lstart + llen;
-	// 				bix += 2;
-	// 			}
-	// 		}
-	// 	}
+	// final int numVals = getNumValues();
+	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
+	// // L3 cache alignment, see comment rightMultByVector OLE column group
+	// // core difference of RLE to OLE is that runs are not segment alignment,
+	// // which requires care of handling runs crossing cache-buckets
+	// final int blksz = CompressionSettings.BITMAP_BLOCK_SZ * 2;
+
+	// // step 1: prepare position and value arrays
+
+	// // current pos / values per RLE list
+
+	// // step 2: cache conscious matrix-vector via horizontal scans
+	// for(int bi = rl; bi < ru; bi += blksz) {
+	// int[] astart = new int[numVals];
+	// int[] apos = skipScan(numVals, rl, astart);
+	// double[] aval = preaggValues(numVals, b, dictVals);
+	// 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 & bix < bimax) {
+	// int lstart = _data[boff + bix];
+	// int llen = _data[boff + bix + 1];
+	// int len = Math.min(start + lstart + llen, bimax) - Math.max(bi, start + lstart);
+	// if(len > 0) {
+	// LinearAlgebraUtils.vectAdd(val, c, Math.max(bi, start + lstart), len);
+	// }
+	// 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 = sumValues(k, b, dictVals);
+	// int bix = 0;
+	// int start = 0;
+
+	// // scan to beginning offset if necessary
+	// if(rl > 0) { // rl aligned with blksz
+	// 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;
+	// }
+	// }
+
+	// // compute partial results, not aligned
+	// while(bix < blen) {
+	// int lstart = _data[boff + bix];
+	// int llen = _data[boff + bix + 1];
+	// LinearAlgebraUtils.vectAdd(val, c, Math.max(rl, start + lstart),
+	// Math.min(start + lstart + llen, ru) - Math.max(rl, start + lstart));
+	// if(start + lstart + llen >= ru)
+	// break;
+	// start += lstart + llen;
+	// bix += 2;
+	// }
+	// }
+	// }
 	// }
 
 	// @Override
-	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int rl,
-	// 	int ru) {
-	// 	final int nrVals = getNumValues();
-	// 	for(int k = 0; k < nrVals; k++) {
-	// 		int boff = _ptr[k];
-	// 		int blen = len(k);
-	// 		int bix = 0;
-	// 		int start = 0;
-
-	// 		// scan to beginning offset if necessary
-	// 		if(rl > 0) { // rl aligned with blksz
-	// 			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;
-	// 			}
-	// 		}
-	// 		// compute partial results, not aligned
-	// 		while(bix < blen) {
-	// 			int lstart = _data[boff + bix];
-	// 			int llen = _data[boff + bix + 1];
-	// 			LinearAlgebraUtils.vectListAdd(preAggregatedB, c, Math.max(rl, start + lstart),
-	// 				Math.min(start + lstart + llen, ru), outputColumns, thatNrColumns, k);
-	// 			if(start + lstart + llen >= ru)
-	// 				break;
-	// 			start += lstart + llen;
-	// 			bix += 2;
-	// 		}
-	// 	}
+	// public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int
+	// rl,
+	// int ru) {
+	// final int nrVals = getNumValues();
+	// for(int k = 0; k < nrVals; k++) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+	// int bix = 0;
+	// int start = 0;
+
+	// // scan to beginning offset if necessary
+	// if(rl > 0) { // rl aligned with blksz
+	// 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;
+	// }
+	// }
+	// // compute partial results, not aligned
+	// while(bix < blen) {
+	// int lstart = _data[boff + bix];
+	// int llen = _data[boff + bix + 1];
+	// LinearAlgebraUtils.vectListAdd(preAggregatedB, c, Math.max(rl, start + lstart),
+	// Math.min(start + lstart + llen, ru), outputColumns, thatNrColumns, k);
+	// if(start + lstart + llen >= ru)
+	// break;
+	// start += lstart + llen;
+	// bix += 2;
+	// }
+	// }
 	// }
 
 	// @Override
 	// public void leftMultByRowVector(double[] a, double[] c, int numVals, double[] values) {
-	// 	final int numCols = getNumCols();
-
-	// 	if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// 		double[] cvals = preAggregate(a, 0);
-	// 		postScaling(values, cvals, c, numVals);
-	// 	}
-	// 	else {
-	// 		// iterate over all values and their bitmaps
-	// 		for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
-	// 			int boff = _ptr[k];
-	// 			int blen = len(k);
-
-	// 			double vsum = 0;
-	// 			int curRunEnd = 0;
-	// 			for(int bix = 0; bix < blen; bix += 2) {
-	// 				int curRunStartOff = curRunEnd + _data[boff + bix];
-	// 				int curRunLen = _data[boff + bix + 1];
-	// 				vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff, curRunLen);
-	// 				curRunEnd = curRunStartOff + curRunLen;
-	// 			}
-
-	// 			// scale partial results by values and write results
-	// 			for(int j = 0; j < numCols; j++)
-	// 				c[_colIndexes[j]] += vsum * values[valOff + j];
-	// 		}
-	// 	}
+	// final int numCols = getNumCols();
+
+	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
+	// double[] cvals = preAggregate(a, 0);
+	// postScaling(values, cvals, c, numVals);
+	// }
+	// else {
+	// // iterate over all values and their bitmaps
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += numCols) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+
+	// double vsum = 0;
+	// int curRunEnd = 0;
+	// for(int bix = 0; bix < blen; bix += 2) {
+	// int curRunStartOff = curRunEnd + _data[boff + bix];
+	// int curRunLen = _data[boff + bix + 1];
+	// vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff, curRunLen);
+	// curRunEnd = curRunStartOff + curRunLen;
+	// }
+
+	// // scale partial results by values and write results
+	// for(int j = 0; j < numCols; j++)
+	// c[_colIndexes[j]] += vsum * values[valOff + j];
+	// }
+	// }
 	// }
 
 	// @Override
 	// public void leftMultByMatrix(final double[] a, final double[] c, final double[] values, final int numRows,
-	// 	final int numCols, int rl, final int ru, final int vOff) {
-
-	// 	final int numVals = getNumValues();
-	// 	if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
-	// 		for(int i = rl; i < ru; i++) {
-	// 			double[] cvals = preAggregate(a, i);
-	// 			postScaling(values, cvals, c, numVals, i, numCols);
-	// 		}
-	// 	}
-	// 	else {
-	// 		// iterate over all values and their bitmaps
-	// 		for(int i = rl, off = vOff * _numRows; i < ru; i++, off += _numRows) {
-	// 			int offC = i * numCols;
-	// 			int valOff = 0;
-	// 			for(int k = 0; k < numVals; k++) {
-	// 				int boff = _ptr[k];
-	// 				int blen = len(k);
-
-	// 				double vsum = 0;
-	// 				int curRunEnd = 0;
-	// 				for(int bix = 0; bix < blen; bix += 2) {
-	// 					int curRunStartOff = curRunEnd + _data[boff + bix];
-	// 					int curRunLen = _data[boff + bix + 1];
-	// 					vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff + off, curRunLen);
-	// 					curRunEnd = curRunStartOff + curRunLen;
-	// 				}
-
-	// 				for(int j = 0; j < _colIndexes.length; j++) {
-	// 					int colIx = _colIndexes[j] + offC;
-	// 					// scale partial results by values and write results
-	// 					c[colIx] += vsum * values[valOff++];
-	// 				}
-	// 			}
-	// 		}
-	// 	}
+	// final int numCols, int rl, final int ru, final int vOff) {
+
+	// final int numVals = getNumValues();
+	// if(numVals >= 1 && _numRows > CompressionSettings.BITMAP_BLOCK_SZ) {
+	// for(int i = rl; i < ru; i++) {
+	// double[] cvals = preAggregate(a, i);
+	// postScaling(values, cvals, c, numVals, i, numCols);
+	// }
+	// }
+	// else {
+	// // iterate over all values and their bitmaps
+	// for(int i = rl, off = vOff * _numRows; i < ru; i++, off += _numRows) {
+	// int offC = i * numCols;
+	// int valOff = 0;
+	// for(int k = 0; k < numVals; k++) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+
+	// double vsum = 0;
+	// int curRunEnd = 0;
+	// for(int bix = 0; bix < blen; bix += 2) {
+	// int curRunStartOff = curRunEnd + _data[boff + bix];
+	// int curRunLen = _data[boff + bix + 1];
+	// vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff + off, curRunLen);
+	// curRunEnd = curRunStartOff + curRunLen;
+	// }
+
+	// for(int j = 0; j < _colIndexes.length; j++) {
+	// int colIx = _colIndexes[j] + offC;
+	// // scale partial results by values and write results
+	// c[colIx] += vsum * values[valOff++];
+	// }
+	// }
+	// }
+	// }
 	// }
 
 	// @Override
-	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int row) {
-
-	// 	final int numVals = getNumValues();
-	// 	int sparseEndIndex = sb.size(row) + sb.pos(row);
-	// 	int[] indexes = sb.indexes(row);
-	// 	double[] sparseV = sb.values(row);
-	// 	for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
-	// 		int boff = _ptr[k];
-	// 		int blen = len(k);
-
-	// 		double vsum = 0;
-	// 		int pointSparse = sb.pos(row);
-	// 		int curRunEnd = 0;
-	// 		for(int bix = 0; bix < blen; bix += 2) {
-	// 			int curRunStartOff = curRunEnd + _data[boff + bix];
-	// 			int curRunLen = _data[boff + bix + 1];
-	// 			curRunEnd = curRunStartOff + curRunLen;
-	// 			while(pointSparse < sparseEndIndex && indexes[pointSparse] < curRunStartOff) {
-	// 				pointSparse++;
-	// 			}
-	// 			while(pointSparse != sparseEndIndex && indexes[pointSparse] >= curRunStartOff &&
-	// 				indexes[pointSparse] < curRunEnd) {
-	// 				vsum += sparseV[pointSparse++];
-	// 			}
-	// 			if(pointSparse == sparseEndIndex) {
-	// 				break;
-	// 			}
-	// 		}
-
-	// 		for(int j = 0; j < _colIndexes.length; j++) {
-	// 			int Voff = _colIndexes[j] + row * numCols;
-	// 			c[Voff] += vsum * values[valOff + j];
-	// 		}
-	// 	}
+	// public void leftMultBySparseMatrix(SparseBlock sb, double[] c, double[] values, int numRows, int numCols, int
+	// row) {
+
+	// final int numVals = getNumValues();
+	// int sparseEndIndex = sb.size(row) + sb.pos(row);
+	// int[] indexes = sb.indexes(row);
+	// double[] sparseV = sb.values(row);
+	// for(int k = 0, valOff = 0; k < numVals; k++, valOff += _colIndexes.length) {
+	// int boff = _ptr[k];
+	// int blen = len(k);
+
+	// double vsum = 0;
+	// int pointSparse = sb.pos(row);
+	// int curRunEnd = 0;
+	// for(int bix = 0; bix < blen; bix += 2) {
+	// int curRunStartOff = curRunEnd + _data[boff + bix];
+	// int curRunLen = _data[boff + bix + 1];
+	// curRunEnd = curRunStartOff + curRunLen;
+	// while(pointSparse < sparseEndIndex && indexes[pointSparse] < curRunStartOff) {
+	// pointSparse++;
+	// }
+	// while(pointSparse != sparseEndIndex && indexes[pointSparse] >= curRunStartOff &&
+	// indexes[pointSparse] < curRunEnd) {
+	// vsum += sparseV[pointSparse++];
+	// }
+	// if(pointSparse == sparseEndIndex) {
+	// break;
+	// }
+	// }
+
+	// for(int j = 0; j < _colIndexes.length; j++) {
+	// int Voff = _colIndexes[j] + row * numCols;
+	// c[Voff] += vsum * values[valOff + j];
+	// }
+	// }
 
 	// }
 
@@ -1091,7 +1067,7 @@ public class ColGroupRLE extends ColGroupOffset {
 				lenL = _data[boffL + bixL + 1];
 				final int endL = startL + lenL;
 				for(int i = startL; i < endL; i++)
-					ag.increment(lhs.getIndex(i) + offKr);
+					ag.increment(lhs._data.getIndex(i) + offKr);
 
 			}
 		}
@@ -1169,7 +1145,7 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
@@ -1182,4 +1158,9 @@ public class ColGroupRLE extends ColGroupOffset {
 	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
 		throw new NotImplementedException();
 	}
+
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+		throw new NotImplementedException();
+	}
 }
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 5e2d303..1acbfdc 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
@@ -61,7 +61,7 @@ public class ColGroupSDC extends ColGroupValue {
 	 */
 	protected AMapToData _data;
 
-		/**
+	/**
 	 * Constructor for serialization
 	 * 
 	 * @param numRows Number of rows contained
@@ -405,7 +405,7 @@ public class ColGroupSDC extends ColGroupValue {
 
 		int row;
 		for(; i < this._numRows && it.hasNext(); i++) {
-			int col = lhs.getIndex(i);
+			int col = lhs._data.getIndex(i);
 			if(it.value() == i)
 				row = getIndex(it.getDataIndexAndIncrement());
 			else
@@ -414,7 +414,7 @@ public class ColGroupSDC extends ColGroupValue {
 		}
 		row = offsetToDefault;
 		for(; i < this._numRows; i++) {
-			int col = lhs.getIndex(i);
+			int col = lhs._data.getIndex(i);
 			ag.increment(col + row * nCol);
 		}
 
@@ -612,54 +612,77 @@ public class ColGroupSDC extends ColGroupValue {
 
 		for(; i < _numRows && it.hasNext(); i++) {
 			int to = (it.value() == i) ? getIndex(it.getDataIndexAndIncrement()) : offsetToDefault;
-			that._dict.addToEntry(ret, that.getIndex(i), to, nCol);
+			that._dict.addToEntry(ret, that._data.getIndex(i), to, nCol);
 		}
 
 		for(; i < _numRows; i++)
-			that._dict.addToEntry(ret, that.getIndex(i), offsetToDefault, nCol);
+			that._dict.addToEntry(ret, that._data.getIndex(i), offsetToDefault, nCol);
 
 		return ret;
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		final AIterator itThat = that._indexes.getIterator();
 		final AIterator itThis = _indexes.getIterator();
+		final int nCol = that._colIndexes.length;
 		final int offsetToDefaultThat = that.getNumValues() - 1;
 		final int offsetToDefaultThis = getNumValues() - 1;
-		final int nCol = that._colIndexes.length;
-
-		int i = 0;
-
-		for(; i < _numRows && itThat.hasNext() && itThis.hasNext(); i++) {
-			int to = (itThis.value() == i) ? getIndex(itThis.getDataIndexAndIncrement()) : offsetToDefaultThis;
-			int fr = (itThat.value() == i) ? that.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefaultThat;
-			that._dict.addToEntry(ret, fr, to, nCol);
-		}
 
-		if(itThat.hasNext()) {
-			for(; i < _numRows && itThat.hasNext(); i++) {
-				int fr = (itThat.value() == i) ? that.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefaultThat;
+		if(preModified) {
+			while(itThat.hasNext() && itThis.hasNext()) {
+				if(itThat.value() == itThis.value()) {
+					final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
+					final int to = getIndex(itThis.getDataIndexAndIncrement());
+					that._dict.addToEntry(ret, fr, to, nCol);
+				}
+				else if(itThat.value() < itThis.value()) {
+					final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
+					that._dict.addToEntry(ret, fr, offsetToDefaultThis, nCol);
+				}
+				else
+					itThis.next();
+			}
+	
+			while(itThat.hasNext()) {
+				final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
 				that._dict.addToEntry(ret, fr, offsetToDefaultThis, nCol);
 			}
 		}
+		else {
+			int i = 0;
 
-		if(itThis.hasNext()) {
-			for(; i < _numRows && itThis.hasNext(); i++) {
+			for(; i < _numRows && itThat.hasNext() && itThis.hasNext(); i++) {
 				int to = (itThis.value() == i) ? getIndex(itThis.getDataIndexAndIncrement()) : offsetToDefaultThis;
-				that._dict.addToEntry(ret, offsetToDefaultThat, to, nCol);
+				int fr = (itThat.value() == i) ? that.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefaultThat;
+				that._dict.addToEntry(ret, fr, to, nCol);
+			}
+
+			if(itThat.hasNext()) {
+				for(; i < _numRows && itThat.hasNext(); i++) {
+					int fr = (itThat.value() == i) ? that
+						.getIndex(itThat.getDataIndexAndIncrement()) : offsetToDefaultThat;
+					that._dict.addToEntry(ret, fr, offsetToDefaultThis, nCol);
+				}
 			}
+
+			if(itThis.hasNext()) {
+				for(; i < _numRows && itThis.hasNext(); i++) {
+					int to = (itThis.value() == i) ? getIndex(itThis.getDataIndexAndIncrement()) : offsetToDefaultThis;
+					that._dict.addToEntry(ret, offsetToDefaultThat, to, nCol);
+				}
+			}
+
+			for(; i < _numRows; i++)
+				that._dict.addToEntry(ret, offsetToDefaultThat, offsetToDefaultThis, nCol);
 		}
 
-		for(; i < _numRows; i++)
-			that._dict.addToEntry(ret, offsetToDefaultThat, offsetToDefaultThis, nCol);
-			
 		return ret;
 	}
 
 	@Override
 	public Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
-		
+
 		final AIterator itThat = that._indexes.getIterator();
 		final AIterator itThis = _indexes.getIterator();
 		final int nCol = that._colIndexes.length;
@@ -671,7 +694,7 @@ public class ColGroupSDC extends ColGroupValue {
 				final int to = getIndex(itThis.getDataIndexAndIncrement());
 				that._dict.addToEntry(ret, fr, to, nCol);
 			}
-			else if(itThat.value() < itThis.value()){
+			else if(itThat.value() < itThis.value()) {
 				final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
 				that._dict.addToEntry(ret, fr, defThis, nCol);
 			}
@@ -679,7 +702,7 @@ public class ColGroupSDC extends ColGroupValue {
 				itThis.next();
 		}
 
-		while(itThat.hasNext()){
+		while(itThat.hasNext()) {
 			final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
 			that._dict.addToEntry(ret, fr, defThis, nCol);
 		}
@@ -688,7 +711,12 @@ public class ColGroupSDC extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret){
+	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
+		throw new NotImplementedException();
+	}
+
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
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 82e5c81..85aab80 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
@@ -243,19 +243,24 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	@Override
 	public int[] getCounts(int[] counts) {
-		final AIterator it = _indexes.getIterator();
-
-		while(it.hasNext()) {
-			it.next();
-			counts[1]++;
-		}
-		counts[0] = _numRows - counts[1];
+		counts[0] = _indexes.getSize();
+		counts[1] = _numRows - counts[0];
 		return counts;
 	}
 
 	@Override
 	public int[] getCounts(int rl, int ru, int[] counts) {
-		throw new NotImplementedException("Not Implemented");
+		final AIterator it = _indexes.getIterator();
+		it.skipTo(rl);
+
+		while(it.hasNext() && it.value() < ru) {
+			it.next();
+			counts[0]++;
+		}
+
+		counts[1] = ru - rl - counts[0];
+
+		return counts;
 	}
 
 	public double[] preAggregate(double[] a, int row) {
@@ -275,7 +280,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			for(; i < _numRows; i++, offA++)
 				vals[0] += a[offA];
 		}
-		else{
+		else {
 			for(; i < _numRows && it.hasNext(); i++)
 				if(it.value() == i)
 					vals[1] += a[i];
@@ -294,10 +299,10 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		final int[] indexes = sb.indexes(row);
 		final double[] sparseV = sb.values(row);
 		final AIterator it = _indexes.getIterator();
-		
+
 		for(int i = sb.pos(row); i < sb.size(row) + sb.pos(row); i++) {
 			it.skipTo(indexes[i]);
-			if(it.value() == indexes[i]){
+			if(it.value() == indexes[i]) {
 				vals[0] += sparseV[i];
 				it.next();
 			}
@@ -309,7 +314,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(),  isLossy());
+		long size = ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getNumValues(), isLossy());
 		size += _indexes.getInMemorySize();
 		return size;
 	}
@@ -375,8 +380,8 @@ public class ColGroupSDCSingle extends ColGroupValue {
 
 		int row;
 		for(; i < this._numRows && it.hasNext(); i++) {
-			int col = lhs.getIndex(i);
-			if(it.value() == i){
+			int col = lhs._data.getIndex(i);
+			if(it.value() == i) {
 				row = 1;
 				it.next();
 			}
@@ -387,7 +392,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		}
 		row = 0;
 		for(; i < this._numRows; i++) {
-			int col = lhs.getIndex(i);
+			int col = lhs._data.getIndex(i);
 			if(col < lhs.getNumValues())
 				ag.increment(col + row * nCol);
 		}
@@ -459,7 +464,8 @@ public class ColGroupSDCSingle extends ColGroupValue {
 		final int rhsNV = this.getNumValues();
 		final int retSize = lhsNV * rhsNV;
 		final int nCol = lhs.getNumValues();
-		IPreAggregate ag = PreAggregateFactory.ag(retSize);;
+		IPreAggregate ag = PreAggregateFactory.ag(retSize);
+		;
 		final AIterator lIt = lhs._indexes.getIterator();
 		final AIterator rIt = _indexes.getIterator();
 
@@ -473,7 +479,7 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			}
 			else
 				col = 0;
-			if(rIt.value() == i){
+			if(rIt.value() == i) {
 				row = 1;
 				rIt.next();
 			}
@@ -482,9 +488,9 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			ag.increment(col + row * nCol);
 		}
 
-		if(lIt.hasNext() ) {
+		if(lIt.hasNext()) {
 			row = 1;
-			for(; i < _numRows && lIt.hasNext() ; i++) {
+			for(; i < _numRows && lIt.hasNext(); i++) {
 				if(lIt.value() == i) {
 					col = 1;
 					lIt.next();
@@ -496,10 +502,10 @@ public class ColGroupSDCSingle extends ColGroupValue {
 			}
 		}
 
-		if( rIt.hasNext()) {
+		if(rIt.hasNext()) {
 			col = 1;
 			for(; i < _numRows && rIt.hasNext(); i++) {
-				if(rIt.value()== i) {
+				if(rIt.value() == i) {
 					row = 1;
 					rIt.next();
 				}
@@ -558,18 +564,91 @@ public class ColGroupSDCSingle extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret){
+	public Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret) {
 		throw new NotImplementedException();
 	}
-	
+
 	@Override
-	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret){
+	public Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret) {
 		throw new NotImplementedException();
 	}
 
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified) {
+		final AIterator itThat = that._indexes.getIterator();
+		final AIterator itThis = _indexes.getIterator();
+		final int nCol = that._colIndexes.length;
+
+		if(preModified) {
+
+			while(itThat.hasNext() && itThis.hasNext()) {
+				if(itThat.value() == itThis.value()) {
+					itThat.next();
+					itThis.next();
+					that._dict.addToEntry(ret, 1, 0, nCol);
+				}
+				else if(itThat.value() < itThis.value()) {
+					itThat.next();
+					// that._dict.addToEntry(ret, 0, 1, nCol);
+				}
+				else
+					itThis.next();
+			}
+
+			// while(itThat.hasNext()) {
+			// 	final int fr = that.getIndex(itThat.getDataIndexAndIncrement());
+			// 	that._dict.addToEntry(ret, fr, 1, nCol);
+			// }
+			return ret;
+		}
+		else {
+			int i = 0;
+			for(; i < _numRows && itThat.hasNext() && itThis.hasNext(); i++) {
+				int to = 0;
+				if(itThis.value() == i) {
+					itThis.next();
+					to = 1;
+				}
+				int fr = 0;
+				if(itThat.value() == i) {
+					itThat.next();
+					fr = 1;
+				}
+				that._dict.addToEntry(ret, fr, to, nCol);
+			}
+
+			if(itThat.hasNext()) {
+				for(; i < _numRows && itThat.hasNext(); i++) {
+					int fr = 0;
+					if(itThat.value() == i) {
+						itThat.next();
+						fr = 1;
+					}
+					that._dict.addToEntry(ret, fr, 1, nCol);
+				}
+			}
+
+			if(itThis.hasNext()) {
+				for(; i < _numRows && itThis.hasNext(); i++) {
+					int to = 0;
+					if(itThis.value() == i) {
+						itThis.next();
+						to = 1;
+					}
+					that._dict.addToEntry(ret, 1, to, nCol);
+				}
+			}
+
+			for(; i < _numRows; i++)
+				that._dict.addToEntry(ret, 1, 1, nCol);
+
+			return ret;
+		}
+
+	}
 }
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 faf8955..34d148d 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
@@ -67,7 +67,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		int[] cachedCounts) {
 		super(colIndices, numRows, dict, cachedCounts);
 		_indexes = OffsetFactory.create(indexes, numRows);
-		_zeros = false;
+		_zeros = true;
 	}
 
 	protected ColGroupSDCSingleZeros(int[] colIndices, int numRows, ADictionary dict, AOffset offsets,
@@ -196,25 +196,22 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 
 	@Override
 	public int[] getCounts(int[] counts) {
-		return getCounts(0, _numRows, counts);
+		counts[0] = _indexes.getSize();
+		counts[1] = _numRows - counts[0];
+		return counts;
 	}
 
 	@Override
 	public int[] getCounts(int rl, int ru, int[] counts) {
-		int i = rl;
 		final AIterator it = _indexes.getIterator();
 		it.skipTo(rl);
 
-		int zeros = 0;
 		while(it.hasNext() && it.value() < ru) {
-			int oldI = i;
-			i = it.value();
 			it.next();
-			zeros += i - oldI - 1;
 			counts[0]++;
 		}
 
-		counts[counts.length - 1] += zeros + ru - i;
+		counts[1] = ru - rl - counts[0];
 
 		return counts;
 	}
@@ -287,8 +284,8 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 			return new ColGroupSDCSingleZeros(_colIndexes, _numRows, applyBinaryRowOp(op.fn, v, sparseSafe, left),
 				_indexes, getCachedCounts());
 		else {
-			ADictionary aDictionary = swapEntries(applyBinaryRowOp(op.fn, v, sparseSafe, left));
-			return new ColGroupSDCSingle(_colIndexes, _numRows, aDictionary, _indexes, null);
+			ADictionary aDictionary = applyBinaryRowOp(op.fn, v, sparseSafe, left);
+			return new ColGroupSDCSingle(_colIndexes, _numRows, aDictionary, _indexes, getCachedCounts());
 		}
 	}
 
@@ -348,7 +345,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 		final AIterator it = _indexes.getIterator();
 
 		while(it.hasNext()) {
-			final int col = lhs.getIndex(it.value());
+			final int col = lhs._data.getIndex(it.value());
 			ag.increment(col);
 		}
 		return ag;
@@ -427,7 +424,7 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
@@ -451,4 +448,9 @@ public class ColGroupSDCSingleZeros extends ColGroupValue {
 
 		return ret;
 	}
+
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret, boolean preModified){
+		throw new NotImplementedException();
+	}
 }
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 3f6b8d3..410ec90 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
@@ -358,7 +358,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		final AIterator it = _indexes.getIterator();
 
 		while(it.hasNext()) {
-			final int col = lhs.getIndex(it.value());
+			final int col = lhs._data.getIndex(it.value());
 			final int row = getIndex(it.getDataIndexAndIncrement());
 			ag.increment(col + row * nCol);
 		}
@@ -482,7 +482,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		final int nCol = that._colIndexes.length;
 
 		while(itThis.hasNext()) {
-			final int fr = that.getIndex(itThis.value());
+			final int fr = that._data.getIndex(itThis.value());
 			final int to = getIndex(itThis.getDataIndexAndIncrement());
 			that._dict.addToEntry(ret, fr, to, nCol);
 		}
@@ -491,7 +491,7 @@ public class ColGroupSDCZeros extends ColGroupValue {
 	}
 
 	@Override
-	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret) {
+	public Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified) {
 		throw new NotImplementedException();
 	}
 
@@ -534,4 +534,9 @@ public class ColGroupSDCZeros extends ColGroupValue {
 		}
 		return ret;
 	}
+	
+	@Override
+	public Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary re, boolean preModified){
+		throw new NotImplementedException();
+	}
 }
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 b13078c..a848031 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,7 +25,6 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.commons.lang.NotImplementedException;
-import org.apache.sysds.runtime.DMLCompressionException;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.data.DenseBlockFP64;
@@ -318,16 +317,15 @@ public class ColGroupUncompressed extends AColGroup {
 
 	public void leftMultByMatrix(MatrixBlock matrix, double[] result, int numCols, int rl, int ru) {
 
-		MatrixBlock tmpRet = new MatrixBlock(ru - rl, _data.getNumColumns(), false);
+		final MatrixBlock tmpRet = new MatrixBlock(ru - rl, _data.getNumColumns(), false);
 		tmpRet.allocateDenseBlock();
-		MatrixBlock leftSlice = matrix.slice(rl, ru - 1, false);
+		final MatrixBlock leftSlice = matrix.slice(rl, ru - 1, false);
 		LibMatrixMult.matrixMult(leftSlice, _data, tmpRet);
 		int offT = numCols * rl;
-
 		if(tmpRet.isEmpty())
 			return;
-		if(tmpRet.isInSparseFormat()) {
-			SparseBlock sb = tmpRet.getSparseBlock();
+		else if(tmpRet.isInSparseFormat()) {
+			final SparseBlock sb = tmpRet.getSparseBlock();
 			for(int rowIdx = 0; rowIdx < ru - rl; rowIdx++, offT += numCols) {
 				if(!sb.isEmpty(rowIdx)) {
 					final int apos = sb.pos(rowIdx);
@@ -340,33 +338,10 @@ public class ColGroupUncompressed extends AColGroup {
 			}
 		}
 		else {
-			double[] tmpRetV = tmpRet.getDenseBlockValues();
-			for(int j = rl, offTemp = 0; j < ru; j++, offTemp += _colIndexes.length, offT += numCols) {
+			final double[] tmpRetV = tmpRet.getDenseBlockValues();
+			for(int j = rl, offTemp = 0; j < ru; j++, offTemp += _colIndexes.length, offT += numCols)
 				for(int i = 0; i < _colIndexes.length; i++)
 					result[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
-			}
-		}
-	}
-
-	public void rightMultByMatrix(int[] outputColumns, double[] preAggregatedB, double[] c, int thatNrColumns, int rl,
-		int ru) {
-		throw new NotImplementedException("Should not be called use other matrix function for uncompressed columns");
-	}
-
-	public double computeMxx(double c, Builtin builtin) {
-		throw new NotImplementedException("Not implemented max min on uncompressed");
-	}
-
-	public void leftMultByMatrix(MatrixBlock matrix, MatrixBlock result) {
-		MatrixBlock pret = new MatrixBlock(matrix.getNumRows(), _colIndexes.length, false);
-		LibMatrixMult.matrixMult(matrix, _data, pret);
-
-		// copying partialResult to the proper indices of the result
-		if(!pret.isEmptyBlock(false)) {
-			double[] rsltArr = result.getDenseBlockValues();
-			for(int colIx = 0; colIx < _colIndexes.length; colIx++)
-				rsltArr[_colIndexes[colIx]] = pret.quickGetValue(0, colIx);
-			result.recomputeNonZeros();
 		}
 	}
 
@@ -382,14 +357,20 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public AColGroup binaryRowOp(BinaryOperator op, double[] v, boolean sparseSafe, boolean left) {
-		DenseBlock b = new DenseBlockFP64(new int[] {1, v.length}, v);
-		MatrixBlock that = new MatrixBlock(1, v.length, b);
-		that.setNonZeros(v.length);
+		double[] selectedValues = new double[_colIndexes.length];
+		for(int i = 0; i < _colIndexes.length; i++) {
+			selectedValues[i] = v[_colIndexes[i]];
+		}
+		DenseBlock b = new DenseBlockFP64(new int[] {1, _colIndexes.length}, selectedValues);
+		MatrixBlock that = new MatrixBlock(1, _colIndexes.length, b);
+		that.setNonZeros(_colIndexes.length);
 		MatrixBlock resultBlock = new MatrixBlock();
+
 		if(left)
 			that.binaryOperations(op, _data, resultBlock);
 		else
 			_data.binaryOperations(op, that, resultBlock);
+
 		return new ColGroupUncompressed(_colIndexes, resultBlock, false);
 	}
 
@@ -557,7 +538,10 @@ public class ColGroupUncompressed extends AColGroup {
 
 	@Override
 	public AColGroup copy() {
-		throw new NotImplementedException("Not implemented copy of uncompressed colGroup yet.");
+		MatrixBlock newData = new MatrixBlock(_data.getNumRows(), _data.getNumColumns(), _data.isInSparseFormat());
+		// _data.copy(newData);
+		newData.copy(_data);
+		return new ColGroupUncompressed(_colIndexes, newData, false);
 	}
 
 	@Override
@@ -585,7 +569,49 @@ public class ColGroupUncompressed extends AColGroup {
 		if(lhs instanceof ColGroupEmpty)
 			return;
 		if(lhs instanceof ColGroupUncompressed) {
-			throw new DMLCompressionException("Not Implemented");
+			ColGroupUncompressed lhsUC = (ColGroupUncompressed) lhs;
+			MatrixBlock tmpRet = new MatrixBlock(_colIndexes.length, _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 = new MatrixBlock(lhData.getNumColumns(), lhData.getNumRows(), false);
+				LibMatrixReorg.transpose(lhData, transposed);
+				transposed.setNonZeros(lhData.getNonZeros());
+				// do transposed left hand side, matrix multiplication.
+				LibMatrixMult.matrixMult(transposed, this._data, tmpRet);
+			}
+
+			if(tmpRet.isEmpty())
+				return;
+			else if(tmpRet.isInSparseFormat()) {
+				SparseBlock sb = tmpRet.getSparseBlock();
+				for(int rowIdx = 0, offT = 0; rowIdx < tmpRet.getNumRows(); rowIdx++, offT += numCols) {
+					if(!sb.isEmpty(rowIdx)) {
+						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++)
+							result[offT + _colIndexes[aix[col]]] += avals[col];
+					}
+				}
+			}
+			else {
+				double[] tmpRetV = tmpRet.getDenseBlockValues();
+				for(int j = 0, offTemp = 0, offT = 0;
+					j < tmpRet.getNumRows();
+					j++, offTemp += _colIndexes.length, offT += numCols) {
+					for(int i = 0; i < _colIndexes.length; i++)
+						result[offT + _colIndexes[i]] += tmpRetV[offTemp + i];
+				}
+			}
+
 		}
 		else {
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index 0eeeb13..223c3a0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -30,21 +30,14 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.sysds.runtime.DMLCompressionException;
-import org.apache.sysds.runtime.compress.CompressionSettings;
 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.QDictionary;
 import org.apache.sysds.runtime.compress.colgroup.pre.ArrPreAggregate;
 import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.compress.colgroup.pre.MapPreAggregate;
-import org.apache.sysds.runtime.compress.utils.ABitmap;
-import org.apache.sysds.runtime.compress.utils.Bitmap;
-import org.apache.sysds.runtime.compress.utils.BitmapLossy;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 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.ValueFunction;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -81,34 +74,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		super(numRows);
 	}
 
-	/**
-	 * Main constructor for the ColGroupValues. Used to contain the dictionaries used for the different types of
-	 * ColGroup.
-	 * 
-	 * @param colIndices indices (within the block) of the columns included in this column
-	 * @param numRows    total number of rows in the parent block
-	 * @param ubm        Uncompressed bitmap representation of the block
-	 * @param cs         The Compression settings used for compression
-	 */
-	protected ColGroupValue(int[] colIndices, int numRows, ABitmap ubm, CompressionSettings cs) {
-		super(colIndices, numRows);
-
-		_zeros = ubm.getNumOffsets() < (long) numRows;
-		// sort values by frequency, if requested
-		if(cs.sortValuesByLength && numRows > CompressionSettings.BITMAP_BLOCK_SZ)
-			ubm.sortValuesByFrequency();
-
-		switch(ubm.getType()) {
-			case Full:
-				_dict = new Dictionary(((Bitmap) ubm).getValues());
-				break;
-			case Lossy:
-				_dict = new QDictionary((BitmapLossy) ubm).makeDoubleDictionary();
-				// _lossy = true;
-				break;
-		}
-	}
-
 	protected ColGroupValue(int[] colIndices, int numRows, ADictionary dict, int[] cachedCounts) {
 		super(colIndices, numRows);
 		_dict = dict;
@@ -130,20 +95,21 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 * 
 	 * @return the number of distinct sets of values associated with the bitmaps in this column group
 	 */
-	public int getNumValues() {
+	public final int getNumValues() {
 		return _dict.getNumberOfValues(_colIndexes.length);
 	}
 
 	@Override
-	public double[] getValues() {
+	public final double[] getValues() {
 		return _dict != null ? _dict.getValues() : null;
 	}
 
-	public ADictionary getDictionary() {
+	public final ADictionary getDictionary() {
 		return _dict;
 	}
 
-	public void addMinMax(double[] ret) {
+	@Override
+	public final void addMinMax(double[] ret) {
 		_dict.addMaxAndMin(ret, _colIndexes);
 	}
 
@@ -362,14 +328,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return ret;
 	}
 
-	public double getMin() {
-		return computeMxx(Double.POSITIVE_INFINITY, Builtin.getBuiltinFnObject(BuiltinCode.MIN));
-	}
-
-	public double getMax() {
-		return computeMxx(Double.NEGATIVE_INFINITY, Builtin.getBuiltinFnObject(BuiltinCode.MAX));
-	}
-
 	protected double computeMxx(double c, Builtin builtin) {
 		if(_zeros)
 			c = builtin.execute(c, 0);
@@ -487,11 +445,11 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	@Override
 	public String toString() {
 		StringBuilder sb = new StringBuilder();
+		sb.append(" Is Lossy: " + _dict.isLossy() + " num Rows: " + getNumRows() + " contain zero row:" + _zeros);
 		sb.append(super.toString());
-		sb.append("Is Lossy: " + _dict.isLossy() + " num Rows: " + getNumRows() + " contain zero row:" + _zeros);
 		if(_dict != null) {
 			sb.append(String.format("\n%15s%5d ", "Values:", _dict.getValues().length));
-			_dict.getString(sb, _colIndexes.length);
+			sb.append(_dict.getString(_colIndexes.length));
 		}
 		return sb.toString();
 	}
@@ -556,6 +514,30 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return super.clone();
 	}
 
+	public AColGroup copyAndSet(double[] newDictionary) {
+		try {
+			ColGroupValue clone = (ColGroupValue) this.clone();
+			clone.setDictionary(new Dictionary(newDictionary));
+			return clone;
+		}
+		catch(CloneNotSupportedException e) {
+			e.printStackTrace();
+		}
+		return null;
+	}
+
+	public AColGroup copyAndSet(ADictionary newDictionary) {
+		try {
+			ColGroupValue clone = (ColGroupValue) this.clone();
+			clone.setDictionary(newDictionary);
+			return clone;
+		}
+		catch(CloneNotSupportedException e) {
+			e.printStackTrace();
+		}
+		return null;
+	}
+
 	public AColGroup copyAndSet(int[] colIndexes, double[] newDictionary) {
 		try {
 			ColGroupValue clone = (ColGroupValue) this.clone();
@@ -594,7 +576,6 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return null;
 	}
 
-
 	@Override
 	protected AColGroup sliceSingleColumn(int idx) {
 		ColGroupValue ret = (ColGroupValue) copy();
@@ -667,12 +648,12 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	 */
 	protected void postScaling(double[] dictValues, double[] vals, double[] c, int numVals, int row, int totalCols,
 		int offT) {
-		final int ncol = getNumCols();
+		final int nCol = getNumCols();
 		int valOff = 0;
 
 		for(int k = 0; k < numVals; k++) {
 			double aval = vals[k];
-			for(int j = 0; j < ncol; j++) {
+			for(int j = 0; j < nCol; j++) {
 				int colIx = _colIndexes[j] + row * totalCols;
 				c[offT + colIx] += aval * dictValues[valOff++];
 			}
@@ -771,21 +752,27 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 	/**
 	 * Pre aggregate into a dictionary. It is assumed that "that" have more distinct values than, "this".
 	 * 
-	 * @param that the other column group whose indexes are used for aggregation.
+	 * @param that      the other column group whose indexes are used for aggregation.
+	 * @param preModify specifies if the matrix in this
 	 * @return A aggregate dictionary
 	 */
-	public Dictionary preAggregateThatIndexStructure(ColGroupValue that) {
+	public Dictionary preAggregateThatIndexStructure(ColGroupValue that, boolean preModify) {
+		int outputLength = that._colIndexes.length * this.getNumValues();
+		Dictionary ret = new Dictionary(new double[outputLength]);
 
-		Dictionary ret = new Dictionary(new double[that._colIndexes.length * this.getNumValues()]);
+		// if(preModify)
+		// LOG.error(preModify + " " + that.getClass().getSimpleName() + " in " + this.getClass().getSimpleName());
 
 		if(that instanceof ColGroupDDC)
 			return preAggregateThatDDCStructure((ColGroupDDC) that, ret);
 		else if(that instanceof ColGroupSDC)
-			return preAggregateThatSDCStructure((ColGroupSDC) that, ret);
-		else if(that instanceof ColGroupSDCZeros)
-			return preAggregateThatSDCZerosStructure((ColGroupSDCZeros) that, ret);
+			return preAggregateThatSDCStructure((ColGroupSDC) that, ret, preModify);
+		else if(that instanceof ColGroupSDCSingle)
+			return preAggregateThatSDCSingleStructure((ColGroupSDCSingle) that, ret, preModify);
 		else if(that instanceof ColGroupSDCSingleZeros)
 			return preAggregateThatSDCSingleZerosStructure((ColGroupSDCSingleZeros) that, ret);
+		else if(that instanceof ColGroupSDCZeros)
+			return preAggregateThatSDCZerosStructure((ColGroupSDCZeros) that, ret);
 		else if(that instanceof ColGroupConst)
 			return preAggregateThatConstStructure((ColGroupConst) that, ret);
 
@@ -795,15 +782,17 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 
 	public abstract Dictionary preAggregateThatDDCStructure(ColGroupDDC that, Dictionary ret);
 
-	public abstract Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret);
+	public abstract Dictionary preAggregateThatSDCStructure(ColGroupSDC that, Dictionary ret, boolean preModified);
 
 	public abstract Dictionary preAggregateThatSDCZerosStructure(ColGroupSDCZeros that, Dictionary ret);
 
 	public abstract Dictionary preAggregateThatSDCSingleZerosStructure(ColGroupSDCSingleZeros that, Dictionary ret);
 
+	public abstract Dictionary preAggregateThatSDCSingleStructure(ColGroupSDCSingle that, Dictionary ret,
+		boolean preModified);
+
 	public Dictionary preAggregateThatConstStructure(ColGroupConst that, Dictionary ret) {
 		computeColSums(ret.getValues(), false);
-		LOG.error(Arrays.toString(ret.getValues()));
 		return ret;
 	}
 
@@ -835,53 +824,82 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		final int lCol = lhs._colIndexes.length;
 		final int rCol = this._colIndexes.length;
 
+		final double threshold = 0.2;
+
 		if(sameIndexStructure(lhs)) {
 			int[] agI = getCounts();
 			for(int a = 0, off = 0; a < nvL; a++, off += nvL + 1)
 				leftMultDictEntry(agI[a], off, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, result);
 		}
 		else if(lhs instanceof ColGroupConst || this instanceof ColGroupConst) {
-			IPreAggregate ag = preAggregate(lhs);
-			if(ag == null)
-				return;
-			else if(ag instanceof MapPreAggregate)
-				leftMultMapPreAggregate(nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, result,
-					(MapPreAggregate) ag);
-			else
-				leftMultArrayPreAggregate(nvL, nvR, lCol, rCol, lhs, numCols, lhValues, rhValues, result,
-					((ArrPreAggregate) ag).getArr());
+			double[] r = this instanceof ColGroupConst ? rhValues : this._dict.colSum(getCounts(), rCol);
+			double[] l = lhs instanceof ColGroupConst ? lhValues : lhs._dict.colSum(lhs.getCounts(), lCol);
+			vectorVectorMultiply(l, lhs._colIndexes, r, this._colIndexes, result, numCols);
 		}
 		else {
-			if(nvR * rCol < nvL * lCol) {
-				Dictionary preAgg = lhs.preAggregateThatIndexStructure(this);
+			int[] countsRight = getCounts();
+			int mostFrequentRight = Math.max(countsRight[0], countsRight[countsRight.length - 1]);
+			double percentageRight = (double) mostFrequentRight / this._numRows;
+			double skipRight = percentageRight * rCol;
+			int[] countsLeft = lhs.getCounts();
+			int mostFrequentLeft = Math.max(countsLeft[0], countsLeft[countsLeft.length - 1]);
+			double percentageLeft = (double) mostFrequentLeft / this._numRows;
+			double skipLeft = percentageLeft * lCol;
+
+			if(skipRight > threshold && percentageRight > percentageLeft && !(this instanceof ColGroupDDC)) {
+				double[] mct = this._dict.getMostCommonTuple(this.getCounts(), rCol);
+				double[] lhsSum = lhs._dict.colSum(lhs.getCounts(), lCol);
+				if(mct != null)
+					vectorVectorMultiply(lhsSum, lhs._colIndexes, mct, this._colIndexes, result, numCols);
+
+				ColGroupValue thisM = (mct != null) ? (ColGroupValue) this
+					.copyAndSet(this._dict.subtractTuple(mct)) : this;
+				Dictionary preAgg = lhs.preAggregateThatIndexStructure(thisM, true);
+				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
+					this._colIndexes, result, numCols);
+			}
+			else if(skipLeft > threshold && !(lhs instanceof ColGroupDDC)) {
+				double[] mct = lhs._dict.getMostCommonTuple(lhs.getCounts(), lCol);
+				double[] thisColSum = this._dict.colSum(getCounts(), rCol);
+				if(mct != null)
+					vectorVectorMultiply(mct, lhs._colIndexes, thisColSum, this._colIndexes, result, numCols);
+
+				ColGroupValue lhsM = (mct != null) ? (ColGroupValue) lhs.copyAndSet(lhs._dict.subtractTuple(mct)) : lhs;
+				Dictionary preAgg = this.preAggregateThatIndexStructure(lhsM, true);
+				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
+					this._colIndexes, result, numCols);
+			}
+			else if(nvR * rCol < nvL * lCol) {
+				Dictionary preAgg = lhs.preAggregateThatIndexStructure(this, false);
 				matrixMultDictionariesAndOutputToColIndexes(lhValues, preAgg.getValues(), lhs._colIndexes,
 					this._colIndexes, result, numCols);
 			}
 			else {
-				Dictionary preAgg = this.preAggregateThatIndexStructure(lhs);
+				Dictionary preAgg = this.preAggregateThatIndexStructure(lhs, false);
 				matrixMultDictionariesAndOutputToColIndexes(preAgg.getValues(), rhValues, lhs._colIndexes,
 					this._colIndexes, result, numCols);
 			}
 		}
 	}
 
-	private void leftMultMapPreAggregate(final int nvL, final int lCol, final int rCol, final ColGroupValue lhs,
-		final int numCols, double[] lhValues, double[] rhValues, double[] c, MapPreAggregate agM) {
-		final int[] map = agM.getMap();
-		final int aggSize = agM.getSize();
-		for(int k = 0; k < aggSize; k += 2)
-			leftMultDictEntry(map[k + 1], map[k], nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-		leftMultDictEntry(agM.getMapFreeValue(), 0, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-	}
+	// private void leftMultMapPreAggregate(final int nvL, final int lCol, final int rCol, final ColGroupValue lhs,
+	// final int numCols, double[] lhValues, double[] rhValues, double[] c, MapPreAggregate agM) {
+	// final int[] map = agM.getMap();
+	// final int aggSize = agM.getSize();
+	// for(int k = 0; k < aggSize; k += 2)
+	// leftMultDictEntry(map[k + 1], map[k], nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
+	// leftMultDictEntry(agM.getMapFreeValue(), 0, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
+	// }
 
-	private void leftMultArrayPreAggregate(final int nvL, final int nvR, final int lCol, final int rCol,
-		final ColGroupValue lhs, final int numCols, double[] lhValues, double[] rhValues, double[] c, int[] arr) {
-		for(int a = 0; a < nvL * nvR; a++)
-			leftMultDictEntry(arr[a], a, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
-	}
+	// private void leftMultArrayPreAggregate(final int nvL, final int nvR, final int lCol, final int rCol,
+	// final ColGroupValue lhs, final int numCols, double[] lhValues, double[] rhValues, double[] c, int[] arr) {
+	// for(int a = 0; a < nvL * nvR; a++)
+	// leftMultDictEntry(arr[a], a, nvL, lCol, rCol, lhs, numCols, lhValues, rhValues, c);
+	// }
 
 	private void leftMultDictEntry(final int m, final int a, final int nvL, final int lCol, final int rCol,
-		final ColGroupValue lhs, final int numCols, double[] lhValues, double[] rhValues, double[] c) {
+		final ColGroupValue lhs, final int numCols, final double[] lhValues, final double[] rhValues,
+		final double[] c) {
 
 		if(m > 0) {
 			final int lhsRowOffset = (a % nvL) * lCol;
@@ -930,27 +948,90 @@ public abstract class ColGroupValue extends ColGroupCompressed implements Clonea
 		return _dict.getNumberNonZeros(counts, _colIndexes.length);
 	}
 
-	private static void matrixMultDictionariesAndOutputToColIndexes(double[] left, double[] right, int[] colsLeft,
-		int[] colsRight, double[] result, int outCols) {
-		final int rows = left.length / colsLeft.length;
-		for(int k = 0; k < rows; k++) {
-			final int offL = k * colsLeft.length;
-			final int offR = k * colsRight.length;
-			for(int i = 0; i < colsLeft.length; i++) {
-				final int offOut = colsLeft[i] * outCols;
-				final double vl = left[offL + i];
-				if(vl != 0)
-					for(int j = 0; j < colsRight.length; j++) {
-						final double vr = right[offR + j];
-						result[offOut + colsRight[j]] += vl * vr;
-					}
-			}
+	private static void vectorVectorMultiply(final double[] left, final int[] leftRows, final double[] right,
+		final int[] rightColumns, final double[] result, final int outCols) {
+		if(left.length != leftRows.length) {
+			// LOG.error(Arrays.toString(left));
+			// LOG.error(Arrays.toString(right));
+			// LOG.error(Arrays.toString(leftRows));
+			// LOG.error(Arrays.toString(rightColumns));
+			throw new DMLCompressionException(
+				"Error left length " + left.length + " not equal columns length" + leftRows.length);
+		}
+		if(right.length != rightColumns.length)
+			throw new DMLCompressionException(
+				"Error right not equal length " + right.length + "  " + rightColumns.length);
+		for(int row = 0; row < leftRows.length; row++) {
+			final int outputRowOffset = leftRows[row] * outCols;
+			final double vLeft = left[row];
+			for(int col = 0; col < rightColumns.length; col++)
+				result[outputRowOffset + rightColumns[col]] += vLeft * right[col];
 		}
 	}
 
-	@Override
-	public int getNumRows() {
-		return _numRows;
+	private static boolean logMM = true;
+
+	/**
+	 * Matrix Multiply the two matrices, note that the left side is transposed,
+	 * 
+	 * making the multiplication a: t(left) %*% right
+	 * 
+	 * @param left      The left side matrix, transposed linearized row major
+	 * @param right     The right hand side linearized row major
+	 * @param rowsLeft  The number of rows and the row indexes on the left hand side
+	 * @param colsRight The number of columns and the column indexes on the right hand side
+	 * @param result    The result matrix to put the results into, linearized row major
+	 * @param outCols   The output columns count, to know how much to offset into with results.
+	 */
+	private static void matrixMultDictionariesAndOutputToColIndexes(double[] left, double[] right, int[] rowsLeft,
+		int[] colsRight, double[] result, int outCols) {
+
+		try {
+			final int rows = left.length / rowsLeft.length;
+			if(rows != right.length / colsRight.length)
+				throw new DMLCompressionException(
+					"Not equal number of rows: " + rows + " " + right.length / colsRight.length);
+			for(int k = 0; k < rows; k++) {
+				final int offL = k * rowsLeft.length;
+				final int offR = k * colsRight.length;
+				// final int offL = k * colsRight.length;
+				// final int offR = k * rowsLeft.length;
+				// if(offR < right.length && offL < left.length)
+				for(int i = 0; i < rowsLeft.length; i++) {
+					final int offOut = rowsLeft[i] * outCols;
+					final double vl = left[offL + i];
+					if(vl != 0)
+						for(int j = 0; j < colsRight.length; j++) {
+							final double vr = right[offR + j];
+							result[offOut + colsRight[j]] += vl * vr;
+						}
+				}
+			}
+		}
+		catch(Exception e) {
+
+			if(logMM) {
+				StringBuilder sb = new StringBuilder();
+				sb.append("\nLeft (transposed):\n");
+				for(int i = 0; i < rowsLeft.length; i++) {
+					for(int j = i * rowsLeft.length; j < (i + 1) * rowsLeft.length; j++)
+						sb.append(left[j] + ", ");
+					sb.append("\n");
+				}
+				LOG.error(sb);
+
+				sb = new StringBuilder();
+				sb.append("\nRight:\n");
+				for(int i = 0; i < colsRight.length; i++) {
+					for(int j = i * colsRight.length; j < (i + 1) * colsRight.length; j++)
+						sb.append(right[j] + ", ");
+					sb.append("\n");
+				}
+				LOG.error(sb);
+				logMM = false;
+			}
+			throw new DMLCompressionException("MM of pre aggregated colGroups failed", e);
+		}
 	}
 
 	@Override
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 9b084e8..1aeda85 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
@@ -164,7 +164,7 @@ public abstract class ADictionary {
 	 * 
 	 * @return the long count of bytes to store the dictionary.
 	 */
-	public long getExactSizeOnDisk(){
+	public long getExactSizeOnDisk() {
 		return 1;
 	}
 
@@ -212,13 +212,15 @@ public abstract class ADictionary {
 	 */
 	public abstract double sumRow(int k, boolean square, int nrColumns);
 
+	public abstract double[] colSum(int[] counts, int nCol);
+
 	public abstract void colSum(double[] c, int[] counts, int[] colIndexes, boolean square);
 
 	public abstract double sum(int[] counts, int ncol);
 
 	public abstract double sumsq(int[] counts, int ncol);
 
-	public abstract StringBuilder getString(StringBuilder sb, int colIndexes);
+	public abstract String getString(int colIndexes);
 
 	/**
 	 * This method adds the max and min values contained in the dictionary to corresponding cells in the ret variable.
@@ -252,6 +254,8 @@ public abstract class ADictionary {
 
 	public abstract long getNumberNonZeros(int[] counts, int nCol);
 
+	public abstract long getNumberNonZerosContained();
+
 	/**
 	 * Copies and adds the dictionary entry from this dictionary to the d dictionary
 	 * 
@@ -268,4 +272,23 @@ public abstract class ADictionary {
 		else
 			return new Dictionary(((Bitmap) ubm).getValues());
 	}
+
+	/**
+	 * Get the most common tuple element contained in the dictionary
+	 * 
+	 * returns null if that tuple is all zero values.
+	 * 
+	 * @param counts The counts of the individual tuples contained, managed by the column group.
+	 * @return a new double array containing the most common value
+	 */
+	public abstract double[] getMostCommonTuple(int[] counts, int nCol);
+
+	/**
+	 * Allocate a new dictionary where the tuple given is subtracted from all tuples in the previous dictionary.
+	 * 
+	 * @param tuple a double list representing a tuple, it is given that the tuple with is the same as this
+	 *              dictionaries.
+	 * @return a new instance of dictionary with the tuple subtracted.
+	 */
+	public abstract ADictionary subtractTuple(double[] tuple);
 }
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 26d090d..b4f843f 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
@@ -85,9 +85,8 @@ public class Dictionary extends ADictionary {
 	@Override
 	public double aggregate(double init, Builtin fn) {
 		// full aggregate can disregard tuple boundaries
-		int len = size();
 		double ret = init;
-		for(int i = 0; i < len; i++)
+		for(int i = 0; i < _values.length; i++)
 			ret = fn.execute(ret, _values[i]);
 		return ret;
 	}
@@ -249,12 +248,23 @@ public class Dictionary extends ADictionary {
 	}
 
 	@Override
+	public double[] colSum(int[] counts, int nCol) {
+		final double[] res = new double[nCol];
+		int idx = 0;
+		for(int k = 0; k < _values.length / nCol; k++) {
+			final int cntk = counts[k];
+			for(int j = 0; j < nCol; j++)
+				res[j] += _values[idx++] * cntk;
+		}
+		return res;
+	}
+
+	@Override
 	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
 		if(_values == null)
 			return;
-
 		for(int k = 0; k < _values.length / colIndexes.length; k++) {
-			int cntk = counts[k];
+			final int cntk = counts[k];
 			for(int j = 0; j < colIndexes.length; j++) {
 				double v = _values[k * colIndexes.length + j];
 				if(square)
@@ -328,17 +338,22 @@ public class Dictionary extends ADictionary {
 		}
 	}
 
-	public StringBuilder getString(StringBuilder sb, int colIndexes) {
-		sb.append("[");
-		for(int i = 0; i < _values.length - 1; i++) {
-			sb.append(_values[i]);
-			sb.append((i) % (colIndexes) == colIndexes - 1 ? ", " : ": ");
-		}
-		if(_values != null && _values.length > 0) {
-			sb.append(_values[_values.length - 1]);
+	public String getString(int colIndexes) {
+		StringBuilder sb = new StringBuilder();
+		if(colIndexes == 1)
+			sb.append(Arrays.toString(_values));
+		else {
+			sb.append("[");
+			for(int i = 0; i < _values.length - 1; i++) {
+				sb.append(_values[i]);
+				sb.append((i) % (colIndexes) == colIndexes - 1 ? "\n: " : ", ");
+			}
+			if(_values != null && _values.length > 0) {
+				sb.append(_values[_values.length - 1]);
+			}
+			sb.append("]");
 		}
-		sb.append("]");
-		return sb;
+		return sb.toString();
 	}
 
 	public ADictionary sliceOutColumnRange(int idxStart, int idxEnd, int previousNumberOfColumns) {
@@ -419,4 +434,46 @@ public class Dictionary extends ADictionary {
 	public boolean isLossy() {
 		return false;
 	}
+
+	@Override
+	public long getNumberNonZerosContained() {
+		long count = 0;
+		for(double v : _values) {
+			if(v != 0.0)
+				count++;
+		}
+		return count;
+	}
+
+	@Override
+	public double[] getMostCommonTuple(int[] counts, int nCol) {
+		int maxIndex = 0;
+		int maxCount = 0;
+		for(int i = 0; i < counts.length; i++) {
+			if(counts[i] > maxCount) {
+				maxCount = counts[i];
+				maxIndex = i;
+			}
+		}
+		final double[] tuple = new double[nCol];
+		boolean allZero = true;
+		for(int i = maxIndex * nCol, off = 0; i < (maxIndex + 1) * nCol && i < _values.length; i++, off++) {
+			final double v = _values[i];
+			if(v != 0) {
+				tuple[off] = _values[i];
+				allZero = false;
+			}
+		}
+
+		return allZero ? null : tuple;
+	}
+
+	@Override
+	public ADictionary subtractTuple(double[] tuple) {
+		double[] newValues = new double[_values.length];
+		for(int i = 0; i < _values.length; i++) {
+			newValues[i] = _values[i] - tuple[i % tuple.length];
+		}
+		return new Dictionary(newValues);
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
index 524bb32..a8559db 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/dictionary/QDictionary.java
@@ -314,6 +314,20 @@ public class QDictionary extends ADictionary {
 		}
 	}
 
+
+	@Override
+	public double[] colSum(int[] counts, int nCol){
+		throw new NotImplementedException("Not Implemented");
+		// final double[] res = new double[counts.length];
+		// int idx = 0;
+		// for(int k = 0; k< _values.length / counts.length; k++){
+		// 	final int cntk = counts[k];
+		// 	for(int j = 0; j< counts.length; j++){
+		// 		res[j] += _values[idx++] * cntk;
+		// 	}
+		// }
+		// return res;
+	}
 	@Override
 	public void colSum(double[] c, int[] counts, int[] colIndexes, boolean square) {
 		throw new NotImplementedException("Not Implemented");
@@ -398,12 +412,13 @@ public class QDictionary extends ADictionary {
 		}
 	}
 
-	public StringBuilder getString(StringBuilder sb, int colIndexes) {
+	public String getString( int colIndexes) {
+		StringBuilder sb = new StringBuilder();
 		for(int i = 0; i < size(); i++) {
 			sb.append(_values[i]);
 			sb.append((i) % (colIndexes) == colIndexes - 1 ? "\n" : " ");
 		}
-		return sb;
+		return sb.toString();
 	}
 
 	public Dictionary makeDoubleDictionary() {
@@ -470,4 +485,24 @@ public class QDictionary extends ADictionary {
 	public boolean isLossy() {
 		return false;
 	}
+
+	@Override
+	public long getNumberNonZerosContained(){
+		long count = 0;
+		for(double v : _values){
+			if(v != 0.0)
+				count++;
+		}
+		return count;
+	}
+
+	@Override
+	public double[] getMostCommonTuple(int[] counts, int nCol){
+		return null;
+	}
+
+	@Override
+	public ADictionary subtractTuple(double[] tuple){
+		throw new NotImplementedException();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
index 13b2007..f6c734c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibCompAgg.java
@@ -83,7 +83,6 @@ public class CLALibCompAgg {
 			else
 				aggregateUnaryNormalCompressedMatrixBlock(inputMatrix, outputMatrix, op, blen, indexesIn, inCP);
 		}
-
 		outputMatrix.recomputeNonZeros();
 
 		return outputMatrix;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
index b505c71..2779a68 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibLeftMultBy.java
@@ -35,10 +35,8 @@ import org.apache.sysds.runtime.compress.colgroup.AColGroup;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup.CompressionType;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
 import org.apache.sysds.runtime.compress.colgroup.pre.IPreAggregate;
-import org.apache.sysds.runtime.functionobjects.SwapIndex;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
-import org.apache.sysds.runtime.matrix.operators.ReorgOperator;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class CLALibLeftMultBy {
@@ -46,16 +44,18 @@ public class CLALibLeftMultBy {
 
 	public static MatrixBlock leftMultByMatrixTransposed(CompressedMatrixBlock m1, MatrixBlock m2, MatrixBlock ret,
 		int k) {
+		if(m2.isEmpty())
+			return ret;
 		MatrixBlock transposed = new MatrixBlock(m2.getNumColumns(), m2.getNumRows(), false);
 		LibMatrixReorg.transpose(m2, transposed);
 		ret = leftMultByMatrix(m1, transposed, ret, k);
 		ret.recomputeNonZeros();
 		return ret;
-		// return LibMatrixReorg.transpose(ret, new MatrixBlock(ret.getNumColumns(), ret.getNumRows(), false));
 	}
 
 	public static MatrixBlock leftMultByMatrixTransposed(CompressedMatrixBlock m1, CompressedMatrixBlock m2,
 		MatrixBlock ret, int k) {
+
 		prepareReturnMatrix(m1, m2, ret, true);
 		leftMultByCompressedTransposedMatrix(m1.getColGroups(), m2, ret, k, m1.getNumColumns(), m1.getMaxNumValues(),
 			m1.isOverlapping());
@@ -66,24 +66,14 @@ public class CLALibLeftMultBy {
 
 	public static MatrixBlock leftMultByMatrix(CompressedMatrixBlock m1, MatrixBlock m2, MatrixBlock ret, int k) {
 		prepareReturnMatrix(m1, m2, ret, false);
-		ret = leftMultByMatrix(m1.getColGroups(), m2, ret, false, m1.getNumColumns(), m1.isOverlapping(), k,
-			m1.getMaxNumValues());
+		if(m2.isEmpty())
+			return ret;
+		ret = leftMultByMatrix(m1.getColGroups(), m2, ret, k, m1.getNumColumns(), m1.getMaxNumValues(),
+			m1.isOverlapping());
 		ret.recomputeNonZeros();
 		return ret;
 	}
 
-	private static MatrixBlock leftMultByMatrix(List<AColGroup> groups, MatrixBlock that, MatrixBlock ret,
-		boolean doTranspose, int numCols, boolean overlapping, int k, Pair<Integer, int[]> v) {
-
-		if(doTranspose) {
-			ReorgOperator r_op = new ReorgOperator(SwapIndex.getSwapIndexFnObject(), k);
-			that = that.reorgOperations(r_op, new MatrixBlock(), 0, 0, 0);
-		}
-
-		return leftMultByMatrix(groups, that, ret, k, numCols, v, overlapping);
-
-	}
-
 	private static MatrixBlock prepareReturnMatrix(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret,
 		boolean doTranspose) {
 		int numRowsOutput = doTranspose ? m2.getNumColumns() : m2.getNumRows();
@@ -95,9 +85,9 @@ public class CLALibLeftMultBy {
 		return ret;
 	}
 
-	public static void leftMultByTransposeSelf(CompressedMatrixBlock mb, MatrixBlock result, int k) {
+	// public static void leftMultByTransposeSelf(CompressedMatrixBlock mb, MatrixBlock result, int k) {
 
-	}
+	// }
 
 	public static void leftMultByTransposeSelf(List<AColGroup> groups, MatrixBlock result, int k, int numColumns,
 		Pair<Integer, int[]> v, boolean overlapping) {
@@ -109,11 +99,18 @@ public class CLALibLeftMultBy {
 			return;
 		}
 
-		if(k <= 1 || overlapping) {
-			if(overlapping)
-				LOG.warn(
-					"Inefficient TSMM with overlapping matrix Could be implemented multi-threaded but is not yet.");
+		if(overlapping) {
+			LOG.warn("Inefficient TSMM with overlapping matrix could be implemented multi-threaded but is not yet.");
 			leftMultByCompressedTransposedMatrix(groups, groups, result);
+
+			result.recomputeNonZeros();
+			return;
+		}
+
+		if(k <= 1) {
+			for(int i = 0; i < groups.size(); i++)
+				leftMultByCompressedTransposedMatrix(groups.get(i), groups, result.getDenseBlockValues(),
+					result.getNumRows(), result.getNumColumns(), i, groups.size());
 		}
 		else {
 			try {
@@ -132,8 +129,8 @@ public class CLALibLeftMultBy {
 				throw new DMLRuntimeException(e);
 			}
 		}
-		copyToUpperTriangle(result.getDenseBlockValues(), numColumns);
 
+		copyToUpperTriangle(result.getDenseBlockValues(), numColumns);
 		result.recomputeNonZeros();
 	}
 
@@ -162,9 +159,10 @@ public class CLALibLeftMultBy {
 		List<AColGroup> thatCGs = that.getColGroups();
 		Pair<Integer, int[]> thatV = that.getMaxNumValues();
 
-		if(k <= 1 || overlapping || that.isOverlapping()){
+		if(k <= 1 || overlapping || that.isOverlapping()) {
 			if(overlapping || that.isOverlapping())
-				LOG.warn("Inefficient Compressed multiplication with overlapping matrix could be implemented multi-threaded but is not yet.");
+				LOG.warn(
+					"Inefficient Compressed multiplication with overlapping matrix could be implemented multi-threaded but is not yet.");
 			leftMultByCompressedTransposedMatrix(colGroups, thatCGs, ret);
 		}
 		else
@@ -220,6 +218,7 @@ public class CLALibLeftMultBy {
 
 			}
 			catch(Exception e) {
+				e.printStackTrace();
 				throw new DMLRuntimeException(e);
 			}
 			return null;
@@ -229,7 +228,6 @@ public class CLALibLeftMultBy {
 	private static void leftMultByCompressedTransposedMatrix(List<AColGroup> thisCG, List<AColGroup> thatCG,
 		MatrixBlock ret) {
 		double[] c = ret.getDenseBlockValues();
-
 		for(AColGroup lhs : thatCG) {
 			leftMultByCompressedTransposedMatrix(lhs, thisCG, c, ret.getNumRows(), ret.getNumColumns(), 0,
 				thisCG.size());
@@ -256,16 +254,9 @@ public class CLALibLeftMultBy {
 		ret.allocateDenseBlock();
 		double[] retV = ret.getDenseBlockValues();
 
-		// for(int b = 0; b < db.numBlocks(); b++) {
-		// int blockSize = db.blockSize(b);
-		// blockU = Math.min(blockL + blockSize, ret.getNumRows());
-
-		if(k == 1) {
-			LOG.trace("Single treaded left matrix multiplication");
-			for(int j = 0; j < colGroups.size(); j++) {
+		if(k == 1)
+			for(int j = 0; j < colGroups.size(); j++)
 				colGroups.get(j).leftMultByMatrix(that, retV, numColumns);
-			}
-		}
 		else {
 			try {
 				ExecutorService pool = CommonThreadPool.get(k);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
index 5417470..5b0a105 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibRightMultBy.java
@@ -20,7 +20,9 @@
 package org.apache.sysds.runtime.compress.lib;
 
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -32,7 +34,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
-import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupEmpty;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
@@ -55,34 +57,18 @@ public class CLALibRightMultBy {
 		that = that instanceof CompressedMatrixBlock ? ((CompressedMatrixBlock) that).decompress(k) : that;
 
 		MatrixBlock m = rightMultByMatrixOverlapping(colGroups, that, ret, k, v);
+
 		if(m instanceof CompressedMatrixBlock)
 			if(allowOverlappingOutput(colGroups, allowOverlap))
 				return m;
-			else {
-				CompressedMatrixBlock outBlock = (CompressedMatrixBlock) m;
-				ColGroupUncompressed uccg = findUncompressedColGroup(outBlock.getColGroups());
-				if(uccg == null)
-					return outBlock.decompress(k);
-				else{
-					MatrixBlock outputBlock = uccg.getData();
-					outputBlock.sparseToDense();
-					return outBlock.decompress(outputBlock, k);
-				}
-			}
+			else
+				return ((CompressedMatrixBlock) m).decompress(k);
 		else
 			return m;
 	}
 
-	private static ColGroupUncompressed findUncompressedColGroup(List<AColGroup> colGroups){
-		for(AColGroup g: colGroups){
-			if(g instanceof ColGroupUncompressed)
-				return (ColGroupUncompressed) g;
-		}
-		return null;
-	}
-
 	private static boolean allowOverlappingOutput(List<AColGroup> colGroups, boolean allowOverlap) {
-		
+
 		if(!allowOverlap) {
 			LOG.debug("Not Overlapping because it is not allowed");
 			return false;
@@ -91,18 +77,18 @@ public class CLALibRightMultBy {
 			return true;
 		// int distinctCount = 0;
 		// for(AColGroup g : colGroups) {
-		// 	if(g instanceof ColGroupCompressed)
-		// 		distinctCount += ((ColGroupCompressed) g).getNumValues();
-		// 	else {
-		// 		LOG.debug("Not Overlapping because there is an un-compressed column group");
-		// 		return false;
-		// 	}
+		// if(g instanceof ColGroupCompressed)
+		// distinctCount += ((ColGroupCompressed) g).getNumValues();
+		// else {
+		// LOG.debug("Not Overlapping because there is an un-compressed column group");
+		// return false;
+		// }
 		// }
 		// final int threshold = colGroups.get(0).getNumRows() / 2;
 		// boolean allow = distinctCount <= threshold;
 		// if(LOG.isDebugEnabled() && !allow)
-		// 	LOG.debug("Not Allowing Overlap because of number of distinct items in compression: " + distinctCount
-		// 		+ " is greater than threshold: " + threshold);
+		// LOG.debug("Not Allowing Overlap because of number of distinct items in compression: " + distinctCount
+		// + " is greater than threshold: " + threshold);
 		// return allow;
 
 	}
@@ -122,9 +108,15 @@ public class CLALibRightMultBy {
 		CompressedMatrixBlock ret, int k, Pair<Integer, int[]> v) {
 
 		List<AColGroup> retCg = new ArrayList<>();
+		boolean containsNull = false;
 		if(k == 1) {
-			for(AColGroup g : colGroups)
-				retCg.add(g.rightMultByMatrix(that));
+			for(AColGroup g : colGroups) {
+				AColGroup retG = g.rightMultByMatrix(that);
+				if(retG != null)
+					retCg.add(retG);
+				else
+					containsNull = true;
+			}
 		}
 		else {
 			ExecutorService pool = CommonThreadPool.get(k);
@@ -132,10 +124,12 @@ public class CLALibRightMultBy {
 				List<Callable<AColGroup>> tasks = new ArrayList<>(colGroups.size());
 				for(AColGroup g : colGroups)
 					tasks.add(new RightMatrixMultTask(g, that));
-				for(Future<AColGroup> fg : pool.invokeAll(tasks)){
+				for(Future<AColGroup> fg : pool.invokeAll(tasks)) {
 					AColGroup g = fg.get();
 					if(g != null)
 						retCg.add(g);
+					else
+						containsNull = true;
 				}
 			}
 			catch(InterruptedException | ExecutionException e) {
@@ -145,9 +139,32 @@ public class CLALibRightMultBy {
 		ret.allocateColGroupList(retCg);
 		if(retCg.size() > 1)
 			ret.setOverlapping(true);
+
+		if(containsNull) {
+			ColGroupEmpty cge = findEmptyColumnsAndMakeEmptyColGroup(retCg, ret.getNumColumns());
+			if(cge != null)
+				retCg.add(cge);
+		}
 		return ret;
 	}
 
+	private static ColGroupEmpty findEmptyColumnsAndMakeEmptyColGroup(List<AColGroup> colGroups, int nCols) {
+		Set<Integer> emptyColumns = new HashSet<Integer>(nCols);
+		for(int i = 0; i < nCols; i++)
+			emptyColumns.add(i);
+
+		for(AColGroup g : colGroups)
+			for(int c : g.getColIndices())
+				emptyColumns.remove(c);
+
+		if(emptyColumns.size() != 0) {
+			int[] emptyColumnsFinal = emptyColumns.stream().mapToInt(Integer::intValue).toArray();
+			return new ColGroupEmpty(emptyColumnsFinal, colGroups.get(0).getNumRows());
+		}
+		else
+			return null;
+	}
+
 	private static class RightMatrixMultTask implements Callable<AColGroup> {
 		private final AColGroup _colGroup;
 		private final MatrixBlock _b;
@@ -163,7 +180,6 @@ public class CLALibRightMultBy {
 				return _colGroup.rightMultByMatrix(_b);
 			}
 			catch(Exception e) {
-				LOG.error(e);
 				throw new DMLRuntimeException(e);
 			}
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
index 11e0b8b..e3ef846 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/lib/CLALibScalar.java
@@ -33,6 +33,7 @@ import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 import org.apache.sysds.runtime.compress.colgroup.AColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupCompressed;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupConst;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupOLE;
 import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
@@ -180,7 +181,7 @@ public class CLALibScalar {
 				tasks.add(new ScalarTask(uc, sop));
 			}
 			else {
-				int nv = ((ColGroupValue) grp).getNumValues() * grp.getColIndices().length;
+				int nv = ((ColGroupCompressed) grp).getNumValues() * grp.getColIndices().length;
 				if(nv < MINIMUM_PARALLEL_SIZE && !(grp instanceof ColGroupOLE)) {
 					small.add(grp);
 				}
diff --git a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixValue.java b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixValue.java
index c9838d5..98a5536 100644
--- a/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixValue.java
+++ b/src/main/java/org/apache/sysds/runtime/matrix/data/MatrixValue.java
@@ -97,7 +97,7 @@ public abstract class MatrixValue implements WritableComparable
 	public abstract void reset(int rl, int cl, double v);
 
 	/**
-	 * Copy this MatrixValue into that MatrixValue.
+	 * Copy that MatrixValue into this MatrixValue.
 	 * 
 	 * If the MatrixValue is a MatrixBlock evaluate the sparsity of the original matrix,
 	 * and copy into either a sparse or a dense matrix.
@@ -107,7 +107,7 @@ public abstract class MatrixValue implements WritableComparable
 	public abstract void copy(MatrixValue that);
 
 	/**
-	 * Copy this MatrixValue into that MatrixValue. But select sparse destination block depending on boolean parameter.
+	 * Copy that MatrixValue into this MatrixValue. But select sparse destination block depending on boolean parameter.
 	 * 
 	 * @param that object to copy the values into.
 	 * @param sp boolean specifying if output should be forced sparse or dense. (only applicable if the 'that' is a MatrixBlock)
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
index a93edd2..1167512 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedTestBase.java
@@ -85,7 +85,7 @@ public abstract class CompressedTestBase extends TestBase {
 
 	protected static OverLapping[] overLapping = new OverLapping[] {
 		// OverLapping.COL,
-		OverLapping.PLUS, OverLapping.MATRIX, OverLapping.NONE,
+		OverLapping.PLUS, OverLapping.MATRIX, OverLapping.NONE, OverLapping.APPEND_EMPTY, OverLapping.APPEND_CONST,
 		// OverLapping.MATRIX_PLUS,
 		// OverLapping.SQUASH,
 		// OverLapping.MATRIX_MULT_NEGATIVE
@@ -217,11 +217,23 @@ public abstract class CompressedTestBase extends TestBase {
 					lossyTolerance = lossyTolerance * 160;
 					cols = 2;
 					break;
+				case APPEND_EMPTY:
+					tmp = new MatrixBlock(rows, 1, 0);
+					break;
+				case APPEND_CONST:
+					tmp = new MatrixBlock(rows, 1, 0)
+						.scalarOperations(new RightScalarOperator(Plus.getPlusFnObject(), 1), new MatrixBlock());
+					break;
 				default:
 					break;
 			}
 			if(cmb instanceof CompressedMatrixBlock) {
-				if(tmp != null) {
+				if(tmp != null && ov == OverLapping.APPEND_EMPTY || ov == OverLapping.APPEND_CONST) {
+					mb = mb.append(tmp, new MatrixBlock());
+					cmb = cmb.append(tmp, new MatrixBlock());
+					cols += tmp.getNumColumns();
+				}
+				else if(tmp != null) {
 					// Make Operator
 					AggregateBinaryOperator abop = InstructionUtils.getMatMultOperator(_k);
 
@@ -337,10 +349,10 @@ public abstract class CompressedTestBase extends TestBase {
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector1 = DataConverter
-				.convertToMatrixBlock(TestUtils.generateTestMatrix(cols, 1, 0.9, 1.1, 1.0, 3));
+				.convertToMatrixBlock(TestUtils.generateTestMatrix(cols, 1, 0.9, 1.5, 1.0, 3));
 
 			MatrixBlock vector2 = (ctype == ChainType.XtwXv) ? DataConverter
-				.convertToMatrixBlock(TestUtils.generateTestMatrix(rows, 1, 0.9, 1.1, 1.0, 3)) : null;
+				.convertToMatrixBlock(TestUtils.generateTestMatrix(rows, 1, 0.9, 1.5, 1.0, 3)) : null;
 
 			// matrix-vector uncompressed
 			MatrixBlock ret1 = mb.chainMatrixMultOperations(vector1, vector2, new MatrixBlock(), ctype, _k);
@@ -681,7 +693,6 @@ public abstract class CompressedTestBase extends TestBase {
 		try {
 			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
-			// ChainType ctype = ChainType.XtwXv;
 			for(MMTSJType mType : new MMTSJType[] {MMTSJType.LEFT,
 				// MMTSJType.RIGHT
 			}) {
@@ -691,8 +702,6 @@ public abstract class CompressedTestBase extends TestBase {
 				// matrix-vector compressed
 				MatrixBlock ret2 = cmb.transposeSelfMatrixMultOperations(new MatrixBlock(), mType, _k);
 
-				// LOG.error(ret1);
-				// LOG.error(ret2);
 				// compare result with input
 				compareResultMatrices(ret1, ret2, 100);
 			}
diff --git a/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java b/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
index 0fcff40..c936c1b 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/TestConstants.java
@@ -57,7 +57,7 @@ public class TestConstants {
 	}
 
 	public enum OverLapping {
-		COL, MATRIX, NONE, MATRIX_PLUS, MATRIX_MULT_NEGATIVE, SQUASH, PLUS;
+		COL, MATRIX, NONE, MATRIX_PLUS, MATRIX_MULT_NEGATIVE, SQUASH, PLUS, APPEND_EMPTY, APPEND_CONST;
 
 		public static boolean effectOnOutput(OverLapping opcode) {
 			switch(opcode) {