You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2020/04/25 19:58:37 UTC

[systemml] branch master updated: [SYSTEMDS-273] Refactor Matrix Compression Package

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7e2a070  [SYSTEMDS-273] Refactor Matrix Compression Package
7e2a070 is described below

commit 7e2a0704b47f99d6c041ef83724962c33dda5a44
Author: Sebastian <ba...@tugraz.at>
AuthorDate: Sat Apr 25 20:26:02 2020 +0200

    [SYSTEMDS-273] Refactor Matrix Compression Package
    
    Major refactoring of compressed Matrix Block to "simplify"
    responsibilities. The main goal of the refactoring was to reduce the
    amount of bindings to CompressedMatrixBlock from the different types of
    ColGroups, to enable easy addition of new types of groups. (without
    reducing performance!)
    
    A list of new groups and other tasks associated with this is added to
    /docs/tasks.txt .
    
    Furthermore, the tests added for the compression work based on an
    estimation tolerance, allowing the estimations inside the system to be
    off by a set factor. In the tests, this has to be expanded upon to
    enable better precision in how the estimations are calculated.
    
    Bellow is an extract of the different changes:
    
    - Separated sub-parts of compression into different packages.
    - Array memory footprint worst case calculations.
    - Moved Compressed Size Estimation Calculation to specific ColGroups
    
    - Extensive testing of size Estimation of ColGroups and compression
      - Jol Memory Estimate tests for compression blocks
      - Using worst case Jol Estimate JVM using uncompressed 64-bit JVM
      - Ideal input generator for testing Col groups compression.
    
    - Factory pattern added for selected constructors
      - ColGroups
        - NameChange from ColGroupCompressor to ColGroupFactory
      - CompressedMatrixBlock
    
    - Enable the parallel execution of the ColGrouping (previously disabled)
    
    - Settings File added for Compression to enable selection of specific
      compression types. This can be extended with settings loaded from
      DMLConfig
      - This is constructed using a builder pattern such that invocation is
      private.
    
    - Added abstract compressed block for overwriting default MatrixBlock
    
    - Add Test Libs to pom.xml
      - Memory estimator framework JOL from OpenJDK to measure object sizes
    
    Closes #872.
---
 docs/Tasks.txt                                     |   20 +
 pom.xml                                            |    9 +
 ...orFactory.java => DMLCompressionException.java} |   25 +-
 .../compress/AbstractCompressedMatrixBlock.java    |  512 +++++++++
 .../sysds/runtime/compress/BitmapEncoder.java      |   65 +-
 .../runtime/compress/CompressedMatrixBlock.java    | 1089 ++------------------
 .../compress/CompressedMatrixBlockFactory.java     |  258 +++++
 .../runtime/compress/CompressionSettings.java      |   83 ++
 .../compress/CompressionSettingsBuilder.java       |   93 ++
 .../runtime/compress/CompressionStatistics.java    |  118 ++-
 .../runtime/compress/ReaderColumnSelection.java    |   14 +-
 .../compress/ReaderColumnSelectionDense.java       |   28 +-
 .../compress/ReaderColumnSelectionDenseSample.java |   31 +-
 .../compress/ReaderColumnSelectionSparse.java      |   69 +-
 .../sysds/runtime/compress/UncompressedBitmap.java |   67 +-
 .../cocode/ColumnGroupPartitionerBinPacking.java   |    1 -
 .../cocode/ColumnGroupPartitionerStatic.java       |    1 -
 .../runtime/compress/cocode/PlanningCoCoder.java   |   42 +-
 .../compress/cocode/PlanningCoCodingGroup.java     |    6 +-
 .../runtime/compress/{ => colgroup}/ColGroup.java  |  102 +-
 .../ColGroupConverter.java}                        |   24 +-
 .../compress/{ => colgroup}/ColGroupDDC.java       |   57 +-
 .../compress/{ => colgroup}/ColGroupDDC1.java      |   49 +-
 .../compress/{ => colgroup}/ColGroupDDC2.java      |   50 +-
 .../runtime/compress/colgroup/ColGroupFactory.java |  282 +++++
 .../runtime/compress/colgroup/ColGroupIO.java      |  121 +++
 .../compress/{ => colgroup}/ColGroupOLE.java       |   60 +-
 .../compress/{ => colgroup}/ColGroupOffset.java    |   85 +-
 .../compress/{ => colgroup}/ColGroupRLE.java       |   39 +-
 .../runtime/compress/colgroup/ColGroupSizes.java   |  149 +++
 .../{ => colgroup}/ColGroupUncompressed.java       |   58 +-
 .../compress/{ => colgroup}/ColGroupValue.java     |   79 +-
 .../compress/{ => colgroup}/DenseRowIterator.java  |   12 +-
 .../compress/{ => colgroup}/RowIterator.java       |   10 +-
 .../compress/{ => colgroup}/SparseRowIterator.java |    9 +-
 .../estim/CompressedSizeEstimationFactors.java     |  147 +++
 .../compress/estim/CompressedSizeEstimator.java    |  288 ++++--
 .../estim/CompressedSizeEstimatorExact.java        |   23 +-
 .../estim/CompressedSizeEstimatorFactory.java      |   11 +-
 .../estim/CompressedSizeEstimatorSample.java       |  468 +--------
 .../runtime/compress/estim/CompressedSizeInfo.java |   61 +-
 .../compress/estim/CompressedSizeInfoColGroup.java |  129 +++
 .../sample/FrequencyCount.java}                    |   51 +-
 .../estim/sample/GuaranteedErrorEstimator.java     |   66 ++
 .../compress/estim/sample/HassAndStokes.java       |  239 +++++
 .../compress/estim/sample/ShlosserEstimator.java   |   51 +
 .../estim/sample/ShlosserJackknifeEstimator.java   |   78 ++
 .../estim/sample/SmoothedJackknifeEstimator.java   |  108 ++
 .../compress/utils/ColumnGroupIterator.java        |    8 +-
 .../CustomHashMap.java}                            |   19 +-
 .../sysds/runtime/compress/utils/DblArray.java     |   22 +-
 .../compress/utils/DblArrayIntListHashMap.java     |   44 +-
 .../compress/utils/DoubleIntListHashMap.java       |   43 +-
 .../sysds/runtime/compress/utils/IntArrayList.java |   39 +-
 .../runtime/compress/utils/LinearAlgebraUtils.java |  177 ++--
 .../runtime/compress/utils/Py4jConverterUtils.java |   70 +-
 .../instructions/cp/CompressionCPInstruction.java  |   15 +-
 .../instructions/cp/EvalNaryCPInstruction.java     |    4 +-
 .../spark/CompressionSPInstruction.java            |    4 +-
 .../org/apache/sysds/utils/MemoryEstimates.java    |   86 ++
 src/test/java/org/apache/sysds/test/TestUtils.java |   14 +-
 .../test/applications/ApplicationTestBase.java     |  128 ---
 .../compress/AbstractCompressedUnaryTests.java     |  165 +++
 .../component/compress/CompressedMatrixTest.java   |  309 +++---
 .../component/compress/CompressedTestBase.java     |  188 ++--
 .../component/compress/CompressedVectorTest.java   |   67 +-
 .../compress/CompressibleInputGenerator.java       |  173 ++++
 .../compress/ParCompressedMatrixTest.java          |  147 +--
 .../sysds/test/component/compress/TestBase.java    |  129 +++
 .../{ => component/compress}/TestConstants.java    |   47 +-
 .../compress/colgroup/JolEstimateDDCTest.java      |   97 ++
 .../compress/colgroup/JolEstimateOLETest.java      |  138 +++
 .../compress/colgroup/JolEstimateRLETest.java      |  177 ++++
 .../compress/colgroup/JolEstimateTest.java         |  174 ++++
 .../compress/colgroup/JolEstimateTestEmpty.java    |  125 +++
 .../colgroup/JolEstimateUncompressedTest.java      |   80 ++
 .../test/component/misc/MemoryEstimateTest.java    |   98 ++
 77 files changed, 5406 insertions(+), 2818 deletions(-)

diff --git a/docs/Tasks.txt b/docs/Tasks.txt
index e63544c..3a7abe7 100644
--- a/docs/Tasks.txt
+++ b/docs/Tasks.txt
@@ -212,6 +212,19 @@ SYSTEMDS-260 Misc Tools
 SYSTEMDS-270 Compressed Matrix Blocks
  * 271 Reintroduce compressed matrix blocks from SystemML             OK
  * 272 Simplify and speedup compression tests                         OK
+ * 273 Refactor compressed Matrix Block to simplify responsibilities  OK
+ * 273a Redesign allocation of ColGroups in ColGroupFactory
+ * 274 Make the DDC Compression dictionary share correctly
+ * 275 Include compressionSettings in DMLConfiguration
+ * 276 Allow Uncompressed Columns to be in sparse formats
+ * 277 Sampling based estimators fix
+ * 278 Compression-CoCode algorithm optimization
+ * 278a Return ColGroups estimated compression ratio to Factory
+ * 279 Add missing standard lossless compression techniques
+ * 279a ColGroup FOR (Frame of reference) encoding
+ * 279b ColGroup DEL (Delta) encoding
+ * MINOR Reduce memory usage for compression statistics.
+ * MINOR Make ContainsAZero() method in UncompressedBitMap
 
 SYSTEMDS-280 New Rewrites
  * 281 New rewrites sum(rmEmpty), sum(table)                          OK
@@ -268,5 +281,12 @@ SYSTEMDS-360 Privacy/Data Exchange Constraints
  * 364 Error handling violated privacy constraints
  * 365 Extended privacy/data exchange constraints
 
+SYSTEMDS-370 Lossy Compression Blocks
+ * 361 ColGroup Quantization
+ * 362 ColGroup Base Data change (from Double to ??)
+
+SYSTEMDS-380 Memory Footprint
+ * 371 Matrix Block Memory footprint update
+
 Others:
  * Break append instruction to cbind and rbind 
diff --git a/pom.xml b/pom.xml
index d1ef1e5..4fad885 100644
--- a/pom.xml
+++ b/pom.xml
@@ -262,6 +262,8 @@
 					<forkCount>1C</forkCount>
 					<argLine>-Xms4g -Xmx4g</argLine>
 					<reuseForks>false</reuseForks>
+					<reportFormat>plain</reportFormat>
+					<trimStackTrace>true</trimStackTrace>
 				</configuration>
 			</plugin>
 
@@ -937,6 +939,13 @@
 		</dependency>
 
 		<dependency>
+			<groupId>org.openjdk.jol</groupId>
+			<artifactId>jol-core</artifactId>
+			<version>0.10</version>
+			<scope>test</scope>
+		</dependency>
+		
+		<dependency>
 			<!--Used for annotations in tests to execute tests in thread safe manner-->
 			<groupId>com.github.stephenc.jcip</groupId>
 			<artifactId>jcip-annotations</artifactId>
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/DMLCompressionException.java
similarity index 61%
copy from src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
copy to src/main/java/org/apache/sysds/runtime/DMLCompressionException.java
index 1e4b1d1..ac8b2b0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/DMLCompressionException.java
@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
+ * 
  *   http://www.apache.org/licenses/LICENSE-2.0
- *
+ * 
  * Unless required by applicable law or agreed to in writing,
  * software distributed under the License is distributed on an
  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -17,15 +17,20 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress.estim;
+package org.apache.sysds.runtime;
 
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+public class DMLCompressionException extends DMLRuntimeException{
+	private static final long serialVersionUID = 1L;
 
-public class CompressedSizeEstimatorFactory {
-	public static final boolean EXTRACT_SAMPLE_ONCE = true;
+	public DMLCompressionException(String string) {
+		super(string);
+	}
+	
+	public DMLCompressionException(Exception e) {
+		super(e);
+	}
 
-	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, int numRows, long seed, double sampling_ratio) {
-		return (sampling_ratio == 1.0) ? new CompressedSizeEstimatorExact(data) : new CompressedSizeEstimatorSample(
-			data, (int) Math.ceil(numRows * sampling_ratio), seed);
+	public DMLCompressionException(String string, Exception ex){
+		super(string,ex);
 	}
-}
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/sysds/runtime/compress/AbstractCompressedMatrixBlock.java b/src/main/java/org/apache/sysds/runtime/compress/AbstractCompressedMatrixBlock.java
new file mode 100644
index 0000000..bf86ede
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/AbstractCompressedMatrixBlock.java
@@ -0,0 +1,512 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.math3.random.Well1024a;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
+import org.apache.sysds.runtime.controlprogram.caching.CacheBlock;
+import org.apache.sysds.runtime.controlprogram.caching.MatrixObject.UpdateType;
+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.MatrixBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
+import org.apache.sysds.runtime.matrix.data.MatrixValue;
+import org.apache.sysds.runtime.matrix.data.RandomMatrixGenerator;
+import org.apache.sysds.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysds.runtime.matrix.operators.AggregateTernaryOperator;
+import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
+import org.apache.sysds.runtime.matrix.operators.CMOperator;
+import org.apache.sysds.runtime.matrix.operators.COVOperator;
+import org.apache.sysds.runtime.matrix.operators.Operator;
+import org.apache.sysds.runtime.matrix.operators.QuaternaryOperator;
+import org.apache.sysds.runtime.matrix.operators.ReorgOperator;
+import org.apache.sysds.runtime.matrix.operators.TernaryOperator;
+import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
+import org.apache.sysds.runtime.util.IndexRange;
+import org.apache.sysds.runtime.util.SortUtils;
+
+public abstract class AbstractCompressedMatrixBlock extends MatrixBlock {
+
+	private static final Log LOG = LogFactory.getLog(AbstractCompressedMatrixBlock.class.getName());
+
+	protected List<ColGroup> _colGroups;
+
+	/**
+	 * Constructor for building an empty Compressed Matrix block object.
+	 */
+	public AbstractCompressedMatrixBlock() {
+		super();
+	}
+
+	/**
+	 * Main constructor for building a block from scratch.
+	 * 
+	 * @param rl     number of rows in the block
+	 * @param cl     number of columns
+	 * @param sparse true if the UNCOMPRESSED representation of the block should be sparse
+	 */
+	public AbstractCompressedMatrixBlock(int rl, int cl, boolean sparse) {
+		super(rl, cl, sparse);
+	}
+
+	/**
+	 * "Copy" constructor to populate this compressed block with the uncompressed contents of a conventional block. Does
+	 * <b>not</b> compress the block. Only creates a shallow copy, and only does deep copy on compression.
+	 * 
+	 * @param that matrix block
+	 */
+	public AbstractCompressedMatrixBlock(MatrixBlock that) {
+		super(that.getNumRows(), that.getNumColumns(), that.isInSparseFormat());
+
+		// shallow copy (deep copy on compression, prevents unnecessary copy)
+		if(isInSparseFormat())
+			sparseBlock = that.getSparseBlock();
+		else
+			denseBlock = that.getDenseBlock();
+		nonZeros = that.getNonZeros();
+	}
+
+	public abstract boolean isCompressed();
+
+	public abstract MatrixBlock decompress();
+
+	@Override
+	public boolean isEmptyBlock(boolean safe) {
+		if(!isCompressed())
+			return super.isEmptyBlock(safe);
+		return(_colGroups == null || getNonZeros() == 0);
+	}
+
+	public static long estimateOriginalSizeInMemory(int nrows, int ncols, double sparsity) {
+		// Estimate the original Size.
+		// Unlike the other Estimation this one takes the original estimation
+		// but also includes the small overhead of different arrays.
+		// TODO: Make the original Memory estimates better for MatrixBlocks.
+		long size = MatrixBlock.estimateSizeInMemory(nrows, ncols, sparsity);
+
+		size += 4; // rlen
+		size += 4; // clen
+		size += 1; // a single boolean fills 8 bits !
+		size += 8; // NonZeros.
+
+		size += 8; // Object reference DenseBlock
+		size += 8; // Object reference Sparse Block
+
+		size += 4; // estimated NNzs Per Row
+
+		if(size % 8 != 0)
+			size += 8 - size % 8; // Add padding
+
+		return size;
+	}
+
+	//////////////////////////////////////////
+	// Graceful fallback to uncompressed linear algebra
+
+	@Override
+	public MatrixBlock unaryOperations(UnaryOperator op, MatrixValue result) {
+		printDecompressWarning("unaryOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.unaryOperations(op, result);
+	}
+
+	@Override
+	public MatrixBlock binaryOperations(BinaryOperator op, MatrixValue thatValue, MatrixValue result) {
+		printDecompressWarning("binaryOperations", (MatrixBlock) thatValue);
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(thatValue);
+		return left.binaryOperations(op, right, result);
+	}
+
+	@Override
+	public void binaryOperationsInPlace(BinaryOperator op, MatrixValue thatValue) {
+		printDecompressWarning("binaryOperationsInPlace", (MatrixBlock) thatValue);
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(thatValue);
+		left.binaryOperationsInPlace(op, right);
+	}
+
+	@Override
+	public void incrementalAggregate(AggregateOperator aggOp, MatrixValue correction, MatrixValue newWithCorrection,
+		boolean deep) {
+		throw new DMLRuntimeException("CompressedMatrixBlock: incrementalAggregate not supported.");
+	}
+
+	@Override
+	public void incrementalAggregate(AggregateOperator aggOp, MatrixValue newWithCorrection) {
+		throw new DMLRuntimeException("CompressedMatrixBlock: incrementalAggregate not supported.");
+	}
+
+	@Override
+	public MatrixBlock reorgOperations(ReorgOperator op, MatrixValue ret, int startRow, int startColumn, int length) {
+		printDecompressWarning("reorgOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.reorgOperations(op, ret, startRow, startColumn, length);
+	}
+
+	@Override
+	public MatrixBlock append(MatrixBlock that, MatrixBlock ret, boolean cbind) {
+		if(cbind) // use supported operation
+			return append(that, ret);
+		printDecompressWarning("append-rbind", that);
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(that);
+		return left.append(right, ret, cbind);
+	}
+
+	@Override
+	public void append(MatrixValue v2, ArrayList<IndexedMatrixValue> outlist, int blen, boolean cbind, boolean m2IsLast,
+		int nextNCol) {
+		printDecompressWarning("append", (MatrixBlock) v2);
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(v2);
+		left.append(right, outlist, blen, cbind, m2IsLast, nextNCol);
+	}
+
+	@Override
+	public void permutationMatrixMultOperations(MatrixValue m2Val, MatrixValue out1Val, MatrixValue out2Val) {
+		permutationMatrixMultOperations(m2Val, out1Val, out2Val, 1);
+	}
+
+	@Override
+	public void permutationMatrixMultOperations(MatrixValue m2Val, MatrixValue out1Val, MatrixValue out2Val, int k) {
+		printDecompressWarning("permutationMatrixMultOperations", (MatrixBlock) m2Val);
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(m2Val);
+		left.permutationMatrixMultOperations(right, out1Val, out2Val, k);
+	}
+
+	@Override
+	public MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, int rl, int ru, int cl, int cu, MatrixBlock ret,
+		UpdateType update) {
+		printDecompressWarning("leftIndexingOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(rhsMatrix);
+		return left.leftIndexingOperations(right, rl, ru, cl, cu, ret, update);
+	}
+
+	@Override
+	public MatrixBlock leftIndexingOperations(ScalarObject scalar, int rl, int cl, MatrixBlock ret, UpdateType update) {
+		printDecompressWarning("leftIndexingOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.leftIndexingOperations(scalar, rl, cl, ret, update);
+	}
+
+	@Override
+	public MatrixBlock slice(int rl, int ru, int cl, int cu, CacheBlock ret) {
+		printDecompressWarning("slice");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.slice(rl, ru, cl, cu, ret);
+	}
+
+	@Override
+	public void slice(ArrayList<IndexedMatrixValue> outlist, IndexRange range, int rowCut, int colCut, int blen,
+		int boundaryRlen, int boundaryClen) {
+		printDecompressWarning("slice");
+		try {
+			MatrixBlock tmp = isCompressed() ? decompress() : this;
+			tmp.slice(outlist, range, rowCut, colCut, blen, boundaryRlen, boundaryClen);
+		}
+		catch(DMLRuntimeException ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+
+	@Override
+	public MatrixBlock zeroOutOperations(MatrixValue result, IndexRange range, boolean complementary) {
+		printDecompressWarning("zeroOutOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.zeroOutOperations(result, range, complementary);
+	}
+
+	@Override
+	public CM_COV_Object cmOperations(CMOperator op) {
+		printDecompressWarning("cmOperations");
+		if(!isCompressed() || isEmptyBlock())
+			return super.cmOperations(op);
+		ColGroup grp = _colGroups.get(0);
+		if(grp instanceof ColGroupUncompressed)
+			return ((ColGroupUncompressed) grp).getData().cmOperations(op);
+
+		ColGroupValue grpVal = (ColGroupValue) grp;
+		MatrixBlock vals = grpVal.getValuesAsBlock();
+		MatrixBlock counts = ColGroupValue.getCountsAsBlock(grpVal.getCounts(true));
+		return vals.cmOperations(op, counts);
+	}
+
+	@Override
+	public CM_COV_Object cmOperations(CMOperator op, MatrixBlock weights) {
+		printDecompressWarning("cmOperations");
+		MatrixBlock right = getUncompressed(weights);
+		if(!isCompressed() || isEmptyBlock())
+			return super.cmOperations(op, right);
+		ColGroup grp = _colGroups.get(0);
+		if(grp instanceof ColGroupUncompressed)
+			return ((ColGroupUncompressed) grp).getData().cmOperations(op);
+		return decompress().cmOperations(op, right);
+	}
+
+	@Override
+	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that) {
+		printDecompressWarning("covOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(that);
+		return left.covOperations(op, right);
+	}
+
+	@Override
+	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that, MatrixBlock weights) {
+		printDecompressWarning("covOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(that);
+		MatrixBlock right2 = getUncompressed(weights);
+		return left.covOperations(op, right1, right2);
+	}
+
+	@Override
+	public MatrixBlock sortOperations(MatrixValue weights, MatrixBlock result) {
+		printDecompressWarning("sortOperations");
+		MatrixBlock right = getUncompressed(weights);
+		if(!isCompressed())
+			return super.sortOperations(right, result);
+		ColGroup grp = _colGroups.get(0);
+		if(grp instanceof ColGroupUncompressed)
+			return ((ColGroupUncompressed) grp).getData().sortOperations(right, result);
+
+		if(right == null) {
+			ColGroupValue grpVal = (ColGroupValue) grp;
+			MatrixBlock vals = grpVal.getValuesAsBlock();
+			int[] counts = grpVal.getCounts(true);
+			double[] data = (vals.getDenseBlock() != null) ? vals.getDenseBlockValues() : null;
+			SortUtils.sortByValue(0, vals.getNumRows(), data, counts);
+			MatrixBlock counts2 = ColGroupValue.getCountsAsBlock(counts);
+			return vals.sortOperations(counts2, result);
+		}
+		else
+			return decompress().sortOperations(right, result);
+	}
+
+	@Override
+	public MatrixBlock aggregateBinaryOperations(MatrixIndexes m1Index, MatrixBlock m1Value, MatrixIndexes m2Index,
+		MatrixBlock m2Value, MatrixBlock result, AggregateBinaryOperator op) {
+		printDecompressWarning("aggregateBinaryOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(m2Value);
+		return left.aggregateBinaryOperations(m1Index, left, m2Index, right, result, op);
+	}
+
+	@Override
+	public MatrixBlock aggregateTernaryOperations(MatrixBlock m1, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret,
+		AggregateTernaryOperator op, boolean inCP) {
+		printDecompressWarning("aggregateTernaryOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(m2);
+		MatrixBlock right2 = getUncompressed(m3);
+		return left.aggregateTernaryOperations(left, right1, right2, ret, op, inCP);
+	}
+
+	@Override
+	public MatrixBlock uaggouterchainOperations(MatrixBlock mbLeft, MatrixBlock mbRight, MatrixBlock mbOut,
+		BinaryOperator bOp, AggregateUnaryOperator uaggOp) {
+		printDecompressWarning("uaggouterchainOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(mbRight);
+		return left.uaggouterchainOperations(left, right, mbOut, bOp, uaggOp);
+	}
+
+	@Override
+	public MatrixBlock groupedAggOperations(MatrixValue tgt, MatrixValue wghts, MatrixValue ret, int ngroups,
+		Operator op) {
+		return groupedAggOperations(tgt, wghts, ret, ngroups, op, 1);
+	}
+
+	@Override
+	public MatrixBlock groupedAggOperations(MatrixValue tgt, MatrixValue wghts, MatrixValue ret, int ngroups,
+		Operator op, int k) {
+		printDecompressWarning("groupedAggOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(wghts);
+		return left.groupedAggOperations(left, right, ret, ngroups, op, k);
+	}
+
+	@Override
+	public MatrixBlock removeEmptyOperations(MatrixBlock ret, boolean rows, boolean emptyReturn, MatrixBlock select) {
+		printDecompressWarning("removeEmptyOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.removeEmptyOperations(ret, rows, emptyReturn, select);
+	}
+
+	@Override
+	public MatrixBlock removeEmptyOperations(MatrixBlock ret, boolean rows, boolean emptyReturn) {
+		printDecompressWarning("removeEmptyOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.removeEmptyOperations(ret, rows, emptyReturn);
+	}
+
+	@Override
+	public MatrixBlock rexpandOperations(MatrixBlock ret, double max, boolean rows, boolean cast, boolean ignore,
+		int k) {
+		printDecompressWarning("rexpandOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.rexpandOperations(ret, max, rows, cast, ignore, k);
+	}
+
+	@Override
+	public MatrixBlock replaceOperations(MatrixValue result, double pattern, double replacement) {
+		printDecompressWarning("replaceOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		return tmp.replaceOperations(result, pattern, replacement);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, double scalar, MatrixValue that, CTableMap resultMap,
+		MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(that);
+		left.ctableOperations(op, scalar, right, resultMap, resultBlock);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, double scalar, double scalar2, CTableMap resultMap,
+		MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		tmp.ctableOperations(op, scalar, scalar2, resultMap, resultBlock);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, MatrixIndexes ix1, double scalar, boolean left, int brlen,
+		CTableMap resultMap, MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock tmp = isCompressed() ? decompress() : this;
+		tmp.ctableOperations(op, ix1, scalar, left, brlen, resultMap, resultBlock);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, MatrixValue that, double scalar, boolean ignoreZeros, CTableMap resultMap,
+		MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right = getUncompressed(that);
+		left.ctableOperations(op, right, scalar, ignoreZeros, resultMap, resultBlock);
+	}
+
+	@Override
+	public MatrixBlock ctableSeqOperations(MatrixValue that, double scalar, MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock right = getUncompressed(that);
+		return this.ctableSeqOperations(right, scalar, resultBlock);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, MatrixValue that, MatrixValue that2, CTableMap resultMap) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(that);
+		MatrixBlock right2 = getUncompressed(that2);
+		left.ctableOperations(op, right1, right2, resultMap);
+	}
+
+	@Override
+	public void ctableOperations(Operator op, MatrixValue that, MatrixValue that2, CTableMap resultMap,
+		MatrixBlock resultBlock) {
+		printDecompressWarning("ctableOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(that);
+		MatrixBlock right2 = getUncompressed(that2);
+		left.ctableOperations(op, right1, right2, resultMap, resultBlock);
+	}
+
+	@Override
+	public MatrixBlock ternaryOperations(TernaryOperator op, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret) {
+		printDecompressWarning("ternaryOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(m2);
+		MatrixBlock right2 = getUncompressed(m3);
+		return left.ternaryOperations(op, right1, right2, ret);
+	}
+
+	@Override
+	public MatrixBlock quaternaryOperations(QuaternaryOperator qop, MatrixBlock um, MatrixBlock vm, MatrixBlock wm,
+		MatrixBlock out) {
+		return quaternaryOperations(qop, um, vm, wm, out, 1);
+	}
+
+	@Override
+	public MatrixBlock quaternaryOperations(QuaternaryOperator qop, MatrixBlock um, MatrixBlock vm, MatrixBlock wm,
+		MatrixBlock out, int k) {
+		printDecompressWarning("quaternaryOperations");
+		MatrixBlock left = isCompressed() ? decompress() : this;
+		MatrixBlock right1 = getUncompressed(um);
+		MatrixBlock right2 = getUncompressed(vm);
+		MatrixBlock right3 = getUncompressed(wm);
+		return left.quaternaryOperations(qop, right1, right2, right3, out, k);
+	}
+
+	@Override
+	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed) {
+		throw new DMLRuntimeException("CompressedMatrixBlock: randOperationsInPlace not supported.");
+	}
+
+	@Override
+	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed, int k) {
+		throw new DMLRuntimeException("CompressedMatrixBlock: randOperationsInPlace not supported.");
+	}
+
+	@Override
+	public MatrixBlock seqOperationsInPlace(double from, double to, double incr) {
+		// output should always be uncompressed
+		throw new DMLRuntimeException("CompressedMatrixBlock: seqOperationsInPlace not supported.");
+	}
+
+	private static boolean isCompressed(MatrixBlock mb) {
+		return(mb instanceof CompressedMatrixBlock && ((CompressedMatrixBlock) mb).isCompressed());
+	}
+
+	private static MatrixBlock getUncompressed(MatrixValue mVal) {
+		return isCompressed((MatrixBlock) mVal) ? ((CompressedMatrixBlock) mVal).decompress() : (MatrixBlock) mVal;
+	}
+
+	private void printDecompressWarning(String operation) {
+		if(isCompressed()) {
+			LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
+		}
+	}
+
+	private void printDecompressWarning(String operation, MatrixBlock m2) {
+		if(isCompressed() || isCompressed(m2)) {
+			LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
+		}
+	}
+
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/sysds/runtime/compress/BitmapEncoder.java b/src/main/java/org/apache/sysds/runtime/compress/BitmapEncoder.java
index a826616..9f1a7d0 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/BitmapEncoder.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/BitmapEncoder.java
@@ -30,12 +30,14 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 /**
  * Static functions for encoding bitmaps in various ways.
- * 
  */
 public class BitmapEncoder {
 	/** Size of the blocks used in a blocked bitmap representation. */
+	// Note it is one more than Character.MAX_VALUE.
 	public static final int BITMAP_BLOCK_SZ = 65536;
 
+	public static boolean MATERIALIZE_ZEROS = false;
+
 	public static int getAlignedBlocksize(int blklen) {
 		return blklen + ((blklen % BITMAP_BLOCK_SZ != 0) ? BITMAP_BLOCK_SZ - blklen % BITMAP_BLOCK_SZ : 0);
 	}
@@ -43,43 +45,44 @@ public class BitmapEncoder {
 	/**
 	 * Generate uncompressed bitmaps for a set of columns in an uncompressed matrix block.
 	 * 
-	 * @param colIndices indexes (within the block) of the columns to extract
-	 * @param rawblock   an uncompressed matrix block; can be dense or sparse
+	 * @param colIndices   Indexes (within the block) of the columns to extract
+	 * @param rawBlock     An uncompressed matrix block; can be dense or sparse
+	 * @param compSettings The compression settings used for the compression.
 	 * @return uncompressed bitmap representation of the columns
 	 */
-	public static UncompressedBitmap extractBitmap(int[] colIndices, MatrixBlock rawblock) {
+	public static UncompressedBitmap extractBitmap(int[] colIndices, MatrixBlock rawBlock,
+		CompressionSettings compSettings) {
 		// note: no sparse column selection reader because low potential
 		// single column selection
 		if(colIndices.length == 1) {
-			return extractBitmap(colIndices[0], rawblock, !CompressedMatrixBlock.MATERIALIZE_ZEROS);
+			return extractBitmap(colIndices[0], rawBlock, !MATERIALIZE_ZEROS, compSettings);
 		}
 		// multiple column selection (general case)
 		else {
 			ReaderColumnSelection reader = null;
-			if(rawblock.isInSparseFormat() && CompressedMatrixBlock.TRANSPOSE_INPUT)
-				reader = new ReaderColumnSelectionSparse(rawblock, colIndices,
-					!CompressedMatrixBlock.MATERIALIZE_ZEROS);
+			if(rawBlock.isInSparseFormat() && compSettings.transposeInput)
+				reader = new ReaderColumnSelectionSparse(rawBlock, colIndices, !MATERIALIZE_ZEROS, compSettings);
 			else
-				reader = new ReaderColumnSelectionDense(rawblock, colIndices, !CompressedMatrixBlock.MATERIALIZE_ZEROS);
+				reader = new ReaderColumnSelectionDense(rawBlock, colIndices, !MATERIALIZE_ZEROS, compSettings);
 
-			return extractBitmap(colIndices, rawblock, reader);
+			return extractBitmap(colIndices, rawBlock, reader);
 		}
 	}
 
-	public static UncompressedBitmap extractBitmapFromSample(int[] colIndices, MatrixBlock rawblock,
-		int[] sampleIndexes) {
+	public static UncompressedBitmap extractBitmapFromSample(int[] colIndices, MatrixBlock rawBlock,
+		int[] sampleIndexes, CompressionSettings compSettings) {
 		// note: no sparse column selection reader because low potential
 
 		// single column selection
 		if(colIndices.length == 1) {
-			return extractBitmap(colIndices[0], rawblock, sampleIndexes, !CompressedMatrixBlock.MATERIALIZE_ZEROS);
+			return extractBitmap(colIndices[0], rawBlock, sampleIndexes, !MATERIALIZE_ZEROS, compSettings);
 		}
 		// multiple column selection (general case)
 		else {
 			return extractBitmap(colIndices,
-				rawblock,
-				new ReaderColumnSelectionDenseSample(rawblock, colIndices, sampleIndexes,
-					!CompressedMatrixBlock.MATERIALIZE_ZEROS));
+				rawBlock,
+				new ReaderColumnSelectionDenseSample(rawBlock, colIndices, sampleIndexes, !MATERIALIZE_ZEROS,
+					compSettings));
 		}
 	}
 
@@ -88,7 +91,6 @@ public class BitmapEncoder {
 	 * 
 	 * @param offsets uncompressed offset list
 	 * @param len     logical length of the given offset list
-	 * 
 	 * @return compressed version of said bitmap
 	 */
 	public static char[] genRLEBitmap(int[] offsets, int len) {
@@ -163,8 +165,14 @@ public class BitmapEncoder {
 			}
 		}
 
-		// Close out the last run
 		if(curRunLen >= 1) {
+			// Edge case, if the last run overlaps the character length bound.
+			if(curRunOff + curRunLen > Character.MAX_VALUE) {
+				buf.add(Character.MAX_VALUE);
+				buf.add((char) 0);
+				curRunOff -= Character.MAX_VALUE;
+			}
+
 			buf.add((char) curRunOff);
 			buf.add((char) curRunLen);
 		}
@@ -226,16 +234,17 @@ public class BitmapEncoder {
 		return encodedBlocks;
 	}
 
-	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawblock, boolean skipZeros) {
+	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawBlock, boolean skipZeros,
+		CompressionSettings compSettings) {
 		// probe map for distinct items (for value or value groups)
 		DoubleIntListHashMap distinctVals = new DoubleIntListHashMap();
 
 		// scan rows and probe/build distinct items
-		final int m = CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.getNumColumns() : rawblock.getNumRows();
+		final int m = compSettings.transposeInput ? rawBlock.getNumColumns() : rawBlock.getNumRows();
 
-		if(rawblock.isInSparseFormat() // SPARSE
-			&& CompressedMatrixBlock.TRANSPOSE_INPUT) {
-			SparseBlock a = rawblock.getSparseBlock();
+		if(rawBlock.isInSparseFormat() // SPARSE
+			&& compSettings.transposeInput) {
+			SparseBlock a = rawBlock.getSparseBlock();
 			if(a != null && !a.isEmpty(colIndex)) {
 				int apos = a.pos(colIndex);
 				int alen = a.size(colIndex);
@@ -277,7 +286,7 @@ public class BitmapEncoder {
 		else // GENERAL CASE
 		{
 			for(int i = 0; i < m; i++) {
-				double val = CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.quickGetValue(colIndex, i) : rawblock
+				double val = compSettings.transposeInput ? rawBlock.quickGetValue(colIndex, i) : rawBlock
 					.quickGetValue(i, colIndex);
 				if(val != 0 || !skipZeros) {
 					IntArrayList lstPtr = distinctVals.get(val);
@@ -293,8 +302,8 @@ public class BitmapEncoder {
 		return new UncompressedBitmap(distinctVals);
 	}
 
-	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawblock, int[] sampleIndexes,
-		boolean skipZeros) {
+	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawBlock, int[] sampleIndexes,
+		boolean skipZeros, CompressionSettings compSettings) {
 		// note: general case only because anyway binary search for small samples
 
 		// probe map for distinct items (for value or value groups)
@@ -304,7 +313,7 @@ public class BitmapEncoder {
 		final int m = sampleIndexes.length;
 		for(int i = 0; i < m; i++) {
 			int rowIndex = sampleIndexes[i];
-			double val = CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.quickGetValue(colIndex, rowIndex) : rawblock
+			double val = compSettings.transposeInput ? rawBlock.quickGetValue(colIndex, rowIndex) : rawBlock
 				.quickGetValue(rowIndex, colIndex);
 			if(val != 0 || !skipZeros) {
 				IntArrayList lstPtr = distinctVals.get(val);
@@ -319,7 +328,7 @@ public class BitmapEncoder {
 		return new UncompressedBitmap(distinctVals);
 	}
 
-	private static UncompressedBitmap extractBitmap(int[] colIndices, MatrixBlock rawblock,
+	private static UncompressedBitmap extractBitmap(int[] colIndices, MatrixBlock rawBlock,
 		ReaderColumnSelection rowReader) {
 		// probe map for distinct items (for value or value groups)
 		DblArrayIntListHashMap distinctVals = new DblArrayIntListHashMap();
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 aee8ac4..3ad65c5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -26,11 +26,8 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -38,22 +35,22 @@ import java.util.concurrent.Future;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math3.random.Well1024a;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.lops.MMTSJ.MMTSJType;
 import org.apache.sysds.lops.MapMultChain.ChainType;
 import org.apache.sysds.runtime.DMLRuntimeException;
-import org.apache.sysds.runtime.compress.ColGroup.CompressionType;
-import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorFactory;
-import org.apache.sysds.runtime.compress.utils.ConverterUtils;
-import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupConverter;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupDDC;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupDDC1;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupIO;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupOffset;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupUncompressed;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupValue;
+import org.apache.sysds.runtime.compress.colgroup.DenseRowIterator;
+import org.apache.sysds.runtime.compress.colgroup.SparseRowIterator;
 import org.apache.sysds.runtime.compress.utils.ColumnGroupIterator;
-import org.apache.sysds.runtime.controlprogram.caching.CacheBlock;
+import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.controlprogram.caching.MatrixObject.UpdateType;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
 import org.apache.sysds.runtime.data.SparseBlock;
@@ -66,73 +63,27 @@ import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.ReduceAll;
 import org.apache.sysds.runtime.functionobjects.ReduceCol;
-import org.apache.sysds.runtime.instructions.cp.CM_COV_Object;
 import org.apache.sysds.runtime.instructions.cp.KahanObject;
-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.IJV;
 import org.apache.sysds.runtime.matrix.data.LibMatrixBincell;
 import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
 import org.apache.sysds.runtime.matrix.data.MatrixValue;
-import org.apache.sysds.runtime.matrix.data.RandomMatrixGenerator;
-// import org.apache.sysds.runtime.matrix.data.SparseBlock;
-// import org.apache.sysds.runtime.matrix.data.SparseRow;
-// import org.apache.sysds.runtime.matrix.data.SparseRowVector;
-// import org.apache.sysds.runtime.matrix.mapred.IndexedMatrixValue;
 import org.apache.sysds.runtime.matrix.operators.AggregateBinaryOperator;
-import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
-import org.apache.sysds.runtime.matrix.operators.AggregateTernaryOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.matrix.operators.BinaryOperator;
-import org.apache.sysds.runtime.matrix.operators.CMOperator;
-import org.apache.sysds.runtime.matrix.operators.COVOperator;
-import org.apache.sysds.runtime.matrix.operators.Operator;
-import org.apache.sysds.runtime.matrix.operators.QuaternaryOperator;
-import org.apache.sysds.runtime.matrix.operators.ReorgOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
-import org.apache.sysds.runtime.matrix.operators.TernaryOperator;
-import org.apache.sysds.runtime.matrix.operators.UnaryOperator;
 import org.apache.sysds.runtime.util.CommonThreadPool;
-import org.apache.sysds.runtime.util.IndexRange;
-import org.apache.sysds.runtime.util.SortUtils;
-
-public class CompressedMatrixBlock extends MatrixBlock {
-
-	// ------------------------------
-	// Logging parameters:
-	// local debug flag
-	private static final boolean LOCAL_DEBUG = false;
-	// DEBUG/TRACE for details
-	private static final Level LOCAL_DEBUG_LEVEL = Level.DEBUG;
 
+public class CompressedMatrixBlock extends AbstractCompressedMatrixBlock {
 	private static final Log LOG = LogFactory.getLog(CompressedMatrixBlock.class.getName());
-
-	static {
-		// for internal debugging only
-		if(LOCAL_DEBUG) {
-			Logger.getLogger("org.apache.sysds.runtime.compress").setLevel(LOCAL_DEBUG_LEVEL);
-		}
-	}
-	// ------------------------------
-
-	// Fields
-
 	private static final long serialVersionUID = 7319372019143154058L;
-	public static final boolean TRANSPOSE_INPUT = true;
-	public static final boolean MATERIALIZE_ZEROS = false;
-	public static final long MIN_PAR_AGG_THRESHOLD = 16 * 1024 * 1024; // 16MB
-	public static final boolean INVESTIGATE_ESTIMATES = false;
-	public static boolean ALLOW_DDC_ENCODING = true;
-	public static final boolean ALLOW_SHARED_DDC1_DICTIONARY = true;
-	protected ArrayList<ColGroup> _colGroups = null;
+
+	// Threshold for when to parallelize the aggregation functions.
+	private static final long MIN_PAR_AGG_THRESHOLD = 16 * 1024 * 1024; // 16MB
 	protected CompressionStatistics _stats = null;
 	protected boolean _sharedDDC1Dict = false;
-	protected long seed = -1; // I the seed is -1 then the system used system millisecond time and class hash for
-								// seeding.
-	protected double sampling_ratio = 0.05;
 
 	/**
 	 * Constructor for building an empty Compressed Matrix block object.
@@ -169,443 +120,25 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		nonZeros = that.getNonZeros();
 	}
 
-	/**
-	 * Obtain the column groups.
-	 * 
-	 * @return the column groups constructed by the compression process.
-	 * 
-	 */
-	public ArrayList<ColGroup> getColGroups() {
-		return _colGroups;
-	}
-
-	public int getNumColGroups() {
-		return _colGroups.size();
-	}
-
-	public void setSeed(long seed) {
-		this.seed = seed;
-	}
-
-	public void setSamplingRatio(double sampling_ratio) {
-		this.sampling_ratio = sampling_ratio;
+	public CompressionStatistics getCompressionStatistics() {
+		return _stats;
 	}
 
-	/**
-	 * 
-	 * @return true if block is compressed.
-	 */
 	public boolean isCompressed() {
 		return(_colGroups != null);
 	}
 
 	public boolean isSingleUncompressedGroup() {
-		return(_colGroups != null && _colGroups.size() == 1 && _colGroups.get(0) instanceof ColGroupUncompressed);
-	}
-
-	private void allocateColGroupList() {
-		_colGroups = new ArrayList<>();
-	}
-
-	@Override
-	public boolean isEmptyBlock(boolean safe) {
-		if(!isCompressed())
-			return super.isEmptyBlock(safe);
-		return(_colGroups == null || getNonZeros() == 0);
-	}
-
-	/**
-	 * A single threaded default execution of the compression.
-	 * 
-	 * @return a compressed representation of the original matrix block object.
-	 */
-	public MatrixBlock compress() {
-		// Default sequential execution of compression
-		return compress(1);
-	}
-
-	/**
-	 * The main method for compressing the input matrix.
-	 * 
-	 * @param k the number of threads used to execute the compression.
-	 * @return A compressed matrix block.
-	 */
-	public MatrixBlock compress(int k) {
-		// check for redundant compression
-		if(isCompressed()) {
-			throw new DMLRuntimeException("Redundant compression, block already compressed.");
-		}
-
-		Timing time = new Timing(true);
-		_stats = new CompressionStatistics();
-
-		// SAMPLE-BASED DECISIONS:
-		// Decisions such as testing if a column is amenable to bitmap
-		// compression or evaluating co-coding potentials are made based on a
-		// subset of the rows. For large data sets, sampling might take a
-		// significant amount of time. So, we generate only one sample and use
-		// it for the entire compression process.
-
-		// prepare basic meta data and deep copy / transpose input
-		final int numRows = getNumRows();
-		final int numCols = getNumColumns();
-		final boolean sparse = isInSparseFormat();
-		MatrixBlock rawblock = !TRANSPOSE_INPUT ? new MatrixBlock(this) : LibMatrixReorg
-			.transpose(this, new MatrixBlock(numCols, numRows, sparse), k);
-
-		// construct sample-based size estimator
-		CompressedSizeEstimator bitmapSizeEstimator = CompressedSizeEstimatorFactory
-			.getSizeEstimator(rawblock, numRows, seed, sampling_ratio);
-
-		// PHASE 1: Classify columns by compression type
-		// We start by determining which columns are amenable to compression
-		List<Integer> colsC = new ArrayList<>();
-		List<Integer> colsUC = new ArrayList<>();
-		HashMap<Integer, Double> compRatios = new HashMap<>();
-
-		// Classify columns according to ratio (size uncompressed / size compressed),
-		// where a column is compressible if ratio > 1.
-		CompressedSizeInfo[] sizeInfos = (k > 1) ? computeCompressedSizeInfos(bitmapSizeEstimator,
-			numCols,
-			k) : computeCompressedSizeInfos(bitmapSizeEstimator, numCols);
-		long nnzUC = 0;
-		for(int col = 0; col < numCols; col++) {
-			double uncompSize = getUncompressedSize(numRows,
-				1,
-				OptimizerUtils.getSparsity(numRows, 1, sizeInfos[col].getEstNnz()));
-			double compRatio = uncompSize / sizeInfos[col].getMinSize();
-			if(compRatio > 1) {
-				colsC.add(col);
-				compRatios.put(col, compRatio);
-			}
-			else {
-				colsUC.add(col);
-				nnzUC += sizeInfos[col].getEstNnz();
-			}
-		}
-
-		// correction of column classification (reevaluate dense estimates if necessary)
-		boolean sparseUC = MatrixBlock.evalSparseFormatInMemory(numRows, colsUC.size(), nnzUC);
-		if(!sparseUC && !colsUC.isEmpty()) {
-			for(int i = 0; i < colsUC.size(); i++) {
-				int col = colsUC.get(i);
-				double uncompSize = getUncompressedSize(numRows, 1, 1.0);
-				double compRatio = uncompSize / sizeInfos[col].getMinSize();
-				if(compRatio > 1) {
-					colsC.add(col);
-					colsUC.remove(i);
-					i--;
-					compRatios.put(col, compRatio);
-					nnzUC -= sizeInfos[col].getEstNnz();
-				}
-			}
-		}
-
-		if(LOG.isTraceEnabled()) {
-			LOG.trace("C: " + Arrays.toString(colsC.toArray(new Integer[0])));
-			LOG.trace(
-				"-- compression ratios: " + Arrays.toString(colsC.stream().map(c -> compRatios.get(c)).toArray()));
-			LOG.trace("UC: " + Arrays.toString(colsUC.toArray(new Integer[0])));
-			LOG.trace(
-				"-- compression ratios: " + Arrays.toString(colsUC.stream().map(c -> compRatios.get(c)).toArray()));
-		}
-
-		if(LOG.isDebugEnabled()) {
-			_stats.timePhase1 = time.stop();
-			LOG.debug("Compression statistics:");
-			LOG.debug("--compression phase 1: " + _stats.timePhase1);
-		}
-
-		if(colsC.isEmpty()) {
-			LOG.warn("Abort block compression because all columns are incompressible.");
-			return new MatrixBlock().copyShallow(this);
-		}
-
-		// PHASE 2: Grouping columns
-		// Divide the bitmap columns into column groups.
-		List<int[]> bitmapColGrps = PlanningCoCoder
-			.findCocodesByPartitioning(bitmapSizeEstimator, colsC, sizeInfos, numRows, k);
-
-		if(LOG.isDebugEnabled()) {
-			_stats.timePhase2 = time.stop();
-			LOG.debug("--compression phase 2: " + _stats.timePhase2);
-		}
-
-		if(INVESTIGATE_ESTIMATES) {
-			double est = 0;
-			for(int[] groupIndices : bitmapColGrps)
-				est += bitmapSizeEstimator.estimateCompressedColGroupSize(groupIndices).getMinSize();
-			est += MatrixBlock.estimateSizeInMemory(numRows,
-				colsUC.size(),
-				OptimizerUtils.getSparsity(numRows, colsUC.size(), nnzUC));
-			_stats.estSize = est;
-		}
-
-		// PHASE 3: Compress and correct sample-based decisions
-		ColGroup[] colGroups = (k > 1) ? compressColGroups(rawblock,
-			bitmapSizeEstimator,
-			compRatios,
-			numRows,
-			bitmapColGrps,
-			colsUC.isEmpty(),
-			k) : compressColGroups(rawblock, bitmapSizeEstimator, compRatios, numRows, bitmapColGrps, colsUC.isEmpty());
-		allocateColGroupList();
-		HashSet<Integer> remainingCols = seq(0, numCols - 1, 1);
-		for(int j = 0; j < colGroups.length; j++) {
-			if(colGroups[j] != null) {
-				for(int col : colGroups[j].getColIndices())
-					remainingCols.remove(col);
-				_colGroups.add(colGroups[j]);
-			}
-		}
-
-		if(LOG.isDebugEnabled()) {
-			_stats.timePhase3 = time.stop();
-			LOG.debug("--compression phase 3: " + _stats.timePhase3);
-		}
-
-		// PHASE 4: Best-effort dictionary sharing for DDC1 single-col groups
-		double[] dict = createSharedDDC1Dictionary(_colGroups);
-		if(dict != null) {
-			applySharedDDC1Dictionary(_colGroups, dict);
-			_sharedDDC1Dict = true;
-		}
-
-		if(LOG.isDebugEnabled()) {
-			_stats.timePhase4 = time.stop();
-			LOG.debug("--compression phase 4: " + _stats.timePhase4);
-		}
-
-		// Phase 5: Cleanup
-		// The remaining columns are stored uncompressed as one big column group
-		if(!remainingCols.isEmpty()) {
-			ArrayList<Integer> list = new ArrayList<>(remainingCols);
-			ColGroupUncompressed ucgroup = new ColGroupUncompressed(list, rawblock);
-			_colGroups.add(ucgroup);
-		}
-
-		_stats.size = estimateCompressedSizeInMemory();
-		_stats.ratio = estimateSizeInMemory() / _stats.size;
-
-		if(_stats.ratio < 1) {
-			LOG.warn("Abort block compression because compression ratio is less than 1.");
-			return new MatrixBlock().copyShallow(this);
-		}
-
-		// final cleanup (discard uncompressed block)
-		rawblock.cleanupBlock(true, true);
-		this.cleanupBlock(true, true);
-
-		_stats.timePhase5 = time.stop();
-		int[] counts = getColGroupCounts(_colGroups);
-		LOG.info("--num col groups: " + _colGroups.size() + ", -- num input cols: " + numCols);
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("--compression phase 5: " + _stats.timePhase5);
-
-			LOG.debug("--col groups types (OLE,RLE,DDC1,DDC2,UC): " + counts[2] + "," + counts[1] + "," + counts[3]
-				+ "," + counts[4] + "," + counts[0]);
-			LOG.debug("--col groups sizes (OLE,RLE,DDC1,DDC2,UC): " + counts[7] + "," + counts[6] + "," + counts[8]
-				+ "," + counts[9] + "," + counts[5]);
-			LOG.debug("--compressed size: " + _stats.size);
-			LOG.debug("--compression ratio: " + _stats.ratio);
-		}
-
-		return this;
-	}
-
-	public CompressionStatistics getCompressionStatistics() {
-		return _stats;
+		return(_colGroups != null && _colGroups.size() == 1 &&
+			_colGroups.get(0).getCompType() == CompressionType.UNCOMPRESSED);
 	}
 
-	/**
-	 * Get array of counts regarding col group types. The position corresponds with the enum ordinal.
-	 * 
-	 * @param colgroups list of column groups
-	 * @return counts
-	 */
-	private static int[] getColGroupCounts(ArrayList<ColGroup> colgroups) {
-		int[] ret = new int[10]; // 5 x count, 5 x num_columns
-		for(ColGroup c : colgroups) {
-			ret[c.getCompType().ordinal()]++;
-			ret[5 + c.getCompType().ordinal()] += c.getNumCols();
-		}
-		return ret;
-	}
-
-	private static CompressedSizeInfo[] computeCompressedSizeInfos(CompressedSizeEstimator estim, int clen) {
-		CompressedSizeInfo[] ret = new CompressedSizeInfo[clen];
-		for(int col = 0; col < clen; col++)
-			ret[col] = estim.estimateCompressedColGroupSize(new int[] {col});
-		return ret;
-	}
-
-	private static CompressedSizeInfo[] computeCompressedSizeInfos(CompressedSizeEstimator estim, int clen, int k) {
-		try {
-			ExecutorService pool = CommonThreadPool.get(k);
-			ArrayList<SizeEstimTask> tasks = new ArrayList<>();
-			for(int col = 0; col < clen; col++)
-				tasks.add(new SizeEstimTask(estim, col));
-			List<Future<CompressedSizeInfo>> rtask = pool.invokeAll(tasks);
-			ArrayList<CompressedSizeInfo> ret = new ArrayList<>();
-			for(Future<CompressedSizeInfo> lrtask : rtask)
-				ret.add(lrtask.get());
-			pool.shutdown();
-			return ret.toArray(new CompressedSizeInfo[0]);
-		}
-		catch(Exception ex) {
-			throw new DMLRuntimeException(ex);
-		}
+	public void allocateColGroupList(List<ColGroup> colGroups) {
+		_colGroups = colGroups;
 	}
 
-	private static ColGroup[] compressColGroups(MatrixBlock in, CompressedSizeEstimator estim,
-		HashMap<Integer, Double> compRatios, int rlen, List<int[]> groups, boolean denseEst) {
-		ColGroup[] ret = new ColGroup[groups.size()];
-		for(int i = 0; i < groups.size(); i++)
-			ret[i] = compressColGroup(in, estim, compRatios, rlen, groups.get(i), denseEst);
-
-		return ret;
-	}
-
-	private static ColGroup[] compressColGroups(MatrixBlock in, CompressedSizeEstimator estim,
-		HashMap<Integer, Double> compRatios, int rlen, List<int[]> groups, boolean denseEst, int k) {
-		try {
-			ExecutorService pool = CommonThreadPool.get(k);
-			ArrayList<CompressTask> tasks = new ArrayList<>();
-			for(int[] colIndexes : groups)
-				tasks.add(new CompressTask(in, estim, compRatios, rlen, colIndexes, denseEst));
-			List<Future<ColGroup>> rtask = pool.invokeAll(tasks);
-			ArrayList<ColGroup> ret = new ArrayList<>();
-			for(Future<ColGroup> lrtask : rtask)
-				ret.add(lrtask.get());
-			pool.shutdown();
-			return ret.toArray(new ColGroup[0]);
-		}
-		catch(Exception ex) {
-			throw new DMLRuntimeException(ex);
-		}
-	}
-
-	private static ColGroup compressColGroup(MatrixBlock in, CompressedSizeEstimator estim,
-		HashMap<Integer, Double> compRatios, int rlen, int[] colIndexes, boolean denseEst) {
-		int[] allGroupIndices = null;
-		int allColsCount = colIndexes.length;
-		CompressedSizeInfo sizeInfo;
-		// The compression type is decided based on a full bitmap since it
-		// will be reused for the actual compression step.
-		UncompressedBitmap ubm = null;
-		PriorityQueue<CompressedColumn> compRatioPQ = null;
-		boolean skipGroup = false;
-		while(true) {
-			// exact big list and observe compression ratio
-			ubm = BitmapEncoder.extractBitmap(colIndexes, in);
-			sizeInfo = estim.estimateCompressedColGroupSize(ubm);
-			double sp2 = denseEst ? 1.0 : OptimizerUtils.getSparsity(rlen, 1, ubm.getNumOffsets());
-			double compRatio = getUncompressedSize(rlen, colIndexes.length, sp2) / sizeInfo.getMinSize();
-
-			if(compRatio > 1) {
-				break; // we have a good group
-			}
-
-			// modify the group
-			if(compRatioPQ == null) {
-				// first modification
-				allGroupIndices = colIndexes.clone();
-				compRatioPQ = new PriorityQueue<>();
-				for(int i = 0; i < colIndexes.length; i++)
-					compRatioPQ.add(new CompressedColumn(i, compRatios.get(colIndexes[i])));
-			}
-
-			// index in allGroupIndices
-			int removeIx = compRatioPQ.poll().colIx;
-			allGroupIndices[removeIx] = -1;
-			allColsCount--;
-			if(allColsCount == 0) {
-				skipGroup = true;
-				break;
-			}
-			colIndexes = new int[allColsCount];
-			// copying the values that do not equal -1
-			int ix = 0;
-			for(int col : allGroupIndices)
-				if(col != -1)
-					colIndexes[ix++] = col;
-		}
-
-		// add group to uncompressed fallback
-		if(skipGroup)
-			return null;
-
-		// create compressed column group
-		long rleSize = sizeInfo.getRLESize();
-		long oleSize = sizeInfo.getOLESize();
-		long ddcSize = sizeInfo.getDDCSize();
-
-		if(ALLOW_DDC_ENCODING && ddcSize < rleSize && ddcSize < oleSize) {
-			if(ubm.getNumValues() <= 255)
-				return new ColGroupDDC1(colIndexes, rlen, ubm);
-			else
-				return new ColGroupDDC2(colIndexes, rlen, ubm);
-		}
-		else if(rleSize < oleSize)
-			return new ColGroupRLE(colIndexes, rlen, ubm);
-		else
-			return new ColGroupOLE(colIndexes, rlen, ubm);
-	}
-
-	/**
-	 * Compute a conservative estimate of the uncompressed size of a column group.
-	 * 
-	 * @param rlen     row length
-	 * @param clen     column length
-	 * @param sparsity the sparsity
-	 * @return estimate of uncompressed size of column group
-	 */
-	private static double getUncompressedSize(int rlen, int clen, double sparsity) {
-		// we estimate the uncompressed size as the minimum of dense representation
-		// and representation in csr, which moderately overestimates sparse representations
-		// of single columns but helps avoid anomalies with sparse columns that are
-		// eventually represented in dense
-		return Math.min(8d * rlen * clen, 4d * rlen + 12d * rlen * clen * sparsity);
-	}
-
-	private static double[] createSharedDDC1Dictionary(ArrayList<ColGroup> colGroups) {
-		if(!ALLOW_DDC_ENCODING || !ALLOW_SHARED_DDC1_DICTIONARY)
-			return null;
-
-		// create joint dictionary
-		HashSet<Double> tmp = new HashSet<>();
-		int numQual = 0;
-		for(ColGroup grp : colGroups)
-			if(grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
-				ColGroupDDC1 grpDDC1 = (ColGroupDDC1) grp;
-				for(double val : grpDDC1.getValues())
-					tmp.add(val);
-				numQual++;
-			}
-
-		// abort shared dictionary creation if empty or too large
-		int maxSize = tmp.contains(0d) ? 256 : 255;
-		if(tmp.isEmpty() || tmp.size() > maxSize || numQual < 2)
-			return null;
-		LOG.debug("Created shared directionary for " + numQual + " DDC1 single column groups.");
-
-		// build consolidated dictionary
-		return tmp.stream().mapToDouble(Double::doubleValue).toArray();
-	}
-
-	private static void applySharedDDC1Dictionary(ArrayList<ColGroup> colGroups, double[] dict) {
-		// create joint mapping table
-		HashMap<Double, Integer> map = new HashMap<>();
-		for(int i = 0; i < dict.length; i++)
-			map.put(dict[i], i);
-
-		// recode data of all relevant DDC1 groups
-		for(ColGroup grp : colGroups)
-			if(grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
-				ColGroupDDC1 grpDDC1 = (ColGroupDDC1) grp;
-				grpDDC1.recodeData(map);
-				grpDDC1.setValues(dict);
-			}
+	public List<ColGroup> getColGroups() {
+		return _colGroups;
 	}
 
 	/**
@@ -697,41 +230,41 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	 */
 	public long estimateCompressedSizeInMemory() {
 		if(!isCompressed())
-			return 0;
-		// basic data inherited from MatrixBlock
-		long total = MatrixBlock.estimateSizeInMemory(0, 0, 0);
-		// adding the size of colGroups ArrayList overhead
-		// object overhead (32B) + int size (4B) + int modCount (4B) + Object[]
-		// elementData overhead + reference (32+8)B +reference ofr each Object (8B)
-		total += 80 + 8 * _colGroups.size();
+			return Long.MAX_VALUE;
+
+		long total = baseSizeInMemory();
+
+		// TODO scale up based on number of col groups.
 		for(ColGroup grp : _colGroups)
 			total += grp.estimateInMemorySize();
-		// correction for shared DDC1 dictionary
+
+		// Correction for shared DDC1 dictionary
+		// TODO Fix DDC Sharing.
 		if(_sharedDDC1Dict) {
 			boolean seenDDC1 = false;
 			for(ColGroup grp : _colGroups)
-				if(grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
+				if(grp.getNumCols() == 1 && grp.getCompType() == CompressionType.DDC) {
 					if(seenDDC1)
-						total -= ((ColGroupDDC1) grp).getValuesSize();
+						total -= grp.getValuesSize();
 					seenDDC1 = true;
 				}
 		}
 		return total;
 	}
 
-	private static class CompressedColumn implements Comparable<CompressedColumn> {
-		final int colIx;
-		final double compRatio;
+	public static long baseSizeInMemory() {
+		long total = 16; // Object header
 
-		public CompressedColumn(int colIx, double compRatio) {
-			this.colIx = colIx;
-			this.compRatio = compRatio;
-		}
+		total += 40; // Matrix Block elements
+		total += 8; // Col Group Ref
+		total += 1 + 7; // Booleans plus padding
 
-		@Override
-		public int compareTo(CompressedColumn o) {
-			return (int) Math.signum(compRatio - o.compRatio);
-		}
+		// TODO: Reduce memory usage from CompressionStatistics
+		total += 8; // Stats object reference
+		total += CompressionStatistics.getSizeInMemory();
+
+		total += 40; // Col Group Array List
+		return total;
 	}
 
 	@Override
@@ -790,53 +323,12 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			super.readFields(in);
 			return;
 		}
-
 		// deserialize compressed block
 		rlen = in.readInt();
 		clen = in.readInt();
 		nonZeros = in.readLong();
 		_sharedDDC1Dict = in.readBoolean();
-		int ncolGroups = in.readInt();
-
-		_colGroups = new ArrayList<>(ncolGroups);
-		double[] sharedDict = null;
-		for(int i = 0; i < ncolGroups; i++) {
-			CompressionType ctype = CompressionType.values()[in.readByte()];
-			ColGroup grp = null;
-
-			// create instance of column group
-			switch(ctype) {
-				case UNCOMPRESSED:
-					grp = new ColGroupUncompressed();
-					break;
-				case OLE_BITMAP:
-					grp = new ColGroupOLE();
-					break;
-				case RLE_BITMAP:
-					grp = new ColGroupRLE();
-					break;
-				case DDC1:
-					grp = new ColGroupDDC1();
-					break;
-				case DDC2:
-					grp = new ColGroupDDC2();
-					break;
-			}
-
-			// deserialize and add column group (flag for shared dictionary passed
-			// and numCols evaluated in DDC1 because numCols not available yet
-			grp.readFields(in, sharedDict != null);
-
-			// use shared DDC1 dictionary if applicable
-			if(_sharedDDC1Dict && grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
-				if(sharedDict == null)
-					sharedDict = ((ColGroupValue) grp).getValues();
-				else
-					((ColGroupValue) grp).setValues(sharedDict);
-			}
-
-			_colGroups.add(grp);
-		}
+		_colGroups = ColGroupIO.readGroups(in, _sharedDDC1Dict);
 	}
 
 	@Override
@@ -854,15 +346,8 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		out.writeInt(clen);
 		out.writeLong(nonZeros);
 		out.writeBoolean(_sharedDDC1Dict);
-		out.writeInt(_colGroups.size());
 
-		boolean skipDict = false;
-		for(ColGroup grp : _colGroups) {
-			boolean shared = (grp instanceof ColGroupDDC1 && _sharedDDC1Dict && grp.getNumCols() == 1);
-			out.writeByte(grp.getCompType().ordinal());
-			grp.write(out, skipDict & shared); // delegate serialization
-			skipDict |= shared;
-		}
+		ColGroupIO.writeGroups(out, _sharedDDC1Dict, _colGroups);
 	}
 
 	/**
@@ -890,7 +375,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	public Iterator<IJV> getIterator(int rl, int ru, boolean inclZeros) {
-		return getIterator(rl, ru, 0, getNumColGroups(), inclZeros);
+		return getIterator(rl, ru, 0, _colGroups.size(), inclZeros);
 	}
 
 	public Iterator<IJV> getIterator(int rl, int ru, int cgl, int cgu, boolean inclZeros) {
@@ -967,17 +452,17 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 
 		// shallow copy of lhs column groups
-		ret2.allocateColGroupList();
+		ret2.allocateColGroupList(new ArrayList<ColGroup>());
 		ret2._colGroups.addAll(_colGroups);
 
 		// copy of rhs column groups w/ col index shifting
 		if(!(that instanceof CompressedMatrixBlock)) {
-			that = new CompressedMatrixBlock(that);
-			((CompressedMatrixBlock) that).compress();
+			that = CompressedMatrixBlockFactory.compress(that);
 		}
-		ArrayList<ColGroup> inColGroups = ((CompressedMatrixBlock) that)._colGroups;
+
+		List<ColGroup> inColGroups = ((CompressedMatrixBlock) that)._colGroups;
 		for(ColGroup group : inColGroups) {
-			ColGroup tmp = ConverterUtils.copyColGroup(group);
+			ColGroup tmp = ColGroupConverter.copyColGroup(group);
 			tmp.shiftColIndices(clen);
 			ret2._colGroups.add(tmp);
 		}
@@ -1004,7 +489,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			throw new DMLRuntimeException(
 				"Invalid weight vector (column vector expected, but ncol=" + w.getNumColumns() + ")");
 
-		// single-threaded mmchain of single uncompressed colgroup
+		// single-threaded MMChain of single uncompressed ColGroup
 		if(isSingleUncompressedGroup()) {
 			return ((ColGroupUncompressed) _colGroups.get(0)).getData().chainMatrixMultOperations(v, w, out, ctype);
 		}
@@ -1030,8 +515,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 		leftMultByVectorTranspose(_colGroups, tmp, out, true, true);
 
-		// System.out.println("Compressed MMChain in "+time.stop());
-
 		return out;
 	}
 
@@ -1053,7 +536,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			throw new DMLRuntimeException(
 				"Invalid weight vector (column vector expected, but ncol=" + w.getNumColumns() + ")");
 
-		// multi-threaded mmchain of single uncompressed colgroup
+		// multi-threaded MMChain of single uncompressed ColGroup
 		if(isSingleUncompressedGroup()) {
 			return ((ColGroupUncompressed) _colGroups.get(0)).getData().chainMatrixMultOperations(v, w, out, ctype, k);
 		}
@@ -1093,7 +576,8 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return super.aggregateBinaryOperations(m1, m2, ret, op);
 		}
 
-		// multi-threaded mm of single uncompressed colgroup
+		// Should not happen that it is a single uncompressed group.
+		// multi-threaded MM of single uncompressed ColGroup
 		if(isSingleUncompressedGroup()) {
 			MatrixBlock tmp = ((ColGroupUncompressed) _colGroups.get(0)).getData();
 			return tmp.aggregateBinaryOperations(this == m1 ? tmp : m1, this == m2 ? tmp : m2, ret, op);
@@ -1174,8 +658,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		return ret;
 	}
 
-	// Aggregate Unary operations no longer exists. most likely not needed anymore.
-
 	@Override
 	public MatrixBlock aggregateUnaryOperations(AggregateUnaryOperator op, MatrixValue result, int blen,
 		MatrixIndexes indexesIn, boolean inCP) {
@@ -1323,13 +805,16 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		return aggregateUnaryOperations(op, result, blen, indexesIn, false);
 	}
 
-	private static void aggregateUnaryOperations(AggregateUnaryOperator op, ArrayList<ColGroup> groups, MatrixBlock ret,
+	private static void aggregateUnaryOperations(AggregateUnaryOperator op, List<ColGroup> groups, MatrixBlock ret,
 		int rl, int ru) {
-		boolean cacheDDC1 = ColGroupValue.LOW_LEVEL_OPT && op.indexFn instanceof ReduceCol &&
+
+		// Seems misplaced logic for when to use CacheDDC
+		boolean cacheDDC1 = op.indexFn instanceof ReduceCol &&
 			op.aggOp.increOp.fn instanceof KahanPlus // rowSums
 			&& ColGroupOffset.ALLOW_CACHE_CONSCIOUS_ROWSUMS && ru - rl > ColGroupOffset.WRITE_CACHE_BLKSZ / 2;
 
 		// process cache-conscious DDC1 groups (adds to output)
+		// TODO: Fix such that is is able to sharing even if ColGroupDDC2
 		if(cacheDDC1) {
 			ArrayList<ColGroupDDC1> tmp = new ArrayList<>();
 			for(ColGroup grp : groups)
@@ -1354,7 +839,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return super.transposeSelfMatrixMultOperations(out, tstype);
 		}
 
-		// single-threaded tsmm of single uncompressed colgroup
+		// single-threaded transpose self MM of single uncompressed ColGroup
 		if(isSingleUncompressedGroup()) {
 			return ((ColGroupUncompressed) _colGroups.get(0)).getData().transposeSelfMatrixMultOperations(out, tstype);
 		}
@@ -1393,7 +878,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 			return super.transposeSelfMatrixMultOperations(out, tstype, k);
 		}
 
-		// multi-threaded tsmm of single uncompressed colgroup
+		// multi-threaded transpose self MM of single uncompressed ColGroup
 		if(isSingleUncompressedGroup()) {
 			return ((ColGroupUncompressed) _colGroups.get(0)).getData()
 				.transposeSelfMatrixMultOperations(out, tstype, k);
@@ -1500,11 +985,11 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 	}
 
-	private static void rightMultByVector(ArrayList<ColGroup> groups, MatrixBlock vect, MatrixBlock ret, boolean inclUC,
+	private static void rightMultByVector(List<ColGroup> groups, MatrixBlock vect, MatrixBlock ret, boolean inclUC,
 		int rl, int ru) {
 		ColGroupValue.setupThreadLocalMemory(getMaxNumValues(groups));
 
-		boolean cacheDDC1 = ColGroupValue.LOW_LEVEL_OPT && ru - rl > ColGroupOffset.WRITE_CACHE_BLKSZ;
+		boolean cacheDDC1 = ru - rl > ColGroupOffset.WRITE_CACHE_BLKSZ;
 
 		// process uncompressed column group (overwrites output)
 		if(inclUC) {
@@ -1625,7 +1110,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		result.recomputeNonZeros();
 	}
 
-	private static void leftMultByTransposeSelf(ArrayList<ColGroup> groups, MatrixBlock result, int gl, int gu) {
+	private static void leftMultByTransposeSelf(List<ColGroup> groups, MatrixBlock result, int gl, int gu) {
 		final int numRows = groups.get(0).getNumRows();
 		final int numGroups = groups.size();
 		final boolean containsUC = containsUncompressedColGroup(groups);
@@ -1719,7 +1204,7 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		return null;
 	}
 
-	private static boolean containsUncompressedColGroup(ArrayList<ColGroup> groups) {
+	private static boolean containsUncompressedColGroup(List<ColGroup> groups) {
 		for(ColGroup grp : groups)
 			if(grp instanceof ColGroupUncompressed)
 				return true;
@@ -1752,13 +1237,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	private static class RightMatrixMultTask implements Callable<Long> {
-		private final ArrayList<ColGroup> _groups;
+		private final List<ColGroup> _groups;
 		private final MatrixBlock _vect;
 		private final MatrixBlock _ret;
 		private final int _rl;
 		private final int _ru;
 
-		protected RightMatrixMultTask(ArrayList<ColGroup> groups, MatrixBlock vect, MatrixBlock ret, int rl, int ru) {
+		protected RightMatrixMultTask(List<ColGroup> groups, MatrixBlock vect, MatrixBlock ret, int rl, int ru) {
 			_groups = groups;
 			_vect = vect;
 			_ret = ret;
@@ -1774,12 +1259,12 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	private static class MatrixMultTransposeTask implements Callable<Object> {
-		private final ArrayList<ColGroup> _groups;
+		private final List<ColGroup> _groups;
 		private final MatrixBlock _ret;
 		private final int _gl;
 		private final int _gu;
 
-		protected MatrixMultTransposeTask(ArrayList<ColGroup> groups, MatrixBlock ret, int gl, int gu) {
+		protected MatrixMultTransposeTask(List<ColGroup> groups, MatrixBlock ret, int gl, int gu) {
 			_groups = groups;
 			_ret = ret;
 			_gl = gl;
@@ -1794,13 +1279,13 @@ public class CompressedMatrixBlock extends MatrixBlock {
 	}
 
 	private static class UnaryAggregateTask implements Callable<MatrixBlock> {
-		private final ArrayList<ColGroup> _groups;
+		private final List<ColGroup> _groups;
 		private final int _rl;
 		private final int _ru;
 		private final MatrixBlock _ret;
 		private final AggregateUnaryOperator _op;
 
-		protected UnaryAggregateTask(ArrayList<ColGroup> groups, MatrixBlock ret, int rl, int ru,
+		protected UnaryAggregateTask(List<ColGroup> groups, MatrixBlock ret, int rl, int ru,
 			AggregateUnaryOperator op) {
 			_groups = groups;
 			_op = op;
@@ -1829,45 +1314,6 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 	}
 
-	private static class SizeEstimTask implements Callable<CompressedSizeInfo> {
-		private final CompressedSizeEstimator _estim;
-		private final int _col;
-
-		protected SizeEstimTask(CompressedSizeEstimator estim, int col) {
-			_estim = estim;
-			_col = col;
-		}
-
-		@Override
-		public CompressedSizeInfo call() {
-			return _estim.estimateCompressedColGroupSize(new int[] {_col});
-		}
-	}
-
-	private static class CompressTask implements Callable<ColGroup> {
-		private final MatrixBlock _in;
-		private final CompressedSizeEstimator _estim;
-		private final HashMap<Integer, Double> _compRatios;
-		private final int _rlen;
-		private final int[] _colIndexes;
-		private final boolean _denseEst;
-
-		protected CompressTask(MatrixBlock in, CompressedSizeEstimator estim, HashMap<Integer, Double> compRatios,
-			int rlen, int[] colIndexes, boolean denseEst) {
-			_in = in;
-			_estim = estim;
-			_compRatios = compRatios;
-			_rlen = rlen;
-			_colIndexes = colIndexes;
-			_denseEst = denseEst;
-		}
-
-		@Override
-		public ColGroup call() {
-			return compressColGroup(_in, _estim, _compRatios, _rlen, _colIndexes, _denseEst);
-		}
-	}
-
 	private static class DecompressTask implements Callable<Object> {
 		private final List<ColGroup> _colGroups;
 		private final MatrixBlock _ret;
@@ -1906,391 +1352,4 @@ public class CompressedMatrixBlock extends MatrixBlock {
 		}
 	}
 
-	//////////////////////////////////////////
-	// Graceful fallback to uncompressed linear algebra
-
-	@Override
-	public MatrixBlock unaryOperations(UnaryOperator op, MatrixValue result) {
-		printDecompressWarning("unaryOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.unaryOperations(op, result);
-	}
-
-	@Override
-	public MatrixBlock binaryOperations(BinaryOperator op, MatrixValue thatValue, MatrixValue result) {
-		printDecompressWarning("binaryOperations", (MatrixBlock) thatValue);
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(thatValue);
-		return left.binaryOperations(op, right, result);
-	}
-
-	@Override
-	public void binaryOperationsInPlace(BinaryOperator op, MatrixValue thatValue) {
-		printDecompressWarning("binaryOperationsInPlace", (MatrixBlock) thatValue);
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(thatValue);
-		left.binaryOperationsInPlace(op, right);
-	}
-
-	@Override
-	public void incrementalAggregate(AggregateOperator aggOp, MatrixValue correction, MatrixValue newWithCorrection,
-		boolean deep) {
-		throw new DMLRuntimeException("CompressedMatrixBlock: incrementalAggregate not supported.");
-	}
-
-	@Override
-	public void incrementalAggregate(AggregateOperator aggOp, MatrixValue newWithCorrection) {
-		throw new DMLRuntimeException("CompressedMatrixBlock: incrementalAggregate not supported.");
-	}
-
-	@Override
-	public MatrixBlock reorgOperations(ReorgOperator op, MatrixValue ret, int startRow, int startColumn, int length) {
-		printDecompressWarning("reorgOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.reorgOperations(op, ret, startRow, startColumn, length);
-	}
-
-	@Override
-	public MatrixBlock append(MatrixBlock that, MatrixBlock ret, boolean cbind) {
-		if(cbind) // use supported operation
-			return append(that, ret);
-		printDecompressWarning("append-rbind", that);
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(that);
-		return left.append(right, ret, cbind);
-	}
-
-	@Override
-	public void append(MatrixValue v2, ArrayList<IndexedMatrixValue> outlist, int blen, boolean cbind, boolean m2IsLast,
-		int nextNCol) {
-		printDecompressWarning("append", (MatrixBlock) v2);
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(v2);
-		left.append(right, outlist, blen, cbind, m2IsLast, nextNCol);
-	}
-
-	@Override
-	public void permutationMatrixMultOperations(MatrixValue m2Val, MatrixValue out1Val, MatrixValue out2Val) {
-		permutationMatrixMultOperations(m2Val, out1Val, out2Val, 1);
-	}
-
-	@Override
-	public void permutationMatrixMultOperations(MatrixValue m2Val, MatrixValue out1Val, MatrixValue out2Val, int k) {
-		printDecompressWarning("permutationMatrixMultOperations", (MatrixBlock) m2Val);
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(m2Val);
-		left.permutationMatrixMultOperations(right, out1Val, out2Val, k);
-	}
-
-	@Override
-	public MatrixBlock leftIndexingOperations(MatrixBlock rhsMatrix, int rl, int ru, int cl, int cu, MatrixBlock ret,
-		UpdateType update) {
-		printDecompressWarning("leftIndexingOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(rhsMatrix);
-		return left.leftIndexingOperations(right, rl, ru, cl, cu, ret, update);
-	}
-
-	@Override
-	public MatrixBlock leftIndexingOperations(ScalarObject scalar, int rl, int cl, MatrixBlock ret, UpdateType update) {
-		printDecompressWarning("leftIndexingOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.leftIndexingOperations(scalar, rl, cl, ret, update);
-	}
-
-	@Override
-	public MatrixBlock slice(int rl, int ru, int cl, int cu, CacheBlock ret) {
-		printDecompressWarning("slice");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.slice(rl, ru, cl, cu, ret);
-	}
-
-	@Override
-	public void slice(ArrayList<IndexedMatrixValue> outlist, IndexRange range, int rowCut, int colCut, int blen,
-		int boundaryRlen, int boundaryClen) {
-		printDecompressWarning("slice");
-		try {
-			MatrixBlock tmp = isCompressed() ? decompress() : this;
-			tmp.slice(outlist, range, rowCut, colCut, blen, boundaryRlen, boundaryClen);
-		}
-		catch(DMLRuntimeException ex) {
-			throw new RuntimeException(ex);
-		}
-	}
-
-	@Override
-	public MatrixBlock zeroOutOperations(MatrixValue result, IndexRange range, boolean complementary) {
-		printDecompressWarning("zeroOutOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.zeroOutOperations(result, range, complementary);
-	}
-
-	@Override
-	public CM_COV_Object cmOperations(CMOperator op) {
-		printDecompressWarning("cmOperations");
-		if(!isCompressed() || isEmptyBlock())
-			return super.cmOperations(op);
-		ColGroup grp = _colGroups.get(0);
-		if(grp instanceof ColGroupUncompressed)
-			return ((ColGroupUncompressed) grp).getData().cmOperations(op);
-
-		ColGroupValue grpVal = (ColGroupValue) grp;
-		MatrixBlock vals = grpVal.getValuesAsBlock();
-		MatrixBlock counts = ColGroupValue.getCountsAsBlock(grpVal.getCounts(true));
-		return vals.cmOperations(op, counts);
-	}
-
-	@Override
-	public CM_COV_Object cmOperations(CMOperator op, MatrixBlock weights) {
-		printDecompressWarning("cmOperations");
-		MatrixBlock right = getUncompressed(weights);
-		if(!isCompressed() || isEmptyBlock())
-			return super.cmOperations(op, right);
-		ColGroup grp = _colGroups.get(0);
-		if(grp instanceof ColGroupUncompressed)
-			return ((ColGroupUncompressed) grp).getData().cmOperations(op);
-		return decompress().cmOperations(op, right);
-	}
-
-	@Override
-	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that) {
-		printDecompressWarning("covOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(that);
-		return left.covOperations(op, right);
-	}
-
-	@Override
-	public CM_COV_Object covOperations(COVOperator op, MatrixBlock that, MatrixBlock weights) {
-		printDecompressWarning("covOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(that);
-		MatrixBlock right2 = getUncompressed(weights);
-		return left.covOperations(op, right1, right2);
-	}
-
-	@Override
-	public MatrixBlock sortOperations(MatrixValue weights, MatrixBlock result) {
-		printDecompressWarning("sortOperations");
-		MatrixBlock right = getUncompressed(weights);
-		if(!isCompressed())
-			return super.sortOperations(right, result);
-		ColGroup grp = _colGroups.get(0);
-		if(grp instanceof ColGroupUncompressed)
-			return ((ColGroupUncompressed) grp).getData().sortOperations(right, result);
-
-		if(right == null) {
-			ColGroupValue grpVal = (ColGroupValue) grp;
-			MatrixBlock vals = grpVal.getValuesAsBlock();
-			int[] counts = grpVal.getCounts(true);
-			double[] data = (vals.getDenseBlock() != null) ? vals.getDenseBlockValues() : null;
-			SortUtils.sortByValue(0, vals.getNumRows(), data, counts);
-			MatrixBlock counts2 = ColGroupValue.getCountsAsBlock(counts);
-			return vals.sortOperations(counts2, result);
-		}
-		else
-			return decompress().sortOperations(right, result);
-	}
-
-	@Override
-	public MatrixBlock aggregateBinaryOperations(MatrixIndexes m1Index, MatrixBlock m1Value, MatrixIndexes m2Index,
-		MatrixBlock m2Value, MatrixBlock result, AggregateBinaryOperator op) {
-		printDecompressWarning("aggregateBinaryOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(m2Value);
-		return left.aggregateBinaryOperations(m1Index, left, m2Index, right, result, op);
-	}
-
-	@Override
-	public MatrixBlock aggregateTernaryOperations(MatrixBlock m1, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret,
-		AggregateTernaryOperator op, boolean inCP) {
-		printDecompressWarning("aggregateTernaryOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(m2);
-		MatrixBlock right2 = getUncompressed(m3);
-		return left.aggregateTernaryOperations(left, right1, right2, ret, op, inCP);
-	}
-
-	@Override
-	public MatrixBlock uaggouterchainOperations(MatrixBlock mbLeft, MatrixBlock mbRight, MatrixBlock mbOut,
-		BinaryOperator bOp, AggregateUnaryOperator uaggOp) {
-		printDecompressWarning("uaggouterchainOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(mbRight);
-		return left.uaggouterchainOperations(left, right, mbOut, bOp, uaggOp);
-	}
-
-	@Override
-	public MatrixBlock groupedAggOperations(MatrixValue tgt, MatrixValue wghts, MatrixValue ret, int ngroups,
-		Operator op) {
-		return groupedAggOperations(tgt, wghts, ret, ngroups, op, 1);
-	}
-
-	@Override
-	public MatrixBlock groupedAggOperations(MatrixValue tgt, MatrixValue wghts, MatrixValue ret, int ngroups,
-		Operator op, int k) {
-		printDecompressWarning("groupedAggOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(wghts);
-		return left.groupedAggOperations(left, right, ret, ngroups, op, k);
-	}
-
-	@Override
-	public MatrixBlock removeEmptyOperations(MatrixBlock ret, boolean rows, boolean emptyReturn, MatrixBlock select) {
-		printDecompressWarning("removeEmptyOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.removeEmptyOperations(ret, rows, emptyReturn, select);
-	}
-
-	@Override
-	public MatrixBlock removeEmptyOperations(MatrixBlock ret, boolean rows, boolean emptyReturn) {
-		printDecompressWarning("removeEmptyOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.removeEmptyOperations(ret, rows, emptyReturn);
-	}
-
-	@Override
-	public MatrixBlock rexpandOperations(MatrixBlock ret, double max, boolean rows, boolean cast, boolean ignore,
-		int k) {
-		printDecompressWarning("rexpandOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.rexpandOperations(ret, max, rows, cast, ignore, k);
-	}
-
-	@Override
-	public MatrixBlock replaceOperations(MatrixValue result, double pattern, double replacement) {
-		printDecompressWarning("replaceOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		return tmp.replaceOperations(result, pattern, replacement);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, double scalar, MatrixValue that, CTableMap resultMap,
-		MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(that);
-		left.ctableOperations(op, scalar, right, resultMap, resultBlock);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, double scalar, double scalar2, CTableMap resultMap,
-		MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		tmp.ctableOperations(op, scalar, scalar2, resultMap, resultBlock);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, MatrixIndexes ix1, double scalar, boolean left, int brlen,
-		CTableMap resultMap, MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock tmp = isCompressed() ? decompress() : this;
-		tmp.ctableOperations(op, ix1, scalar, left, brlen, resultMap, resultBlock);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, MatrixValue that, double scalar, boolean ignoreZeros, CTableMap resultMap,
-		MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right = getUncompressed(that);
-		left.ctableOperations(op, right, scalar, ignoreZeros, resultMap, resultBlock);
-	}
-
-	@Override
-	public MatrixBlock ctableSeqOperations(MatrixValue that, double scalar, MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock right = getUncompressed(that);
-		return this.ctableSeqOperations(right, scalar, resultBlock);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, MatrixValue that, MatrixValue that2, CTableMap resultMap) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(that);
-		MatrixBlock right2 = getUncompressed(that2);
-		left.ctableOperations(op, right1, right2, resultMap);
-	}
-
-	@Override
-	public void ctableOperations(Operator op, MatrixValue that, MatrixValue that2, CTableMap resultMap,
-		MatrixBlock resultBlock) {
-		printDecompressWarning("ctableOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(that);
-		MatrixBlock right2 = getUncompressed(that2);
-		left.ctableOperations(op, right1, right2, resultMap, resultBlock);
-	}
-
-	@Override
-	public MatrixBlock ternaryOperations(TernaryOperator op, MatrixBlock m2, MatrixBlock m3, MatrixBlock ret) {
-		printDecompressWarning("ternaryOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(m2);
-		MatrixBlock right2 = getUncompressed(m3);
-		return left.ternaryOperations(op, right1, right2, ret);
-	}
-
-	@Override
-	public MatrixBlock quaternaryOperations(QuaternaryOperator qop, MatrixBlock um, MatrixBlock vm, MatrixBlock wm,
-		MatrixBlock out) {
-		return quaternaryOperations(qop, um, vm, wm, out, 1);
-	}
-
-	@Override
-	public MatrixBlock quaternaryOperations(QuaternaryOperator qop, MatrixBlock um, MatrixBlock vm, MatrixBlock wm,
-		MatrixBlock out, int k) {
-		printDecompressWarning("quaternaryOperations");
-		MatrixBlock left = isCompressed() ? decompress() : this;
-		MatrixBlock right1 = getUncompressed(um);
-		MatrixBlock right2 = getUncompressed(vm);
-		MatrixBlock right3 = getUncompressed(wm);
-		return left.quaternaryOperations(qop, right1, right2, right3, out, k);
-	}
-
-	@Override
-	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed) {
-		throw new DMLRuntimeException("CompressedMatrixBlock: randOperationsInPlace not supported.");
-	}
-
-	@Override
-	public MatrixBlock randOperationsInPlace(RandomMatrixGenerator rgen, Well1024a bigrand, long bSeed, int k) {
-		throw new DMLRuntimeException("CompressedMatrixBlock: randOperationsInPlace not supported.");
-	}
-
-	@Override
-	public MatrixBlock seqOperationsInPlace(double from, double to, double incr) {
-		// output should always be uncompressed
-		throw new DMLRuntimeException("CompressedMatrixBlock: seqOperationsInPlace not supported.");
-	}
-
-	private static boolean isCompressed(MatrixBlock mb) {
-		return(mb instanceof CompressedMatrixBlock && ((CompressedMatrixBlock) mb).isCompressed());
-	}
-
-	private static MatrixBlock getUncompressed(MatrixValue mVal) {
-		return isCompressed((MatrixBlock) mVal) ? ((CompressedMatrixBlock) mVal).decompress() : (MatrixBlock) mVal;
-	}
-
-	private void printDecompressWarning(String operation) {
-		if(isCompressed()) {
-			LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
-		}
-	}
-
-	private void printDecompressWarning(String operation, MatrixBlock m2) {
-		if(isCompressed() || isCompressed(m2)) {
-			LOG.warn("Operation '" + operation + "' not supported yet - decompressing for ULA operations.");
-		}
-	}
-
-	private static HashSet<Integer> seq(int from, int to, int incr) {
-		HashSet<Integer> ret = new HashSet<>();
-		for(int i = from; i <= to; i += incr)
-			ret.add(i);
-		return ret;
-	}
-
 }
\ No newline at end of file
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
new file mode 100644
index 0000000..9ee52ed
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlockFactory.java
@@ -0,0 +1,258 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupDDC1;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupFactory;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorFactory;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysds.runtime.controlprogram.parfor.stat.Timing;
+import org.apache.sysds.runtime.matrix.data.LibMatrixReorg;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+
+/**
+ * Factory pattern to construct a CompressedMatrixBlock.
+ */
+public class CompressedMatrixBlockFactory {
+	// local debug flag
+	private static final boolean LOCAL_DEBUG = false;
+
+	// DEBUG/TRACE for details
+	private static final Level LOCAL_DEBUG_LEVEL = Level.DEBUG;
+
+	static {
+		// for internal debugging only
+		if(LOCAL_DEBUG) {
+			Logger.getLogger("org.apache.sysds.runtime.compress").setLevel(LOCAL_DEBUG_LEVEL);
+		}
+	}
+
+	private static final Log LOG = LogFactory.getLog(CompressedMatrixBlockFactory.class.getName());
+	private static final CompressionSettings defaultCompressionSettings = new CompressionSettingsBuilder().create();
+
+	public static MatrixBlock compress(MatrixBlock mb) {
+		// Default sequential execution of compression
+		return compress(mb, 1, defaultCompressionSettings);
+	}
+
+	public static MatrixBlock compress(MatrixBlock mb, CompressionSettings customSettings) {
+		return compress(mb, 1, customSettings);
+	}
+
+	public static MatrixBlock compress(MatrixBlock mb, int k) {
+		return compress(mb, k, defaultCompressionSettings);
+	}
+
+	/**
+	 * The main method for compressing the input matrix.
+	 * 
+	 * SAMPLE-BASED DECISIONS: Decisions such as testing if a column is amenable to bitmap compression or evaluating
+	 * co-coding potentials are made based on a subset of the rows. For large data sets, sampling might take a
+	 * significant amount of time. So, we generate only one sample and use it for the entire compression process.
+	 * 
+	 * Once the compression plan is selected based on sampling, the plan is verified and decisions are overwritten by
+	 * full estimates.
+	 * 
+	 * @param mb           The matrix block to compress
+	 * @param k            The number of threads used to execute the compression
+	 * @param compSettings The Compression settings used
+	 * @return A compressed matrix block.
+	 */
+	public static MatrixBlock compress(MatrixBlock mb, int k, CompressionSettings compSettings) {
+		// Check for redundant compression
+		if(mb instanceof CompressedMatrixBlock && ((CompressedMatrixBlock) mb).isCompressed()) {
+			throw new DMLRuntimeException("Redundant compression, block already compressed.");
+		}
+
+		Timing time = new Timing(true);
+		CompressionStatistics _stats = new CompressionStatistics();
+
+		// Prepare basic meta data and deep copy / transpose input
+		int numRows = mb.getNumRows();
+		int numCols = mb.getNumColumns();
+		boolean sparse = mb.isInSparseFormat();
+
+		// Transpose the MatrixBlock if the TransposeInput flag is set.
+		// This gives better cache consciousness, at a small upfront cost.
+		MatrixBlock rawBlock = !compSettings.transposeInput ? new MatrixBlock(mb) : LibMatrixReorg
+			.transpose(mb, new MatrixBlock(numCols, numRows, sparse), k);
+
+		// Construct sample-based size estimator
+		CompressedSizeEstimator sizeEstimator = CompressedSizeEstimatorFactory.getSizeEstimator(rawBlock, compSettings);
+
+		// --------------------------------------------------
+		// PHASE 1: Classify columns by compression type
+		// Start by determining which columns are amenable to compression
+
+		// Classify columns according to ratio (size uncompressed / size compressed),
+		// where a column is compressible if ratio > 1.
+
+		CompressedSizeInfo sizeInfos = sizeEstimator.computeCompressedSizeInfos(k);
+
+		if(compSettings.investigateEstimate)
+			_stats.estimatedSizeCols = sizeInfos.memoryEstimate();
+
+		_stats.setNextTimePhase(time.stop());
+		LOG.debug("Compression statistics:");
+		LOG.debug("--compression phase 1: " + _stats.getLastTimePhase());
+
+		if(sizeInfos.colsC.isEmpty()) {
+			LOG.warn("Abort block compression because all columns are incompressible.");
+			return new MatrixBlock().copyShallow(mb);
+		}
+		// --------------------------------------------------
+
+		// --------------------------------------------------
+		// PHASE 2: Grouping columns
+		// Divide the columns into column groups.
+		List<int[]> coCodeColGroups = PlanningCoCoder.findCocodesByPartitioning(sizeEstimator, sizeInfos, numRows, k);
+		_stats.setNextTimePhase(time.stop());
+		LOG.debug("--compression phase 2: " + _stats.getLastTimePhase());
+
+		// TODO: Make second estimate of memory usage if the ColGroups are as above?
+		// This should already be done inside the PlanningCoCoder, and therefore this information
+		// should be returned there, and not estimated twice.
+		// if(INVESTIGATE_ESTIMATES) {
+		// _stats.estimatedSizeColGroups = memoryEstimateIfColsAre(coCodeColGroups);
+		// }
+		// --------------------------------------------------
+
+		// --------------------------------------------------
+		// PHASE 3: Compress and correct sample-based decisions
+		ColGroup[] colGroups = ColGroupFactory
+			.compressColGroups(rawBlock, sizeInfos.compRatios, coCodeColGroups, compSettings, k);
+
+		// Make Compression happen!
+		CompressedMatrixBlock res = new CompressedMatrixBlock(mb);
+		List<ColGroup> colGroupList = ColGroupFactory.assignColumns(numCols, colGroups, rawBlock, compSettings);
+		res.allocateColGroupList(colGroupList);
+		_stats.setNextTimePhase(time.stop());
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("--compression phase 3: " + _stats.getLastTimePhase());
+		}
+		// --------------------------------------------------
+
+		// --------------------------------------------------
+		// PHASE 4: Best-effort dictionary sharing for DDC1 single-col groups
+		// TODO FIX DDC Sharing
+		double[] dict = (!(compSettings.validCompressions.contains(CompressionType.DDC)) ||
+			!(compSettings.allowSharedDDCDictionary)) ? null : createSharedDDC1Dictionary(colGroupList);
+		if(dict != null) {
+			applySharedDDC1Dictionary(colGroupList, dict);
+			res._sharedDDC1Dict = true;
+		}
+		_stats.setNextTimePhase(time.stop());
+		if(LOG.isDebugEnabled()) {
+			LOG.debug("--compression phase 4: " + _stats.getLastTimePhase());
+		}
+		// --------------------------------------------------
+
+		// --------------------------------------------------
+		// Phase 5: Cleanup
+		// The remaining columns are stored uncompressed as one big column group
+		_stats.size = res.estimateCompressedSizeInMemory();
+		_stats.originalSize = mb.estimateSizeInMemory();
+		_stats.ratio = _stats.originalSize / (double) _stats.size;
+
+		if(_stats.ratio < 1) {
+			LOG.warn("Abort block compression because compression ratio is less than 1.");
+			return new MatrixBlock().copyShallow(mb);
+		}
+
+		// Final cleanup (discard uncompressed block)
+		rawBlock.cleanupBlock(true, true);
+		res.cleanupBlock(true, true);
+
+		_stats.setNextTimePhase(time.stop());
+		_stats.setColGroupsCounts(colGroupList);
+
+		LOG.info("--num col groups: " + colGroupList.size() + ", -- num input cols: " + numCols);
+		LOG.debug("--compression phase 5: " + _stats.getLastTimePhase());
+		LOG.debug("--col groups types " + _stats.getGroupsTypesString());
+		LOG.debug("--col groups sizes " + _stats.getGroupsSizesString());
+		LOG.debug("--compressed size: " + _stats.size);
+		LOG.debug("--compression ratio: " + _stats.ratio);
+
+		// Set the statistics object.
+		// For better compression ratios this could be removed, since it is around 64 Bytes.
+		res._stats = _stats;
+
+		return res;
+		// --------------------------------------------------
+	}
+
+
+	/**
+	 * Dictionary sharing between DDC ColGroups.
+	 * 
+	 * FYI DOES NOT WORK FOR ALL CASES!
+	 * @param colGroups The List of all ColGroups.
+	 * @return the shared value list for the DDC ColGroups.
+	 */
+	private static double[] createSharedDDC1Dictionary(List<ColGroup> colGroups) {
+		// create joint dictionary
+		HashSet<Double> tmp = new HashSet<>();
+		int numQual = 0;
+		for(final ColGroup grp : colGroups)
+			if(grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
+				final ColGroupDDC1 grpDDC1 = (ColGroupDDC1) grp;
+				for(final double val : grpDDC1.getValues())
+					tmp.add(val);
+				numQual++;
+			}
+
+		// abort shared dictionary creation if empty or too large
+		int maxSize = tmp.contains(0d) ? 256 : 255;
+		if(tmp.isEmpty() || tmp.size() > maxSize || numQual < 2)
+			return null;
+		LOG.debug("Created shared directionary for " + numQual + " DDC1 single column groups.");
+
+		// build consolidated dictionary
+		return tmp.stream().mapToDouble(Double::doubleValue).toArray();
+	}
+
+	private static void applySharedDDC1Dictionary(List<ColGroup> colGroups, double[] dict) {
+		// create joint mapping table
+		HashMap<Double, Integer> map = new HashMap<>();
+		for(int i = 0; i < dict.length; i++)
+			map.put(dict[i], i);
+
+		// recode data of all relevant DDC1 groups
+		for(ColGroup grp : colGroups)
+			if(grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
+				ColGroupDDC1 grpDDC1 = (ColGroupDDC1) grp;
+				grpDDC1.recodeData(map);
+				grpDDC1.setValues(dict);
+			}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
new file mode 100644
index 0000000..892027f
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettings.java
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.util.List;
+
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+
+/**
+ * Compression Settings class, used as a bundle of parameters inside the Compression framework.
+ * See CompressionSettingsBuilder for default non static parameters.
+ */
+public class CompressionSettings {
+
+	// Sorting of values by physical length helps by 10-20%, especially for serial, while
+	// slight performance decrease for parallel incl multi-threaded, hence not applied for
+	// distributed operations (also because compression time + garbage collection increases)
+	public static final boolean SORT_VALUES_BY_LENGTH = true;
+
+	// The sampling ratio used when choosing ColGroups.
+	// Note that, default behavior is to use exact estimator if the number of elements is below 1000.
+	public final double samplingRatio;
+
+	// Share DDC Dictionaries between ColGroups.
+	// TODO FIX DDC Dictionarie sharing.
+	public final boolean allowSharedDDCDictionary;
+
+	// Transpose input matrix, to optimize performance, this reallocate the matrix to a more cache conscious allocation
+	// for iteration in columns.
+	public final boolean transposeInput;
+
+	// If the seed is -1 then the system used system millisecond time and class hash for seeding.
+	public final int seed;
+
+	// Investigate the estimate.
+	public final boolean investigateEstimate;
+
+	// Removed the option of LOW_LEVEL_OPT, (only effecting OLE and RLE.)
+	// public final boolean LOW_LEVEL_OPT;
+
+	// Valid Compressions List, containing the ColGroup CompressionTypes that are allowed to be used for the compression
+	// Default is to always allow for Uncompromisable ColGroup.
+	public final List<CompressionType> validCompressions;
+
+	protected CompressionSettings(double samplingRatio, boolean allowSharedDDCDictionary, boolean transposeInput,
+		int seed, boolean investigateEstimate, List<CompressionType> validCompressions) {
+		this.samplingRatio = samplingRatio;
+		this.allowSharedDDCDictionary = allowSharedDDCDictionary;
+		this.transposeInput = transposeInput;
+		this.seed = seed;
+		this.investigateEstimate = investigateEstimate;
+		this.validCompressions = validCompressions;
+	}
+
+
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append("\n" + super.toString());
+		sb.append("\n Valid Compressions: " + validCompressions);
+		sb.append("\n DDC1 share dict: " + allowSharedDDCDictionary);
+		// 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
new file mode 100644
index 0000000..9285249
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionSettingsBuilder.java
@@ -0,0 +1,93 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+
+/**
+ * Builder pattern for Compression Settings.
+ * See CompressionSettings for details on values.
+ */
+public class CompressionSettingsBuilder {
+	private double samplingRatio = 0.05;
+	private boolean allowSharedDDCDictionary = true;
+	private boolean transposeInput = true;
+	private int seed = -1;
+	private boolean investigateEstimate = false;
+	private List<CompressionType> validCompressions = new ArrayList<>();
+
+	public CompressionSettingsBuilder() {
+		validCompressions.add(CompressionType.DDC);
+		validCompressions.add(CompressionType.OLE);
+		validCompressions.add(CompressionType.RLE);
+		validCompressions.add(CompressionType.UNCOMPRESSED);
+	}
+	
+	public CompressionSettingsBuilder copySettings(CompressionSettings that){
+		this.samplingRatio = that.samplingRatio;
+		this.allowSharedDDCDictionary = that.allowSharedDDCDictionary;
+		this.transposeInput = that.transposeInput;
+		this.seed = that.seed;
+		this.investigateEstimate = that.investigateEstimate;
+		this.validCompressions = new ArrayList<>(that.validCompressions);
+		return this;
+	}
+
+	public CompressionSettingsBuilder setSamplingRatio(double samplingRatio) {
+		this.samplingRatio = samplingRatio;
+		return this;
+	}
+
+	public CompressionSettingsBuilder setAllowSharedDDCDictionary(boolean allowSharedDDCDictionary) {
+		this.allowSharedDDCDictionary = allowSharedDDCDictionary;
+		return this;
+	}
+
+	public CompressionSettingsBuilder setTransposeInput(boolean transposeInput) {
+		this.transposeInput = transposeInput;
+		return this;
+	}
+
+	public CompressionSettingsBuilder setSeed(int seed) {
+		this.seed = seed;
+		return this;
+	}
+
+	public CompressionSettingsBuilder setInvestigateEstimate(boolean investigateEstimate) {
+		this.investigateEstimate = investigateEstimate;
+		return this;
+	}
+
+	public CompressionSettingsBuilder setValidCompressions(List<CompressionType> validCompressions) {
+		// should always contain Uncompressed as an option.
+		if(!validCompressions.contains(CompressionType.UNCOMPRESSED))
+			validCompressions.add(CompressionType.UNCOMPRESSED);
+		this.validCompressions = validCompressions;
+		return this;
+	}
+
+	public CompressionSettings create() {
+		return new CompressionSettings(samplingRatio, allowSharedDDCDictionary, transposeInput, seed,
+			investigateEstimate, validCompressions);
+	}
+}
\ No newline at end of file
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 394b359..aa831d7 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressionStatistics.java
@@ -19,25 +19,113 @@
 
 package org.apache.sysds.runtime.compress;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+
 public class CompressionStatistics {
-	public double timePhase1 = -1;
-	public double timePhase2 = -1;
-	public double timePhase3 = -1;
-	public double timePhase4 = -1;
-	public double timePhase5 = -1;
-	public double estSize = -1;
-	public double size = -1;
-	public double ratio = -1;
+
+	private ArrayList<Double> timePhases = new ArrayList<>();
+	public double ratio;
+	public long originalSize;
+	public long estimatedSizeColGroups;
+	public long estimatedSizeCols;
+	public long size;
+
+	private Map<CompressionType, int[]> colGroupCounts;
 
 	public CompressionStatistics() {
-		// do nothing
 	}
 
-	public CompressionStatistics(double t1, double t2, double t3, double t4, double t5) {
-		timePhase1 = t1;
-		timePhase2 = t2;
-		timePhase3 = t3;
-		timePhase4 = t4;
-		timePhase5 = t5;
+	public void setNextTimePhase(double time) {
+		timePhases.add(time);
+	}
+
+	public double getLastTimePhase() {
+		return timePhases.get(timePhases.size() - 1);
+	}
+
+	/**
+	 * Set array of counts regarding col group types. 
+	 * 
+	 * The position corresponds with the enum ordinal.
+	 * 
+	 * @param colGroups list of ColGroups used in compression.
+	 */
+	public void setColGroupsCounts(List<ColGroup> colGroups) {
+		HashMap<CompressionType, int[]> ret = new HashMap<>();
+		for(ColGroup c : colGroups) {
+			CompressionType ct = c.getCompType();
+			int colCount = c.getNumCols();
+			int[] values;
+			if(ret.containsKey(ct)) {
+				values = ret.get(ct);
+				values[0] += 1;
+				values[1] += colCount;
+			}
+			else {
+				values = new int[] {1, colCount};
+			}
+			ret.put(ct, values);
+		}
+		this.colGroupCounts = ret;
+	}
+
+	public Map<CompressionType, int[]> getColGroups() {
+		return colGroupCounts;
+	}
+
+	public ArrayList<Double> getTimeArrayList() {
+		return timePhases;
+	}
+
+	public String getGroupsTypesString() {
+		StringBuilder sb = new StringBuilder();
+
+		for(CompressionType ctKey : colGroupCounts.keySet()) {
+			sb.append(ctKey + ":" + colGroupCounts.get(ctKey)[0] + " ");
+		}
+		return sb.toString();
+	}
+
+	public String getGroupsSizesString() {
+		StringBuilder sb = new StringBuilder();
+		for(CompressionType ctKey : colGroupCounts.keySet()) {
+
+			sb.append(ctKey + ":" + colGroupCounts.get(ctKey)[1] + " ");
+		}
+		return sb.toString();
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append("Compression Statistics:\n");
+		sb.append("\t" + getGroupsTypesString() + "\n");
+		sb.append("\t" + getGroupsSizesString() + "\n");
+		return sb.toString();
+	}
+
+	public static long getSizeInMemory() {
+		long total = 16; // header
+		total += 8; // compression ratio
+		total += 8; // original size
+		total += 8; // estimated size col groups
+		total += 8; // estimated size cols
+		total += 8; // actual size
+
+		total += 8; // Array list Time phases
+		total += 8; // Map colGroup Counts
+
+		// TODO what happens if we scale number of col Groups...
+		// TODO Reduce memory usage for compression statistics.
+		total += 64; // HashMap col Groups.
+		total += 40; // ArrayList time phases
+
+		return total;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java
index c809851..7f064ac 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java
@@ -23,22 +23,23 @@ import org.apache.sysds.runtime.compress.utils.DblArray;
 
 /**
  * Base class for all column selection readers.
- * 
  */
-public abstract class ReaderColumnSelection 
-{
+public abstract class ReaderColumnSelection {
 	protected int[] _colIndexes = null;
 	protected int _numRows = -1;
 	protected int _lastRow = -1;
 	protected boolean _skipZeros = false;
-	
-	protected ReaderColumnSelection(int[] colIndexes, int numRows, boolean skipZeros) {
+
+	protected CompressionSettings _compSettings;
+
+	protected ReaderColumnSelection(int[] colIndexes, int numRows, boolean skipZeros, CompressionSettings compSettings) {
 		_colIndexes = colIndexes;
 		_numRows = numRows;
 		_lastRow = -1;
 		_skipZeros = skipZeros;
+		_compSettings = compSettings;
 	}
-	
+
 	/**
 	 * Gets the next row, null when no more rows.
 	 * 
@@ -49,7 +50,6 @@ public abstract class ReaderColumnSelection
 	public int getCurrentRowIndex() {
 		return _lastRow;
 	}
-	
 
 	/**
 	 * Resets the reader to the first row.
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDense.java b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDense.java
index ec3df7d..76ef66f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDense.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDense.java
@@ -22,18 +22,16 @@ package org.apache.sysds.runtime.compress;
 import org.apache.sysds.runtime.compress.utils.DblArray;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
-public class ReaderColumnSelectionDense extends ReaderColumnSelection 
-{
+public class ReaderColumnSelectionDense extends ReaderColumnSelection {
 	protected MatrixBlock _data;
-	
+
 	// reusable return
 	private DblArray nonZeroReturn;
 	private DblArray reusableReturn;
 	private double[] reusableArr;
 
-	public ReaderColumnSelectionDense(MatrixBlock data, int[] colIndices, boolean skipZeros) {
-		super(colIndices, CompressedMatrixBlock.TRANSPOSE_INPUT ? 
-				data.getNumColumns() : data.getNumRows(), skipZeros);
+	public ReaderColumnSelectionDense(MatrixBlock data, int[] colIndices, boolean skipZeros, CompressionSettings compSettings) {
+		super(colIndices, compSettings.transposeInput ? data.getNumColumns() : data.getNumRows(), skipZeros, compSettings);
 		_data = data;
 		reusableArr = new double[colIndices.length];
 		reusableReturn = new DblArray(reusableArr);
@@ -41,23 +39,23 @@ public class ReaderColumnSelectionDense extends ReaderColumnSelection
 
 	@Override
 	public DblArray nextRow() {
-		if( _skipZeros) {
-			while ((nonZeroReturn = getNextRow()) != null
-				&& DblArray.isZero(nonZeroReturn)){} 
+		if(_skipZeros) {
+			while((nonZeroReturn = getNextRow()) != null && DblArray.isZero(nonZeroReturn)) {
+			}
 			return nonZeroReturn;
-		} else {
+		}
+		else {
 			return getNextRow();
 		}
 	}
 
 	private DblArray getNextRow() {
-		if(_lastRow == _numRows-1)
+		if(_lastRow == _numRows - 1)
 			return null;
 		_lastRow++;
-		for (int i = 0; i < _colIndexes.length; i++) {
-			reusableArr[i] = CompressedMatrixBlock.TRANSPOSE_INPUT ? 
-					_data.quickGetValue( _colIndexes[i], _lastRow ) : 
-					_data.quickGetValue( _lastRow, _colIndexes[i] );
+		for(int i = 0; i < _colIndexes.length; i++) {
+			reusableArr[i] = _compSettings.transposeInput ? _data.quickGetValue(_colIndexes[i],
+				_lastRow) : _data.quickGetValue(_lastRow, _colIndexes[i]);
 		}
 		return reusableReturn;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDenseSample.java b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDenseSample.java
index 2f0dcc8..7fd4b72 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDenseSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionDenseSample.java
@@ -26,10 +26,9 @@ import org.apache.sysds.runtime.matrix.data.MatrixBlock;
  * 
  * considers only a subset of row indexes
  */
-public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection 
-{
+public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection {
 	protected MatrixBlock _data;
-	
+
 	private int[] _sampleIndexes;
 	private int lastIndex = -1;
 
@@ -38,9 +37,9 @@ public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection
 	private DblArray reusableReturn;
 	private double[] reusableArr;
 
-	public ReaderColumnSelectionDenseSample(MatrixBlock data, int[] colIndexes, int[] sampleIndexes, boolean skipZeros) 
-	{
-		super(colIndexes, -1, skipZeros);
+	public ReaderColumnSelectionDenseSample(MatrixBlock data, int[] colIndexes, int[] sampleIndexes,
+		boolean skipZeros, CompressionSettings compSettings) {
+		super(colIndexes, -1, skipZeros, compSettings);
 		_data = data;
 		_sampleIndexes = sampleIndexes;
 		reusableArr = new double[colIndexes.length];
@@ -49,23 +48,23 @@ public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection
 
 	@Override
 	public DblArray nextRow() {
-		if (_skipZeros) {
-			while ((nonZeroReturn = getNextRow()) != null
-					&& DblArray.isZero(nonZeroReturn)){}
+		if(_skipZeros) {
+			while((nonZeroReturn = getNextRow()) != null && DblArray.isZero(nonZeroReturn)) {
+			}
 			return nonZeroReturn;
-		} else {
+		}
+		else {
 			return getNextRow();
 		}
 	}
 
 	private DblArray getNextRow() {
-		if (lastIndex == _sampleIndexes.length - 1)
+		if(lastIndex == _sampleIndexes.length - 1)
 			return null;
 		lastIndex++;
-		for (int i = 0; i < _colIndexes.length; i++) {
-			reusableArr[i] = CompressedMatrixBlock.TRANSPOSE_INPUT ? 
-					_data.quickGetValue(_colIndexes[i], _sampleIndexes[lastIndex]) :
-					_data.quickGetValue(_sampleIndexes[lastIndex], _colIndexes[i]);
+		for(int i = 0; i < _colIndexes.length; i++) {
+			reusableArr[i] = _compSettings.transposeInput ? _data.quickGetValue(_colIndexes[i],
+				_sampleIndexes[lastIndex]) : _data.quickGetValue(_sampleIndexes[lastIndex], _colIndexes[i]);
 		}
 		return reusableReturn;
 	}
@@ -74,7 +73,7 @@ public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection
 	public int getCurrentRowIndex() {
 		return _sampleIndexes[lastIndex];
 	}
-	
+
 	@Override
 	public void reset() {
 		lastIndex = -1;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionSparse.java b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionSparse.java
index e157617..abdb723 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionSparse.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelectionSparse.java
@@ -26,15 +26,12 @@ import org.apache.sysds.runtime.data.SparseRow;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 /**
- * Used to extract the values at certain indexes from each row in a sparse
- * matrix
+ * Used to extract the values at certain indexes from each row in a sparse matrix
  * 
- * Keeps returning all-zeros arrays until reaching the last possible index. The
- * current compression algorithm treats the zero-value in a sparse matrix like
- * any other value.
+ * Keeps returning all-zeros arrays until reaching the last possible index. The current compression algorithm treats the
+ * zero-value in a sparse matrix like any other value.
  */
-public class ReaderColumnSelectionSparse extends ReaderColumnSelection 
-{
+public class ReaderColumnSelectionSparse extends ReaderColumnSelection {
 	private final DblArray ZERO_DBL_ARRAY;
 	private DblArray nonZeroReturn;
 
@@ -45,62 +42,58 @@ public class ReaderColumnSelectionSparse extends ReaderColumnSelection
 	// current sparse row positions
 	private SparseRow[] sparseCols = null;
 	private int[] sparsePos = null;
-	
-	public ReaderColumnSelectionSparse(MatrixBlock data, int[] colIndexes, boolean skipZeros) 
-	{
-		super(colIndexes, CompressedMatrixBlock.TRANSPOSE_INPUT ? 
-				data.getNumColumns() : data.getNumRows(), skipZeros);
+
+	public ReaderColumnSelectionSparse(MatrixBlock data, int[] colIndexes, boolean skipZeros, CompressionSettings compSettings) {
+		super(colIndexes, compSettings.transposeInput ? data.getNumColumns() : data.getNumRows(), skipZeros, compSettings);
 		ZERO_DBL_ARRAY = new DblArray(new double[colIndexes.length], true);
 		reusableArr = new double[colIndexes.length];
 		reusableReturn = new DblArray(reusableArr);
-		
-		if( !CompressedMatrixBlock.TRANSPOSE_INPUT ){
+
+		if(!_compSettings.transposeInput) {
 			throw new RuntimeException("SparseColumnSelectionReader should not be used without transposed input.");
 		}
-		
+
 		sparseCols = new SparseRow[colIndexes.length];
 		sparsePos = new int[colIndexes.length];
-		if( data.getSparseBlock()!=null )
-		for( int i=0; i<colIndexes.length; i++ )
-			sparseCols[i] = data.getSparseBlock().get(colIndexes[i]);
+		if(data.getSparseBlock() != null)
+			for(int i = 0; i < colIndexes.length; i++)
+				sparseCols[i] = data.getSparseBlock().get(colIndexes[i]);
 	}
 
 	@Override
 	public DblArray nextRow() {
 		if(_skipZeros) {
-			while ((nonZeroReturn = getNextRow()) != null
-				&& nonZeroReturn == ZERO_DBL_ARRAY){}
+			while((nonZeroReturn = getNextRow()) != null && nonZeroReturn == ZERO_DBL_ARRAY) {
+			}
 			return nonZeroReturn;
-		} else {
+		}
+		else {
 			return getNextRow();
 		}
 	}
 
-	private DblArray getNextRow() 
-	{
-		if(_lastRow == _numRows-1)
+	private DblArray getNextRow() {
+		if(_lastRow == _numRows - 1)
 			return null;
 		_lastRow++;
-		
-		if( !CompressedMatrixBlock.TRANSPOSE_INPUT ){
+
+		if(!_compSettings.transposeInput) {
 			throw new RuntimeException("SparseColumnSelectionReader should not be used without transposed input.");
 		}
-		
-		//move pos to current row if necessary (for all columns)
-		for( int i=0; i<_colIndexes.length; i++ )
-			if( sparseCols[i] != null && (sparseCols[i].indexes().length<=sparsePos[i] 
-				|| sparseCols[i].indexes()[sparsePos[i]]<_lastRow) )
-			{
+
+		// move pos to current row if necessary (for all columns)
+		for(int i = 0; i < _colIndexes.length; i++)
+			if(sparseCols[i] != null &&
+				(sparseCols[i].indexes().length <= sparsePos[i] || sparseCols[i].indexes()[sparsePos[i]] < _lastRow)) {
 				sparsePos[i]++;
 			}
-		
-		//extract current values
+
+		// extract current values
 		Arrays.fill(reusableArr, 0);
 		boolean zeroResult = true;
-		for( int i=0; i<_colIndexes.length; i++ ) 
-			if( sparseCols[i] != null && sparseCols[i].indexes().length>sparsePos[i]
-				&&sparseCols[i].indexes()[sparsePos[i]]==_lastRow )
-			{
+		for(int i = 0; i < _colIndexes.length; i++)
+			if(sparseCols[i] != null && sparseCols[i].indexes().length > sparsePos[i] &&
+				sparseCols[i].indexes()[sparsePos[i]] == _lastRow) {
 				reusableArr[i] = sparseCols[i].values()[sparsePos[i]];
 				zeroResult = false;
 			}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/UncompressedBitmap.java b/src/main/java/org/apache/sysds/runtime/compress/UncompressedBitmap.java
index 5292aaf..0b1aa8f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/UncompressedBitmap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/UncompressedBitmap.java
@@ -29,37 +29,34 @@ import org.apache.sysds.runtime.compress.utils.DoubleIntListHashMap.DIListEntry;
 import org.apache.sysds.runtime.compress.utils.IntArrayList;
 import org.apache.sysds.runtime.util.SortUtils;
 
-/** 
- * Uncompressed representation of one or more columns in bitmap format. 
- * 
+/**
+ * Uncompressed representation of one or more columns in bitmap format.
  */
-public final class UncompressedBitmap 
-{
+public final class UncompressedBitmap {
+	
 	private final int _numCols;
 
-	/** Distinct values that appear in the column. Linearized as value groups <v11 v12> <v21 v22>.*/
+	/** Distinct values that appear in the column. Linearized as value groups <v11 v12> <v21 v22>. */
 	private double[] _values;
 
 	/** Bitmaps (as lists of offsets) for each of the values. */
 	private IntArrayList[] _offsetsLists;
 
-	public UncompressedBitmap( DblArrayIntListHashMap distinctVals, int numColumns ) 
-	{
+	public UncompressedBitmap(DblArrayIntListHashMap distinctVals, int numColumns) {
 		// added for one pass bitmap construction
 		// Convert inputs to arrays
 		int numVals = distinctVals.size();
-		_values = new double[numVals*numColumns];
+		_values = new double[numVals * numColumns];
 		_offsetsLists = new IntArrayList[numVals];
 		int bitmapIx = 0;
-		for( DArrayIListEntry val : distinctVals.extractValues()) {
-			System.arraycopy(val.key.getData(), 0, _values, bitmapIx*numColumns, numColumns);
+		for(DArrayIListEntry val : distinctVals.extractValues()) {
+			System.arraycopy(val.key.getData(), 0, _values, bitmapIx * numColumns, numColumns);
 			_offsetsLists[bitmapIx++] = val.value;
 		}
 		_numCols = numColumns;
 	}
 
-	public UncompressedBitmap( DoubleIntListHashMap distinctVals ) 
-	{
+	public UncompressedBitmap(DoubleIntListHashMap distinctVals) {
 		// added for one pass bitmap construction
 		// Convert inputs to arrays
 		int numVals = distinctVals.size();
@@ -72,12 +69,12 @@ public final class UncompressedBitmap
 		}
 		_numCols = 1;
 	}
-	
+
 	public int getNumColumns() {
 		return _numCols;
 	}
-	
-	/** 
+
+	/**
 	 * Get all values without unnecessary allocations and copies.
 	 * 
 	 * @return dictionary of value tuples
@@ -85,22 +82,22 @@ public final class UncompressedBitmap
 	public double[] getValues() {
 		return _values;
 	}
-	
+
 	/**
 	 * Obtain tuple of column values associated with index.
 	 * 
-	 * @param ix   index of a particular distinct value
+	 * @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);
+		return Arrays.copyOfRange(_values, ix * _numCols, (ix + 1) * _numCols);
 	}
 
 	/**
 	 * Obtain number of distinct values in the column.
 	 * 
-	 * @return number of distinct values in the column; this number is also the
-	 *         number of bitmaps, since there is one bitmap per value
+	 * @return number of distinct values in the column; this number is also the number of bitmaps, since there is one
+	 *         bitmap per value
 	 */
 	public int getNumValues() {
 		return _values.length / _numCols;
@@ -112,37 +109,37 @@ public final class UncompressedBitmap
 
 	public long getNumOffsets() {
 		long ret = 0;
-		for( IntArrayList offlist : _offsetsLists )
+		for(IntArrayList offlist : _offsetsLists)
 			ret += offlist.size();
 		return ret;
 	}
-	
+
 	public int getNumOffsets(int ix) {
 		return _offsetsLists[ix].size();
 	}
-	
+
 	public void sortValuesByFrequency() {
 		int numVals = getNumValues();
 		int numCols = getNumColumns();
-		
+
 		double[] freq = new double[numVals];
 		int[] pos = new int[numVals];
-		
-		//populate the temporary arrays
-		for(int i=0; i<numVals; i++) {
+
+		// populate the temporary arrays
+		for(int i = 0; i < numVals; i++) {
 			freq[i] = getNumOffsets(i);
 			pos[i] = i;
 		}
-		
-		//sort ascending and reverse (descending)
+
+		// sort ascending and reverse (descending)
 		SortUtils.sortByValue(0, numVals, freq, pos);
 		ArrayUtils.reverse(pos);
-		
-		//create new value and offset list arrays
-		double[] lvalues = new double[numVals*numCols];
+
+		// create new value and offset list arrays
+		double[] lvalues = new double[numVals * numCols];
 		IntArrayList[] loffsets = new IntArrayList[numVals];
-		for(int i=0; i<numVals; i++) {
-			System.arraycopy(_values, pos[i]*numCols, lvalues, i*numCols, numCols);
+		for(int i = 0; i < numVals; i++) {
+			System.arraycopy(_values, pos[i] * numCols, lvalues, i * numCols, numCols);
 			loffsets[i] = _offsetsLists[pos[i]];
 		}
 		_values = lvalues;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
index e627734..435a1fb 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerBinPacking.java
@@ -32,7 +32,6 @@ import org.apache.sysds.runtime.util.SortUtils;
 
 /**
  * Column group partitioning with bin packing heuristic.
- * 
  */
 public class ColumnGroupPartitionerBinPacking extends ColumnGroupPartitioner {
 	private static final boolean FIRST_FIT_DEC = true;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerStatic.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
index 5468966..eb5fab6 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/ColumnGroupPartitionerStatic.java
@@ -27,7 +27,6 @@ import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.GroupableColInfo
 
 /**
  * Column group partitioning with static distribution heuristic.
- * 
  */
 public class ColumnGroupPartitionerStatic extends ColumnGroupPartitioner {
 	private static final int MAX_COL_PER_GROUP = 20;
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 91d971d..080bbcc 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
@@ -32,6 +32,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 import org.apache.sysds.runtime.util.CommonThreadPool;
 
 public class PlanningCoCoder {
@@ -44,19 +45,34 @@ public class PlanningCoCoder {
 		BIN_PACKING, STATIC,
 	}
 
-	public static List<int[]> findCocodesByPartitioning(CompressedSizeEstimator sizeEstimator, List<Integer> cols,
-		CompressedSizeInfo[] colInfos, int numRows, int k) {
-		// filtering out non-groupable columns as singleton groups
+	/**
+	 * Main entry point of CoCode.
+	 * 
+	 * This package groups together ColGroups across columns, to improve compression further,
+	 * 
+	 * @param sizeEstimator The size estimator used for estimating ColGroups potential sizes.
+	 * @param colInfos      The information already gathered on the individual ColGroups of columns.
+	 * @param numRows       The number of rows in the input matrix.
+	 * @param k             The concurrency degree allowed for this operation.
+	 * @return The Estimated (hopefully) best groups of ColGroups.
+	 */
+	public static List<int[]> findCocodesByPartitioning(CompressedSizeEstimator sizeEstimator,
+		CompressedSizeInfo colInfos, int numRows, int k) {
+		// filtering out non-group-able columns as singleton groups
 		// weight is the ratio of its cardinality to the number of rows
+
+		List<Integer> cols = colInfos.colsC;
+		CompressedSizeInfoColGroup[] colGroups = colInfos.compressionInfo;
+
 		int numCols = cols.size();
 		List<Integer> groupCols = new ArrayList<>();
 		HashMap<Integer, GroupableColInfo> groupColsInfo = new HashMap<>();
 		for(int i = 0; i < numCols; i++) {
 			int colIx = cols.get(i);
-			double cardinality = colInfos[colIx].getEstCard();
+			double cardinality = colGroups[colIx].getEstCard();
 			double weight = cardinality / numRows;
 			groupCols.add(colIx);
-			groupColsInfo.put(colIx, new GroupableColInfo(weight, colInfos[colIx].getMinSize()));
+			groupColsInfo.put(colIx, new GroupableColInfo(weight, colGroups[colIx].getMinSize()));
 		}
 
 		// use column group partitioner to create partitions of columns
@@ -71,7 +87,7 @@ public class PlanningCoCoder {
 	}
 
 	private static List<int[]> getCocodingGroupsBruteForce(List<int[]> bins,
-		HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator estim, int rlen) {
+		HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator estimator, int rlen) {
 		List<int[]> retGroups = new ArrayList<>();
 		for(int[] bin : bins) {
 			// building an array of singleton CoCodingGroup
@@ -79,7 +95,7 @@ public class PlanningCoCoder {
 			for(int col : bin)
 				sgroups.add(new PlanningCoCodingGroup(col, groupColsInfo.get(col)));
 			// brute force co-coding
-			PlanningCoCodingGroup[] outputGroups = findCocodesBruteForce(estim,
+			PlanningCoCodingGroup[] outputGroups = findCocodesBruteForce(estimator,
 				rlen,
 				sgroups.toArray(new PlanningCoCodingGroup[0]));
 			for(PlanningCoCodingGroup grp : outputGroups)
@@ -90,7 +106,7 @@ public class PlanningCoCoder {
 	}
 
 	private static List<int[]> getCocodingGroupsBruteForce(List<int[]> bins,
-		HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator estim, int rlen, int k) {
+		HashMap<Integer, GroupableColInfo> groupColsInfo, CompressedSizeEstimator estimator, int rlen, int k) {
 		List<int[]> retGroups = new ArrayList<>();
 		try {
 			ExecutorService pool = CommonThreadPool.get(k);
@@ -100,7 +116,7 @@ public class PlanningCoCoder {
 				ArrayList<PlanningCoCodingGroup> sgroups = new ArrayList<>();
 				for(int col : bin)
 					sgroups.add(new PlanningCoCodingGroup(col, groupColsInfo.get(col)));
-				tasks.add(new CocodeTask(estim, sgroups, rlen));
+				tasks.add(new CocodeTask(estimator, sgroups, rlen));
 			}
 			List<Future<PlanningCoCodingGroup[]>> rtask = pool.invokeAll(tasks);
 			for(Future<PlanningCoCodingGroup[]> lrtask : rtask)
@@ -119,12 +135,14 @@ public class PlanningCoCoder {
 	 * Identify columns to code together. Uses a greedy approach that merges pairs of column groups into larger groups.
 	 * Each phase of the greedy algorithm considers all combinations of pairs to merge.
 	 * 
+	 * TODO Find better faster ways of finding cocodes than brute force.
+	 * 
 	 * @param sizeEstimator  compressed size estimator
 	 * @param numRowsWeight  number of rows weight
 	 * @param singltonGroups planning co-coding groups
-	 * @return
+	 * @return A PlanningCoCodingGroup.
 	 */
-	private static PlanningCoCodingGroup[] findCocodesBruteForce(CompressedSizeEstimator estim, int numRows,
+	private static PlanningCoCodingGroup[] findCocodesBruteForce(CompressedSizeEstimator estimator, int numRows,
 		PlanningCoCodingGroup[] singletonGroups) {
 		if(LOG.isTraceEnabled())
 			LOG.trace("Cocoding: process " + singletonGroups.length);
@@ -150,7 +168,7 @@ public class PlanningCoCoder {
 						continue;
 
 					// memoization or newly created group (incl bitmap extraction)
-					PlanningCoCodingGroup c1c2 = memo.getOrCreate(c1, c2, estim, numRows);
+					PlanningCoCodingGroup c1c2 = memo.getOrCreate(c1, c2, estimator, numRows);
 
 					// keep best merged group only
 					if(tmp == null || c1c2.getChangeInSize() < tmp.getChangeInSize() ||
diff --git a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCodingGroup.java b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCodingGroup.java
index 2a9d226..1bf1332 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCodingGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/cocode/PlanningCoCodingGroup.java
@@ -23,11 +23,10 @@ import java.util.Arrays;
 
 import org.apache.sysds.runtime.compress.cocode.PlanningCoCoder.GroupableColInfo;
 import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
-import org.apache.sysds.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
 
 /**
  * Class to represent information about co-coding a group of columns.
- * 
  */
 public class PlanningCoCodingGroup {
 	private int[] _colIndexes;
@@ -62,7 +61,8 @@ public class PlanningCoCodingGroup {
 		_colIndexes = getMergedIndexes(grp1._colIndexes, grp2._colIndexes);
 
 		// estimating size info
-		CompressedSizeInfo groupSizeInfo = estim.estimateCompressedColGroupSize(_colIndexes);
+		CompressedSizeInfoColGroup groupSizeInfo = estim.estimateCompressedColGroupSize(_colIndexes);
+		
 		_estSize = groupSizeInfo.getMinSize();
 		_cardRatio = groupSizeInfo.getEstCard() / numRows;
 
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroup.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroup.java
similarity index 76%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroup.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroup.java
index 3957e23..6d27dff 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroup.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroup.java
@@ -17,15 +17,17 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Iterator;
-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.IJV;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
@@ -34,53 +36,72 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 /**
  * Class that stores information about a column group within a compressed matrix block. There are subclasses specific to
  * each compression type.
- * 
  */
 public abstract class ColGroup implements Serializable {
+	protected static final Log LOG = LogFactory.getLog(ColGroup.class.getName());
 	private static final long serialVersionUID = 2439785418908671481L;
 
+	/**
+	 * Public Group types supported
+	 * 
+	 * Note For instance DDC is called DDC not DDC1, or DDC2 which is a specific subtype of the DDC.
+	 */
 	public enum CompressionType {
 		UNCOMPRESSED, // uncompressed sparse/dense
-		RLE_BITMAP, // RLE bitmap
-		OLE_BITMAP, // OLE bitmap
-		DDC1, // DDC 1 byte
-		DDC2; // DDC 2 byte
+		RLE, // RLE bitmap
+		OLE, // OLE bitmap
+		DDC, // Dictionary encoding
+		// QUANTIZE, // Quantize the double values to int 8.
 	}
 
 	/**
-	 * Offsets of the columns that make up the column group. Zero-based, and relative to the matrix block.
+	 * Concrete ColGroupType
+	 * 
+	 * Protected such that outside the ColGroup package it should be unknown which specific subtype is used.
 	 */
+	protected enum ColGroupType {
+		UNCOMPRESSED, // uncompressed sparse/dense
+		RLE, // RLE bitmap
+		OLE, // OLE bitmap
+		DDC1, DDC2,
+	}
+
+	/** The ColGroup Indexes 0 offset, contained in the ColGroup */
 	protected int[] _colIndexes;
 
+	/** ColGroup Implementation Contains zero values */
+	protected boolean _zeros;
+
 	/** Number of rows in the matrix, for use by child classes. */
 	protected int _numRows;
 
+	/** Empty constructor, used for serializing into an empty new object of ColGroup. */
+	protected ColGroup() {
+		this._colIndexes = null;
+		this._numRows = -1;
+	}
+
 	/**
 	 * Main constructor.
 	 * 
 	 * @param colIndices offsets of the columns in the matrix block that make up the group
-	 * @param numRows    total number of rows in the parent block
+	 * @param numRows    total number of rows in the block
 	 */
 	protected ColGroup(int[] colIndices, int numRows) {
+		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");
+		}
+		if(numRows < 1) {
+			throw new DMLRuntimeException(numRows + " is an invalid number of rows in a ColGroup");
+		}
 		_colIndexes = colIndices;
 		_numRows = numRows;
 	}
 
 	/**
-	 * Convenience constructor for converting indices to a more compact format.
-	 * 
-	 * @param colIndicesList list of column indices
-	 * @param numRows        total number of rows in the parent block
-	 */
-	protected ColGroup(List<Integer> colIndicesList, int numRows) {
-		_colIndexes = new int[colIndicesList.size()];
-		int i = 0;
-		for(Integer index : colIndicesList)
-			_colIndexes[i++] = index;
-		_numRows = numRows;
-	}
-
-	/**
 	 * Obtain the offsets of the columns in the matrix block that make up the group
 	 * 
 	 * @return offsets of the columns in the matrix block that make up the group
@@ -99,6 +120,11 @@ public abstract class ColGroup implements Serializable {
 		return _colIndexes[colNum];
 	}
 
+	/**
+	 * Get number of rows contained in the ColGroup.
+	 * 
+	 * @return An integer that is the number of rows.
+	 */
 	public int getNumRows() {
 		return _numRows;
 	}
@@ -119,6 +145,14 @@ public abstract class ColGroup implements Serializable {
 	 */
 	public abstract CompressionType getCompType();
 
+	/**
+	 * Internally get the specific type of ColGroup, this could be extracted from the object but that does not allow for
+	 * nice switches in the code.
+	 * 
+	 * @return ColGroupType of the object.
+	 */
+	protected abstract ColGroupType getColGroupType();
+
 	public void shiftColIndices(int offset) {
 		for(int i = 0; i < _colIndexes.length; i++)
 			_colIndexes[i] += offset;
@@ -130,11 +164,7 @@ public abstract class ColGroup implements Serializable {
 	 * @return an upper bound on the number of bytes used to store this ColGroup in memory.
 	 */
 	public long estimateInMemorySize() {
-		// object (12B padded to factors of 8), int numRows (4B),
-		// array reference colIndices (8B)
-		// + array object overhead if exists (32B) + 4B per element
-		long size = 24;
-		return (_colIndexes == null) ? size : size + 32 + 4 * _colIndexes.length;
+		return ColGroupSizes.estimateInMemorySizeGroup(_colIndexes.length);
 	}
 
 	/**
@@ -218,6 +248,13 @@ public abstract class ColGroup implements Serializable {
 	public abstract double get(int r, int c);
 
 	/**
+	 * Get the number of values. contained inside the ColGroup.
+	 * 
+	 * @return value at the row/column position
+	 */
+	public abstract long getValuesSize();
+
+	/**
 	 * Multiply the slice of the matrix that this column group represents by a vector on the right.
 	 * 
 	 * @param vector vector to multiply by (tall vector)
@@ -245,6 +282,13 @@ public abstract class ColGroup implements Serializable {
 	 */
 	public abstract ColGroup scalarOperation(ScalarOperator op);
 
+	/**
+	 * Unary Aggregate operator, since aggregate operators require new object output, the output becomes an uncompressed
+	 * matrix.
+	 * 
+	 * @param op     The operator used
+	 * @param result the output matrix block.
+	 */
 	public abstract void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result);
 
 	/**
@@ -275,7 +319,7 @@ public abstract class ColGroup implements Serializable {
 	 * @param rl   row lower bound, inclusive
 	 * @param ru   row upper bound, exclusive
 	 */
-	protected abstract void countNonZerosPerRow(int[] rnnz, int rl, int ru);
+	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
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/ConverterUtils.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConverter.java
similarity index 83%
rename from src/main/java/org/apache/sysds/runtime/compress/utils/ConverterUtils.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConverter.java
index c33154b..a3b434a 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/ConverterUtils.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupConverter.java
@@ -17,19 +17,17 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress.utils;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.Arrays;
 
-import org.apache.sysds.runtime.compress.ColGroup;
-import org.apache.sysds.runtime.compress.ColGroupDDC1;
-import org.apache.sysds.runtime.compress.ColGroupOLE;
-import org.apache.sysds.runtime.compress.ColGroupRLE;
-import org.apache.sysds.runtime.compress.ColGroupUncompressed;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.DataConverter;
 
-public class ConverterUtils {
+/**
+ * Utility functions for ColGroup to convert ColGroups or MatrixBlocks. to other representations.
+ */
+public class ColGroupConverter {
 	/**
 	 * Copy col group instance with deep copy of column indices but shallow copy of actual contents;
 	 * 
@@ -68,10 +66,22 @@ public class ConverterUtils {
 		return ret;
 	}
 
+	/**
+	 * Extract the double array primitive from a Matrix Block that is an vector.
+	 * 
+	 * @param vector The Matrix block vector
+	 * @return The double array primitive
+	 */
 	public static double[] getDenseVector(MatrixBlock vector) {
 		return DataConverter.convertToDoubleVector(vector, false);
 	}
 
+	/**
+	 * Extracts the Uncompressed MatrixBlock representation of a Col Group
+	 * 
+	 * @param group an ColGroup to decompress
+	 * @return A MatrixBlock.
+	 */
 	public static MatrixBlock getUncompressedColBlock(ColGroup group) {
 		return (group instanceof ColGroupUncompressed) ? ((ColGroupUncompressed) group)
 			.getData() : new ColGroupUncompressed(Arrays.asList(group)).getData();
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
similarity index 80%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
index 9938942..f2cad3e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC.java
@@ -17,23 +17,17 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.Arrays;
 import java.util.Iterator;
 
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.KahanFunction;
-import org.apache.sysds.runtime.functionobjects.KahanPlus;
-import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
-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.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.instructions.cp.KahanObject;
 import org.apache.sysds.runtime.matrix.data.IJV;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
-import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 
 /**
  * Class to encapsulate information about a column group that is encoded with dense dictionary encoding (DDC).
@@ -44,11 +38,16 @@ import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 public abstract class ColGroupDDC extends ColGroupValue {
 	private static final long serialVersionUID = -3204391646123465004L;
 
+	@Override
+	public CompressionType getCompType() {
+		return CompressionType.DDC;
+	}
+
 	public ColGroupDDC() {
 		super();
 	}
 
-	public ColGroupDDC(int[] colIndices, int numRows, UncompressedBitmap ubm) {
+	protected ColGroupDDC(int[] colIndices, int numRows, UncompressedBitmap ubm) {
 		super(colIndices, numRows, ubm);
 	}
 
@@ -102,7 +101,7 @@ public abstract class ColGroupDDC extends ColGroupValue {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		int ncol = getNumCols();
 		for(int i = rl; i < ru; i++) {
 			int lnnz = 0;
@@ -112,34 +111,6 @@ public abstract class ColGroupDDC extends ColGroupValue {
 		}
 	}
 
-	@Override
-	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result, int rl, int ru) {
-		// sum and sumsq (reduceall/reducerow over tuples and counts)
-		if(op.aggOp.increOp.fn instanceof KahanPlus || op.aggOp.increOp.fn instanceof KahanPlusSq) {
-			KahanFunction kplus = (op.aggOp.increOp.fn instanceof KahanPlus) ? KahanPlus
-				.getKahanPlusFnObject() : KahanPlusSq.getKahanPlusSqFnObject();
-
-			if(op.indexFn instanceof ReduceAll)
-				computeSum(result, kplus);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowSums(result, kplus, rl, ru);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColSums(result, kplus);
-		}
-		// 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;
-
-			if(op.indexFn instanceof ReduceAll)
-				computeMxx(result, builtin, false);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowMxx(result, builtin, rl, ru);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColMxx(result, builtin, false);
-		}
-	}
 
 	protected void computeSum(MatrixBlock result, KahanFunction kplus) {
 		int nrow = getNumRows();
@@ -236,7 +207,7 @@ public abstract class ColGroupDDC extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		return super.estimateInMemorySize();
+		return ColGroupSizes.estimateInMemorySizeDDC(getNumCols(), getNumValues());
 	}
 
 	@Override
@@ -308,4 +279,12 @@ public abstract class ColGroupDDC extends ColGroupValue {
 				buff[_colIndexes[j]] = _values[off + j];
 		}
 	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		return sb.toString();
+	}
+
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC1.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC1.java
similarity index 91%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC1.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC1.java
index b7f7d74..bf873c8 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC1.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC1.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -25,7 +25,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
 
-import org.apache.sysds.runtime.compress.utils.ConverterUtils;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.functionobjects.KahanFunction;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
@@ -42,16 +42,17 @@ public class ColGroupDDC1 extends ColGroupDDC {
 
 	private byte[] _data;
 
-	public ColGroupDDC1() {
+	protected ColGroupDDC1() {
 		super();
 	}
 
-	public ColGroupDDC1(int[] colIndices, int numRows, UncompressedBitmap ubm) {
+	protected ColGroupDDC1(int[] colIndices, int numRows, UncompressedBitmap ubm) {
 		super(colIndices, numRows, ubm);
-		_data = new byte[numRows];
 
 		int numVals = ubm.getNumValues();
 		int numCols = ubm.getNumColumns();
+		
+		_data = new byte[numRows];
 
 		// materialize zero values, if necessary
 		if(ubm.getNumOffsets() < (long) numRows * numCols) {
@@ -72,22 +73,26 @@ public class ColGroupDDC1 extends ColGroupDDC {
 		}
 	}
 
-	public ColGroupDDC1(int[] colIndices, int numRows, double[] values, byte[] data) {
+	// Internal Constructor, to be used when copying a DDC Colgroup, and for scalar operations
+	protected ColGroupDDC1(int[] colIndices, int numRows, double[] values, byte[] data) {
 		super(colIndices, numRows, values);
 		_data = data;
 	}
 
+	
 	@Override
-	public CompressionType getCompType() {
-		return CompressionType.DDC1;
+	protected ColGroupType getColGroupType(){
+		return ColGroupType.DDC1;
 	}
 
 	/**
 	 * Getter method to get the data, contained in The DDC ColGroup.
+	 * 
 	 * Not safe if modifications is made to the byte list.
-	 * @return The contained data 
+	 * 
+	 * @return The contained data
 	 */
-	public  byte[] getData(){
+	public byte[] getData() {
 		return _data;
 	}
 
@@ -195,13 +200,7 @@ public class ColGroupDDC1 extends ColGroupDDC {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-
-		// adding data size
-		if(_data != null)
-			size += _data.length;
-
-		return size;
+		return ColGroupSizes.estimateInMemorySizeDDC1(getNumCols(), getNumValues(), _data.length);
 	}
 
 	@Override
@@ -239,7 +238,7 @@ public class ColGroupDDC1 extends ColGroupDDC {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int ncol = getNumCols();
 		final int numVals = getNumValues();
 
@@ -256,7 +255,7 @@ public class ColGroupDDC1 extends ColGroupDDC {
 
 	@Override
 	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] b = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
@@ -277,7 +276,7 @@ public class ColGroupDDC1 extends ColGroupDDC {
 	}
 
 	public static void rightMultByVector(ColGroupDDC1[] grps, MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] b = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 
 		// prepare distinct values once
@@ -303,7 +302,7 @@ public class ColGroupDDC1 extends ColGroupDDC {
 
 	@Override
 	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result) {
-		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] a = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int nrow = getNumRows();
 		final int numVals = getNumValues();
@@ -422,4 +421,12 @@ public class ColGroupDDC1 extends ColGroupDDC {
 		// as zero are represented, it is sufficient to simply apply the scalar op
 		return new ColGroupDDC1(_colIndexes, _numRows, applyScalarOp(op), _data);
 	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append(" DataLength: " + this._data.length);
+		return sb.toString();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC2.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC2.java
similarity index 89%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC2.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC2.java
index 72603f0..ba28dbe 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupDDC2.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupDDC2.java
@@ -17,14 +17,14 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.sysds.runtime.compress.utils.ConverterUtils;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.functionobjects.KahanFunction;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
@@ -43,11 +43,11 @@ public class ColGroupDDC2 extends ColGroupDDC {
 
 	private char[] _data;
 
-	public ColGroupDDC2() {
+	protected ColGroupDDC2() {
 		super();
 	}
 
-	public ColGroupDDC2(int[] colIndices, int numRows, UncompressedBitmap ubm) {
+	protected ColGroupDDC2(int[] colIndices, int numRows, UncompressedBitmap ubm) {
 		super(colIndices, numRows, ubm);
 		_data = new char[numRows];
 
@@ -73,14 +73,27 @@ public class ColGroupDDC2 extends ColGroupDDC {
 		}
 	}
 
-	public ColGroupDDC2(int[] colIndices, int numRows, double[] values, char[] data) {
+	// Internal Constructor, to be used when copying a DDC Colgroup, and for scalar operations
+	protected ColGroupDDC2(int[] colIndices, int numRows, double[] values, char[] data) {
 		super(colIndices, numRows, values);
 		_data = data;
 	}
 
 	@Override
-	public CompressionType getCompType() {
-		return CompressionType.DDC2;
+	protected ColGroupType getColGroupType(){
+		return ColGroupType.DDC1;
+	}
+
+	/**
+	 * Getter method to get the data, contained in The DDC ColGroup.
+	 * 
+	 * Not safe if modifications is made to the byte list.
+	 * 
+	 * @return The contained data
+	 */
+
+	public char[] getData() {
+		return _data;
 	}
 
 	@Override
@@ -161,13 +174,8 @@ public class ColGroupDDC2 extends ColGroupDDC {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-
-		// adding data size
-		if(_data != null)
-			size += 2 * _data.length;
-
-		return size;
+		// LOG.debug(this.toString());
+		return ColGroupSizes.estimateInMemorySizeDDC2(getNumCols(), getNumValues(), _data.length);
 	}
 
 	@Override
@@ -205,7 +213,7 @@ public class ColGroupDDC2 extends ColGroupDDC {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int ncol = getNumCols();
 		final int numVals = getNumValues();
 
@@ -222,7 +230,7 @@ public class ColGroupDDC2 extends ColGroupDDC {
 
 	@Override
 	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] b = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
@@ -243,7 +251,7 @@ public class ColGroupDDC2 extends ColGroupDDC {
 
 	@Override
 	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result) {
-		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] a = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int nrow = getNumRows();
 		final int ncol = getNumCols();
@@ -356,4 +364,12 @@ public class ColGroupDDC2 extends ColGroupDDC {
 		// as zero are represented, it is sufficient to simply apply the scalar op
 		return new ColGroupDDC2(_colIndexes, _numRows, applyScalarOp(op), _data);
 	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append(" DataLength: " + this._data.length);
+		return sb.toString();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java
new file mode 100644
index 0000000..cc2009a
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupFactory.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.runtime.compress.colgroup;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.sysds.runtime.DMLCompressionException;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeEstimatorExact;
+import org.apache.sysds.runtime.compress.estim.CompressedSizeInfoColGroup;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.util.CommonThreadPool;
+
+/**
+ * Factory pattern for constructing ColGroups.
+ */
+public class ColGroupFactory {
+
+	/**
+	 * The actual compression method, that handles the logic of compressing multiple columns together. This method also
+	 * have the responsibility of correcting any estimation errors previously made.
+	 * 
+	 * @param in           The input matrix, that could have been transposed if CompSettings was set to do that
+	 * @param compRatios   The previously computed Compression ratings of individual col indexes.
+	 * @param groups       The column groups to consider compressing together.
+	 * @param compSettings The compression settings to construct the compression based on.
+	 * @param k            The number of parallelism used.
+	 * @return A Resulting array of ColGroups, containing the compressed information from the input matrix block.
+	 */
+	public static ColGroup[] compressColGroups(MatrixBlock in, HashMap<Integer, Double> compRatios, List<int[]> groups,
+		CompressionSettings compSettings, int k) {
+
+		if(k == 1) {
+			compressColGroups(in, compRatios, groups, compSettings);
+		}
+
+		try {
+			ExecutorService pool = CommonThreadPool.get(k);
+			ArrayList<CompressTask> tasks = new ArrayList<>();
+			for(int[] colIndexes : groups)
+				tasks.add(new CompressTask(in, compRatios, colIndexes, compSettings));
+			List<Future<ColGroup>> rtask = pool.invokeAll(tasks);
+			ArrayList<ColGroup> ret = new ArrayList<>();
+			for(Future<ColGroup> lrtask : rtask)
+				ret.add(lrtask.get());
+			pool.shutdown();
+			return ret.toArray(new ColGroup[0]);
+		}
+		catch(Exception ex) {
+			throw new DMLRuntimeException(ex);
+		}
+	}
+
+	private static ColGroup[] compressColGroups(MatrixBlock in, HashMap<Integer, Double> compRatios, List<int[]> groups,
+		CompressionSettings compSettings) {
+		ColGroup[] ret = new ColGroup[groups.size()];
+		for(int i = 0; i < groups.size(); i++)
+			ret[i] = compressColGroup(in, compRatios, groups.get(i), compSettings);
+
+		return ret;
+	}
+
+	private static class CompressedColumn implements Comparable<CompressedColumn> {
+		final int colIx;
+		final double compRatio;
+
+		public CompressedColumn(int colIx, double compRatio) {
+			this.colIx = colIx;
+			this.compRatio = compRatio;
+		}
+
+		public static PriorityQueue<CompressedColumn> makePriorityQue(HashMap<Integer, Double> compRatios,
+			int[] colIndexes) {
+			PriorityQueue<CompressedColumn> compRatioPQ;
+
+			// first modification
+			compRatioPQ = new PriorityQueue<>();
+			for(int i = 0; i < colIndexes.length; i++)
+				compRatioPQ.add(new CompressedColumn(i, compRatios.get(colIndexes[i])));
+
+			return compRatioPQ;
+		}
+
+		@Override
+		public int compareTo(CompressedColumn o) {
+			return (int) Math.signum(compRatio - o.compRatio);
+		}
+	}
+
+	private static class CompressTask implements Callable<ColGroup> {
+		private final MatrixBlock _in;
+		private final HashMap<Integer, Double> _compRatios;
+		private final int[] _colIndexes;
+		private final CompressionSettings _compSettings;
+
+		protected CompressTask(MatrixBlock in, HashMap<Integer, Double> compRatios, int[] colIndexes,
+			CompressionSettings compSettings) {
+			_in = in;
+			_compRatios = compRatios;
+			_colIndexes = colIndexes;
+			_compSettings = compSettings;
+		}
+
+		@Override
+		public ColGroup call() {
+			return compressColGroup(_in, _compRatios, _colIndexes, _compSettings);
+		}
+	}
+
+	private static ColGroup compressColGroup(MatrixBlock in, HashMap<Integer, Double> compRatios, int[] colIndexes,
+		CompressionSettings compSettings) {
+
+		int[] allGroupIndices = colIndexes.clone();
+
+		CompressedSizeInfoColGroup sizeInfo;
+		// The compression type is decided based on a full bitmap since it
+		// will be reused for the actual compression step.
+		UncompressedBitmap ubm = null;
+		PriorityQueue<CompressedColumn> compRatioPQ = CompressedColumn.makePriorityQue(compRatios, colIndexes);
+
+		// TODO: Use sample based estimator still here.
+		// Switching to exact estimator here, when doing the actual compression.
+		// FYI, this was also how it was doing it before, under the covers.
+		// This is because the ubm is extracted for the entire column, (because it is going to be used for the later
+		// compression i guess)
+		CompressedSizeEstimator estimator = new CompressedSizeEstimatorExact(in, compSettings);
+
+		while(true) {
+
+			// STEP 1.
+			// Extract the entire input column list and observe compression ratio
+			ubm = BitmapEncoder.extractBitmap(colIndexes, in, compSettings);
+			sizeInfo = new CompressedSizeInfoColGroup(estimator.estimateCompressedColGroupSize(ubm),
+				compSettings.validCompressions);
+
+			// Throw error if for some reason the compression observed is 0.
+			if(sizeInfo.getMinSize() == 0) {
+				throw new DMLRuntimeException("Size info of compressed Col Group is 0");
+			}
+
+			// STEP 2.
+			// Calculate the compression ratio compared to an uncompressed ColGroup type.
+			double compRatio = sizeInfo.getCompressionSize(CompressionType.UNCOMPRESSED) / sizeInfo.getMinSize();
+
+			// STEP 3.
+			// Finish the search and close this compression if the group show good compression.
+
+			// Seems a little early to stop here. Maybe reconsider how to decide when to stop.
+			// Also when comparing to the case of 1.0 compression ratio, it could be that we chose to compress a group
+			// worse than the individual columns.
+
+			// Furthermore performance of a compressed representation that does not compress much, is decremental to
+			// overall performance.
+			if(compRatio > 1.0) {
+				int rlen = compSettings.transposeInput ? in.getNumColumns() : in.getNumRows();
+				return compress(colIndexes, rlen, ubm, sizeInfo.getBestCompressionType(), compSettings, in);
+			}
+			else {
+				// STEP 4.
+				// Try to remove the least compressible column from the columns to compress.
+				// Then repeat from Step 1.
+
+				allGroupIndices[compRatioPQ.poll().colIx] = -1;
+
+				if(colIndexes.length - 1 == 0) {
+					return null;
+				}
+
+				colIndexes = new int[colIndexes.length - 1];
+				// copying the values that do not equal -1
+				int ix = 0;
+				for(int col : allGroupIndices)
+					if(col != -1)
+						colIndexes[ix++] = col;
+			}
+		}
+	}
+
+	/**
+	 * Method for compressing an ColGroup.
+	 * 
+	 * @param colIndexes     The Column indexes to compress
+	 * @param rlen           The number of rows in the columns
+	 * @param ubm            The uncompressedBitmap containing all the data needed for the compression (unless
+	 *                       Uncompressed ColGroup)
+	 * @param compType       The CompressionType selected
+	 * @param compSettings   The compression Settings used for the given compression
+	 * @param rawMatrixBlock The copy of the original input (maybe transposed) MatrixBlock
+	 * @return A Compressed ColGroup
+	 */
+	public static ColGroup compress(int[] colIndexes, int rlen, UncompressedBitmap ubm, CompressionType compType,
+		CompressionSettings compSettings, MatrixBlock rawMatrixBlock) {
+
+		switch(compType) {
+			case DDC:
+				if(ubm.getNumValues() < 256) {
+					return new ColGroupDDC1(colIndexes, rlen, ubm);
+				}
+				else {
+					return new ColGroupDDC2(colIndexes, rlen, ubm);
+				}
+			case RLE:
+				return new ColGroupRLE(colIndexes, rlen, ubm);
+			case OLE:
+				return new ColGroupOLE(colIndexes, rlen, ubm);
+			case UNCOMPRESSED:
+				return new ColGroupUncompressed(colIndexes, rawMatrixBlock, compSettings);
+			default:
+				throw new DMLCompressionException("Not implemented ColGroup Type compressed in factory.");
+		}
+	}
+
+	/**
+	 * 
+	 * Method for producing the final ColGroupList stored inside the CompressedMatrixBlock.
+	 * 
+	 * TODO Redesign this method such that it does not utilize the null pointers to decide on which ColGroups should be
+	 * incompressable. This is done by changing both this method and compressColGroup inside this class.
+	 * 
+	 * @param numCols The number of columns in input matrix
+	 * @param colGroups The colgroups made to assign
+	 * @param rawBlock The (maybe transposed) original MatrixBlock
+	 * @param compSettings The Compressionsettings used.
+	 * @return return the final ColGroupList.
+	 */
+	public static List<ColGroup> assignColumns(int numCols, ColGroup[] colGroups, MatrixBlock rawBlock,
+		CompressionSettings compSettings) {
+
+		List<ColGroup> _colGroups = new ArrayList<>();
+		HashSet<Integer> remainingCols = seq(0, numCols - 1, 1);
+		for(int j = 0; j < colGroups.length; j++) {
+			if(colGroups[j] != null) {
+				for(int col : colGroups[j].getColIndices())
+					remainingCols.remove(col);
+				_colGroups.add(colGroups[j]);
+			}
+		}
+
+		if(!remainingCols.isEmpty()) {
+			int[] list = remainingCols.stream().mapToInt(i -> i).toArray();
+			ColGroupUncompressed ucgroup = new ColGroupUncompressed(list, rawBlock, compSettings);
+			_colGroups.add(ucgroup);
+		}
+		return _colGroups;
+	}
+
+	private static HashSet<Integer> seq(int from, int to, int incr) {
+		HashSet<Integer> ret = new HashSet<>();
+		for(int i = from; i <= to; i += incr)
+			ret.add(i);
+		return ret;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java
new file mode 100644
index 0000000..b1c7502
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupIO.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.runtime.compress.colgroup;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.ColGroupType;
+
+/**
+ * This has the IO responsibility of ColGroups, such that it enables to read and write ColGroups to and from a DataInput
+ * and DataOutput
+ */
+public class ColGroupIO {
+
+	/**
+	 * Read groups from a file. Note that the information about how many should be in the file already.
+	 * 
+	 * @param in              The Data input object to read from.
+	 * @param _sharedDDC1Dict Boolean flag to specify if the DCC should share dictionary.
+	 * @return Return a List containing the ColGroups from the DataInput.
+	 * @throws IOException Throws IO Exception if the in refuses to read data.
+	 */
+	public static List<ColGroup> readGroups(DataInput in, boolean _sharedDDC1Dict) throws IOException {
+
+		// Read in how many colGroups there are
+		int nColGroups = in.readInt();
+
+		// Allocate that amount into an ArrayList
+		List<ColGroup> _colGroups = new ArrayList<>(nColGroups);
+		double[] sharedDict = null;
+
+		// Read each ColGroup one at a time.
+		for(int i = 0; i < nColGroups; i++) {
+			ColGroupType ctype = ColGroupType.values()[in.readByte()];
+			ColGroup grp = null;
+
+			// create instance of column group
+			switch(ctype) {
+				case UNCOMPRESSED:
+					grp = new ColGroupUncompressed();
+					break;
+				case OLE:
+					grp = new ColGroupOLE();
+					break;
+				case RLE:
+					grp = new ColGroupRLE();
+					break;
+				case DDC1:
+					grp = new ColGroupDDC1();
+					break;
+				case DDC2:
+					grp = new ColGroupDDC2();
+					break;
+				default:
+					throw new DMLRuntimeException("Unsupported ColGroup Type used");
+			}
+
+			// Deserialize and add column group (flag for shared dictionary passed
+			// and numCols evaluated in DDC1 because numCols not available yet
+			grp.readFields(in, sharedDict != null);
+
+			// use shared DDC1 dictionary if applicable
+			if(_sharedDDC1Dict && grp.getNumCols() == 1 && grp instanceof ColGroupDDC1) {
+				if(sharedDict == null)
+					sharedDict = ((ColGroupValue) grp).getValues();
+				else
+					((ColGroupValue) grp).setValues(sharedDict);
+			}
+
+			_colGroups.add(grp);
+		}
+
+		return _colGroups;
+	}
+
+	/**
+	 * Writes the ColGroups out to the DataOutput.
+	 * 
+	 * @param out             The DataOutput the ColGroups are written to
+	 * @param _sharedDDC1Dict Boolean flag specifying if the DDC share dictionaries.
+	 * @param _colGroups      List of the ColGroups to write to file.
+	 * @throws IOException Throws IO Exception if the out refuses to write.
+	 */
+	public static void writeGroups(DataOutput out, boolean _sharedDDC1Dict, List<ColGroup> _colGroups)
+		throws IOException
+	{
+		// Write out how many ColGroups we save.
+		out.writeInt(_colGroups.size());
+
+		boolean skipDict = false;
+		for(ColGroup grp : _colGroups) {
+			// TODO save DDC Dict sharing smarter.
+			boolean shared = (grp instanceof ColGroupDDC1 && _sharedDDC1Dict && grp.getNumCols() == 1);
+			out.writeByte(grp.getCompType().ordinal());
+			grp.write(out, skipDict & shared); 
+			skipDict |= shared;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupOLE.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
similarity index 93%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupOLE.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
index 5a20ced..405b379 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupOLE.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOLE.java
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.Arrays;
 import java.util.Iterator;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysds.runtime.compress.utils.ConverterUtils;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -37,14 +38,15 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 /**
  * Class to encapsulate information about a column group that is encoded with simple lists of offsets for each set of
  * distinct values.
- * 
  */
 public class ColGroupOLE extends ColGroupOffset {
 	private static final long serialVersionUID = -9157676271360528008L;
 
 	private static final Log LOG = LogFactory.getLog(ColGroupOLE.class.getName());
 
-	public ColGroupOLE() {
+	protected int[] _skiplist;
+
+	protected ColGroupOLE() {
 		super();
 	}
 
@@ -55,7 +57,7 @@ public class ColGroupOLE extends ColGroupOffset {
 	 * @param numRows    total number of rows in the parent block
 	 * @param ubm        Uncompressed bitmap representation of the block
 	 */
-	public ColGroupOLE(int[] colIndices, int numRows, UncompressedBitmap ubm) {
+	protected ColGroupOLE(int[] colIndices, int numRows, UncompressedBitmap ubm) {
 		super(colIndices, numRows, ubm);
 
 		// compress the bitmaps
@@ -69,9 +71,11 @@ public class ColGroupOLE extends ColGroupOffset {
 
 		// compact bitmaps to linearized representation
 		createCompressedBitmaps(numVals, totalLen, lbitmaps);
-		// TODO FIX Skiplist properly... Had to move it out since L2SVM test crash in edge cases. Make Conditions Consistant, or move allocation outside.
+
+		// TODO FIX Skiplist construction Since it is not needed in all cases.
+
 		_skiplist = new int[numVals];
-		if(LOW_LEVEL_OPT && CREATE_SKIPLIST && numRows > 2 * BitmapEncoder.BITMAP_BLOCK_SZ) {
+		if( CREATE_SKIP_LIST && numRows > 2 * BitmapEncoder.BITMAP_BLOCK_SZ) {
 			int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 			// _skiplist = new int[numVals];
 			int rl = (getNumRows() / 2 / blksz) * blksz;
@@ -92,7 +96,7 @@ public class ColGroupOLE extends ColGroupOffset {
 			LOG.warn("OLE group larger than UC dense: " + estimateInMemorySize() + " " + ucSize);
 	}
 
-	public ColGroupOLE(int[] colIndices, int numRows, boolean zeros, double[] values, char[] bitmaps,
+	protected ColGroupOLE(int[] colIndices, int numRows, boolean zeros, double[] values, char[] bitmaps,
 		int[] bitmapOffs) {
 		super(colIndices, numRows, zeros, values);
 		_data = bitmaps;
@@ -101,12 +105,17 @@ public class ColGroupOLE extends ColGroupOffset {
 
 	@Override
 	public CompressionType getCompType() {
-		return CompressionType.OLE_BITMAP;
+		return CompressionType.OLE;
+	}
+
+	@Override
+	protected ColGroupType getColGroupType(){
+		return ColGroupType.OLE;
 	}
 
 	@Override
 	public void decompressToBlock(MatrixBlock target, int rl, int ru) {
-		if(LOW_LEVEL_OPT && getNumValues() > 1) {
+		if( getNumValues() > 1) {
 			final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 			final int numCols = getNumCols();
 			final int numVals = getNumValues();
@@ -140,7 +149,7 @@ public class ColGroupOLE extends ColGroupOffset {
 
 	@Override
 	public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-		if(LOW_LEVEL_OPT && getNumValues() > 1) {
+		if( getNumValues() > 1) {
 			final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 			final int numCols = getNumCols();
 			final int numVals = getNumValues();
@@ -245,6 +254,14 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
+	public long estimateInMemorySize() {
+		// LOG.debug(this.toString());
+		// Note 0 is because the size can be calculated based on the given values,
+		// And because the fourth argument is only needed in estimation, not when an OLE ColGroup is created.
+		return ColGroupSizes.estimateInMemorySizeOLE(getNumCols(), _values.length, _data.length, 0);
+	}
+
+	@Override
 	public ColGroup scalarOperation(ScalarOperator op) {
 		double val0 = op.executeScalar(0);
 
@@ -274,7 +291,7 @@ public class ColGroupOLE extends ColGroupOffset {
 
 	@Override
 	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] b = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
@@ -286,7 +303,7 @@ public class ColGroupOLE extends ColGroupOffset {
 			sb[j] = b[_colIndexes[j]];
 		}
 
-		if(LOW_LEVEL_OPT && numVals > 1 && _numRows > blksz) {
+		if( numVals > 1 && _numRows > blksz) {
 			// 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)
@@ -358,14 +375,14 @@ public class ColGroupOLE extends ColGroupOffset {
 
 	@Override
 	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result) {
-		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] a = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
 		final int n = getNumRows();
 
-		if(LOW_LEVEL_OPT && numVals > 1 && _numRows > blksz) {
+		if( numVals > 1 && _numRows > blksz) {
 			// cache blocking config (see matrix-vector mult for explanation)
 			final int blksz2 = ColGroupOffset.READ_CACHE_BLKSZ;
 
@@ -482,7 +499,7 @@ public class ColGroupOLE extends ColGroupOffset {
 		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 		final int numVals = getNumValues();
 
-		if(ALLOW_CACHE_CONSCIOUS_ROWSUMS && LOW_LEVEL_OPT && numVals > 1 && _numRows > blksz) {
+		if(ALLOW_CACHE_CONSCIOUS_ROWSUMS && numVals > 1 && _numRows > blksz) {
 			final int blksz2 = ColGroupOffset.WRITE_CACHE_BLKSZ / 2;
 
 			// step 1: prepare position and value arrays
@@ -640,7 +657,7 @@ public class ColGroupOLE extends ColGroupOffset {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
 		final int blksz2 = ColGroupOffset.WRITE_CACHE_BLKSZ;
 		final int numVals = getNumValues();
@@ -739,6 +756,15 @@ public class ColGroupOLE extends ColGroupOffset {
 		return new OLERowIterator(rl, ru);
 	}
 
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append(String.format("\n%15s%5d ", "SkipList:", this._skiplist.length));
+		sb.append(Arrays.toString(this._skiplist));
+		return sb.toString();
+	}
+
 	private class OLEValueIterator implements Iterator<Integer> {
 		private final int _ru;
 		private final int _boff;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupOffset.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
similarity index 85%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupOffset.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
index c30ce9a..b51bd38 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupOffset.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupOffset.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -26,18 +26,13 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.functionobjects.Builtin;
-import org.apache.sysds.runtime.functionobjects.KahanFunction;
-import org.apache.sysds.runtime.functionobjects.KahanPlus;
-import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
-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.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.matrix.data.IJV;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
-import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 
 /**
  * Base class for column groups encoded with various types of bitmap encoding.
@@ -49,7 +44,7 @@ import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 public abstract class ColGroupOffset extends ColGroupValue {
 	private static final long serialVersionUID = -1635828933479403125L;
 
-	protected static final boolean CREATE_SKIPLIST = true;
+	protected static final boolean CREATE_SKIP_LIST = true;
 
 	protected static final int READ_CACHE_BLKSZ = 2 * BitmapEncoder.BITMAP_BLOCK_SZ;
 	public static final int WRITE_CACHE_BLKSZ = 2 * BitmapEncoder.BITMAP_BLOCK_SZ;
@@ -58,9 +53,6 @@ public abstract class ColGroupOffset extends ColGroupValue {
 	/** Bitmaps, one per uncompressed value in {@link #_values}. */
 	protected int[] _ptr; // bitmap offsets per value
 	protected char[] _data; // linearized bitmaps (variable length)
-	protected boolean _zeros; // contains zero values
-
-	protected int[] _skiplist;
 
 	public ColGroupOffset() {
 		super();
@@ -110,16 +102,13 @@ public abstract class ColGroupOffset extends ColGroupValue {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-
-		// adding bitmaps size
-		size += 16; // array references
-		if(_data != null) {
-			size += 32 + _ptr.length * 4; // offsets
-			size += 32 + _data.length * 2; // bitmaps
+		// Could use a ternary operator, but it looks odd with our code formatter here.
+		if(_data == null) {
+			return ColGroupSizes.estimateInMemorySizeOffset(getNumCols(), _colIndexes.length, 0, 0);
+		}
+		else {
+			return ColGroupSizes.estimateInMemorySizeOffset(getNumCols(), _values.length, _ptr.length, _data.length);
 		}
-
-		return size;
 	}
 
 	// generic decompression for OLE/RLE, to be overwritten for performance
@@ -342,43 +331,6 @@ public abstract class ColGroupOffset extends ColGroupValue {
 		return ret;
 	}
 
-	@Override
-	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result, int rl, int ru) {
-		// sum and sumsq (reduceall/reducerow over tuples and counts)
-		if(op.aggOp.increOp.fn instanceof KahanPlus || op.aggOp.increOp.fn instanceof KahanPlusSq) {
-			KahanFunction kplus = (op.aggOp.increOp.fn instanceof KahanPlus) ? KahanPlus
-				.getKahanPlusFnObject() : KahanPlusSq.getKahanPlusSqFnObject();
-
-			if(op.indexFn instanceof ReduceAll)
-				computeSum(result, kplus);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowSums(result, kplus, rl, ru);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColSums(result, kplus);
-		}
-		// 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;
-
-			if(op.indexFn instanceof ReduceAll)
-				computeMxx(result, builtin, _zeros);
-			else if(op.indexFn instanceof ReduceCol)
-				computeRowMxx(result, builtin, rl, ru);
-			else if(op.indexFn instanceof ReduceRow)
-				computeColMxx(result, builtin, _zeros);
-		}
-	}
-
-	protected abstract void computeSum(MatrixBlock result, KahanFunction kplus);
-
-	protected abstract void computeRowSums(MatrixBlock result, KahanFunction kplus, int rl, int ru);
-
-	protected abstract void computeColSums(MatrixBlock result, KahanFunction kplus);
-
-	protected abstract void computeRowMxx(MatrixBlock result, Builtin builtin, int rl, int ru);
-
 	protected abstract boolean[] computeZeroIndicatorVector();
 
 	@Override
@@ -404,6 +356,23 @@ public abstract class ColGroupOffset extends ColGroupValue {
 	 */
 	public abstract Iterator<Integer> getIterator(int k, int rl, int ru);
 
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append(String.format("\n%15s%5d ", "Pointers:" , this._ptr.length ));
+		sb.append(Arrays.toString(this._ptr));
+		sb.append(String.format("\n%15s%5d ", "Data:" , this._data.length));
+		sb.append("[");
+		for(int x = 0; x < _data.length; x++) {
+			sb.append(((int) _data[x]));
+			if(x != _data.length - 1)
+				sb.append(", ");
+		}
+		sb.append("]");
+		return sb.toString();
+	}
+
 	protected class OffsetValueIterator implements Iterator<IJV> {
 		// iterator configuration
 		private final int _rl;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupRLE.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
similarity index 95%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupRLE.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
index ec817ca..de10bed 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupRLE.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupRLE.java
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.Arrays;
 import java.util.Iterator;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.sysds.runtime.compress.utils.ConverterUtils;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.compress.utils.LinearAlgebraUtils;
 import org.apache.sysds.runtime.data.DenseBlock;
 import org.apache.sysds.runtime.functionobjects.Builtin;
@@ -41,7 +42,7 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	private static final Log LOG = LogFactory.getLog(ColGroupRLE.class.getName());
 
-	public ColGroupRLE() {
+	protected ColGroupRLE() {
 		super();
 	}
 
@@ -52,7 +53,7 @@ public class ColGroupRLE extends ColGroupOffset {
 	 * @param numRows    total number of rows in the parent block
 	 * @param ubm        Uncompressed bitmap representation of the block
 	 */
-	public ColGroupRLE(int[] colIndices, int numRows, UncompressedBitmap ubm) {
+	protected ColGroupRLE(int[] colIndices, int numRows, UncompressedBitmap ubm) {
 		super(colIndices, numRows, ubm);
 
 		// compress the bitmaps
@@ -68,12 +69,13 @@ public class ColGroupRLE extends ColGroupOffset {
 		createCompressedBitmaps(numVals, totalLen, lbitmaps);
 
 		// debug output
-		double ucSize = MatrixBlock.estimateSizeDenseInMemory(numRows, colIndices.length);
+		double ucSize = ColGroupSizes.estimateInMemorySizeUncompressed(numRows, colIndices.length, 1.0);
 		if(estimateInMemorySize() > ucSize)
-			LOG.warn("RLE group larger than UC dense: " + estimateInMemorySize() + " " + ucSize);
+			LOG.warn(
+				String.format("RLE group larger than UC dense: %8d Uncompressed: %8d", estimateInMemorySize(), (int)ucSize));
 	}
 
-	public ColGroupRLE(int[] colIndices, int numRows, boolean zeros, double[] values, char[] bitmaps,
+	protected ColGroupRLE(int[] colIndices, int numRows, boolean zeros, double[] values, char[] bitmaps,
 		int[] bitmapOffs) {
 		super(colIndices, numRows, zeros, values);
 		_data = bitmaps;
@@ -82,12 +84,17 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	@Override
 	public CompressionType getCompType() {
-		return CompressionType.RLE_BITMAP;
+		return CompressionType.RLE;
+	}
+
+	@Override
+	protected ColGroupType getColGroupType() {
+		return ColGroupType.RLE;
 	}
 
 	@Override
 	public void decompressToBlock(MatrixBlock target, int rl, int ru) {
-		if(LOW_LEVEL_OPT && getNumValues() > 1) {
+		if(getNumValues() > 1) {
 			final int blksz = 128 * 1024;
 			final int numCols = getNumCols();
 			final int numVals = getNumValues();
@@ -126,7 +133,7 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	@Override
 	public void decompressToBlock(MatrixBlock target, int[] colixTargets) {
-		if(LOW_LEVEL_OPT && getNumValues() > 1) {
+		if(getNumValues() > 1) {
 			final int blksz = 128 * 1024;
 			final int numCols = getNumCols();
 			final int numVals = getNumValues();
@@ -252,7 +259,7 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	@Override
 	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru) {
-		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] b = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
@@ -263,7 +270,7 @@ public class ColGroupRLE extends ColGroupOffset {
 			sb[j] = b[_colIndexes[j]];
 		}
 
-		if(LOW_LEVEL_OPT && numVals > 1 && _numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
+		if(numVals > 1 && _numRows > BitmapEncoder.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
@@ -346,13 +353,13 @@ public class ColGroupRLE extends ColGroupOffset {
 
 	@Override
 	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result) {
-		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] a = ColGroupConverter.getDenseVector(vector);
 		double[] c = result.getDenseBlockValues();
 		final int numCols = getNumCols();
 		final int numVals = getNumValues();
 		final int n = getNumRows();
 
-		if(LOW_LEVEL_OPT && numVals > 1 && _numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
+		if(numVals > 1 && _numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
 			final int blksz = ColGroupOffset.READ_CACHE_BLKSZ;
 
 			// step 1: prepare position and value arrays
@@ -508,7 +515,7 @@ public class ColGroupRLE extends ColGroupOffset {
 
 		final int numVals = getNumValues();
 
-		if(ALLOW_CACHE_CONSCIOUS_ROWSUMS && LOW_LEVEL_OPT && numVals > 1 && _numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
+		if(ALLOW_CACHE_CONSCIOUS_ROWSUMS && numVals > 1 && _numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
 			final int blksz = ColGroupOffset.WRITE_CACHE_BLKSZ / 2;
 
 			// step 1: prepare position and value arrays
@@ -661,7 +668,7 @@ public class ColGroupRLE extends ColGroupOffset {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		final int numVals = getNumValues();
 		final int numCols = getNumCols();
 
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
new file mode 100644
index 0000000..c53f867
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupSizes.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.runtime.compress.colgroup;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.utils.MemoryEstimates;
+
+public class ColGroupSizes {
+	// ------------------------------
+	// Logging parameters:
+	// local debug flag
+	private static final boolean LOCAL_DEBUG = false;
+	// DEBUG/TRACE for details
+	private static final Level LOCAL_DEBUG_LEVEL = Level.DEBUG;
+
+	protected static final Log LOG = LogFactory.getLog(ColGroupSizes.class.getName());
+
+	static {
+		// for internal debugging only
+		if(LOCAL_DEBUG) {
+			Logger.getLogger("org.apache.sysds.runtime.compress.colgroup").setLevel(LOCAL_DEBUG_LEVEL);
+		}
+	}
+	// ------------------------------
+
+	public static long getEmptyMemoryFootprint(Class<?> colGroupClass) {
+		switch(colGroupClass.getSimpleName()) {
+			case "ColGroup":
+				return estimateInMemorySizeGroup(0);
+			case "ColGroupValue":
+				return estimateInMemorySizeGroupValue(0, 0);
+			case "ColGroupOffset":
+				return estimateInMemorySizeOffset(0, 0, 0, 0);
+			case "ColGroupDDC":
+				return estimateInMemorySizeDDC(0, 0);
+			case "ColGroupDDC1":
+				return estimateInMemorySizeDDC1(0, 0, 0);
+			case "ColGroupDDC2":
+				return estimateInMemorySizeDDC2(0, 0, 0);
+			case "ColGroupOLE":
+				return estimateInMemorySizeOLE(0, 0, 0, 0);
+			case "ColGroupRLE":
+				return estimateInMemorySizeRLE(0, 0, 0, 0);
+			case "ColGroupUncompressed":
+				return estimateInMemorySizeUncompressed(0, 0, 0.0);
+			default:
+				throw new NotImplementedException("Case not implemented");
+		}
+	}
+
+	public static long estimateInMemorySizeGroup(int nrColumns) {
+		long size = 0;
+		size += 16; // Object header
+		size += 4; // int numRows,
+		size += 1; // _zeros boolean reference
+		size += 3; // padding
+		size += MemoryEstimates.intArrayCost(nrColumns);
+		return size;
+	}
+
+	public static long estimateInMemorySizeGroupValue(int nrColumns, long nrValues) {
+		long size = estimateInMemorySizeGroup(nrColumns);
+		size += MemoryEstimates.doubleArrayCost(nrValues);
+		return size;
+	}
+
+	public static long estimateInMemorySizeDDC(int nrCols, int uniqueVals) {
+		long size = estimateInMemorySizeGroupValue(nrCols, uniqueVals);
+		return size;
+	}
+
+	public static long estimateInMemorySizeDDC1(int nrCols, int uniqueVals, int dataLength) {
+		if(uniqueVals > 255)
+			return Long.MAX_VALUE;
+		// LOG.debug("DD1C: " + nrCols + " nr unique: " + uniqueVals + " DataLength: " + dataLength);
+		long size = estimateInMemorySizeDDC(nrCols, uniqueVals);
+		size += MemoryEstimates.byteArrayCost(dataLength);
+		return size;
+	}
+
+	public static long estimateInMemorySizeDDC2(int nrCols, int uniqueVals, int dataLength) {
+		if(uniqueVals > Character.MAX_VALUE)
+			return Long.MAX_VALUE;
+		// LOG.debug("DD2C: " + nrCols + "nr unique: " + uniqueVals +" datalen: "+ dataLength);
+		long size = estimateInMemorySizeDDC(nrCols, uniqueVals);
+		size += MemoryEstimates.charArrayCost(dataLength);
+		return size;
+	}
+
+	public static long estimateInMemorySizeOffset(int nrColumns, long nrValues, int pointers, int offsetLength) {
+		// LOG.debug("OFFSET list: nrC " + nrColumns +"\tnrV " + nrValues + "\tpl "+pointers +"\tdl "+ offsetLength);
+		long size = estimateInMemorySizeGroupValue(nrColumns, nrValues);
+		size += MemoryEstimates.intArrayCost(pointers);
+		size += MemoryEstimates.charArrayCost(offsetLength);
+		return size;
+	}
+
+	public static long estimateInMemorySizeOLE(int nrColumns, int nrValues, int offsetLength, int nrRows) {
+		nrColumns = nrColumns > 0 ? nrColumns : 1;
+		offsetLength += (nrRows / BitmapEncoder.BITMAP_BLOCK_SZ) * 2;
+		long size = 0;
+		// LOG.debug("OLE cols: " + nrColumns + " vals: " + nrValues + " pointers: " + (nrValues / nrColumns + 1)
+		// + " offsetLength: " + (offsetLength) + " runs: " + nrValues / nrColumns);
+		size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues / nrColumns) + 1, offsetLength);
+		size += MemoryEstimates.intArrayCost((int) nrValues / nrColumns);
+		return size;
+	}
+
+	public static long estimateInMemorySizeRLE(int nrColumns, int nrValues, int nrRuns, int nrRows) {
+		nrColumns = nrColumns > 0 ? nrColumns : 1;
+		int offsetLength = (nrRuns) * 2;
+		// LOG.debug("\n\tRLE cols: " + nrColumns + " vals: " + nrValues + " offsetLength: " + offsetLength);
+		long size = estimateInMemorySizeOffset(nrColumns, nrValues, (nrValues / nrColumns) + 1, offsetLength);
+
+		return size;
+	}
+
+	public static long estimateInMemorySizeUncompressed(int nrRows, int nrColumns, double sparsity) {
+		long size = 0;
+		// 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);
+		return size;
+	}
+}
\ No newline at end of file
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupUncompressed.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
similarity index 88%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupUncompressed.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
index 25a3f93..6d0d865 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupUncompressed.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupUncompressed.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -26,6 +26,8 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.sysds.runtime.DMLCompressionException;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.data.SparseBlock;
 import org.apache.sysds.runtime.data.SparseBlock.Type;
 import org.apache.sysds.runtime.functionobjects.ReduceRow;
@@ -52,36 +54,41 @@ public class ColGroupUncompressed extends ColGroup {
 	private MatrixBlock _data;
 
 	public ColGroupUncompressed() {
-		super((int[]) null, -1);
+		super(new int[] {}, -1);
+	}
+
+	public long getValuesSize() {
+		throw new DMLCompressionException("Should not currently be used to estimate uncompressed size.");
 	}
 
 	/**
-	 * Main constructor.
+	 * Main constructor for Uncompressed ColGroup.
 	 * 
-	 * @param colIndicesList indices (relative to the current block) of the columns that this column group represents.
-	 * @param rawblock       the uncompressed block; uncompressed data must be present at the time that the constructor
+	 * @param colIndicesList Indices (relative to the current block) of the columns that this column group represents.
+	 * @param rawBlock       The uncompressed block; uncompressed data must be present at the time that the constructor
 	 *                       is called
+	 * @param compSettings   The Settings for how to compress this block, Here using information about the raw block if
+	 *                       it is transposed.
 	 */
-	@SuppressWarnings("unused")
-	public ColGroupUncompressed(List<Integer> colIndicesList, MatrixBlock rawblock) {
-		super(colIndicesList, CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.getNumColumns() : rawblock.getNumRows());
+	public ColGroupUncompressed(int[] colIndicesList, MatrixBlock rawBlock, CompressionSettings compSettings) {
+		super(colIndicesList, compSettings.transposeInput ? rawBlock.getNumColumns() : rawBlock.getNumRows());
 
 		// prepare meta data
-		int numRows = CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.getNumColumns() : rawblock.getNumRows();
+		int numRows = compSettings.transposeInput ? rawBlock.getNumColumns() : rawBlock.getNumRows();
 
 		// Create a matrix with just the requested rows of the original block
-		_data = new MatrixBlock(numRows, _colIndexes.length, rawblock.isInSparseFormat());
+		_data = new MatrixBlock(numRows, _colIndexes.length, rawBlock.isInSparseFormat());
 
 		// ensure sorted col indices
 		if(!SortUtils.isSorted(0, _colIndexes.length, _colIndexes))
 			Arrays.sort(_colIndexes);
 
 		// special cases empty blocks
-		if(rawblock.isEmptyBlock(false))
+		if(rawBlock.isEmptyBlock(false))
 			return;
 		// special cases full block
-		if(!CompressedMatrixBlock.TRANSPOSE_INPUT && _data.getNumColumns() == rawblock.getNumColumns()) {
-			_data.copy(rawblock);
+		if(!compSettings.transposeInput && _data.getNumColumns() == rawBlock.getNumColumns()) {
+			_data.copy(rawBlock);
 			return;
 		}
 
@@ -90,8 +97,9 @@ public class ColGroupUncompressed extends ColGroup {
 		int n = _colIndexes.length;
 		for(int i = 0; i < m; i++) {
 			for(int j = 0; j < n; j++) {
-				double val = CompressedMatrixBlock.TRANSPOSE_INPUT ? rawblock.quickGetValue(_colIndexes[j],
-					i) : rawblock.quickGetValue(i, _colIndexes[j]);
+				double val = compSettings.transposeInput ?
+					rawBlock.quickGetValue(_colIndexes[j], i) :
+					rawBlock.quickGetValue(i, _colIndexes[j]);
 				_data.appendValue(i, j, val);
 			}
 		}
@@ -143,6 +151,11 @@ public class ColGroupUncompressed extends ColGroup {
 		return CompressionType.UNCOMPRESSED;
 	}
 
+	@Override
+	protected ColGroupType getColGroupType() {
+		return ColGroupType.UNCOMPRESSED;
+	}
+
 	/**
 	 * Access for superclass
 	 * 
@@ -181,9 +194,7 @@ public class ColGroupUncompressed extends ColGroup {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-		// adding the size of colContents
-		return size + 8 + _data.estimateSizeInMemory();
+		return ColGroupSizes.estimateInMemorySizeUncompressed(_numRows, getNumCols(), _data.getSparsity());
 	}
 
 	@Override
@@ -361,7 +372,7 @@ public class ColGroupUncompressed extends ColGroup {
 	}
 
 	@Override
-	protected void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
+	public void countNonZerosPerRow(int[] rnnz, int rl, int ru) {
 		for(int i = rl; i < ru; i++)
 			rnnz[i - rl] += _data.recomputeNonZeros(i, i, 0, _data.getNumColumns() - 1);
 	}
@@ -450,4 +461,13 @@ public class ColGroupUncompressed extends ColGroup {
 			}
 		}
 	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append("\n");
+		sb.append(_data.toString());
+		return sb.toString();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ColGroupValue.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
similarity index 80%
rename from src/main/java/org/apache/sysds/runtime/compress/ColGroupValue.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
index 8c0e6c0..a1c120c 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ColGroupValue.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/ColGroupValue.java
@@ -17,14 +17,22 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
 import java.util.Arrays;
 
+import org.apache.sysds.runtime.DMLScriptException;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.functionobjects.Builtin;
 import org.apache.sysds.runtime.functionobjects.Builtin.BuiltinCode;
 import org.apache.sysds.runtime.functionobjects.KahanFunction;
 import org.apache.sysds.runtime.functionobjects.KahanPlus;
+import org.apache.sysds.runtime.functionobjects.KahanPlusSq;
+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.instructions.cp.KahanObject;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.data.Pair;
@@ -38,13 +46,6 @@ import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 public abstract class ColGroupValue extends ColGroup {
 	private static final long serialVersionUID = 3786247536054353658L;
 
-	public static boolean LOW_LEVEL_OPT = true;
-
-	// sorting of values by physical length helps by 10-20%, especially for serial, while
-	// slight performance decrease for parallel incl multi-threaded, hence not applied for
-	// distributed operations (also because compression time + garbage collection increases)
-	public static final boolean SORT_VALUES_BY_LENGTH = true;
-
 	// thread-local pairs of reusable temporary vectors for positions and values
 	private static ThreadLocal<Pair<int[], double[]>> memPool = new ThreadLocal<Pair<int[], double[]>>() {
 		@Override
@@ -57,7 +58,7 @@ public abstract class ColGroupValue extends ColGroup {
 	protected double[] _values; // linearized <numcol vals> <numcol vals>
 
 	public ColGroupValue() {
-		super((int[]) null, -1);
+		super();
 	}
 
 	/**
@@ -71,7 +72,7 @@ public abstract class ColGroupValue extends ColGroup {
 		super(colIndices, numRows);
 
 		// sort values by frequency, if requested
-		if(LOW_LEVEL_OPT && SORT_VALUES_BY_LENGTH && numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
+		if(CompressionSettings.SORT_VALUES_BY_LENGTH && numRows > BitmapEncoder.BITMAP_BLOCK_SZ) {
 			ubm.sortValuesByFrequency();
 		}
 
@@ -93,13 +94,7 @@ public abstract class ColGroupValue extends ColGroup {
 
 	@Override
 	public long estimateInMemorySize() {
-		long size = super.estimateInMemorySize();
-
-		// adding the size of values
-		size += 8; // array reference
-		size += getValuesSize(); // values
-
-		return size;
+		return ColGroupSizes.estimateInMemorySizeGroupValue(_colIndexes.length, getValuesSize());
 	}
 
 	public long getValuesSize() {
@@ -350,7 +345,44 @@ public abstract class ColGroupValue extends ColGroup {
 	 * @param rl     row lower index, inclusive
 	 * @param ru     row upper index, exclusive
 	 */
-	public abstract void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result, int rl, int ru);
+	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result, int rl, int ru) {
+		// sum and sumsq (reduceall/reducerow over tuples and counts)
+		if(op.aggOp.increOp.fn instanceof KahanPlus || op.aggOp.increOp.fn instanceof KahanPlusSq) {
+			KahanFunction kplus = (op.aggOp.increOp.fn instanceof KahanPlus) ? KahanPlus
+				.getKahanPlusFnObject() : KahanPlusSq.getKahanPlusSqFnObject();
+
+			if(op.indexFn instanceof ReduceAll)
+				computeSum(result, kplus);
+			else if(op.indexFn instanceof ReduceCol)
+				computeRowSums(result, kplus, rl, ru);
+			else if(op.indexFn instanceof ReduceRow)
+				computeColSums(result, kplus);
+		}
+		// 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;
+
+			if(op.indexFn instanceof ReduceAll)
+				computeMxx(result, builtin, _zeros);
+			else if(op.indexFn instanceof ReduceCol)
+				computeRowMxx(result, builtin, rl, ru);
+			else if(op.indexFn instanceof ReduceRow)
+				computeColMxx(result, builtin, _zeros);
+		}
+		else {
+			throw new DMLScriptException("Unknown UnaryAggregate operator on CompressedMatrixBlock");
+		}
+	}
+
+	protected abstract void computeSum(MatrixBlock result, KahanFunction kplus );
+
+	protected abstract void computeRowSums(MatrixBlock result, KahanFunction kplus, int rl, int ru);
+
+	protected abstract void computeColSums(MatrixBlock result, KahanFunction kplus);
+
+	protected abstract void computeRowMxx(MatrixBlock result, Builtin builtin, int rl, int ru);
 
 	// dynamic memory management
 
@@ -392,4 +424,15 @@ public abstract class ColGroupValue extends ColGroup {
 			Arrays.fill(tmp, 0, len, 0);
 		return tmp;
 	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append(super.toString());
+		sb.append(String.format("\n%15s%5d ", "Columns:", this._colIndexes.length));
+		sb.append(Arrays.toString(this._colIndexes));
+		sb.append(String.format("\n%15s%5d ", "Values:", this._values.length));
+		sb.append(Arrays.toString(this._values));
+		return sb.toString();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/DenseRowIterator.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/DenseRowIterator.java
similarity index 83%
rename from src/main/java/org/apache/sysds/runtime/compress/DenseRowIterator.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/DenseRowIterator.java
index a12b0ec..5b593ea 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/DenseRowIterator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/DenseRowIterator.java
@@ -17,16 +17,18 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
-import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
-class DenseRowIterator extends RowIterator<double[]> {
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+
+public class DenseRowIterator extends RowIterator<double[]> {
 
 	private final double[] _ret;
 
-	public DenseRowIterator(int rl, int ru, ArrayList<ColGroup> colGroups, int clen) {
+	public DenseRowIterator(int rl, int ru, List<ColGroup> colGroups, int clen) {
 		super(rl, ru, colGroups);
 		_ret = new double[clen];
 	}
@@ -45,4 +47,4 @@ class DenseRowIterator extends RowIterator<double[]> {
 		_rpos++;
 		return _ret;
 	}
-}
\ No newline at end of file
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/RowIterator.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/RowIterator.java
similarity index 84%
rename from src/main/java/org/apache/sysds/runtime/compress/RowIterator.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/RowIterator.java
index daa26f4..aa6caaf 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/RowIterator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/RowIterator.java
@@ -17,25 +17,25 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
 
-import org.apache.sysds.runtime.compress.ColGroup.ColGroupRowIterator;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.ColGroupRowIterator;
 
 abstract class RowIterator<T> implements Iterator<T> {
 	// iterator configuration
 	protected final int _rl;
 	protected final int _ru;
 
-	private final ArrayList<ColGroup> _colGroups;
+	private final List<ColGroup> _colGroups;
 
 	// iterator state
 	protected ColGroupRowIterator[] _iters = null;
 	protected int _rpos;
 
-	public RowIterator(int rl, int ru, ArrayList<ColGroup> colGroups) {
+	public RowIterator(int rl, int ru, List<ColGroup> colGroups) {
 		_rl = rl;
 		_ru = ru;
 		_colGroups = colGroups;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/SparseRowIterator.java b/src/main/java/org/apache/sysds/runtime/compress/colgroup/SparseRowIterator.java
similarity index 86%
rename from src/main/java/org/apache/sysds/runtime/compress/SparseRowIterator.java
rename to src/main/java/org/apache/sysds/runtime/compress/colgroup/SparseRowIterator.java
index 79a0be6..ae88c6d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/SparseRowIterator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/colgroup/SparseRowIterator.java
@@ -17,18 +17,19 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.colgroup;
 
-import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.sysds.runtime.compress.BitmapEncoder;
 import org.apache.sysds.runtime.data.SparseRow;
 import org.apache.sysds.runtime.data.SparseRowVector;
 
-class SparseRowIterator extends RowIterator<SparseRow> {
+public class SparseRowIterator extends RowIterator<SparseRow> {
 	private final SparseRowVector _ret;
 	private final double[] _tmp;
 
-	public SparseRowIterator(int rl, int ru, ArrayList<ColGroup> colGroups, int clen) {
+	public SparseRowIterator(int rl, int ru, List<ColGroup> colGroups, int clen) {
 		super(rl, ru, colGroups);
 		_ret = new SparseRowVector(clen);
 		_tmp = new double[clen];
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimationFactors.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimationFactors.java
new file mode 100644
index 0000000..52e69d7
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimationFactors.java
@@ -0,0 +1,147 @@
+/*
+ * 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.estim;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Logger;
+import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+/**
+ * Compressed Size Estimation factors. Contains meta information used to estimate the compression sizes of given columns
+ * into given CompressionFormats
+ */
+public class CompressedSizeEstimationFactors implements Comparable<CompressedSizeEstimationFactors> {
+	static {
+		// Set to avoid constructing multiple main loggers.
+		Logger.getLogger("org.apache.sysds.runtime.compress.estim");
+	}
+
+	protected static final Log LOG = LogFactory.getLog(CompressedSizeEstimationFactors.class.getName());
+
+	protected final int numCols; // Number of columns in the compressed group
+	protected final int numVals; // Number of unique values in the compressed group
+	protected final int numOffs; // num OLE offsets
+	protected final int numRuns; // num RLE runs
+	protected final int numSingle; // num singletons
+	protected final int numRows;
+	protected final boolean containsZero;
+
+	protected CompressedSizeEstimationFactors(int numCols, int numVals, int numOffs, int numRuns, int numSingle,
+		int numRows, boolean containsZero) {
+		this.numCols = numCols;
+		this.numVals = numVals;
+		this.numOffs = numOffs;
+		this.numRuns = numRuns;
+		this.numSingle = numSingle;
+		this.numRows = numRows;
+		this.containsZero = containsZero;
+		LOG.debug(this);
+	}
+
+	protected static CompressedSizeEstimationFactors computeSizeEstimationFactors(UncompressedBitmap ubm,
+		boolean inclRLE, int numRows, int numCols) {
+
+		int numVals = ubm.getNumValues();
+
+		// TODO: fix the UncompressedBitmap to contain information of if the specific columns extracted
+		// contains zero values.
+		// This is still not contained in the list because default behavior is to ignore 0 values.
+		boolean containsZero = false;
+
+		int numRuns = 0;
+		int numOffs = 0;
+		int numSingle = 0;
+
+		LOG.debug("NumCols :" + numCols);
+
+		// compute size estimation factors
+		for(int i = 0; i < numVals; i++) {
+			int listSize = ubm.getNumOffsets(i);
+			numOffs += listSize;
+			numSingle += (listSize == 1) ? 1 : 0;
+			if(inclRLE) {
+				int[] list = ubm.getOffsetsList(i).extractValues();
+				int lastOff = -2;
+				numRuns += list[listSize - 1] / (BitmapEncoder.BITMAP_BLOCK_SZ - 1);
+				for(int j = 0; j < listSize; j++) {
+					if(list[j] != lastOff + 1) {
+						numRuns++;
+					}
+					lastOff = list[j];
+				}
+			}
+		}
+
+		return new CompressedSizeEstimationFactors(numCols, numVals * numCols, numOffs + numVals, numRuns, numSingle,
+			numRows, containsZero);
+	}
+
+	protected Iterable<Integer> fieldIterator() {
+		ArrayList<Integer> fields = new ArrayList<>();
+		fields.add(new Integer(numCols));
+		fields.add(numVals);
+		fields.add(numOffs);
+		fields.add(numRuns);
+		fields.add(numSingle);
+		fields.add(numRows);
+		fields.add(containsZero ? 1 : 0);
+		return fields;
+	}
+
+	public int compareTo(CompressedSizeEstimationFactors that) {
+		int diff = 0;
+		Iterator<Integer> thisF = this.fieldIterator().iterator();
+		Iterator<Integer> thatF = that.fieldIterator().iterator();
+
+		while(thisF.hasNext() && thatF.hasNext()) {
+			Integer thisV = thisF.next();
+			Integer thatV = thatF.next();
+
+			if(thisV == thatV) {
+				diff = diff << 1;
+			}
+			else if(thisV > thatV) {
+				diff = diff + 1 << 1;
+			}
+			else {
+				diff = diff - 1 << 1;
+			}
+		}
+		return diff;
+	}
+
+	@Override
+	public String toString() {
+		StringBuilder sb = new StringBuilder();
+		sb.append("\nrows:" + numRows);
+		sb.append("\tcols:" + numCols);
+		sb.append("\tnum Offsets:" + numOffs);
+		sb.append("\tnum Singles:" + numSingle);
+		sb.append("\tnum Runs:" + numRuns);
+		sb.append("\tnum Unique Vals:" + numVals);
+		sb.append("\tcontains a 0: " + containsZero);
+		return sb.toString();
+	}
+}
\ No newline at end of file
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 c2ed94e..9f71f0d 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
@@ -19,134 +19,234 @@
 
 package org.apache.sysds.runtime.compress.estim;
 
-import org.apache.sysds.runtime.compress.BitmapEncoder;
-import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.UncompressedBitmap;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.util.CommonThreadPool;
 
+/**
+ * Main abstract class for estimating size of compressions on columns.
+ */
 public abstract class CompressedSizeEstimator {
 
+	private static final boolean LOCAL_DEBUG = false;
+	private static final Level LOCAL_DEBUG_LEVEL = Level.DEBUG;
+	static {
+		if(LOCAL_DEBUG) {
+			Logger.getLogger("org.apache.sysds.runtime.compress.estim").setLevel(LOCAL_DEBUG_LEVEL);
+		}
+	}
+	protected static final Log LOG = LogFactory.getLog(CompressedSizeEstimator.class.getName());
+
+	/** The Matrix Block to extract the compression estimates from */
 	protected MatrixBlock _data;
+	/** The number of rows in the matrix block, extracted to a field because the matrix could be transposed */
 	protected final int _numRows;
+	/** The number of columns in the matrix block, extracted to a field because the matrix could be transposed */
+	protected final int _numCols;
+	/** The compression settings to use, for estimating the size, and compress the ColGroups. */
+	protected final CompressionSettings _compSettings;
 
-	public CompressedSizeEstimator(MatrixBlock data) {
+	/**
+	 * Main Constructor for Compression Estimator.
+	 * 
+	 * 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.
+	 */
+	protected CompressedSizeEstimator(MatrixBlock data, CompressionSettings compSettings) {
 		_data = data;
-		_numRows = CompressedMatrixBlock.TRANSPOSE_INPUT ? _data.getNumColumns() : _data.getNumRows();
+		_numRows = compSettings.transposeInput ? _data.getNumColumns() : _data.getNumRows();
+		_numCols = compSettings.transposeInput ? _data.getNumRows() : _data.getNumColumns();
+		_compSettings = compSettings;
+	}
+
+	/**
+	 * Single threaded version of extracting Compression Size info
+	 * 
+	 * @return The Compression Size info of each Column compressed isolated.
+	 */
+	public CompressedSizeInfo computeCompressedSizeInfos() {
+		return computeCompressedSizeInfos(1);
 	}
 
-	public int getNumRows() {
-		return _numRows;
+	/**
+	 * Multi threaded version of extracting Compression Size info
+	 * 
+	 * @param k The concurrency degree.
+	 * @return The Compression Size info of each Column compressed isolated.
+	 */
+	public CompressedSizeInfo computeCompressedSizeInfos(int k) {
+		CompressedSizeInfoColGroup[] sizeInfos = estimateIndividualColumnGroupSizes(k);
+		return computeCompressedSizeInfos(sizeInfos);
 	}
 
-	public abstract CompressedSizeInfo estimateCompressedColGroupSize(int[] colIndexes);
-
-	public abstract CompressedSizeInfo estimateCompressedColGroupSize(UncompressedBitmap ubm);
-
-	protected SizeEstimationFactors computeSizeEstimationFactors(UncompressedBitmap ubm, boolean inclRLE) {
-		int numVals = ubm.getNumValues();
-		int numRuns = 0;
-		int numOffs = 0;
-		int numSegs = 0;
-		int numSingle = 0;
-
-		// compute size estimation factors
-		for(int i = 0; i < numVals; i++) {
-			int[] list = ubm.getOffsetsList(i).extractValues();
-			int listSize = ubm.getNumOffsets(i);
-			numOffs += listSize;
-			numSegs += list[listSize - 1] / BitmapEncoder.BITMAP_BLOCK_SZ + 1;
-			numSingle += (listSize == 1) ? 1 : 0;
-			if(inclRLE) {
-				int lastOff = -2;
-				for(int j = 0; j < listSize; j++) {
-					if(list[j] != lastOff + 1) {
-						numRuns++; // new run
-						numRuns += (list[j] - lastOff) / // empty runs
-							BitmapEncoder.BITMAP_BLOCK_SZ;
-					}
-					lastOff = list[j];
+	private CompressedSizeInfo computeCompressedSizeInfos(CompressedSizeInfoColGroup[] sizeInfos) {
+		List<Integer> colsC = new ArrayList<>();
+		List<Integer> colsUC = new ArrayList<>();
+		HashMap<Integer, Double> compRatios = new HashMap<>();
+		int nnzUC = 0;
+
+		for(int col = 0; col < _numCols; col++) {
+			double uncompSize = sizeInfos[col].getCompressionSize(CompressionType.UNCOMPRESSED);
+			double minCompressedSize = (double) sizeInfos[col].getMinSize();
+			double compRatio = uncompSize / minCompressedSize;
+
+			if(compRatio > 1000) {
+
+				LOG.warn("\n\tVery good CompressionRatio: " + compRatio + "\n\tUncompressedSize: " + uncompSize
+					+ "\tCompressedSize: " + minCompressedSize + "\tType: " + sizeInfos[col].getBestCompressionType());
+			}
+
+			if(compRatio > 1) {
+				colsC.add(col);
+				compRatios.put(col, compRatio);
+			}
+			else {
+				colsUC.add(col);
+				// TODO nnzUC not incrementing as intended outside this function.
+				nnzUC += sizeInfos[col].getEstNnz();
+			}
+		}
+
+		// correction of column classification (reevaluate dense estimates if necessary)
+		if(!MatrixBlock.evalSparseFormatInMemory(_numRows, colsUC.size(), nnzUC) && !colsUC.isEmpty()) {
+			for(int i = 0; i < colsUC.size(); i++) {
+				int col = colsUC.get(i);
+				double uncompSize = MatrixBlock.estimateSizeInMemory(_numRows, 1, 1.0);
+				// CompressedMatrixBlock.getUncompressedSize(numRows, 1, 1.0);
+				double compRatio = uncompSize / sizeInfos[col].getMinSize();
+				if(compRatio > 1) {
+					colsC.add(col);
+					colsUC.remove(i);
+					i--;
+					compRatios.put(col, compRatio);
+					nnzUC -= sizeInfos[col].getEstNnz();
 				}
 			}
 		}
 
-		// construct estimation factors
-		return new SizeEstimationFactors(numVals, numSegs, numOffs, numRuns, numSingle);
+		if(LOG.isTraceEnabled()) {
+			LOG.trace("C: " + Arrays.toString(colsC.toArray(new Integer[0])));
+			LOG.trace(
+				"-- compression ratios: " + Arrays.toString(colsC.stream().map(c -> compRatios.get(c)).toArray()));
+			LOG.trace("UC: " + Arrays.toString(colsUC.toArray(new Integer[0])));
+			LOG.trace(
+				"-- compression ratios: " + Arrays.toString(colsUC.stream().map(c -> compRatios.get(c)).toArray()));
+		}
+
+		return new CompressedSizeInfo(sizeInfos, colsC, colsUC, compRatios, nnzUC);
+
+	}
+
+	private CompressedSizeInfoColGroup[] estimateIndividualColumnGroupSizes(int k) {
+		return (k > 1) ? CompressedSizeInfoColGroup(_numCols, k) : CompressedSizeInfoColGroup(_numCols);
 	}
 
 	/**
-	 * Estimates the number of bytes needed to encode this column group in RLE encoding format.
+	 * Method used for compressing into one type of colGroup
 	 * 
-	 * @param numVals number of value tuples
-	 * @param numRuns number of runs
-	 * @param numCols number of columns
-	 * @return number of bytes to encode column group in RLE format
+	 * @return CompressedSizeInfo on a compressed colGroup compressing the entire matrix into a single colGroup type.
 	 */
-	protected static long getRLESize(int numVals, int numRuns, int numCols) {
-		int ret = 0;
-		// distinct value tuples [double per col]
-		ret += 8 * numVals * numCols;
-		// offset/len fields per distinct value tuple [2xint]
-		ret += 8 * numVals;
-		// run data [2xchar]
-		ret += 4 * numRuns;
-		return ret;
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize() {
+		int[] colIndexes = makeColIndexes();
+		return estimateCompressedColGroupSize(colIndexes);
 	}
 
 	/**
-	 * Estimates the number of bytes needed to encode this column group in OLE format.
+	 * Abstract method for extracting Compressed Size Info of specified columns, together in a single ColGroup
 	 * 
-	 * @param numVals number of value tuples
-	 * @param numOffs number of offsets
-	 * @param numSeqs number of segment headers
-	 * @param numCols number of columns
-	 * @return number of bytes to encode column group in RLE format
+	 * @param colIndexes The Colums to group together inside a ColGroup
+	 * @return The CompressedSizeInformation associated with the selected ColGroups.
 	 */
-	protected static long getOLESize(int numVals, float numOffs, int numSeqs, int numCols) {
-		int ret = 0;
-		// distinct value tuples [double per col]
-		ret += 8 * numVals * numCols;
-		// offset/len fields per distinct value tuple [2xint]
-		ret += 8 * numVals;
-		// offset list data [1xchar]
-		ret += 2 * numOffs;
-		// offset list seqment headers [1xchar]
-		ret += 2 * numSeqs;
-		return ret;
-	}
+	public abstract CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes);
 
 	/**
-	 * Estimates the number of bytes needed to encode this column group in DDC1 or DDC2 format.
+	 * 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 numVals number of value tuples
-	 * @param numRows number of rows
-	 * @param numCols number of columns
-	 * @return number of bytes to encode column group in RLE format
+	 * @param ubm the UncompressedBitMap, either extracted from a sample or from the entier dataset
+	 * @return The size factors estimated from the Bit Map.
 	 */
-	protected static long getDDCSize(int numVals, int numRows, int numCols) {
-		if(numVals > Character.MAX_VALUE - 1)
-			return Long.MAX_VALUE;
-
-		int ret = 0;
-		// distinct value tuples [double per col]
-		ret += 8 * numVals * numCols;
-		// data [byte or char per row]
-		ret += ((numVals > 255) ? 2 : 1) * numRows;
+	public CompressedSizeEstimationFactors estimateCompressedColGroupSize(UncompressedBitmap ubm) {
+		return CompressedSizeEstimationFactors.computeSizeEstimationFactors(ubm,
+			_compSettings.validCompressions.contains(CompressionType.RLE),
+			_numRows,
+			ubm.getNumColumns());
+	}
+
+	// ------------------------------------------------
+	// PARALLEL CODE
+	// ------------------------------------------------
+
+	private CompressedSizeInfoColGroup[] CompressedSizeInfoColGroup(int clen) {
+		CompressedSizeInfoColGroup[] ret = new CompressedSizeInfoColGroup[clen];
+		for(int col = 0; col < clen; col++)
+			ret[col] = estimateCompressedColGroupSize(new int[] {col});
 		return ret;
 	}
 
-	protected static class SizeEstimationFactors {
-		protected int numVals; // num value tuples
-		protected int numSegs; // num OLE segments
-		protected int numOffs; // num OLE offsets
-		protected int numRuns; // num RLE runs
-		protected int numSingle; // num singletons
-
-		protected SizeEstimationFactors(int numvals, int numsegs, int numoffs, int numruns, int numsingle) {
-			numVals = numvals;
-			numSegs = numsegs;
-			numOffs = numoffs;
-			numRuns = numruns;
-			numSingle = numsingle;
+	private CompressedSizeInfoColGroup[] CompressedSizeInfoColGroup(int clen, int k) {
+		try {
+			ExecutorService pool = CommonThreadPool.get(k);
+			ArrayList<SizeEstimationTask> tasks = new ArrayList<>();
+			for(int col = 0; col < clen; col++)
+				tasks.add(new SizeEstimationTask(this, col));
+			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.toArray(new CompressedSizeInfoColGroup[0]);
+		}
+		catch(InterruptedException | ExecutionException e) {
+			throw new DMLRuntimeException(e);
+		}
+	}
+
+	private static class SizeEstimationTask implements Callable<CompressedSizeInfoColGroup> {
+		private final CompressedSizeEstimator _estimator;
+		private final int _col;
+
+		protected SizeEstimationTask(CompressedSizeEstimator estimator, int col) {
+			_estimator = estimator;
+			_col = col;
+		}
+
+		@Override
+		public CompressedSizeInfoColGroup call() {
+			return _estimator.estimateCompressedColGroupSize(new int[] {_col});
+		}
+	}
+
+	// ------------------------------------------------
+	// PARALLEL CODE END
+	// ------------------------------------------------
+
+	// UTIL
+
+	private int[] makeColIndexes() {
+		int[] colIndexes = new int[_numCols];
+		for(int i = 0; i < _numCols; i++) {
+			colIndexes[i] = i;
 		}
+		return colIndexes;
 	}
 }
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 27216ae..6911e69 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -20,32 +20,23 @@
 package org.apache.sysds.runtime.compress.estim;
 
 import org.apache.sysds.runtime.compress.BitmapEncoder;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.UncompressedBitmap;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 /**
  * Exact compressed size estimator (examines entire dataset).
- * 
  */
 public class CompressedSizeEstimatorExact extends CompressedSizeEstimator {
-	public CompressedSizeEstimatorExact(MatrixBlock data) {
-		super(data);
-	}
 
-	@Override
-	public CompressedSizeInfo estimateCompressedColGroupSize(int[] colIndexes) {
-		return estimateCompressedColGroupSize(BitmapEncoder.extractBitmap(colIndexes, _data));
+	public CompressedSizeEstimatorExact(MatrixBlock data, CompressionSettings compSettings) {
+		super(data, compSettings);
 	}
 
 	@Override
-	public CompressedSizeInfo estimateCompressedColGroupSize(UncompressedBitmap ubm) {
-		// compute size estimation factors
-		SizeEstimationFactors fact = computeSizeEstimationFactors(ubm, true);
-
-		// construct new size info summary
-		return new CompressedSizeInfo(fact.numVals, fact.numOffs,
-			getRLESize(fact.numVals, fact.numRuns, ubm.getNumColumns()),
-			getOLESize(fact.numVals, fact.numOffs, fact.numSegs, ubm.getNumColumns()),
-			getDDCSize(fact.numVals, _numRows, ubm.getNumColumns()));
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
+		LOG.debug("CompressedSizeEstimatorExact: " + colIndexes.length);
+		UncompressedBitmap entireBitMap = BitmapEncoder.extractBitmap(colIndexes, _data, _compSettings);
+		return new CompressedSizeInfoColGroup(estimateCompressedColGroupSize(entireBitMap), _compSettings.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 1e4b1d1..8976c0d 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
@@ -19,13 +19,18 @@
 
 package org.apache.sysds.runtime.compress.estim;
 
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 public class CompressedSizeEstimatorFactory {
+
 	public static final boolean EXTRACT_SAMPLE_ONCE = true;
 
-	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, int numRows, long seed, double sampling_ratio) {
-		return (sampling_ratio == 1.0) ? new CompressedSizeEstimatorExact(data) : new CompressedSizeEstimatorSample(
-			data, (int) Math.ceil(numRows * sampling_ratio), seed);
+	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, CompressionSettings compSettings) {
+		long elements = compSettings.transposeInput ? data.getNumColumns() : data.getNumRows();
+		elements = data.getNonZeros() / (compSettings.transposeInput ? data.getNumRows() : data.getNumColumns());
+		return (compSettings.samplingRatio >= 1.0 || elements < 1000) ? new CompressedSizeEstimatorExact(data,
+			compSettings) : new CompressedSizeEstimatorSample(data, compSettings,
+				(int) Math.ceil(elements * compSettings.samplingRatio));
 	}
 }
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 bbafa1b..82c90f5 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -24,25 +24,15 @@ import java.util.HashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math3.analysis.UnivariateFunction;
-import org.apache.commons.math3.analysis.solvers.UnivariateSolverUtils;
-import org.apache.commons.math3.distribution.ChiSquaredDistribution;
+import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.compress.BitmapEncoder;
-import org.apache.sysds.runtime.compress.ReaderColumnSelection;
-import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.compress.UncompressedBitmap;
-import org.apache.sysds.runtime.compress.utils.DblArray;
+import org.apache.sysds.runtime.compress.estim.sample.HassAndStokes;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.UtilFunctions;
 
 public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
-	private final static double SHLOSSER_JACKKNIFE_ALPHA = 0.975;
-	public static final double HAAS_AND_STOKES_ALPHA1 = 0.9; // 0.9 recommended in paper
-	public static final double HAAS_AND_STOKES_ALPHA2 = 30; // 30 recommended in paper
-	public static final int HAAS_AND_STOKES_UJ2A_C = 50; // 50 recommend in paper
-	public static final boolean HAAS_AND_STOKES_UJ2A_CUT2 = true; // cut frequency in half
-	public static final boolean HAAS_AND_STOKES_UJ2A_SOLVE = true; // true recommended
-	public static final int MAX_SOLVE_CACHE_SIZE = 64 * 1024; // global 2MB cache
 
 	private static final Log LOG = LogFactory.getLog(CompressedSizeEstimatorSample.class.getName());
 
@@ -51,20 +41,25 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 
 	/**
 	 * CompressedSizeEstimatorSample, samples from the input data and estimates the size of the compressed matrix.
-	 * @param data the input data sampled from
-	 * @param sampleSize size of the sampling used
-	 * @param seed Seed for the sampling of the matrix if set to -1 random seed based on system time and class hash is selected
+	 * 
+	 * @param data         The input data sampled from
+	 * @param compSettings The Settings used for the sampling, and compression, contains information such as seed.
+	 * @param sampleSize   Size of the sampling used
 	 */
-	public CompressedSizeEstimatorSample(MatrixBlock data, int sampleSize, long seed) {
-		super(data);
+	public CompressedSizeEstimatorSample(MatrixBlock data, CompressionSettings compSettings, int sampleSize) {
+		super(data, compSettings);
 		// get sample of rows, incl eager extraction
-		_sampleRows = getSortedUniformSample(_numRows, sampleSize, seed);
+		if(_numRows < sampleSize) {
+			throw new DMLRuntimeException("SampleSize should always be less than number of rows");
+		}
+
+		_sampleRows = getSortedUniformSample(_numRows, sampleSize, _compSettings.seed);
+
 		if(CompressedSizeEstimatorFactory.EXTRACT_SAMPLE_ONCE) {
 			MatrixBlock select = new MatrixBlock(_numRows, 1, false);
 			for(int i = 0; i < sampleSize; i++)
 				select.quickSetValue(_sampleRows[i], 0, 1);
-			_data = _data
-				.removeEmptyOperations(new MatrixBlock(), !CompressedMatrixBlock.TRANSPOSE_INPUT, true, select);
+			_data = _data.removeEmptyOperations(new MatrixBlock(), !_compSettings.transposeInput, true, select);
 		}
 
 		// establish estimator-local cache for numeric solve
@@ -72,15 +67,17 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	}
 
 	@Override
-	public CompressedSizeInfo estimateCompressedColGroupSize(int[] colIndexes) {
+	public CompressedSizeInfoColGroup estimateCompressedColGroupSize(int[] colIndexes) {
 		int sampleSize = _sampleRows.length;
 		int numCols = colIndexes.length;
 		int[] sampleRows = _sampleRows;
 
 		// extract statistics from sample
 		UncompressedBitmap ubm = CompressedSizeEstimatorFactory.EXTRACT_SAMPLE_ONCE ? BitmapEncoder
-			.extractBitmap(colIndexes, _data) : BitmapEncoder.extractBitmapFromSample(colIndexes, _data, sampleRows);
-		SizeEstimationFactors fact = computeSizeEstimationFactors(ubm, false);
+			.extractBitmap(colIndexes, _data, _compSettings) : BitmapEncoder
+				.extractBitmapFromSample(colIndexes, _data, sampleRows, _compSettings);
+		CompressedSizeEstimationFactors fact = CompressedSizeEstimationFactors
+			.computeSizeEstimationFactors(ubm, false, _numRows, numCols);
 
 		// estimate number of distinct values (incl fixes for anomalies w/ large sample fraction)
 		int totalCardinality = getNumDistinctValues(ubm, _numRows, sampleRows, _solveCache);
@@ -102,32 +99,25 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 
 		// 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));
-		int totalNumSeg = fact.numSegs + numUnseenSeg;
-		int totalNumRuns = getNumRuns(ubm, sampleSize, _numRows, sampleRows) + numUnseenSeg;
+		// int numUnseenSeg = (int) (unseenVals * Math.ceil((double) _numRows / BitmapEncoder.BITMAP_BLOCK_SZ / 2));
+		int totalNumRuns = getNumRuns(ubm, sampleSize, _numRows, sampleRows);
 
-		// construct new size info summary
-		return new CompressedSizeInfo(totalCardinality, numNonZeros,
-			getRLESize(totalCardinality, totalNumRuns, numCols),
-			getOLESize(totalCardinality, numNonZeros, totalNumSeg, numCols),
-			getDDCSize(totalCardinality, _numRows, numCols));
-	}
+		// TODO. Make it possible to detect if the values contains a 0.
+		// Same case as in the Exact estimator, there is no way of knowing currently if a specific column or row
+		// contains
+		// a 0.
+		boolean containsZero = false;
 
-	@Override
-	public CompressedSizeInfo estimateCompressedColGroupSize(UncompressedBitmap ubm) {
-		// compute size estimation factors
-		SizeEstimationFactors fact = computeSizeEstimationFactors(ubm, true);
+		CompressedSizeEstimationFactors totalFacts = new CompressedSizeEstimationFactors(numCols, totalCardinality,
+			numNonZeros, totalNumRuns, fact.numSingle, _numRows, containsZero);
 
 		// construct new size info summary
-		return new CompressedSizeInfo(fact.numVals, fact.numOffs,
-			getRLESize(fact.numVals, fact.numRuns, ubm.getNumColumns()),
-			getOLESize(fact.numVals, fact.numOffs, fact.numSegs, ubm.getNumColumns()),
-			getDDCSize(fact.numVals, _numRows, ubm.getNumColumns()));
+		return new CompressedSizeInfoColGroup(totalFacts, _compSettings.validCompressions);
 	}
 
 	private static int getNumDistinctValues(UncompressedBitmap ubm, int numRows, int[] sampleRows,
 		HashMap<Integer, Double> solveCache) {
-		return haasAndStokes(ubm, numRows, sampleRows.length, solveCache);
+		return HassAndStokes.haasAndStokes(ubm, numRows, sampleRows.length, solveCache);
 	}
 
 	private static int getNumRuns(UncompressedBitmap ubm, int sampleSize, int totalNumRows, int[] sampleRows) {
@@ -300,400 +290,8 @@ public class CompressedSizeEstimatorSample extends CompressedSizeEstimator {
 	 */
 	private static int[] getSortedUniformSample(int range, int smplSize, long seed) {
 		if(smplSize == 0)
-			return new int[] {};
+			throw new DMLRuntimeException("Sample Size of 0 is invalid");
 		return UtilFunctions.getSortedSampleIndexes(range, smplSize, seed);
 	}
 
-	/////////////////////////////////////////////////////
-	// Sample Cardinality Estimator library
-	/////////////////////////////////////////
-
-	/**
-	 * M. Charikar, S. Chaudhuri, R. Motwani, and V. R. Narasayya, Towards estimation error guarantees for distinct
-	 * values, PODS'00.
-	 * 
-	 * @param nRows            number of rows
-	 * @param sampleSize       sample size
-	 * @param sampleRowsReader a reader for the sampled rows
-	 * @return error estimator
-	 */
-	@SuppressWarnings("unused")
-	private static int guaranteedErrorEstimator(int nRows, int sampleSize, ReaderColumnSelection sampleRowsReader) {
-		HashMap<DblArray, Integer> valsCount = getValCounts(sampleRowsReader);
-		// number of values that occur only once
-		int singltonValsCount = 0;
-		int otherValsCount = 0;
-		for(Integer c : valsCount.values()) {
-			if(c == 1)
-				singltonValsCount++;
-			else
-				otherValsCount++;
-		}
-		return (int) Math.round(otherValsCount + singltonValsCount * Math.sqrt(((double) nRows) / sampleSize));
-	}
-
-	/**
-	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. Sampling-Based Estimation of the Number of
-	 * Distinct Values of an Attribute. VLDB'95, Section 3.2.
-	 * 
-	 * @param nRows            number of rows
-	 * @param sampleSize       sample size
-	 * @param sampleRowsReader reader
-	 * @return estimator
-	 */
-	private static int shlosserEstimator(UncompressedBitmap ubm, int nRows, int sampleSize) {
-		double q = ((double) sampleSize) / nRows;
-		double oneMinusQ = 1 - q;
-
-		int numVals = ubm.getNumValues();
-		int[] freqCounts = getFreqCounts(ubm);
-
-		double numerSum = 0, denomSum = 0;
-		int iPlusOne = 1;
-		for(int i = 0; i < freqCounts.length; i++, iPlusOne++) {
-			numerSum += Math.pow(oneMinusQ, iPlusOne) * freqCounts[i];
-			denomSum += iPlusOne * q * Math.pow(oneMinusQ, i) * freqCounts[i];
-		}
-		int estimate = (int) Math.round(numVals + freqCounts[0] * numerSum / denomSum);
-		return estimate < 1 ? 1 : estimate;
-	}
-
-	/**
-	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. Sampling-Based Estimation of the Number of
-	 * Distinct Values of an Attribute. VLDB'95, Section 4.3.
-	 * 
-	 * @param nRows            number of rows
-	 * @param sampleSize       sample size
-	 * @param sampleRowsReader row reader
-	 * @return estimator
-	 */
-	private static int smoothedJackknifeEstimator(UncompressedBitmap ubm, int nRows, int sampleSize) {
-		int numVals = ubm.getNumValues();
-		int[] freqCounts = getFreqCounts(ubm);
-		// all values in the sample are zeros
-		if(freqCounts.length == 0)
-			return 0;
-		// nRows is N and sampleSize is n
-
-		int d = numVals;
-		double f1 = freqCounts[0];
-		int Nn = nRows * sampleSize;
-		double D0 = (d - f1 / sampleSize) / (1 - (nRows - sampleSize + 1) * f1 / Nn);
-		double NTilde = nRows / D0;
-		/*-
-		 *
-		 * h (as defined in eq. 5 in the paper) can be implemented as:
-		 * 
-		 * double h = Gamma(nRows - NTilde + 1) x Gamma.gamma(nRows -sampleSize + 1) 
-		 * 		     ----------------------------------------------------------------
-		 *  		Gamma.gamma(nRows - sampleSize - NTilde + 1) x Gamma.gamma(nRows + 1)
-		 * 
-		 * 
-		 * However, for large values of nRows, Gamma.gamma returns NAN
-		 * (factorial of a very large number).
-		 * 
-		 * The following implementation solves this problem by levaraging the
-		 * cancelations that show up when expanding the factorials in the
-		 * numerator and the denominator.
-		 * 
-		 * 
-		 * 		min(A,D-1) x [min(A,D-1) -1] x .... x B
-		 * h = -------------------------------------------
-		 * 		C x [C-1] x .... x max(A+1,D)
-		 * 
-		 * where A = N-\tilde{N}
-		 *       B = N-\tilde{N} - n + a
-		 *       C = N
-		 *       D = N-n+1
-		 *       
-		 * 		
-		 *
-		 */
-		double A = nRows - NTilde;
-		double B = A - sampleSize + 1;
-		double C = nRows;
-		double D = nRows - sampleSize + 1;
-		A = Math.min(A, D - 1);
-		D = Math.max(A + 1, D);
-		double h = 1;
-
-		for(; A >= B || C >= D; A--, C--) {
-			if(A >= B)
-				h *= A;
-			if(C >= D)
-				h /= C;
-		}
-		// end of h computation
-
-		double g = 0, gamma = 0;
-		// k here corresponds to k+1 in the paper (the +1 comes from replacing n
-		// with n-1)
-		for(int k = 2; k <= sampleSize + 1; k++) {
-			g += 1.0 / (nRows - NTilde - sampleSize + k);
-		}
-		for(int i = 1; i <= freqCounts.length; i++) {
-			gamma += i * (i - 1) * freqCounts[i - 1];
-		}
-		gamma *= (nRows - 1) * D0 / Nn / (sampleSize - 1);
-		gamma += D0 / nRows - 1;
-
-		double estimate = (d + nRows * h * g * gamma) / (1 - (nRows - NTilde - sampleSize + 1) * f1 / Nn);
-		return estimate < 1 ? 1 : (int) Math.round(estimate);
-	}
-
-	/**
-	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. 1995. Sampling-Based Estimation of the Number
-	 * of Distinct Values of an Attribute. VLDB'95, Section 5.2, recommended estimator by the authors
-	 * 
-	 * @param nRows            number of rows
-	 * @param sampleSize       sample size
-	 * @param sampleRowsReader row reader
-	 * @return estimator
-	 */
-	@SuppressWarnings("unused")
-	private static int shlosserJackknifeEstimator(UncompressedBitmap ubm, int nRows, int sampleSize) {
-		int numVals = ubm.getNumValues();
-		CriticalValue cv = computeCriticalValue(sampleSize);
-
-		// uniformity chi-square test
-		double nBar = ((double) sampleSize) / numVals;
-		// test-statistic
-		double u = 0;
-		for(int i = 0; i < numVals; i++) {
-			u += Math.pow(ubm.getNumOffsets(i) - nBar, 2);
-		}
-		u /= nBar;
-		if(sampleSize != cv.usedSampleSize)
-			computeCriticalValue(sampleSize);
-		if(u < cv.uniformityCriticalValue) // uniform
-			return smoothedJackknifeEstimator(ubm, nRows, sampleSize);
-		else
-			return shlosserEstimator(ubm, nRows, sampleSize);
-	}
-
-	private static CriticalValue computeCriticalValue(int sampleSize) {
-		ChiSquaredDistribution chiSqr = new ChiSquaredDistribution(sampleSize - 1);
-		return new CriticalValue(chiSqr.inverseCumulativeProbability(SHLOSSER_JACKKNIFE_ALPHA), sampleSize);
-	}
-
-	/**
-	 * Haas, Peter J., and Lynne Stokes. "Estimating the number of classes in a finite population." Journal of the
-	 * American Statistical Association 93.444 (1998): 1475-1487.
-	 * 
-	 * The hybrid estimator given by Eq. 33 in Section 6
-	 * 
-	 * @param nRows            number of rows
-	 * @param sampleSize       sample size
-	 * @param solveCache
-	 * @param sampleRowsReader row reader
-	 * @return estimator
-	 */
-	private static int haasAndStokes(UncompressedBitmap ubm, int nRows, int sampleSize,
-		HashMap<Integer, Double> solveCache) {
-		// obtain value and frequency histograms
-		int numVals = ubm.getNumValues();
-		int[] freqCounts = getFreqCounts(ubm);
-
-		// all values in the sample are zeros.
-		if(numVals == 0)
-			return 1;
-
-		double q = ((double) sampleSize) / nRows;
-		double f1 = freqCounts[0];
-
-		// compute basic Duj1 estimate
-		double duj1 = getDuj1Estimate(q, f1, sampleSize, numVals);
-
-		// compute gamma based on Duj1
-		double gamma = getGammaSquared(duj1, freqCounts, sampleSize, nRows);
-		double d = -1;
-
-		// core hybrid estimator based on gamma
-		if(gamma < HAAS_AND_STOKES_ALPHA1)
-			d = getDuj2Estimate(q, f1, sampleSize, numVals, gamma);
-		else if(gamma < HAAS_AND_STOKES_ALPHA2)
-			d = getDuj2aEstimate(q, freqCounts, sampleSize, numVals, gamma, nRows, solveCache);
-		else
-			d = getSh3Estimate(q, freqCounts, numVals);
-
-		// round and ensure min value 1
-		return Math.max(1, (int) Math.round(d));
-	}
-
-	private static HashMap<DblArray, Integer> getValCounts(ReaderColumnSelection sampleRowsReader) {
-		HashMap<DblArray, Integer> valsCount = new HashMap<>();
-		DblArray val = null;
-		Integer cnt;
-		while(null != (val = sampleRowsReader.nextRow())) {
-			cnt = valsCount.get(val);
-			if(cnt == null)
-				cnt = 0;
-			cnt++;
-			valsCount.put(new DblArray(val), cnt);
-		}
-		return valsCount;
-	}
-
-	/**
-	 * Creates an inverted histogram, where freqCounts[i-1] indicates how many values occurred with a frequency i. Note
-	 * that freqCounts[0] represents the special values of the number of singletons.
-	 * 
-	 * @param ubm uncompressed bitmap
-	 * @return frequency counts
-	 */
-	private static int[] getFreqCounts(UncompressedBitmap ubm) {
-		// determine max frequency
-		int numVals = ubm.getNumValues();
-		int maxCount = 0;
-		for(int i = 0; i < numVals; i++)
-			maxCount = Math.max(maxCount, ubm.getNumOffsets(i));
-
-		// create frequency histogram
-		int[] freqCounts = new int[maxCount];
-		for(int i = 0; i < numVals; i++)
-			freqCounts[ubm.getNumOffsets(i) - 1]++;
-
-		return freqCounts;
-
-	}
-
-	/**
-	 * Computes the "unsmoothed first-order jackknife estimator" (Eq 11).
-	 * 
-	 */
-	private static double getDuj1Estimate(double q, double f1, int n, int dn) {
-		return dn / (1 - ((1 - q) * f1) / n);
-	}
-
-	/**
-	 * Computes the "unsmoothed second-order jackknife estimator" (Eq 18b).
-	 * 
-	 */
-	private static double getDuj2Estimate(double q, double f1, int n, int dn, double gammaDuj1) {
-		return (dn - (1 - q) * f1 * Math.log(1 - q) * gammaDuj1 / q) / (1 - ((1 - q) * f1) / n);
-	}
-
-	/**
-	 * Computes the "unsmoothed second-order jackknife estimator" with additional stabilization procedure, which removes
-	 * the classes whose frequency exceed c, computes Duj2 over the reduced sample, and finally adds the removed
-	 * frequencies.
-	 * 
-	 */
-	private static double getDuj2aEstimate(double q, int f[], int n, int dn, double gammaDuj1, int N,
-		HashMap<Integer, Double> solveCache) {
-		int c = HAAS_AND_STOKES_UJ2A_CUT2 ? f.length / 2 + 1 : HAAS_AND_STOKES_UJ2A_C + 1;
-
-		// compute adjusted sample size after removing classes that
-		// exceed a fixed frequency c
-		int nB = 0, cardB = 0;
-		for(int i = c; i <= f.length; i++)
-			if(f[i - 1] != 0) {
-				nB += f[i - 1] * i; // numVals times frequency
-				cardB += f[i - 1];
-			}
-
-		// fallback to Duj2 over full sample if only high frequency columns
-		if(n - nB == 0)
-			return getDuj2Estimate(q, f[0], n, dn, gammaDuj1);
-
-		// compute reduced population size via numeric solve
-		int updatedN = N;
-		for(int i = c; i <= f.length; i++)
-			if(f[i - 1] != 0)
-				updatedN -= f[i - 1] *
-					(!HAAS_AND_STOKES_UJ2A_SOLVE ? i / q : getMethodOfMomentsEstimate(i, q, 1, N, solveCache));
-
-		// remove classes that exceed a fixed frequency c
-		for(int i = c; i <= f.length; i++)
-			f[i - 1] = 0;
-
-		// compute duj2a over reduced sample
-		double updatedDuj1 = getDuj1Estimate(q, f[0], n - nB, dn - cardB);
-		double updatedGammaDuj1 = getGammaSquared(updatedDuj1, f, n - nB, updatedN);
-		double duj2 = getDuj2Estimate(q, f[0], n - nB, dn - cardB, updatedGammaDuj1);
-		return duj2 + cardB;
-	}
-
-	/**
-	 * Computed the "shlosser third-order estimator". (Eq 30b)
-	 * 
-	 * Note that this estimator can show anomalies with NaN as the results due to terms such as Math.pow(1+q, i) which
-	 * exceed Double.MAX_VALUE even for moderately large i, e.g., q=0.05 at around 14K.
-	 * 
-	 */
-	private static double getSh3Estimate(double q, int[] f, double dn) {
-		double fraq11 = 0, fraq12 = 0, fraq21 = 0, fraq22 = 0;
-		for(int i = 1; i <= f.length; i++)
-			if(f[i - 1] != 0) {
-				fraq11 += i * q * q * Math.pow(1 - q * q, i - 1) * f[i - 1];
-				// NOTE: numerically unstable due to Math.pow(1+q, i) overflows
-				// fraq12 += Math.pow(1 - q, i) * (Math.pow(1+q, i)-1) * f[i-1];
-				fraq12 += (Math.pow(1 - q * q, i) - Math.pow(1 - q, i)) * f[i - 1];
-				fraq21 += Math.pow(1 - q, i) * f[i - 1];
-				fraq22 += i * q * Math.pow(1 - q, i - 1) * f[i - 1];
-			}
-		return dn + f[0] * fraq11 / fraq12 * Math.pow(fraq21 / fraq22, 2);
-	}
-
-	/**
-	 * Computes the "squared coefficient of variation" based on a given initial estimate D (Eq 16).
-	 * 
-	 */
-	private static double getGammaSquared(double D, int[] f, int n, int N) {
-		double gamma = 0;
-		for(int i = 1; i <= f.length; i++)
-			if(f[i - 1] != 0)
-				gamma += i * (i - 1) * f[i - 1];
-		gamma *= D / n / n;
-		gamma += D / N - 1;
-		return Math.max(0, gamma);
-	}
-
-	/**
-	 * Solves the method-of-moments estimate numerically. We use a cache on the same observed instances in the sample as
-	 * q is constant and min/max are chosen conservatively.
-	 * 
-	 */
-	private static double getMethodOfMomentsEstimate(int nj, double q, double min, double max,
-		HashMap<Integer, Double> solveCache) {
-		if(solveCache.containsKey(nj))
-			return solveCache.get(nj);
-
-		double est = UnivariateSolverUtils.solve(new MethodOfMomentsFunction(nj, q), min, max, 1e-9);
-
-		if(solveCache.size() < MAX_SOLVE_CACHE_SIZE)
-			solveCache.put(nj, est);
-
-		return est;
-	}
-
-	/*
-	 * In the shlosserSmoothedJackknifeEstimator as long as the sample size did not change, we will have the same
-	 * critical value each time the estimator is used (given that alpha is the same). We cache the critical value to
-	 * avoid recomputing it in each call.
-	 */
-	private static class CriticalValue {
-		public final double uniformityCriticalValue;
-		public final int usedSampleSize;
-
-		public CriticalValue(double cv, int size) {
-			uniformityCriticalValue = cv;
-			usedSampleSize = size;
-		}
-	}
-
-	private static class MethodOfMomentsFunction implements UnivariateFunction {
-		private final int _nj;
-		private final double _q;
-
-		public MethodOfMomentsFunction(int nj, double q) {
-			_nj = nj;
-			_q = q;
-		}
-
-		@Override
-		public double value(double x) {
-			return _q * x / (1 - Math.pow(1 - _q, x)) - _nj;
-		}
-	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
index b18fcac..62d3c5e 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfo.java
@@ -19,48 +19,49 @@
 
 package org.apache.sysds.runtime.compress.estim;
 
+import java.util.HashMap;
+import java.util.List;
+
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
 
 /**
- * 
- * A helper reusable object for maintaining bitmap sizes
+ * A helper reusable object for maintaining information about estimated compression
  */
 public class CompressedSizeInfo {
-	private final int _estCard;
-	private final int _estNnz;
-	private final long _rleSize;
-	private final long _oleSize;
-	private final long _ddcSize;
-
-	public CompressedSizeInfo(int estCard, int estNnz, long rleSize, long oleSize, long ddcSize) {
-		_estCard = estCard;
-		_estNnz = estNnz;
-		_rleSize = rleSize;
-		_oleSize = oleSize;
-		_ddcSize = ddcSize;
-	}
 
-	public long getRLESize() {
-		return _rleSize;
-	}
+	public CompressedSizeInfoColGroup[] compressionInfo;
+	public List<Integer> colsC;
+	public List<Integer> colsUC;
+	public HashMap<Integer, Double> compRatios;
+	public int nnzUC;
 
-	public long getOLESize() {
-		return _oleSize;
+	public CompressedSizeInfo(CompressedSizeInfoColGroup[] compressionInfo, List<Integer> colsC, List<Integer> colsUC,
+		HashMap<Integer, Double> compRatios, int nnzUC) {
+		this.compressionInfo = compressionInfo;
+		this.colsC = colsC;
+		this.colsUC = colsUC;
+		this.compRatios = compRatios;
+		this.nnzUC = nnzUC;
 	}
 
-	public long getDDCSize() {
-		return CompressedMatrixBlock.ALLOW_DDC_ENCODING ? _ddcSize : Long.MAX_VALUE;
+	public CompressedSizeInfoColGroup getGroupInfo(int index) {
+		return compressionInfo[index];
 	}
 
-	public long getMinSize() {
-		return Math.min(Math.min(getRLESize(), getOLESize()), getDDCSize());
-	}
+	/**
+	 * Method for returning the calculated memory usage from this specific compression plan.
+	 * @return The in memory estimate as a long counting bytes.
+	 */
+	public long memoryEstimate() {
+		// Basic data inherited from MatrixBlock + CompressedMatrixBlock
+		long est = CompressedMatrixBlock.baseSizeInMemory();
+		// Memory usage from all Compression Groups.
+		for(CompressedSizeInfoColGroup csi : compressionInfo) {
+			est += csi.getMinSize();
+		}
 
-	public int getEstCard() {
-		return _estCard;
+		return est;
 	}
 
-	public int getEstNnz() {
-		return _estNnz;
-	}
+	
 }
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
new file mode 100644
index 0000000..ba66a81
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeInfoColGroup.java
@@ -0,0 +1,129 @@
+/*
+ * 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.estim;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+import org.apache.sysds.runtime.compress.colgroup.ColGroupSizes;
+
+/**
+ * Information collected about a specific ColGroup's compression size.
+ */
+public class CompressedSizeInfoColGroup {
+
+	private final int _numVals;
+	private final int _numOffs;
+	private final long _minSize;
+	private final CompressionType _bestCompressionType;
+	private final Map<CompressionType, Long> _sizes;
+
+	public CompressedSizeInfoColGroup(CompressedSizeEstimationFactors fact,
+		List<CompressionType> validCompressionTypes) {
+		_numVals = fact.numVals;
+		_numOffs = fact.numOffs;
+		_sizes = calculateCompressionSizes(fact, validCompressionTypes);
+		Map.Entry<CompressionType, Long> bestEntry = null;
+		for(Map.Entry<CompressionType, Long> ent : _sizes.entrySet()) {
+			if(bestEntry == null || ent.getValue() < bestEntry.getValue())
+				bestEntry = ent;
+		}
+		_bestCompressionType = bestEntry.getKey();
+		_minSize = bestEntry.getValue();
+	}
+
+	public long getCompressionSize(CompressionType ct) {
+		return _sizes.get(ct);
+	}
+
+	public CompressionType getBestCompressionType() {
+		return _bestCompressionType;
+	}
+
+	public Map<CompressionType, Long> getAllCompressionSizes() {
+		return _sizes;
+	}
+
+	public long getMinSize() {
+		return _minSize;
+	}
+
+	/**
+	 * Note cardinality is the same as number of distinct values.
+	 * 
+	 * @return cardinality or number of distinct values.
+	 */
+	public int getEstCard() {
+		return _numVals;
+	}
+
+	/**
+	 * Number of offsets, or number of non zero values.
+	 * 
+	 * @return Number of non zeros or number of values.
+	 */
+	public int getEstNnz() {
+		return _numOffs;
+	}
+
+	private static Map<CompressionType, Long> calculateCompressionSizes(CompressedSizeEstimationFactors fact,
+		List<CompressionType> validCompressionTypes) {
+		Map<CompressionType, Long> res = new HashMap<>();
+		for(CompressionType ct : validCompressionTypes) {
+			res.put(ct, getCompressionSize(ct, fact));
+		}
+		return res;
+	}
+
+	private static Long getCompressionSize(CompressionType ct, CompressedSizeEstimationFactors fact) {
+		long size = 0;
+		switch(ct) {
+			case DDC:
+				if(fact.numVals < 256) {
+					size = ColGroupSizes.estimateInMemorySizeDDC1(fact.numCols,
+						fact.numVals + (fact.containsZero ? 1 : 0),
+						fact.numRows);
+				}
+				else {
+					size = ColGroupSizes.estimateInMemorySizeDDC2(fact.numCols,
+						fact.numVals + (fact.containsZero ? 1 : 0),
+						fact.numRows);
+				}
+				break;
+			case RLE:
+				size = ColGroupSizes.estimateInMemorySizeRLE(fact.numCols, fact.numVals, fact.numRuns, fact.numRows);
+				break;
+			case OLE:
+				size = ColGroupSizes.estimateInMemorySizeOLE(fact.numCols, fact.numVals, fact.numOffs, fact.numRows);
+				break;
+			case UNCOMPRESSED:
+				size = ColGroupSizes.estimateInMemorySizeUncompressed(fact.numRows,
+					fact.numCols,
+					((double) fact.numVals / (fact.numRows * fact.numCols)));
+				break;
+			default:
+				throw new NotImplementedException("The col compression Type is not yet supported");
+		}
+		return size;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/FrequencyCount.java
similarity index 50%
copy from src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java
copy to src/main/java/org/apache/sysds/runtime/compress/estim/sample/FrequencyCount.java
index c809851..6685f11 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/ReaderColumnSelection.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/FrequencyCount.java
@@ -17,44 +17,31 @@
  * under the License.
  */
 
-package org.apache.sysds.runtime.compress;
+package org.apache.sysds.runtime.compress.estim.sample;
 
-import org.apache.sysds.runtime.compress.utils.DblArray;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+public class FrequencyCount {
 
-/**
- * Base class for all column selection readers.
- * 
- */
-public abstract class ReaderColumnSelection 
-{
-	protected int[] _colIndexes = null;
-	protected int _numRows = -1;
-	protected int _lastRow = -1;
-	protected boolean _skipZeros = false;
-	
-	protected ReaderColumnSelection(int[] colIndexes, int numRows, boolean skipZeros) {
-		_colIndexes = colIndexes;
-		_numRows = numRows;
-		_lastRow = -1;
-		_skipZeros = skipZeros;
-	}
-	
 	/**
-	 * Gets the next row, null when no more rows.
+	 * Creates an inverted histogram, where freqCounts[i-1] indicates how many values occurred with a frequency i. Note
+	 * that freqCounts[0] represents the special values of the number of singletons.
 	 * 
-	 * @return next row
+	 * @param ubm uncompressed bitmap
+	 * @return frequency counts
 	 */
-	public abstract DblArray nextRow();
+	protected static int[] get(UncompressedBitmap ubm) {
+		// determine max frequency
+		int numVals = ubm.getNumValues();
+		int maxCount = 0;
+		for(int i = 0; i < numVals; i++)
+			maxCount = Math.max(maxCount, ubm.getNumOffsets(i));
 
-	public int getCurrentRowIndex() {
-		return _lastRow;
-	}
-	
+		// create frequency histogram
+		int[] freqCounts = new int[maxCount];
+		for(int i = 0; i < numVals; i++)
+			freqCounts[ubm.getNumOffsets(i) - 1]++;
 
-	/**
-	 * Resets the reader to the first row.
-	 */
-	public void reset() {
-		_lastRow = -1;
+		return freqCounts;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/GuaranteedErrorEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/GuaranteedErrorEstimator.java
new file mode 100644
index 0000000..dc3e488
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/GuaranteedErrorEstimator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.estim.sample;
+
+import java.util.HashMap;
+
+import org.apache.sysds.runtime.compress.ReaderColumnSelection;
+import org.apache.sysds.runtime.compress.utils.DblArray;
+
+public class GuaranteedErrorEstimator {
+
+	/**
+	 * M. Charikar, S. Chaudhuri, R. Motwani, and V. R. Narasayya, Towards estimation error guarantees for distinct
+	 * values, PODS'00.
+	 * 
+	 * @param nRows            number of rows
+	 * @param sampleSize       sample size
+	 * @param sampleRowsReader a reader for the sampled rows
+	 * @return error estimator
+	 */
+	@SuppressWarnings("unused")
+	private static int guaranteedErrorEstimator(int nRows, int sampleSize, ReaderColumnSelection sampleRowsReader) {
+		HashMap<DblArray, Integer> valsCount = getValCounts(sampleRowsReader);
+		// number of values that occur only once
+		int singltonValsCount = 0;
+		int otherValsCount = 0;
+		for(Integer c : valsCount.values()) {
+			if(c == 1)
+				singltonValsCount++;
+			else
+				otherValsCount++;
+		}
+		return (int) Math.round(otherValsCount + singltonValsCount * Math.sqrt(((double) nRows) / sampleSize));
+	}
+
+	private static HashMap<DblArray, Integer> getValCounts(ReaderColumnSelection sampleRowsReader) {
+		HashMap<DblArray, Integer> valsCount = new HashMap<>();
+		DblArray val = null;
+		Integer cnt;
+		while(null != (val = sampleRowsReader.nextRow())) {
+			cnt = valsCount.get(val);
+			if(cnt == null)
+				cnt = 0;
+			cnt++;
+			valsCount.put(new DblArray(val), cnt);
+		}
+		return valsCount;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/HassAndStokes.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/HassAndStokes.java
new file mode 100644
index 0000000..785f277
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/HassAndStokes.java
@@ -0,0 +1,239 @@
+/*
+ * 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.estim.sample;
+
+import java.util.HashMap;
+
+import org.apache.commons.math3.analysis.UnivariateFunction;
+import org.apache.commons.math3.analysis.solvers.UnivariateSolverUtils;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+public class HassAndStokes {
+
+	public static final double HAAS_AND_STOKES_ALPHA1 = 0.9; // 0.9 recommended in paper
+	public static final double HAAS_AND_STOKES_ALPHA2 = 30; // 30 recommended in paper
+	public static final int HAAS_AND_STOKES_UJ2A_C = 50; // 50 recommend in paper
+	public static final boolean HAAS_AND_STOKES_UJ2A_CUT2 = true; // cut frequency in half
+	public static final boolean HAAS_AND_STOKES_UJ2A_SOLVE = true; // true recommended
+	public static final int MAX_SOLVE_CACHE_SIZE = 64 * 1024; // global 2MB cache
+
+	/**
+	 * Haas, Peter J., and Lynne Stokes. "Estimating the number of classes in a finite population." Journal of the
+	 * American Statistical Association 93.444 (1998): 1475-1487.
+	 * 
+	 * The hybrid estimator given by Eq. 33 in Section 6
+	 * 
+	 * @param ubm        The Uncompressed Bit map
+	 * @param nRows      The number of rows originally in the input
+	 * @param sampleSize The number of rows used in the sample
+	 * @param solveCache A Hashmap containing information for getDuj2aEstimate
+	 * @return An estimation of distinct elements in the population.
+	 */
+	public static int haasAndStokes(UncompressedBitmap ubm, int nRows, int sampleSize,
+		HashMap<Integer, Double> solveCache) {
+		// obtain value and frequency histograms
+		int numVals = ubm.getNumValues();
+		int[] freqCounts = FrequencyCount.get(ubm);
+
+		// all values in the sample are zeros.
+		if(numVals == 0)
+			return 1;
+
+		double q = ((double) sampleSize) / nRows;
+		double f1 = freqCounts[0];
+
+		// compute basic Duj1 estimate
+		double duj1 = getDuj1Estimate(q, f1, sampleSize, numVals);
+
+		// compute gamma based on Duj1
+		double gamma = getGammaSquared(duj1, freqCounts, sampleSize, nRows);
+		double d = -1;
+
+		// core hybrid estimator based on gamma
+		if(gamma < HAAS_AND_STOKES_ALPHA1)
+			d = getDuj2Estimate(q, f1, sampleSize, numVals, gamma);
+		else if(gamma < HAAS_AND_STOKES_ALPHA2)
+			d = getDuj2aEstimate(q, freqCounts, sampleSize, numVals, gamma, nRows, solveCache);
+		else
+			d = getSh3Estimate(q, freqCounts, numVals);
+
+		// round and ensure min value 1
+		return Math.max(1, (int) Math.round(d));
+	}
+
+	/**
+	 * Computes the "un-smoothed first-order jackknife estimator" (Eq 11).
+	 * 
+	 * @param q  ??
+	 * @param f1 ??
+	 * @param n  ??
+	 * @param dn ??
+	 * @return ??
+	 */
+	private static double getDuj1Estimate(double q, double f1, int n, int dn) {
+		return dn / (1 - ((1 - q) * f1) / n);
+	}
+
+	/**
+	 * Computes the "un-smoothed second-order jackknife estimator" (Eq 18b).
+	 *
+	 * @param q         ??
+	 * @param f1        ??
+	 * @param n         ??
+	 * @param dn        ??
+	 * @param gammaDuj1 ??
+	 * @return ??
+	 */
+	private static double getDuj2Estimate(double q, double f1, int n, int dn, double gammaDuj1) {
+		return (dn - (1 - q) * f1 * Math.log(1 - q) * gammaDuj1 / q) / (1 - ((1 - q) * f1) / n);
+	}
+
+	/**
+	 * Computes the "un-smoothed second-order jackknife estimator" with additional stabilization procedure, which
+	 * removes the classes whose frequency exceed c, computes Duj2 over the reduced sample, and finally adds the removed
+	 * frequencies.
+	 * 
+	 * @param q          ??
+	 * @param f          ??
+	 * @param n          ??
+	 * @param dn         ??
+	 * @param gammaDuj1  ??
+	 * @param N          ??
+	 * @param solveCache ??
+	 * @return ??
+	 */
+	private static double getDuj2aEstimate(double q, int f[], int n, int dn, double gammaDuj1, int N,
+		HashMap<Integer, Double> solveCache) {
+		int c = HAAS_AND_STOKES_UJ2A_CUT2 ? f.length / 2 + 1 : HAAS_AND_STOKES_UJ2A_C + 1;
+
+		// compute adjusted sample size after removing classes that
+		// exceed a fixed frequency c
+		int nB = 0, cardB = 0;
+		for(int i = c; i <= f.length; i++)
+			if(f[i - 1] != 0) {
+				nB += f[i - 1] * i; // numVals times frequency
+				cardB += f[i - 1];
+			}
+
+		// fallback to Duj2 over full sample if only high frequency columns
+		if(n - nB == 0)
+			return getDuj2Estimate(q, f[0], n, dn, gammaDuj1);
+
+		// compute reduced population size via numeric solve
+		int updatedN = N;
+		for(int i = c; i <= f.length; i++)
+			if(f[i - 1] != 0)
+				updatedN -= f[i - 1] *
+					(!HAAS_AND_STOKES_UJ2A_SOLVE ? i / q : getMethodOfMomentsEstimate(i, q, 1, N, solveCache));
+
+		// remove classes that exceed a fixed frequency c
+		for(int i = c; i <= f.length; i++)
+			f[i - 1] = 0;
+
+		// compute duj2a over reduced sample
+		double updatedDuj1 = getDuj1Estimate(q, f[0], n - nB, dn - cardB);
+		double updatedGammaDuj1 = getGammaSquared(updatedDuj1, f, n - nB, updatedN);
+		double duj2 = getDuj2Estimate(q, f[0], n - nB, dn - cardB, updatedGammaDuj1);
+		return duj2 + cardB;
+	}
+
+	/**
+	 * Computes the "squared coefficient of variation" based on a given initial estimate D (Eq 16).
+	 * 
+	 * @param D ??
+	 * @param f ??
+	 * @param n ??
+	 * @param N ??
+	 * @return ??
+	 */
+	private static double getGammaSquared(double D, int[] f, int n, int N) {
+		double gamma = 0;
+		for(int i = 1; i <= f.length; i++)
+			if(f[i - 1] != 0)
+				gamma += i * (i - 1) * f[i - 1];
+		gamma *= D / n / n;
+		gamma += D / N - 1;
+		return Math.max(0, gamma);
+	}
+
+	/**
+	 * Computed the "shlosser third-order estimator". (Eq 30b)
+	 * 
+	 * Note that this estimator can show anomalies with NaN as the results due to terms such as Math.pow(1+q, i) which
+	 * exceed Double.MAX_VALUE even for moderately large i, e.g., q=0.05 at around 14K.
+	 * 
+	 * @param q  ??
+	 * @param f  ??
+	 * @param dn ??
+	 * @return ??
+	 */
+	private static double getSh3Estimate(double q, int[] f, double dn) {
+		double fraq11 = 0, fraq12 = 0, fraq21 = 0, fraq22 = 0;
+		for(int i = 1; i <= f.length; i++)
+			if(f[i - 1] != 0) {
+				fraq11 += i * q * q * Math.pow(1 - q * q, i - 1) * f[i - 1];
+				// NOTE: numerically unstable due to Math.pow(1+q, i) overflows
+				// fraq12 += Math.pow(1 - q, i) * (Math.pow(1+q, i)-1) * f[i-1];
+				fraq12 += (Math.pow(1 - q * q, i) - Math.pow(1 - q, i)) * f[i - 1];
+				fraq21 += Math.pow(1 - q, i) * f[i - 1];
+				fraq22 += i * q * Math.pow(1 - q, i - 1) * f[i - 1];
+			}
+		return dn + f[0] * fraq11 / fraq12 * Math.pow(fraq21 / fraq22, 2);
+	}
+
+	/**
+	 * Solves the method-of-moments estimate numerically. We use a cache on the same observed instances in the sample as
+	 * q is constant and min/max are chosen conservatively.
+	 * 
+	 * @param nj         ??
+	 * @param q          ??
+	 * @param min        ??
+	 * @param max        ??
+	 * @param solveCache ??
+	 * @return ??
+	 */
+	private static double getMethodOfMomentsEstimate(int nj, double q, double min, double max,
+		HashMap<Integer, Double> solveCache) {
+		if(solveCache.containsKey(nj))
+			return solveCache.get(nj);
+
+		double est = UnivariateSolverUtils.solve(new MethodOfMomentsFunction(nj, q), min, max, 1e-9);
+
+		if(solveCache.size() < MAX_SOLVE_CACHE_SIZE)
+			solveCache.put(nj, est);
+
+		return est;
+	}
+
+	private static class MethodOfMomentsFunction implements UnivariateFunction {
+		private final int _nj;
+		private final double _q;
+
+		public MethodOfMomentsFunction(int nj, double q) {
+			_nj = nj;
+			_q = q;
+		}
+
+		@Override
+		public double value(double x) {
+			return _q * x / (1 - Math.pow(1 - _q, x)) - _nj;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserEstimator.java
new file mode 100644
index 0000000..3e9962c
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserEstimator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.estim.sample;
+
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+public class ShlosserEstimator {
+
+	/**
+	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. Sampling-Based Estimation of the Number of
+	 * Distinct Values of an Attribute. VLDB'95, Section 3.2.
+	 * 
+	 * @param ubm        The Uncompressed Bitmap containing the data from the sample
+	 * @param nRows      The original number of rows in the entire input
+	 * @param sampleSize The number of rows in the sample
+	 * @return an estimation of number of distinct values.
+	 */
+	public static int get(UncompressedBitmap ubm, int nRows, int sampleSize) {
+		double q = ((double) sampleSize) / nRows;
+		double oneMinusQ = 1 - q;
+
+		int numVals = ubm.getNumValues();
+		int[] freqCounts = FrequencyCount.get(ubm);
+
+		double numerSum = 0, denomSum = 0;
+		int iPlusOne = 1;
+		for(int i = 0; i < freqCounts.length; i++, iPlusOne++) {
+			numerSum += Math.pow(oneMinusQ, iPlusOne) * freqCounts[i];
+			denomSum += iPlusOne * q * Math.pow(oneMinusQ, i) * freqCounts[i];
+		}
+		int estimate = (int) Math.round(numVals + freqCounts[0] * numerSum / denomSum);
+		return estimate < 1 ? 1 : estimate;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserJackknifeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserJackknifeEstimator.java
new file mode 100644
index 0000000..7c04638
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/ShlosserJackknifeEstimator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.estim.sample;
+
+import org.apache.commons.math3.distribution.ChiSquaredDistribution;
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+public class ShlosserJackknifeEstimator {
+
+	private final static double SHLOSSER_JACKKNIFE_ALPHA = 0.975;
+
+	/**
+	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. 1995. Sampling-Based Estimation of the Number
+	 * of Distinct Values of an Attribute. VLDB'95, Section 5.2, recommended estimator by the authors
+	 * 
+	 * @param ubm        The Uncompressed Bitmap containing the data from the sample
+	 * @param nRows      The original number of rows in the entire input
+	 * @param sampleSize The number of rows in the sample
+	 * @return an estimation of number of distinct values.
+	 */
+	@SuppressWarnings("unused")
+	private static int shlosserJackknifeEstimator(UncompressedBitmap ubm, int nRows, int sampleSize) {
+		int numVals = ubm.getNumValues();
+		CriticalValue cv = computeCriticalValue(sampleSize);
+
+		// uniformity chi-square test
+		double nBar = ((double) sampleSize) / numVals;
+		// test-statistic
+		double u = 0;
+		for(int i = 0; i < numVals; i++) {
+			u += Math.pow(ubm.getNumOffsets(i) - nBar, 2);
+		}
+		u /= nBar;
+		if(sampleSize != cv.usedSampleSize)
+			computeCriticalValue(sampleSize);
+		if(u < cv.uniformityCriticalValue) // uniform
+			return SmoothedJackknifeEstimator.get(ubm, nRows, sampleSize);
+		else
+			return ShlosserEstimator.get(ubm, nRows, sampleSize);
+	}
+
+	private static CriticalValue computeCriticalValue(int sampleSize) {
+		ChiSquaredDistribution chiSqr = new ChiSquaredDistribution(sampleSize - 1);
+		return new CriticalValue(chiSqr.inverseCumulativeProbability(SHLOSSER_JACKKNIFE_ALPHA), sampleSize);
+	}
+
+	/*
+	 * In the shlosserSmoothedJackknifeEstimator as long as the sample size did not change, we will have the same
+	 * critical value each time the estimator is used (given that alpha is the same). We cache the critical value to
+	 * avoid recomputing it in each call.
+	 */
+	private static class CriticalValue {
+		public final double uniformityCriticalValue;
+		public final int usedSampleSize;
+
+		public CriticalValue(double cv, int size) {
+			uniformityCriticalValue = cv;
+			usedSampleSize = size;
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SmoothedJackknifeEstimator.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SmoothedJackknifeEstimator.java
new file mode 100644
index 0000000..6282eb0
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SmoothedJackknifeEstimator.java
@@ -0,0 +1,108 @@
+/*
+ * 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.estim.sample;
+
+import org.apache.sysds.runtime.compress.UncompressedBitmap;
+
+public class SmoothedJackknifeEstimator {
+
+	/**
+	 * Peter J. Haas, Jeffrey F. Naughton, S. Seshadri, and Lynne Stokes. Sampling-Based Estimation of the Number of
+	 * Distinct Values of an Attribute. VLDB'95, Section 4.3.
+	 * 
+	 * @param ubm        The Uncompressed Bitmap containing the data from the sample
+	 * @param nRows      The original number of rows in the entire input
+	 * @param sampleSize The number of rows in the sample
+	 * @return Estimate of the number of distinct values
+	 */
+	public static int get(UncompressedBitmap ubm, int nRows, int sampleSize) {
+		int numVals = ubm.getNumValues();
+		int[] freqCounts = FrequencyCount.get(ubm);
+		// all values in the sample are zeros
+		if(freqCounts.length == 0)
+			return 0;
+		// nRows is N and sampleSize is n
+
+		int d = numVals;
+		double f1 = freqCounts[0];
+		int Nn = nRows * sampleSize;
+		double D0 = (d - f1 / sampleSize) / (1 - (nRows - sampleSize + 1) * f1 / Nn);
+		double NTilde = nRows / D0;
+		/*-
+		 *
+		 * h (as defined in eq. 5 in the paper) can be implemented as:
+		 * 
+		 * double h = Gamma(nRows - NTilde + 1) x Gamma.gamma(nRows -sampleSize + 1) 
+		 * 		     ----------------------------------------------------------------
+		 *  		Gamma.gamma(nRows - sampleSize - NTilde + 1) x Gamma.gamma(nRows + 1)
+		 * 
+		 * 
+		 * However, for large values of nRows, Gamma.gamma returns NAN
+		 * (factorial of a very large number).
+		 * 
+		 * The following implementation solves this problem by levaraging the
+		 * cancelations that show up when expanding the factorials in the
+		 * numerator and the denominator.
+		 * 
+		 * 
+		 * 		min(A,D-1) x [min(A,D-1) -1] x .... x B
+		 * h = -------------------------------------------
+		 * 		C x [C-1] x .... x max(A+1,D)
+		 * 
+		 * where A = N-\tilde{N}
+		 *       B = N-\tilde{N} - n + a
+		 *       C = N
+		 *       D = N-n+1
+		 *       
+		 * 		
+		 *
+		 */
+		double A = nRows - NTilde;
+		double B = A - sampleSize + 1;
+		double C = nRows;
+		double D = nRows - sampleSize + 1;
+		A = Math.min(A, D - 1);
+		D = Math.max(A + 1, D);
+		double h = 1;
+
+		for(; A >= B || C >= D; A--, C--) {
+			if(A >= B)
+				h *= A;
+			if(C >= D)
+				h /= C;
+		}
+		// end of h computation
+
+		double g = 0, gamma = 0;
+		// k here corresponds to k+1 in the paper (the +1 comes from replacing n
+		// with n-1)
+		for(int k = 2; k <= sampleSize + 1; k++) {
+			g += 1.0 / (nRows - NTilde - sampleSize + k);
+		}
+		for(int i = 1; i <= freqCounts.length; i++) {
+			gamma += i * (i - 1) * freqCounts[i - 1];
+		}
+		gamma *= (nRows - 1) * D0 / Nn / (sampleSize - 1);
+		gamma += D0 / nRows - 1;
+
+		double estimate = (d + nRows * h * g * gamma) / (1 - (nRows - NTilde - sampleSize + 1) * f1 / Nn);
+		return estimate < 1 ? 1 : (int) Math.round(estimate);
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/ColumnGroupIterator.java b/src/main/java/org/apache/sysds/runtime/compress/utils/ColumnGroupIterator.java
index 6364ae6..9745f85 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/ColumnGroupIterator.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/ColumnGroupIterator.java
@@ -19,10 +19,10 @@
 
 package org.apache.sysds.runtime.compress.utils;
 
-import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
 
-import org.apache.sysds.runtime.compress.ColGroup;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
 import org.apache.sysds.runtime.matrix.data.IJV;
 
 public class ColumnGroupIterator implements Iterator<IJV> {
@@ -36,9 +36,9 @@ public class ColumnGroupIterator implements Iterator<IJV> {
 	private int _posColGroup = -1;
 	private Iterator<IJV> _iterColGroup = null;
 	private boolean _noNext = false;
-	private ArrayList<ColGroup> _colGroups;
+	private List<ColGroup> _colGroups;
 
-	public ColumnGroupIterator(int rl, int ru, int cgl, int cgu, boolean inclZeros, ArrayList<ColGroup> colGroups) {
+	public ColumnGroupIterator(int rl, int ru, int cgl, int cgu, boolean inclZeros, List<ColGroup> colGroups) {
 		_rl = rl;
 		_ru = ru;
 		_cgu = cgu;
diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/utils/CustomHashMap.java
similarity index 61%
copy from src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
copy to src/main/java/org/apache/sysds/runtime/compress/utils/CustomHashMap.java
index 1e4b1d1..1362c6f 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/estim/CompressedSizeEstimatorFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/CustomHashMap.java
@@ -16,16 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.sysds.runtime.compress.utils;
 
-package org.apache.sysds.runtime.compress.estim;
-
-import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+/**
+ * This class provides a memory-efficient base for Custom HashMaps for restricted use cases.
+ */
+public abstract class CustomHashMap {
+	protected static final int INIT_CAPACITY = 8;
+	protected static final int RESIZE_FACTOR = 2;
+	protected static final float LOAD_FACTOR = 0.75f;
 
-public class CompressedSizeEstimatorFactory {
-	public static final boolean EXTRACT_SAMPLE_ONCE = true;
+	protected int _size = -1;
 
-	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, int numRows, long seed, double sampling_ratio) {
-		return (sampling_ratio == 1.0) ? new CompressedSizeEstimatorExact(data) : new CompressedSizeEstimatorSample(
-			data, (int) Math.ceil(numRows * sampling_ratio), seed);
+	public int size() {
+		return _size;
 	}
 }
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 9f8e0b3..49cb5d0 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
@@ -23,21 +23,19 @@ import java.util.Arrays;
 
 /**
  * Helper class used for bitmap extraction.
- *
  */
-public class DblArray 
-{
+public class DblArray {
 	private double[] _arr = null;
 	private boolean _zero = false;
-	
+
 	public DblArray() {
 		this(null, false);
 	}
-	
+
 	public DblArray(double[] arr) {
 		this(arr, false);
 	}
-	
+
 	public DblArray(DblArray that) {
 		this(Arrays.copyOf(that._arr, that._arr.length), that._zero);
 	}
@@ -46,11 +44,11 @@ public class DblArray
 		_arr = arr;
 		_zero = allZeros;
 	}
-	
+
 	public double[] getData() {
 		return _arr;
 	}
-	
+
 	@Override
 	public int hashCode() {
 		return _zero ? 0 : Arrays.hashCode(_arr);
@@ -58,9 +56,7 @@ public class DblArray
 
 	@Override
 	public boolean equals(Object o) {
-		return ( o instanceof DblArray
-			&& _zero == ((DblArray) o)._zero
-			&& Arrays.equals(_arr, ((DblArray) o)._arr) );
+		return(o instanceof DblArray && _zero == ((DblArray) o)._zero && Arrays.equals(_arr, ((DblArray) o)._arr));
 	}
 
 	@Override
@@ -69,8 +65,8 @@ public class DblArray
 	}
 
 	public static boolean isZero(double[] ds) {
-		for (int i = 0; i < ds.length; i++)
-			if (ds[i] != 0.0)
+		for(int i = 0; i < ds.length; i++)
+			if(ds[i] != 0.0)
 				return false;
 		return true;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayIntListHashMap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayIntListHashMap.java
index 69904dd..24bde6d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayIntListHashMap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DblArrayIntListHashMap.java
@@ -22,31 +22,29 @@ package org.apache.sysds.runtime.compress.utils;
 import java.util.ArrayList;
 
 /**
- * This class provides a memory-efficient replacement for
- * {@code HashMap<DblArray,IntArrayList>} for restricted use cases.
+ * This class provides a memory-efficient replacement for {@code HashMap<DblArray,IntArrayList>} for restricted use
+ * cases.
  * 
+ * TODO: Fix allocation of size such that it contains some amount of overhead from the start, to enable hashmap
+ * performance.
  */
-public class DblArrayIntListHashMap 
-{
-	private static final int INIT_CAPACITY = 8;
-	private static final int RESIZE_FACTOR = 2;
-	private static final float LOAD_FACTOR = 0.75f;
+public class DblArrayIntListHashMap extends CustomHashMap {
 
 	private DArrayIListEntry[] _data = null;
-	private int _size = -1;
 
 	public DblArrayIntListHashMap() {
 		_data = new DArrayIListEntry[INIT_CAPACITY];
 		_size = 0;
 	}
 
-	public int size() {
-		return _size;
+	public DblArrayIntListHashMap(int init_capacity) {
+		_data = new DArrayIListEntry[init_capacity];
+		_size = 0;
 	}
 
 	public IntArrayList get(DblArray key) {
 		// probe for early abort
-		if( _size == 0 )
+		if(_size == 0)
 			return null;
 
 		// compute entry index position
@@ -54,8 +52,8 @@ public class DblArrayIntListHashMap
 		int ix = indexFor(hash, _data.length);
 
 		// find entry
-		for( DArrayIListEntry e = _data[ix]; e != null; e = e.next ) {
-			if( e.key.equals(key) ) {
+		for(DArrayIListEntry e = _data[ix]; e != null; e = e.next) {
+			if(e.key.equals(key)) {
 				return e.value;
 			}
 		}
@@ -72,18 +70,22 @@ public class DblArrayIntListHashMap
 		DArrayIListEntry enew = new DArrayIListEntry(key, value);
 		enew.next = _data[ix]; // colliding entries / null
 		_data[ix] = enew;
+		if(enew.next != null && enew.next.key == key) {
+			enew.next = enew.next.next;
+			_size--;
+		}
 		_size++;
 
 		// resize if necessary
-		if( _size >= LOAD_FACTOR * _data.length )
+		if(_size >= LOAD_FACTOR * _data.length)
 			resize();
 	}
 
 	public ArrayList<DArrayIListEntry> extractValues() {
 		ArrayList<DArrayIListEntry> ret = new ArrayList<>();
-		for( DArrayIListEntry e : _data ) {
-			if( e != null ) {
-				while( e.next != null ) {
+		for(DArrayIListEntry e : _data) {
+			if(e != null) {
+				while(e.next != null) {
 					ret.add(e);
 					e = e.next;
 				}
@@ -96,7 +98,7 @@ public class DblArrayIntListHashMap
 
 	private void resize() {
 		// check for integer overflow on resize
-		if( _data.length > Integer.MAX_VALUE / RESIZE_FACTOR )
+		if(_data.length > Integer.MAX_VALUE / RESIZE_FACTOR)
 			return;
 
 		// resize data array and copy existing contents
@@ -105,9 +107,9 @@ public class DblArrayIntListHashMap
 		_size = 0;
 
 		// rehash all entries
-		for( DArrayIListEntry e : olddata ) {
-			if( e != null ) {
-				while( e.next != null ) {
+		for(DArrayIListEntry e : olddata) {
+			if(e != null) {
+				while(e.next != null) {
 					appendValue(e.key, e.value);
 					e = e.next;
 				}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleIntListHashMap.java b/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleIntListHashMap.java
index f548588..5236196 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleIntListHashMap.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/DoubleIntListHashMap.java
@@ -22,31 +22,28 @@ package org.apache.sysds.runtime.compress.utils;
 import java.util.ArrayList;
 
 /**
- * This class provides a memory-efficient replacement for
- * {@code HashMap<Double,IntArrayList>} for restricted use cases.
+ * This class provides a memory-efficient replacement for {@code HashMap<Double,IntArrayList>} for restricted use cases.
  * 
+ * TODO: Fix allocation of size such that it contains some amount of overhead from the start, to enable hashmap
+ * performance.
  */
-public class DoubleIntListHashMap 
-{
-	private static final int INIT_CAPACITY = 8;
-	private static final int RESIZE_FACTOR = 2;
-	private static final float LOAD_FACTOR = 0.75f;
+public class DoubleIntListHashMap extends CustomHashMap {
 
 	private DIListEntry[] _data = null;
-	private int _size = -1;
 
 	public DoubleIntListHashMap() {
 		_data = new DIListEntry[INIT_CAPACITY];
 		_size = 0;
 	}
 
-	public int size() {
-		return _size;
+	public DoubleIntListHashMap(int init_capacity) {
+		_data = new DIListEntry[init_capacity];
+		_size = 0;
 	}
 
 	public IntArrayList get(double key) {
 		// probe for early abort
-		if( _size == 0 )
+		if(_size == 0)
 			return null;
 
 		// compute entry index position
@@ -54,8 +51,8 @@ public class DoubleIntListHashMap
 		int ix = indexFor(hash, _data.length);
 
 		// find entry
-		for( DIListEntry e = _data[ix]; e != null; e = e.next ) {
-			if( e.key == key ) {
+		for(DIListEntry e = _data[ix]; e != null; e = e.next) {
+			if(e.key == key) {
 				return e.value;
 			}
 		}
@@ -72,18 +69,22 @@ public class DoubleIntListHashMap
 		DIListEntry enew = new DIListEntry(key, value);
 		enew.next = _data[ix]; // colliding entries / null
 		_data[ix] = enew;
+		if(enew.next != null && enew.next.key == key) {
+			enew.next = enew.next.next;
+			_size--;
+		}
 		_size++;
 
 		// resize if necessary
-		if( _size >= LOAD_FACTOR * _data.length )
+		if(_size >= LOAD_FACTOR * _data.length)
 			resize();
 	}
 
 	public ArrayList<DIListEntry> extractValues() {
 		ArrayList<DIListEntry> ret = new ArrayList<>();
-		for( DIListEntry e : _data ) {
-			if (e != null) {
-				while( e.next != null ) {
+		for(DIListEntry e : _data) {
+			if(e != null) {
+				while(e.next != null) {
 					ret.add(e);
 					e = e.next;
 				}
@@ -96,7 +97,7 @@ public class DoubleIntListHashMap
 
 	private void resize() {
 		// check for integer overflow on resize
-		if( _data.length > Integer.MAX_VALUE / RESIZE_FACTOR )
+		if(_data.length > Integer.MAX_VALUE / RESIZE_FACTOR)
 			return;
 
 		// resize data array and copy existing contents
@@ -105,9 +106,9 @@ public class DoubleIntListHashMap
 		_size = 0;
 
 		// rehash all entries
-		for( DIListEntry e : olddata ) {
-			if( e != null ) {
-				while( e.next != null ) {
+		for(DIListEntry e : olddata) {
+			if(e != null) {
+				while(e.next != null) {
 					appendValue(e.key, e.value);
 					e = e.next;
 				}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/IntArrayList.java b/src/main/java/org/apache/sysds/runtime/compress/utils/IntArrayList.java
index 677655b..a9b224d 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/IntArrayList.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/IntArrayList.java
@@ -22,12 +22,9 @@ package org.apache.sysds.runtime.compress.utils;
 import java.util.Arrays;
 
 /**
- * This class provides a memory-efficient replacement for {@code ArrayList<Integer>} for
- * restricted use cases.
- * 
+ * This class provides a memory-efficient replacement for {@code ArrayList<Integer>} for restricted use cases.
  */
-public class IntArrayList 
-{
+public class IntArrayList {
 	private static final int INIT_CAPACITY = 4;
 	private static final int RESIZE_FACTOR = 2;
 
@@ -39,7 +36,7 @@ public class IntArrayList
 		_data = null;
 		_size = 0;
 	}
-	
+
 	public IntArrayList(int value) {
 		this();
 		appendValue(value);
@@ -51,18 +48,18 @@ public class IntArrayList
 
 	public void appendValue(int value) {
 		// embedded value (no array allocation)
-		if( _size == 0 ) {
+		if(_size == 0) {
 			_val0 = value;
 			_size = 1;
 			return;
 		}
 
 		// allocate or resize array if necessary
-		if( _data == null ) {
+		if(_data == null) {
 			_data = new int[INIT_CAPACITY];
 			_data[0] = _val0;
-		} 
-		else if( _size + 1 >= _data.length ) {
+		}
+		else if(_size + 1 >= _data.length) {
 			resize();
 		}
 
@@ -72,31 +69,27 @@ public class IntArrayList
 	}
 
 	/**
-	 * Returns the underlying array of offsets. Note that this array might be 
-	 * physically larger than the actual length of the offset lists. Use size() 
-	 * to obtain the actual length.
+	 * Returns the underlying array of offsets. Note that this array might be physically larger than the actual length
+	 * of the offset lists. Use size() to obtain the actual length.
 	 * 
-	 * @return integer array of offsets, the physical array length
-	 * may be larger than the length of the offset list 
+	 * @return integer array of offsets, the physical array length may be larger than the length of the offset list
 	 */
 	public int[] extractValues() {
-		if( _size == 1 )
-			return new int[] { _val0 };
+		if(_size == 1)
+			return new int[] {_val0};
 		else
 			return _data;
 	}
-	
+
 	public int[] extractValues(boolean trim) {
 		int[] ret = extractValues();
-		return (trim && _size < ret.length) ?
-			Arrays.copyOfRange(ret, 0, _size) : ret;
+		return (trim && _size < ret.length) ? Arrays.copyOfRange(ret, 0, _size) : ret;
 	}
 
 	private void resize() {
 		// check for integer overflow on resize
-		if( _data.length > Integer.MAX_VALUE / RESIZE_FACTOR )
-			throw new RuntimeException(
-					"IntArrayList resize leads to integer overflow: size=" + _size);
+		if(_data.length > Integer.MAX_VALUE / RESIZE_FACTOR)
+			throw new RuntimeException("IntArrayList resize leads to integer overflow: size=" + _size);
 
 		// resize data array and copy existing contents
 		_data = Arrays.copyOf(_data, _data.length * RESIZE_FACTOR);
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/LinearAlgebraUtils.java b/src/main/java/org/apache/sysds/runtime/compress/utils/LinearAlgebraUtils.java
index 53b701a..b24d6f1 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/LinearAlgebraUtils.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/LinearAlgebraUtils.java
@@ -20,140 +20,122 @@
 package org.apache.sysds.runtime.compress.utils;
 
 import org.apache.sysds.runtime.data.DenseBlock;
-// import org.apache.sysds.runtime.matrix.data.DenseBlock;
 import org.apache.sysds.runtime.matrix.data.LibMatrixMult;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
 /**
- * This library contains all vector primitives that are used compressed 
- * linear algebra. For primitives that exist in LibMatrixMult, these 
- * calls are simply forwarded to ensure consistency in performance and 
- * result correctness. 
+ * This library contains all vector primitives that are used in compressed linear algebra. For primitives that exist
+ * in LibMatrixMult, these calls are simply forwarded to ensure consistency in performance and result correctness.
  */
-public class LinearAlgebraUtils 
-{
-	//forwarded calls to LibMatrixMult
-	
+public class LinearAlgebraUtils {
+	// forwarded calls to LibMatrixMult
+
 	public static double dotProduct(double[] a, double[] b, final int len) {
 		return LibMatrixMult.dotProduct(a, b, 0, 0, len);
 	}
 
-	public static double dotProduct( double[] a, double[] b, int ai, int bi, final int len ) {
+	public static double dotProduct(double[] a, double[] b, int ai, int bi, final int len) {
 		return LibMatrixMult.dotProduct(a, b, ai, bi, len);
 	}
-	
-	public static void vectMultiplyAdd( final double aval, double[] b, double[] c, int bi, int ci, final int len ) {
+
+	public static void vectMultiplyAdd(final double aval, double[] b, double[] c, int bi, int ci, final int len) {
 		LibMatrixMult.vectMultiplyAdd(aval, b, c, bi, ci, len);
 	}
-	
-	public static void vectMultiplyAdd( final double aval, double[] b, double[] c, int[] bix, final int bi, final int ci, final int len ) {
+
+	public static void vectMultiplyAdd(final double aval, double[] b, double[] c, int[] bix, final int bi, final int ci,
+		final int len) {
 		LibMatrixMult.vectMultiplyAdd(aval, b, c, bix, bi, ci, len);
 	}
 
-	public static void vectAdd( double[] a, double[] c, int ai, int ci, final int len ) {
+	public static void vectAdd(double[] a, double[] c, int ai, int ci, final int len) {
 		LibMatrixMult.vectAdd(a, c, ai, ci, len);
 	}
 
-	public static void vectAdd( final double aval, double[] c, char[] bix, final int bi, final int ci, final int len )
-	{
-		final int bn = len%8;
-		
-		//rest, not aligned to 8-blocks
-		for( int j = bi; j < bi+bn; j++ )
-			c[ ci + bix[j] ] += aval;
-		
-		//unrolled 8-block (for better instruction-level parallelism)
-		for( int j = bi+bn; j < bi+len; j+=8 )
-		{
-			c[ ci+bix[j+0] ] += aval;
-			c[ ci+bix[j+1] ] += aval;
-			c[ ci+bix[j+2] ] += aval;
-			c[ ci+bix[j+3] ] += aval;
-			c[ ci+bix[j+4] ] += aval;
-			c[ ci+bix[j+5] ] += aval;
-			c[ ci+bix[j+6] ] += aval;
-			c[ ci+bix[j+7] ] += aval;
+	public static void vectAdd(final double aval, double[] c, char[] bix, final int bi, final int ci, final int len) {
+		final int bn = len % 8;
+
+		// rest, not aligned to 8-blocks
+		for(int j = bi; j < bi + bn; j++)
+			c[ci + bix[j]] += aval;
+
+		// unrolled 8-block (for better instruction-level parallelism)
+		for(int j = bi + bn; j < bi + len; j += 8) {
+			c[ci + bix[j + 0]] += aval;
+			c[ci + bix[j + 1]] += aval;
+			c[ci + bix[j + 2]] += aval;
+			c[ci + bix[j + 3]] += aval;
+			c[ci + bix[j + 4]] += aval;
+			c[ci + bix[j + 5]] += aval;
+			c[ci + bix[j + 6]] += aval;
+			c[ci + bix[j + 7]] += aval;
 		}
 	}
 
-	public static void vectAdd( final double aval, double[] c, final int ci, final int len )
-	{
-		final int bn = len%8;
-		
-		//rest, not aligned to 8-blocks
-		for( int j = 0; j < bn; j++ )
-			c[ ci + j ] += aval;
-		
-		//unrolled 8-block (for better instruction-level parallelism)
-		for( int j = bn; j < len; j+=8 )
-		{
-			c[ ci+j+0 ] += aval;
-			c[ ci+j+1 ] += aval;
-			c[ ci+j+2 ] += aval;
-			c[ ci+j+3 ] += aval;
-			c[ ci+j+4 ] += aval;
-			c[ ci+j+5 ] += aval;
-			c[ ci+j+6 ] += aval;
-			c[ ci+j+7 ] += aval;
+	public static void vectAdd(final double aval, double[] c, final int ci, final int len) {
+		final int bn = len % 8;
+
+		// rest, not aligned to 8-blocks
+		for(int j = 0; j < bn; j++)
+			c[ci + j] += aval;
+
+		// unrolled 8-block (for better instruction-level parallelism)
+		for(int j = bn; j < len; j += 8) {
+			c[ci + j + 0] += aval;
+			c[ci + j + 1] += aval;
+			c[ci + j + 2] += aval;
+			c[ci + j + 3] += aval;
+			c[ci + j + 4] += aval;
+			c[ci + j + 5] += aval;
+			c[ci + j + 6] += aval;
+			c[ci + j + 7] += aval;
 		}
 	}
 
-
-	public static double vectSum( double[] a, char[] bix, final int ai, final int bi, final int len )
-	{
+	public static double vectSum(double[] a, char[] bix, final int ai, final int bi, final int len) {
 		double val = 0;
-		final int bn = len%8;
-		
-		//rest, not aligned to 8-blocks
-		for( int j = bi; j < bi+bn; j++ )
-			val += a[ ai + bix[j] ];
-		
-		//unrolled 8-block (for better instruction-level parallelism)
-		for( int j = bi+bn; j < bi+len; j+=8 )
-		{
-			val += a[ ai+bix[j+0] ]
-			     + a[ ai+bix[j+1] ]
-			     + a[ ai+bix[j+2] ]
-			     + a[ ai+bix[j+3] ]
-			     + a[ ai+bix[j+4] ]
-			     + a[ ai+bix[j+5] ]
-			     + a[ ai+bix[j+6] ]
-			     + a[ ai+bix[j+7] ];
+		final int bn = len % 8;
+
+		// rest, not aligned to 8-blocks
+		for(int j = bi; j < bi + bn; j++)
+			val += a[ai + bix[j]];
+
+		// unrolled 8-block (for better instruction-level parallelism)
+		for(int j = bi + bn; j < bi + len; j += 8) {
+			val += a[ai + bix[j + 0]] + a[ai + bix[j + 1]] + a[ai + bix[j + 2]] + a[ai + bix[j + 3]] +
+				a[ai + bix[j + 4]] + a[ai + bix[j + 5]] + a[ai + bix[j + 6]] + a[ai + bix[j + 7]];
 		}
-		
+
 		return val;
 	}
 
-	public static double vectSum( double[] a, int ai, final int len )
-	{
+	public static double vectSum(double[] a, int ai, final int len) {
 		double val = 0;
-		final int bn = len%8;
-		
-		//rest, not aligned to 8-blocks
-		for( int j = ai; j < ai+bn; j++ )
-			val += a[ j ];
-		
-		//unrolled 8-block (for better instruction-level parallelism)
-		for( int j = ai+bn; j < ai+len; j+=8 ) {
-			val += a[ j+0 ] + a[ j+1 ] + a[ j+2 ] + a[ j+3 ]
-			     + a[ j+4 ] + a[ j+5 ] + a[ j+6 ] + a[ j+7 ];
+		final int bn = len % 8;
+
+		// rest, not aligned to 8-blocks
+		for(int j = ai; j < ai + bn; j++)
+			val += a[j];
+
+		// unrolled 8-block (for better instruction-level parallelism)
+		for(int j = ai + bn; j < ai + len; j += 8) {
+			val += a[j + 0] + a[j + 1] + a[j + 2] + a[j + 3] + a[j + 4] + a[j + 5] + a[j + 6] + a[j + 7];
 		}
-		
+
 		return val;
 	}
 
-	public static long copyUpperToLowerTriangle( MatrixBlock ret ) {
+	public static long copyUpperToLowerTriangle(MatrixBlock ret) {
 		return LibMatrixMult.copyUpperToLowerTriangle(ret);
 	}
-	
-	public static void copyNonZerosToUpperTriangle( MatrixBlock ret, MatrixBlock tmp, int ix ) {
+
+	public static void copyNonZerosToUpperTriangle(MatrixBlock ret, MatrixBlock tmp, int ix) {
 		double[] a = tmp.getDenseBlockValues();
 		DenseBlock c = ret.getDenseBlock();
-		for(int i=0; i<tmp.getNumColumns(); i++)
-			if( a[i] != 0 )
-				c.set((ix<i)?ix:i, (ix<i)?i:ix, a[i]);
+		for(int i = 0; i < tmp.getNumColumns(); i++)
+			if(a[i] != 0)
+				c.set((ix < i) ? ix : i, (ix < i) ? i : ix, a[i]);
 	}
-	
+
 	/**
 	 * Obtain the index of the closest element in a to the value x.
 	 * 
@@ -166,13 +148,14 @@ public class LinearAlgebraUtils
 		int low = 0;
 		int high = a.length - 1;
 
-		while (low < high) {
+		while(low < high) {
 			int mid = (low + high) / 2;
 			int d1 = Math.abs(a[mid] - x);
 			int d2 = Math.abs(a[mid + 1] - x);
-			if (d2 <= d1) {
+			if(d2 <= d1) {
 				low = mid + 1;
-			} else {
+			}
+			else {
 				high = mid;
 			}
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/compress/utils/Py4jConverterUtils.java b/src/main/java/org/apache/sysds/runtime/compress/utils/Py4jConverterUtils.java
index 880fe4f..3b91197 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/utils/Py4jConverterUtils.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/utils/Py4jConverterUtils.java
@@ -19,26 +19,26 @@
 
 package org.apache.sysds.runtime.compress.utils;
 
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
 import org.apache.sysds.common.Types;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
 /**
  * Utils for converting python data to java.
  */
 public class Py4jConverterUtils {
-	public static MatrixBlock convertPy4JArrayToMB(byte [] data, int rlen, int clen) {
+	public static MatrixBlock convertPy4JArrayToMB(byte[] data, int rlen, int clen) {
 		return convertPy4JArrayToMB(data, rlen, clen, false, Types.ValueType.FP64);
 	}
-	
-	public static MatrixBlock convertPy4JArrayToMB(byte [] data, int rlen, int clen, Types.ValueType valueType) {
+
+	public static MatrixBlock convertPy4JArrayToMB(byte[] data, int rlen, int clen, Types.ValueType valueType) {
 		return convertPy4JArrayToMB(data, rlen, clen, false, valueType);
 	}
-	
-	public static MatrixBlock convertSciPyCOOToMB(byte [] data, byte [] row, byte [] col, int rlen, int clen, int nnz) {
+
+	public static MatrixBlock convertSciPyCOOToMB(byte[] data, byte[] row, byte[] col, int rlen, int clen, int nnz) {
 		MatrixBlock mb = new MatrixBlock(rlen, clen, true);
 		mb.allocateSparseRowsBlock(false);
 		ByteBuffer buf1 = ByteBuffer.wrap(data);
@@ -57,48 +57,50 @@ public class Py4jConverterUtils {
 		mb.examSparsity();
 		return mb;
 	}
-	
+
 	public static MatrixBlock allocateDenseOrSparse(int rlen, int clen, boolean isSparse) {
 		MatrixBlock ret = new MatrixBlock(rlen, clen, isSparse);
 		ret.allocateBlock();
 		return ret;
 	}
-	
+
 	public static MatrixBlock allocateDenseOrSparse(long rlen, long clen, boolean isSparse) {
 		if(rlen > Integer.MAX_VALUE || clen > Integer.MAX_VALUE) {
-			throw new DMLRuntimeException("Dimensions of matrix are too large to be passed via NumPy/SciPy:" + rlen + " X " + clen);
+			throw new DMLRuntimeException(
+				"Dimensions of matrix are too large to be passed via NumPy/SciPy:" + rlen + " X " + clen);
 		}
 		return allocateDenseOrSparse((int) rlen, (int) clen, isSparse);
 	}
-	
+
 	public static MatrixBlock convertPy4JArrayToMB(byte[] data, int rlen, int clen, boolean isSparse,
-			Types.ValueType valueType) {
+		Types.ValueType valueType) {
 		MatrixBlock mb = new MatrixBlock(rlen, clen, isSparse, -1);
 		if(isSparse) {
 			throw new DMLRuntimeException("Convertion to sparse format not supported");
 		}
 		else {
 			long limit = (long) rlen * clen;
-			if( limit > Integer.MAX_VALUE )
-				throw new DMLRuntimeException("Dense NumPy array of size " + limit + " cannot be converted to MatrixBlock");
-			double [] denseBlock = new double[(int) limit];
+			if(limit > Integer.MAX_VALUE)
+				throw new DMLRuntimeException(
+					"Dense NumPy array of size " + limit + " cannot be converted to MatrixBlock");
+			double[] denseBlock = new double[(int) limit];
 			ByteBuffer buf = ByteBuffer.wrap(data);
 			buf.order(ByteOrder.nativeOrder());
-			switch (valueType) {
+			switch(valueType) {
 				case INT32:
-					for (int i = 0; i < rlen * clen; i++)
+					for(int i = 0; i < rlen * clen; i++)
 						denseBlock[i] = buf.getInt();
 					break;
 				case FP32:
-					for (int i = 0; i < rlen * clen; i++)
+					for(int i = 0; i < rlen * clen; i++)
 						denseBlock[i] = buf.getFloat();
 					break;
 				case FP64:
-					for (int i = 0; i < rlen * clen; i++)
+					for(int i = 0; i < rlen * clen; i++)
 						denseBlock[i] = buf.getDouble();
 					break;
 				default:
-					throw new DMLRuntimeException("Unsupported value type: "+valueType.name());
+					throw new DMLRuntimeException("Unsupported value type: " + valueType.name());
 			}
 			mb.init(denseBlock, rlen, clen);
 		}
@@ -106,34 +108,34 @@ public class Py4jConverterUtils {
 		mb.examSparsity();
 		return mb;
 	}
-	
-	public static byte [] convertMBtoPy4JDenseArr(MatrixBlock mb) {
-		byte [] ret = null;
+
+	public static byte[] convertMBtoPy4JDenseArr(MatrixBlock mb) {
+		byte[] ret = null;
 		if(mb.isInSparseFormat()) {
 			mb.sparseToDense();
 		}
-		
-		long limit = mb.getNumRows()*mb.getNumColumns();
+
+		long limit = mb.getNumRows() * mb.getNumColumns();
 		int times = Double.SIZE / Byte.SIZE;
-		if( limit > Integer.MAX_VALUE / times )
+		if(limit > Integer.MAX_VALUE / times)
 			throw new DMLRuntimeException("MatrixBlock of size " + limit + " cannot be converted to dense numpy array");
 		ret = new byte[(int) (limit * times)];
-		
-		double [] denseBlock = mb.getDenseBlockValues();
+
+		double[] denseBlock = mb.getDenseBlockValues();
 		if(mb.isEmptyBlock()) {
-			for(int i=0;i < limit;i++){
-				ByteBuffer.wrap(ret, i*times, times).order(ByteOrder.nativeOrder()).putDouble(0);
+			for(int i = 0; i < limit; i++) {
+				ByteBuffer.wrap(ret, i * times, times).order(ByteOrder.nativeOrder()).putDouble(0);
 			}
 		}
 		else if(denseBlock == null) {
 			throw new DMLRuntimeException("Error while dealing with empty blocks.");
 		}
 		else {
-			for(int i=0;i < denseBlock.length;i++){
-				ByteBuffer.wrap(ret, i*times, times).order(ByteOrder.nativeOrder()).putDouble(denseBlock[i]);
+			for(int i = 0; i < denseBlock.length; i++) {
+				ByteBuffer.wrap(ret, i * times, times).order(ByteOrder.nativeOrder()).putDouble(denseBlock[i]);
 			}
 		}
-		
+
 		return ret;
 	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/CompressionCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/CompressionCPInstruction.java
index 18c6256..90ea352 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/CompressionCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/CompressionCPInstruction.java
@@ -20,7 +20,7 @@
 package org.apache.sysds.runtime.instructions.cp;
 
 import org.apache.sysds.hops.OptimizerUtils;
-import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.instructions.Instruction;
 import org.apache.sysds.runtime.instructions.InstructionUtils;
@@ -40,15 +40,14 @@ public class CompressionCPInstruction extends ComputationCPInstruction {
 		CPOperand out = new CPOperand(parts[2]);
 		return new CompressionCPInstruction(null, in1, out, opcode, str);
 	}
-	
+
 	@Override
-	public void processInstruction( ExecutionContext ec ) {
-		//get matrix block input
+	public void processInstruction(ExecutionContext ec) {
+		// Get matrix block input
 		MatrixBlock in = ec.getMatrixInput(input1.getName());
-		//compress the matrix block
-		MatrixBlock out = new CompressedMatrixBlock(in)
-			.compress(OptimizerUtils.getConstrainedNumThreads(-1));
-		//set output and release input
+		// Compress the matrix block
+		MatrixBlock out = CompressedMatrixBlockFactory.compress(in, OptimizerUtils.getConstrainedNumThreads(-1));
+		// Set output and release input
 		ec.releaseMatrixInput(input1.getName());
 		ec.setMatrixOutput(output.getName(), out);
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/EvalNaryCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/EvalNaryCPInstruction.java
index ce337b5..4204337 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/EvalNaryCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/EvalNaryCPInstruction.java
@@ -185,7 +185,7 @@ public class EvalNaryCPInstruction extends BuiltinNaryCPInstruction {
 		}
 	}
 	
-	private void checkValidArguments(List<Data> loData, List<String> loNames, List<String> fArgNames) {
+	private static void checkValidArguments(List<Data> loData, List<String> loNames, List<String> fArgNames) {
 		//check number of parameters
 		int listSize = (loNames != null) ? loNames.size() : loData.size();
 		if( listSize != fArgNames.size() )
@@ -203,7 +203,7 @@ public class EvalNaryCPInstruction extends BuiltinNaryCPInstruction {
 		}
 	}
 	
-	private ListObject reorderNamedListForFunctionCall(ListObject in, List<String> fArgNames) {
+	private static ListObject reorderNamedListForFunctionCall(ListObject in, List<String> fArgNames) {
 		List<Data> sortedData = new ArrayList<>();
 		for( String name : fArgNames )
 			sortedData.add(in.getData(name));
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/CompressionSPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/CompressionSPInstruction.java
index c46cf5b..f27325a 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/CompressionSPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/CompressionSPInstruction.java
@@ -21,7 +21,7 @@ package org.apache.sysds.runtime.instructions.spark;
 
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.function.Function;
-import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.controlprogram.context.SparkExecutionContext;
 import org.apache.sysds.runtime.instructions.InstructionUtils;
@@ -62,7 +62,7 @@ public class CompressionSPInstruction extends UnarySPInstruction {
 
 		@Override
 		public MatrixBlock call(MatrixBlock arg0) throws Exception {
-			return new CompressedMatrixBlock(arg0).compress();
+			return CompressedMatrixBlockFactory.compress(arg0);
 		}
 	}
 }
diff --git a/src/main/java/org/apache/sysds/utils/MemoryEstimates.java b/src/main/java/org/apache/sysds/utils/MemoryEstimates.java
new file mode 100644
index 0000000..41ac437
--- /dev/null
+++ b/src/main/java/org/apache/sysds/utils/MemoryEstimates.java
@@ -0,0 +1,86 @@
+/*
+ * 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.utils;
+
+/**
+ * Memory Estimates is a helper class containing static classes that estimate the memory requirements of different types
+ * of objects in java.
+ * All estimates are worst case JVM x86-64bit uncompressed object pointers.
+ */
+public class MemoryEstimates {
+	public static long byteArrayCost(int length) {
+		long size = 0;
+		size += 8; // Byte array Reference
+		size += 20; // Byte array Object header
+		if(length <= 4) { // byte array fills out the first 4 bytes differently than the later bytes.
+			size += 4;
+		}
+		else { // byte array pads to next 8 bytes after the first 4.
+			size += length;
+			int diff = (length - 4) % 8;
+			if(diff > 0) {
+				size += 8 - diff;
+			}
+		}
+		return size;
+	}
+
+	public static long charArrayCost(int length) {
+		long size = 0;
+		size += 8; // char array Reference
+		size += 20; // char array Object header
+		if(length <= 2) { // char array fills out the first 2 chars differently than the later bytes.
+			size += 4;
+		}
+		else {
+			size += length * 2;// 2 bytes per char
+			int diff = (length * 2 - 4) % 8;
+			if(diff > 0) {
+				size += 8 - diff; // next object alignment
+			}
+		}
+		return size;
+	}
+
+	public static long intArrayCost(int length) {
+		long size = 0;
+		size += 8; // _ptr int[] reference
+		size += 20; // int array Object header
+		if(length <= 1) {
+			size += 4;
+		}
+		else {
+			size += length * 4; // offsets 4 bytes per int
+			if(length % 2 == 0) {
+				size += 4;
+			}
+		}
+		return size;
+	}
+
+	public static long doubleArrayCost(long length) {
+		long size = 0;
+		size += 8; // _values double array reference
+		size += 20; // double array object header
+		size += 4; // padding inside double array object to align to 8 bytes.
+		size += 8 * length; // Each double fills 8 Bytes
+		return size;
+	}
+}
\ No newline at end of file
diff --git a/src/test/java/org/apache/sysds/test/TestUtils.java b/src/test/java/org/apache/sysds/test/TestUtils.java
index a23f7b6..530dccb 100644
--- a/src/test/java/org/apache/sysds/test/TestUtils.java
+++ b/src/test/java/org/apache/sysds/test/TestUtils.java
@@ -666,7 +666,7 @@ public class TestUtils
 	}
 
 	public static void compareMatricesBitAvgDistance(double[][] expectedMatrix, double[][] actualMatrix, int rows, int cols,
-		long maxUnitsOfLeastPrecision, long maxAvgDistance){
+		long maxUnitsOfLeastPrecision, long maxAvgDistance, String message){
 		int countErrors = 0;
 		long sumDistance = 0;
 		long distance;
@@ -680,9 +680,15 @@ public class TestUtils
 				}
 			}
 		}
-		long avgDistance = sumDistance / (rows * cols); 
-		assertTrue("" + countErrors + " values are not in equal", countErrors == 0);
-		assertTrue("The avg distance in bits: "+ avgDistance +" was higher than max: " + maxAvgDistance, avgDistance <= maxAvgDistance);
+		long avgDistance = sumDistance / (rows * cols);
+		assertTrue(message + "\n" + countErrors + " values are not in equal", countErrors == 0);
+		assertTrue(message + "\nThe avg distance in bits: "+ avgDistance +" was higher than max: " + maxAvgDistance,
+			avgDistance <= maxAvgDistance);
+	}
+
+	public static void compareMatricesBitAvgDistance(double[][] expectedMatrix, double[][] actualMatrix, int rows,
+		int cols, long maxUnitsOfLeastPrecision, long maxAvgDistance) {
+			compareMatricesBitAvgDistance(expectedMatrix, actualMatrix, rows, cols, maxUnitsOfLeastPrecision, maxAvgDistance, "");
 	}
 
 	/**
diff --git a/src/test/java/org/apache/sysds/test/applications/ApplicationTestBase.java b/src/test/java/org/apache/sysds/test/applications/ApplicationTestBase.java
deleted file mode 100644
index 95f3981..0000000
--- a/src/test/java/org/apache/sysds/test/applications/ApplicationTestBase.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.sysds.test.applications;
-
-import java.util.ArrayList;
-import java.util.Collection;
-
-import org.junit.After;
-import org.junit.runners.Parameterized.Parameters;
-import org.apache.sysds.api.DMLScript;
-import org.apache.sysds.common.Types.ExecMode;
-import org.apache.sysds.test.AutomatedTestBase;
-import org.apache.sysds.test.TestConstants;
-import org.apache.sysds.test.TestConstants.CompressionType;
-import org.apache.sysds.test.TestConstants.MatrixType;
-import org.apache.sysds.test.TestConstants.SparsityType;
-
-public class ApplicationTestBase extends AutomatedTestBase {
-
-	protected static SparsityType[] usedSparsityTypes = new SparsityType[] { // Sparsities Used For testing.
-		SparsityType.DENSE, //
-		SparsityType.SPARSE, //
-		// SparsityType.EMPTY
-	};
-
-	protected static CompressionType[] usedCompressionTypes = new CompressionType[] {CompressionType.LOSSLESS,
-		// CompressionType.LOSSY,
-	};
-
-	protected static MatrixType[] usedMatrixType = new MatrixType[] { // Matrix Input sizes for testing
-		MatrixType.SMALL,
-		// MatrixType.LARGE,
-		MatrixType.FEW_COL,
-		MatrixType.FEW_ROW,
-		// MatrixType.SINGLE_COL,
-		// MatrixType.SINGLE_ROW,
-		MatrixType.L_ROWS,
-		MatrixType.XL_ROWS,
-	};
-
-	protected static ExecMode[] usedExecutionModes = new ExecMode[] { // The used execution modes
-		ExecMode.SINGLE_NODE, ExecMode.HYBRID,
-		// ExecMode.SPARK,
-	};
-	// ExecMode.values()
-
-	protected int id; // Unique ID for each test case in parameterized classes
-
-	protected int rows;
-	protected int cols;
-	protected double sparsity;
-
-	protected ExecMode platformOld;
-	protected boolean sparkConfigOld;
-
-	public ApplicationTestBase(int id, SparsityType sparType, MatrixType matrixType, ExecMode newPlatform) {
-		this.id = id;
-		this.sparsity = TestConstants.getSparsityValue(sparType);
-		this.rows = TestConstants.getNumberOfRows(matrixType);
-		this.cols = TestConstants.getNumberOfColumns(matrixType);
-
-		this.platformOld = rtplatform;
-		rtplatform = newPlatform;
-		this.sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
-
-		if(rtplatform == ExecMode.SPARK || rtplatform == ExecMode.HYBRID)
-			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
-
-	}
-
-	@After
-	public void teardown() {
-		rtplatform = platformOld;
-		DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
-	}
-
-	@Parameters
-	public static Collection<Object[]> data() {
-		ArrayList<Object[]> tests = new ArrayList<>();
-		int id = 0;
-		for(SparsityType st : usedSparsityTypes) { // Test multiple Sparsities
-			for(MatrixType mt : usedMatrixType) {
-				for(ExecMode ex : usedExecutionModes) { // Test all Execution Modes.
-					tests.add(new Object[] {id++, st, mt, ex});
-				}
-			}
-		}
-		return tests;
-	}
-
-	@Override
-	public void setUp() {
-	}
-
-	@Override
-	public void tearDown() {
-	}
-
-	@Override
-	public String toString() {
-		StringBuilder builder = new StringBuilder();
-
-		builder.append("args: ");
-
-		builder.append(String.format("%6s%5s", "Rows:", rows));
-		builder.append(String.format("%6s%5s", "Cols:", cols));
-		builder.append(String.format("%6s%4s", "Spar:", sparsity));
-
-		return builder.toString();
-	}
-}
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
new file mode 100644
index 0000000..a6315e4
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/component/compress/AbstractCompressedUnaryTests.java
@@ -0,0 +1,165 @@
+/*
+ * 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;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.instructions.InstructionUtils;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysds.runtime.util.DataConverter;
+import org.apache.sysds.test.TestUtils;
+import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
+import org.apache.sysds.test.component.compress.TestConstants.SparsityType;
+import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.ValueType;
+import org.junit.Test;
+
+public abstract class AbstractCompressedUnaryTests extends CompressedTestBase {
+
+	public AbstractCompressedUnaryTests(SparsityType sparType, ValueType valType, ValueRange valRange,
+		CompressionSettings compSettings, MatrixTypology matrixTypology) {
+		super(sparType, valType, valRange, compSettings, matrixTypology);
+	}
+
+	enum AggType {
+		ROWSUMS, COLSUMS, SUM, ROWSUMSSQ, COLSUMSSQ, SUMSQ, ROWMAXS, COLMAXS, MAX, ROWMINS, COLMINS, MIN,
+	}
+
+	@Test
+	public void testUnaryOperator_ROWSUMS() {
+		testUnaryOperators(AggType.ROWSUMS);
+	}
+
+	@Test
+	public void testUnaryOperator_COLSUMS() {
+		testUnaryOperators(AggType.COLSUMS);
+	}
+
+	@Test
+	public void testUnaryOperator_SUM() {
+		testUnaryOperators(AggType.SUM);
+	}
+
+	@Test
+	public void testUnaryOperator_ROWSUMSSQ() {
+		testUnaryOperators(AggType.ROWSUMSSQ);
+	}
+
+	@Test
+	public void testUnaryOperator_COLSUMSSQ() {
+		testUnaryOperators(AggType.COLSUMSSQ);
+	}
+
+	@Test
+	public void testUnaryOperator_SUMSQ() {
+		testUnaryOperators(AggType.SUMSQ);
+	}
+
+	@Test
+	public void testUnaryOperator_ROWMAXS() {
+		testUnaryOperators(AggType.ROWMAXS);
+	}
+
+	@Test
+	public void testUnaryOperator_COLMAXS() {
+		testUnaryOperators(AggType.COLMAXS);
+	}
+
+	@Test
+	public void testUnaryOperator_MAX() {
+		testUnaryOperators(AggType.MAX);
+	}
+
+	@Test
+	public void testUnaryOperator_ROWMINS() {
+		testUnaryOperators(AggType.MAX);
+	}
+
+	@Test
+	public void testUnaryOperator_COLMINS() {
+		testUnaryOperators(AggType.MAX);
+	}
+
+	@Test
+	public void testUnaryOperator_MIN() {
+		testUnaryOperators(AggType.MAX);
+	}
+
+	protected AggregateUnaryOperator getUnaryOperator(AggType aggType, int k) {
+		switch(aggType) {
+			case SUM:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uak+", k);
+			case ROWSUMS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uark+", k);
+			case COLSUMS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uack+", k);
+			case SUMSQ:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uasqk+", k);
+			case ROWSUMSSQ:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uarsqk+", k);
+			case COLSUMSSQ:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uacsqk+", k);
+			case MAX:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uamax", k);
+			case ROWMAXS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uarmax", k);
+			case COLMAXS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uacmax", k);
+			case MIN:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uamin", k);
+			case ROWMINS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uarmin", k);
+			case COLMINS:
+				return InstructionUtils.parseBasicAggregateUnaryOperator("uacmin", k);
+			default:
+				throw new NotImplementedException("Not Supported Aggregate Unary operator in test");
+		}
+	}
+
+	public abstract void testUnaryOperators(AggType aggType);
+
+	public void testUnaryOperators(AggType aggType, AggregateUnaryOperator auop) {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return; // Input was not compressed then just pass test
+			// matrix-vector uncompressed
+			MatrixBlock ret1 = mb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, true);
+			// matrix-vector compressed
+			MatrixBlock ret2 = cmb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, true);
+
+			// compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			int dim1 = (aggType == AggType.ROWSUMS || aggType == AggType.ROWSUMSSQ || aggType == AggType.ROWMINS ||
+				aggType == AggType.ROWMINS) ? rows : 1;
+			int dim2 = (aggType == AggType.COLSUMS || aggType == AggType.COLSUMSSQ || aggType == AggType.COLMAXS ||
+				aggType == AggType.COLMINS) ? cols : 1;
+
+			TestUtils.compareMatricesBitAvgDistance(d1, d2, dim1, dim2, 2048, 20, compressionSettings.toString());
+
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
+		}
+	}
+}
\ No newline at end of file
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 9f43035..bbff431 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
@@ -19,20 +19,21 @@
 
 package org.apache.sysds.test.component.compress;
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 import org.apache.sysds.lops.MMTSJ.MMTSJType;
 import org.apache.sysds.lops.MapMultChain.ChainType;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.CompressionStatistics;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.Plus;
-import org.apache.sysds.runtime.instructions.InstructionUtils;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateBinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
@@ -41,49 +42,30 @@ import org.apache.sysds.runtime.matrix.operators.RightScalarOperator;
 import org.apache.sysds.runtime.matrix.operators.ScalarOperator;
 import org.apache.sysds.runtime.util.DataConverter;
 import org.apache.sysds.test.TestUtils;
-import org.apache.sysds.test.TestConstants.CompressionType;
-import org.apache.sysds.test.TestConstants.MatrixType;
-import org.apache.sysds.test.TestConstants.SparsityType;
-import org.apache.sysds.test.TestConstants.ValueType;
-import org.apache.sysds.test.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
+import org.apache.sysds.test.component.compress.TestConstants.SparsityType;
+import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.ValueType;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.openjdk.jol.datamodel.X86_64_DataModel;
+import org.openjdk.jol.info.ClassLayout;
+import org.openjdk.jol.layouters.HotSpotLayouter;
+import org.openjdk.jol.layouters.Layouter;
 
 @RunWith(value = Parameterized.class)
-public class CompressedMatrixTest extends CompressedTestBase {
-
-	// Input
-	protected double[][] input;
-	protected MatrixBlock mb;
-
-	// Compressed Block
-	protected CompressedMatrixBlock cmb;
-
-	// Compression Result
-	protected MatrixBlock cmbResult;
-
-	// Decompressed Result
-	protected MatrixBlock cmbDeCompressed;
-	protected double[][] deCompressed;
-
-	public CompressedMatrixTest(SparsityType sparType, ValueType valType, ValueRange valRange, CompressionType compType,
-		MatrixType matrixType, boolean compress, double samplingRatio) {
-		super(sparType, valType, valRange, compType, matrixType, compress, samplingRatio);
-		input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, 7);
-		mb = getMatrixBlockInput(input);
-		cmb = new CompressedMatrixBlock(mb);
-		cmb.setSeed(1);
-		cmb.setSamplingRatio(samplingRatio);
-		if(compress) {
-			cmbResult = cmb.compress();
-		}
-		cmbDeCompressed = cmb.decompress();
-		deCompressed = DataConverter.convertToDoubleMatrix(cmbDeCompressed);
+public class CompressedMatrixTest extends AbstractCompressedUnaryTests {
+
+	public CompressedMatrixTest(SparsityType sparType, ValueType valType, ValueRange valRange,
+		CompressionSettings compSettings, MatrixTypology matrixTypology) {
+		super(sparType, valType, valRange, compSettings, matrixTypology);
 	}
 
 	@Test
 	public void testConstruction() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock)) {
-				// TODO Compress EVERYTHING!
+			if(!(cmb instanceof CompressedMatrixBlock)) {
 				return; // Input was not compressed then just pass test
 				// Assert.assertTrue("Compression Failed \n" + this.toString(), false);
 			}
@@ -99,7 +81,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testGetValue() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			for(int i = 0; i < rows; i++)
@@ -118,7 +100,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testAppend() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector = DataConverter
@@ -129,7 +111,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 
 			// matrix-vector compressed
 			MatrixBlock ret2 = cmb.append(vector, new MatrixBlock());
-			if(compress)
+			if(ret2 instanceof CompressedMatrixBlock)
 				ret2 = ((CompressedMatrixBlock) ret2).decompress();
 
 			// compare result with input
@@ -146,7 +128,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testMatrixMultChain() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector1 = DataConverter
@@ -170,7 +152,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 				// compare result with input
 				double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
 				double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
-				TestUtils.compareMatricesBitAvgDistance(d1, d2, cols, 1, 512, 15);
+				TestUtils.compareMatricesBitAvgDistance(d1, d2, cols, 1, 512, 32);
 			}
 		}
 		catch(Exception e) {
@@ -182,7 +164,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testTransposeSelfMatrixMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 			// ChainType ctype = ChainType.XtwXv;
 			for(MMTSJType mType : new MMTSJType[] {MMTSJType.LEFT,
@@ -209,7 +191,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testMatrixVectorMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector = DataConverter
@@ -239,7 +221,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testVectorMatrixMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector = DataConverter
@@ -269,7 +251,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testScalarOperationsSparseUnsafe() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			// matrix-scalar uncompressed
@@ -278,7 +260,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 
 			// matrix-scalar compressed
 			MatrixBlock ret2 = cmb.scalarOperations(sop, new MatrixBlock());
-			if(compress)
+			if(ret2 instanceof CompressedMatrixBlock)
 				ret2 = ((CompressedMatrixBlock) ret2).decompress();
 
 			// compare result with input
@@ -296,7 +278,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testScalarOperations() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			// matrix-scalar uncompressed
@@ -305,7 +287,7 @@ public class CompressedMatrixTest extends CompressedTestBase {
 
 			// matrix-scalar compressed
 			MatrixBlock ret2 = cmb.scalarOperations(sop, new MatrixBlock());
-			if(compress)
+			if(ret2 instanceof CompressedMatrixBlock)
 				ret2 = ((CompressedMatrixBlock) ret2).decompress();
 
 			// compare result with input
@@ -320,83 +302,16 @@ public class CompressedMatrixTest extends CompressedTestBase {
 		}
 	}
 
-	// TODO replace with Direction x Types.AggOp
-	enum AggType {
-		ROWSUMS, COLSUMS, SUM, ROWSUMSSQ, COLSUMSSQ, SUMSQ, ROWMAXS, COLMAXS, MAX, ROWMINS, COLMINS, MIN,
-	}
-
-	@Test
-	public void testUnaryOperators() {
-		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
-				return; // Input was not compressed then just pass test
-			for(AggType aggType : AggType.values()) {
-				AggregateUnaryOperator auop = null;
-				switch(aggType) {
-					case SUM:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uak+");
-						break;
-					case ROWSUMS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uark+");
-						break;
-					case COLSUMS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uack+");
-						break;
-					case SUMSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uasqk+");
-						break;
-					case ROWSUMSSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarsqk+");
-						break;
-					case COLSUMSSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacsqk+");
-						break;
-					case MAX:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uamax");
-						break;
-					case ROWMAXS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarmax");
-						break;
-					case COLMAXS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacmax");
-						break;
-					case MIN:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uamin");
-						break;
-					case ROWMINS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarmin");
-						break;
-					case COLMINS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacmin");
-						break;
-				}
-				// matrix-vector uncompressed
-				MatrixBlock ret1 = mb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, true);
-
-				// matrix-vector compressed
-				MatrixBlock ret2 = cmb.aggregateUnaryOperations(auop, new MatrixBlock(), Math.max(rows, cols), null, true);
-
-				// compare result with input
-				double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
-				double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
-				int dim1 = (aggType == AggType.ROWSUMS || aggType == AggType.ROWSUMSSQ || aggType == AggType.ROWMINS ||
-					aggType == AggType.ROWMINS) ? rows : 1;
-				int dim2 = (aggType == AggType.COLSUMS || aggType == AggType.COLSUMSSQ || aggType == AggType.COLMAXS ||
-					aggType == AggType.COLMINS) ? cols : 1;
-
-				TestUtils.compareMatricesBitAvgDistance(d1, d2, dim1, dim2, 1024, 20);
-			}
-		}
-		catch(Exception e) {
-			e.printStackTrace();
-			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
-		}
+	@Override
+	public void testUnaryOperators(AggType aggType) {
+		AggregateUnaryOperator auop = super.getUnaryOperator(aggType, 1);
+		testUnaryOperators(aggType, auop);
 	}
 
 	@Test
 	public void testSerialization() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			// serialize compressed matrix block
@@ -424,4 +339,150 @@ public class CompressedMatrixTest extends CompressedTestBase {
 			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
 		}
 	}
+
+	@Test
+	public void testCompressionRatio() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+			CompressionStatistics cStat = ((CompressedMatrixBlock) cmb).getCompressionStatistics();
+			assertTrue("Compression ration if compressed should be larger than 1", cStat.ratio > 1);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
+		}
+	}
+
+	@Test
+	public void testCompressionEstimationVSCompression() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+			CompressionStatistics cStat = ((CompressedMatrixBlock) cmb).getCompressionStatistics();
+			long colsEstimate = cStat.estimatedSizeCols;
+			// long groupsEstimate = cStat.estimatedSizeColGroups;
+			long actualSize = cStat.size;
+			long originalSize = cStat.originalSize;
+			int allowedTolerance = 0;
+
+			if(compressionSettings.samplingRatio < 1.0) {
+				allowedTolerance = sampleTolerance;
+			}
+
+			StringBuilder builder = new StringBuilder();
+			builder.append("\n\t" + String.format("%-40s - %12d", "Actual compressed size: ", actualSize));
+			// builder.append("\n\t"+String.format("%-40s - %12d","<= estimated ColGroup compressed
+			// size",groupsEstimate));
+			builder.append("\n\t" + String.format("%-40s - %12d with tolerance: %5d",
+				"<= estimated isolated ColGroups: ",
+				colsEstimate,
+				allowedTolerance));
+			builder.append("\n\t" + String.format("%-40s - %12d", "<= Original size: ", originalSize));
+			builder.append("\n\tcol groups types: " + cStat.getGroupsTypesString());
+			builder.append("\n\tcol groups sizes: " + cStat.getGroupsSizesString());
+			builder.append("\n\t" + this.toString());
+			boolean res = actualSize - colsEstimate <= allowedTolerance;
+			assertTrue(builder.toString(), res);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
+		}
+	}
+
+	@Test
+	public void testCompressionEstimationVSJolEstimate() {
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+			CompressionStatistics cStat = ((CompressedMatrixBlock) cmb).getCompressionStatistics();
+			long actualSize = cStat.size;
+			long originalSize = cStat.originalSize;
+			long JolEstimatedSize = getJolSize(((CompressedMatrixBlock) cmb));
+
+			StringBuilder builder = new StringBuilder();
+			builder.append("\n\t" + String.format("%-40s - %12d", "Actual compressed size: ", actualSize));
+			builder.append("\n\t" + String.format("%-40s - %12d", "<= Original size: ", originalSize));
+			builder.append("\n\t" + String.format("%-40s - %12d", "and equal to JOL Size: ", JolEstimatedSize));
+			// builder.append("\n\t " + getJolSizeString(cmb));
+			builder.append("\n\tcol groups types: " + cStat.getGroupsTypesString());
+			builder.append("\n\tcol groups sizes: " + cStat.getGroupsSizesString());
+			builder.append("\n\t" + this.toString());
+
+			assertTrue(builder.toString(), actualSize == JolEstimatedSize && actualSize <= originalSize);
+			// assertTrue(builder.toString(), groupsEstimate < actualSize && colsEstimate < groupsEstimate);
+
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
+		}
+	}
+
+	@Test
+	public void testCompressionScale() {
+		// This test is here for a sanity check such that we verify that the compression ratio from our Matrix
+		// Compressed Block is not unreasonably good.
+		try {
+			if(!(cmb instanceof CompressedMatrixBlock))
+				return;
+
+			CompressionStatistics cStat = ((CompressedMatrixBlock) cmb).getCompressionStatistics();
+
+			double compressRatio = cStat.ratio;
+			long actualSize = cStat.size;
+			long originalSize = cStat.originalSize;
+
+			StringBuilder builder = new StringBuilder();
+			builder.append("Compression Ratio sounds suspiciously good at: " + compressRatio);
+			builder.append("\n\tActual compressed size: " + actualSize);
+			builder.append(" original size: " + originalSize);
+			builder.append("\n\tcol groups types: " + cStat.getGroupsTypesString());
+			builder.append("\n\tcol groups sizes: " + cStat.getGroupsSizesString());
+			builder.append("\n\t" + this.toString());
+
+			assertTrue(builder.toString(), compressRatio < 1000.0);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			throw new RuntimeException(this.toString() + "\n" + e.getMessage(), e);
+		}
+	}
+
+	private static long getJolSize(CompressedMatrixBlock cmb) {
+		Layouter l = new HotSpotLayouter(new X86_64_DataModel());
+		long jolEstimate = 0;
+		CompressionStatistics cStat = cmb.getCompressionStatistics();
+		for(Object ob : new Object[] {cmb, cStat, cStat.getColGroups(), cStat.getTimeArrayList(), cmb.getColGroups()}) {
+			jolEstimate += ClassLayout.parseInstance(ob, l).instanceSize();
+		}
+		for(ColGroup cg : cmb.getColGroups()) {
+			jolEstimate += cg.estimateInMemorySize();
+		}
+		return jolEstimate;
+	}
+
+	@SuppressWarnings("unused")
+	private static String getJolSizeString(CompressedMatrixBlock cmb) {
+		StringBuilder builder = new StringBuilder();
+		Layouter l = new HotSpotLayouter(new X86_64_DataModel());
+		long diff;
+		long jolEstimate = 0;
+		CompressionStatistics cStat = cmb.getCompressionStatistics();
+		for(Object ob : new Object[] {cmb, cStat, cStat.getColGroups(), cStat.getTimeArrayList(), cmb.getColGroups()}) {
+			ClassLayout cl = ClassLayout.parseInstance(ob, l);
+			diff = cl.instanceSize();
+			jolEstimate += diff;
+			builder.append(cl.toPrintable());
+			builder.append("TOTAL MEM: " + jolEstimate + " diff " + diff + "\n");
+		}
+		for(ColGroup cg : cmb.getColGroups()) {
+			diff = cg.estimateInMemorySize();
+			jolEstimate += diff;
+			builder.append(cg.getCompType());
+			builder.append("TOTAL MEM: " + jolEstimate + " diff " + diff + "\n");
+		}
+		return builder.toString();
+	}
 }
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 fa2ec20..9a84f0e 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
@@ -19,105 +19,130 @@
 
 package org.apache.sysds.test.component.compress;
 
+import static org.junit.Assert.assertTrue;
+
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 
-import org.junit.runners.Parameterized.Parameters;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressedMatrixBlockFactory;
+import org.apache.sysds.runtime.compress.CompressionSettings;
+import org.apache.sysds.runtime.compress.CompressionSettingsBuilder;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.util.DataConverter;
-import org.apache.sysds.test.AutomatedTestBase;
-import org.apache.sysds.test.TestUtils;
-import org.apache.sysds.test.TestConstants;
-import org.apache.sysds.test.TestConstants.CompressionType;
-import org.apache.sysds.test.TestConstants.MatrixType;
-import org.apache.sysds.test.TestConstants.SparsityType;
-import org.apache.sysds.test.TestConstants.ValueRange;
-import org.apache.sysds.test.TestConstants.ValueType;
-
-public class CompressedTestBase extends AutomatedTestBase {
-
-	protected static SparsityType[] usedSparsityTypes = new SparsityType[] {SparsityType.DENSE, SparsityType.SPARSE,
-		// SparsityType.EMPTY
+import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
+import org.apache.sysds.test.component.compress.TestConstants.SparsityType;
+import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.ValueType;
+import org.junit.runners.Parameterized.Parameters;
+
+public class CompressedTestBase extends TestBase {
+
+	protected static SparsityType[] usedSparsityTypes = new SparsityType[] { // Sparsity 0.9, 0.1, 0.01 and 0.0
+		// SparsityType.DENSE,
+		SparsityType.SPARSE, SparsityType.ULTRA_SPARSE, SparsityType.EMPTY
 	};
 	protected static ValueType[] usedValueTypes = new ValueType[] {
 		// ValueType.RAND,
-		ValueType.CONST, ValueType.RAND_ROUND_DDC, ValueType.RAND_ROUND_OLE,};
-
-	protected static ValueRange[] usedValueRanges = new ValueRange[] {ValueRange.SMALL, ValueRange.LARGE,};
+		ValueType.CONST, ValueType.RAND_ROUND, ValueType.OLE_COMPRESSIBLE, ValueType.RLE_COMPRESSIBLE,};
 
-	protected static CompressionType[] usedCompressionTypes = new CompressionType[] {CompressionType.LOSSLESS,
-		// CompressionType.LOSSY,
+	protected static ValueRange[] usedValueRanges = new ValueRange[] {ValueRange.SMALL,
+		// ValueRange.LARGE,
 	};
 
-	protected static MatrixType[] usedMatrixType = new MatrixType[] {MatrixType.SMALL,
-		// MatrixType.LARGE,
-		MatrixType.FEW_COL, MatrixType.FEW_ROW,
-		// MatrixType.SINGLE_COL,
-		// MatrixType.SINGLE_ROW,
-		MatrixType.L_ROWS,
-		// MatrixType.XL_ROWS,
+	private static List<CompressionType> DDCOnly = new ArrayList<>();
+	private static List<CompressionType> OLEOnly = new ArrayList<>();
+	private static List<CompressionType> RLEOnly = new ArrayList<>();
+
+	static {
+		DDCOnly.add(CompressionType.DDC);
+		OLEOnly.add(CompressionType.OLE);
+		RLEOnly.add(CompressionType.RLE);
+	}
+
+	private static final int compressionSeed = 7;
+
+	protected static CompressionSettings[] usedCompressionSettings = new CompressionSettings[] {
+		new CompressionSettingsBuilder().setSamplingRatio(0.1).setAllowSharedDDCDictionary(false)
+			.setSeed(compressionSeed).setValidCompressions(DDCOnly).setInvestigateEstimate(true).create(),
+		// TODO: DDC1 sharring does not work correctly in Aggregare Col Max.
+		// The other tests passes fine.
+		// new
+		// CompressionSettingsBuilder().setSamplingRatio(0.1).setAllowSharedDDCDictionary(true).setSeed(compressionSeed).setValidCompressions(DDCOnly)
+		// .create(),
+		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setValidCompressions(OLEOnly)
+			.setInvestigateEstimate(true).create(),
+		new CompressionSettingsBuilder().setSamplingRatio(0.1).setSeed(compressionSeed).setValidCompressions(RLEOnly)
+			.setInvestigateEstimate(true).create(),
+		new CompressionSettingsBuilder().setSamplingRatio(1.0).setSeed(compressionSeed).setInvestigateEstimate(true)
+			.create()};
+
+	protected static MatrixTypology[] usedMatrixTypology = new MatrixTypology[] { // Selected Matrix Types
+		MatrixTypology.SMALL,
+		// MatrixTypology.FEW_COL,
+		MatrixTypology.FEW_ROW,
+		// MatrixTypology.LARGE,
+		// MatrixTypology.SINGLE_COL,
+		// MatrixTypology.SINGLE_ROW,
+		MatrixTypology.L_ROWS,
+		// MatrixTypology.XL_ROWS,
 	};
 
-	public static double[] samplingRatio = {0.05, 1.00};
+	// Compressed Block
+	protected MatrixBlock cmb;
 
-	protected ValueType valType;
-	protected ValueRange valRange;
-	protected CompressionType compType;
-	protected boolean compress;
+	// Decompressed Result
+	protected MatrixBlock cmbDeCompressed;
+	protected double[][] deCompressed;
 
-	protected int rows;
-	protected int cols;
-	protected int min;
-	protected int max;
-	protected double sparsity;
+	// Threads
+	protected int k = 1;
 
-	public CompressedTestBase(SparsityType sparType, ValueType valType, ValueRange valueRange, CompressionType compType,
-		MatrixType matrixType, boolean compress, double samplingRatio) {
-		this.sparsity = TestConstants.getSparsityValue(sparType);
-		this.rows = TestConstants.getNumberOfRows(matrixType);
-		this.cols = TestConstants.getNumberOfColumns(matrixType);
+	protected int sampleTolerance = 1024;
 
-		this.max = TestConstants.getMaxRangeValue(valueRange);
-		if(valType == ValueType.CONST) {
-			min = max;
-		}
-		else {
-			min = TestConstants.getMinRangeValue(valueRange);
-		}
-		this.valRange = valueRange;
-		this.valType = valType;
-		this.compType = compType;
-		this.compress = compress;
-	}
+	public CompressedTestBase(SparsityType sparType, ValueType valType, ValueRange valueRange,
+		CompressionSettings compSettings, MatrixTypology MatrixTypology) {
+		super(sparType, valType, valueRange, compSettings, MatrixTypology);
+		// System.out.println("HERE !");
+		try {
 
-	protected MatrixBlock getMatrixBlockInput(double[][] input) {
-		// generate input data
+			cmb = CompressedMatrixBlockFactory.compress(mb, k, compressionSettings);
+
+			if(cmb instanceof CompressedMatrixBlock) {
+				cmbDeCompressed = ((CompressedMatrixBlock) cmb).decompress();
+				if(cmbDeCompressed != null) {
+
+					deCompressed = DataConverter.convertToDoubleMatrix(cmbDeCompressed);
+				}
+			}
+			else {
+				cmbDeCompressed = null;
+				deCompressed = null;
+			}
 
-		if(valType == ValueType.RAND_ROUND_OLE || valType == ValueType.RAND_ROUND_DDC) {
-			CompressedMatrixBlock.ALLOW_DDC_ENCODING = (valType == ValueType.RAND_ROUND_DDC);
-			input = TestUtils.round(input);
+		}
+		catch(Exception e) {
+			e.printStackTrace();
+			// throw new RuntimeException(
+			// "CompressionTest Init failed with settings: " + this.toString() + "\n" + e.getMessage(), e);
+			assertTrue("\nCompressionTest Init failed with settings: " + this.toString(), false);
 		}
 
-		return DataConverter.convertToMatrixBlock(input);
 	}
 
 	@Parameters
 	public static Collection<Object[]> data() {
 		ArrayList<Object[]> tests = new ArrayList<>();
 
-		// Only add a single selected test of constructor with no compression
-		tests.add(new Object[] {SparsityType.SPARSE, ValueType.RAND, ValueRange.SMALL, CompressionType.LOSSLESS,
-			MatrixType.SMALL, false, 1.0});
-
 		for(SparsityType st : usedSparsityTypes) {
 			for(ValueType vt : usedValueTypes) {
 				for(ValueRange vr : usedValueRanges) {
-					for(CompressionType ct : usedCompressionTypes) {
-						for(MatrixType mt : usedMatrixType) {
-							for(double sr : samplingRatio) {
-								tests.add(new Object[] {st, vt, vr, ct, mt, true, sr});
-							}
+					for(CompressionSettings cs : usedCompressionSettings) {
+						for(MatrixTypology mt : usedMatrixTypology) {
+							tests.add(new Object[] {st, vt, vr, cs, mt,});
+
 						}
 					}
 				}
@@ -126,31 +151,4 @@ public class CompressedTestBase extends AutomatedTestBase {
 
 		return tests;
 	}
-
-	@Override
-	public void setUp() {
-	}
-
-	@Override
-	public void tearDown() {
-	}
-
-	@Override
-	public String toString() {
-		StringBuilder builder = new StringBuilder();
-
-		builder.append("args: ");
-
-		builder.append(String.format("%6s%14s", "Vt:", valType));
-		builder.append(String.format("%6s%8s", "Vr:", valRange));
-		builder.append(String.format("%6s%8s", "CP:", compType));
-		builder.append(String.format("%6s%5s", "CD:", compress));
-		builder.append(String.format("%6s%5s", "Rows:", rows));
-		builder.append(String.format("%6s%5s", "Cols:", cols));
-		builder.append(String.format("%6s%12s", "Min:", min));
-		builder.append(String.format("%6s%12s", "Max:", max));
-		builder.append(String.format("%6s%4s", "Spar:", sparsity));
-
-		return builder.toString();
-	}
 }
diff --git a/src/test/java/org/apache/sysds/test/component/compress/CompressedVectorTest.java b/src/test/java/org/apache/sysds/test/component/compress/CompressedVectorTest.java
index 683abaf..451a234 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/CompressedVectorTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressedVectorTest.java
@@ -19,46 +19,32 @@
 
 package org.apache.sysds.test.component.compress;
 
+import java.util.ArrayList;
+import java.util.Collection;
+
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.functionobjects.CM;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.CMOperator;
 import org.apache.sysds.runtime.matrix.operators.CMOperator.AggregateOperationTypes;
-import org.apache.sysds.runtime.util.DataConverter;
 import org.apache.sysds.test.TestUtils;
-
-import java.util.ArrayList;
-import java.util.Collection;
-
+import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
+import org.apache.sysds.test.component.compress.TestConstants.SparsityType;
+import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.ValueType;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
-import org.apache.sysds.test.TestConstants.CompressionType;
-import org.apache.sysds.test.TestConstants.MatrixType;
-import org.apache.sysds.test.TestConstants.SparsityType;
-import org.apache.sysds.test.TestConstants.ValueType;
-import org.apache.sysds.test.TestConstants.ValueRange;
 
 @RunWith(value = Parameterized.class)
 public class CompressedVectorTest extends CompressedTestBase {
 
-	// Input
-	double[][] input;
-	MatrixBlock mb;
-
-	// Compressed Block
-	CompressedMatrixBlock cmb;
-
-	// Compression Result
-	MatrixBlock cmbResult;
-
-	// Decompressed Result
-	MatrixBlock cmbDeCompressed;
-	double[][] deCompressed;
-
-	protected static MatrixType[] usedMatrixTypeLocal = new MatrixType[] {// types
-		MatrixType.SINGLE_COL, MatrixType.SINGLE_COL_L};
+	protected static MatrixTypology[] usedMatrixTypologyLocal = new MatrixTypology[] {// types
+		MatrixTypology.SINGLE_COL,
+		// MatrixTypology.SINGLE_COL_L
+	};
 
 	@Parameters
 	public static Collection<Object[]> data() {
@@ -66,11 +52,9 @@ public class CompressedVectorTest extends CompressedTestBase {
 		for(SparsityType st : usedSparsityTypes) {
 			for(ValueType vt : usedValueTypes) {
 				for(ValueRange vr : usedValueRanges) {
-					for(CompressionType ct : usedCompressionTypes) {
-						for(MatrixType mt : usedMatrixTypeLocal) {
-							for(double sr : samplingRatio) {
-								tests.add(new Object[] {st, vt, vr, ct, mt, true, sr});
-							}
+					for(CompressionSettings cs : usedCompressionSettings) {
+						for(MatrixTypology mt : usedMatrixTypologyLocal) {
+							tests.add(new Object[] {st, vt, vr, cs, mt});
 						}
 					}
 				}
@@ -79,26 +63,16 @@ public class CompressedVectorTest extends CompressedTestBase {
 		return tests;
 	}
 
-	public CompressedVectorTest(SparsityType sparType, ValueType valType, ValueRange valRange, CompressionType compType,
-		MatrixType matrixType, boolean compress, double samplingRatio) {
-		super(sparType, valType, valRange, compType, matrixType, compress, samplingRatio);
-		input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, 7);
-		mb = getMatrixBlockInput(input);
-		cmb = new CompressedMatrixBlock(mb);
-		cmb.setSeed(1);
-		cmb.setSamplingRatio(samplingRatio);
-		if(compress) {
-			cmbResult = cmb.compress();
-		}
-		cmbDeCompressed = cmb.decompress();
-		deCompressed = DataConverter.convertToDoubleMatrix(cmbDeCompressed);
+	public CompressedVectorTest(SparsityType sparType, ValueType valType, ValueRange valRange,
+		CompressionSettings compSettings, MatrixTypology matrixTypology) {
+		super(sparType, valType, valRange, compSettings, matrixTypology);
 	}
 
 	@Test
 	public void testCentralMoment() throws Exception {
 		// TODO: Make Central Moment Test work on Multi dimensional Matrix
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			// quantile uncompressed
@@ -116,9 +90,6 @@ public class CompressedVectorTest extends CompressedTestBase {
 		catch(Exception e) {
 			throw new Exception(this.toString() + "\n" + e.getMessage(), e);
 		}
-		finally {
-			CompressedMatrixBlock.ALLOW_DDC_ENCODING = true;
-		}
 	}
 
 	@Test
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
new file mode 100644
index 0000000..443c828
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/component/compress/CompressibleInputGenerator.java
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.sysds.runtime.compress.colgroup.ColGroup.CompressionType;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.util.DataConverter;
+
+/**
+ * WARNING, this compressible input generator generates transposed inputs, (rows and cols are switched) this is because
+ * then the test does not need to transpose the input for the colGroups that expect transposed inputs.
+ * 
+ */
+public class CompressibleInputGenerator {
+
+	public static MatrixBlock getInput(int rows, int cols, CompressionType ct, int nrUnique, double sparsity,
+		int seed) {
+		double[][] output = getInputDoubleMatrix(rows, cols, ct, nrUnique, sparsity, seed, false);
+		return DataConverter.convertToMatrixBlock(output);
+	}
+
+	public static double[][] getInputDoubleMatrix(int rows, int cols, CompressionType ct, int nrUnique, double sparsity,
+		int seed, boolean transpose) {
+		double[][] output;
+		switch(ct) {
+			case RLE:
+				output = rle(rows, cols, nrUnique, sparsity, seed, transpose);
+				break;
+			case OLE:
+				output = ole(rows, cols, nrUnique, sparsity, seed, transpose);
+				break;
+			default:
+				throw new NotImplementedException("Not implemented generator.");
+		}
+
+		return output;
+	}
+
+	private static double[][] rle(int rows, int cols, int nrUnique, double sparsity, int seed, boolean transpose) {
+
+		Random r = new Random(seed);
+		List<Double> values = getNRandomValues(nrUnique, r);
+
+		double[][] matrix = transpose ? new double[rows][cols]: new double[cols][rows];
+
+		for(int colNr = 0; colNr < cols; colNr++) {
+			Collections.shuffle(values, r);
+
+			// Generate a Dirichlet distribution, to distribute the values
+			int[] occurences = makeDirichletDistribution(nrUnique, rows, r);
+
+			// double[] col = new double[rows];
+
+			int pointer = 0;
+			int valuePointer = 0;
+			for(int nr : occurences) {
+				int zeros = (int) (Math.floor(nr * (1.0 - sparsity)));
+				int before = (zeros > 0) ? r.nextInt(zeros) : 0;
+				int after = zeros - before;
+				pointer += before;
+				for(int i = before; i < nr - after; i++) {
+					if(transpose){
+						matrix[pointer][colNr] = values.get(valuePointer);
+					}else{
+						matrix[colNr][pointer] = values.get(valuePointer);
+					}
+					pointer++;
+				}
+				pointer += after;
+				valuePointer++;
+				if(valuePointer == values.size() && after == 0) {
+					while(pointer < rows) {
+						if(transpose){
+							matrix[pointer][colNr] = values.get(nrUnique - 1);
+						}else{
+							matrix[colNr][pointer] = values.get(nrUnique - 1);
+						}
+						pointer++;
+					}
+				}
+			}
+		}
+		return matrix;
+	}
+
+	// Note ole compress the best if there are multiple correlated columns.
+	// Therefore the multiple columns are needed for good compressions.
+	// Also Nr Unique is only associated to a specific column in this compression, so the number of
+	// uniques are only in a single column, making actual the nrUnique (cols * nrUnique)
+	// Does not guaranty that all the nr uniques are in use, since the values are randomly selected.
+	private static double[][] ole(int rows, int cols, int nrUnique, double sparsity, int seed, boolean transpose) {
+		// chose some random values
+		Random r = new Random(seed);
+		List<Double> values = getNRandomValues(nrUnique, r);
+		double[][] matrix = transpose ? new double[rows][cols]: new double[cols][rows];
+
+		// Generate the first column.
+		// double[] col1 = new double[rows];
+		// matrix[0] = col1;
+		for(int x = 0; x < rows; x++) {
+			if(r.nextDouble() < sparsity) {
+				if(transpose){
+					matrix[x][0] = values.get(r.nextInt(nrUnique));
+				}else{
+					matrix[0][x] = values.get(r.nextInt(nrUnique));
+				}
+			}
+		}
+
+		// System.out.println(Arrays.toString(matrix[0]));
+
+		for(int y = 1; y < cols; y++) {
+			for(int x = 0; x < rows; x++) {
+				if(r.nextDouble() < sparsity) {
+					if(transpose){
+						matrix[x][y] = Double.longBitsToDouble(Double.doubleToLongBits(matrix[x][0] + y) << 32L);
+					}else{
+						matrix[y][x] = Double.longBitsToDouble(Double.doubleToLongBits(matrix[0][x] + y) << 32L);
+					}
+				}
+			}
+		}
+		// System.out.println(Arrays.toString(matrix[1]));
+
+		return matrix;
+	}
+
+	private static int[] makeDirichletDistribution(int nrUnique, int rows, Random r) {
+		double[] distribution = new double[nrUnique];
+		double sum = 0;
+		for(int i = 0; i < nrUnique; i++) {
+			distribution[i] = r.nextDouble();
+			sum += distribution[i];
+		}
+
+		int[] occurences = new int[nrUnique];
+		for(int i = 0; i < nrUnique; i++) {
+			occurences[i] = (int) (((double) distribution[i] / (double) sum) * (double) rows);
+		}
+		return occurences;
+	}
+
+	private static List<Double> getNRandomValues(int nrUnique, Random r) {
+		List<Double> values = new ArrayList<>();
+		for(int i = 0; i < nrUnique; i++) {
+			values.add((double)Math.round(r.nextDouble()* 1000.0));
+		}
+		return values;
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/component/compress/ParCompressedMatrixTest.java b/src/test/java/org/apache/sysds/test/component/compress/ParCompressedMatrixTest.java
index cf79595..80b82bb 100644
--- a/src/test/java/org/apache/sysds/test/component/compress/ParCompressedMatrixTest.java
+++ b/src/test/java/org/apache/sysds/test/component/compress/ParCompressedMatrixTest.java
@@ -19,66 +19,41 @@
 
 package org.apache.sysds.test.component.compress;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 import org.apache.sysds.lops.MMTSJ.MMTSJType;
 import org.apache.sysds.lops.MapMultChain.ChainType;
 import org.apache.sysds.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysds.runtime.compress.CompressionSettings;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.functionobjects.Multiply;
 import org.apache.sysds.runtime.functionobjects.Plus;
-import org.apache.sysds.runtime.instructions.InstructionUtils;
 import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.matrix.operators.AggregateBinaryOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateOperator;
 import org.apache.sysds.runtime.matrix.operators.AggregateUnaryOperator;
 import org.apache.sysds.runtime.util.DataConverter;
 import org.apache.sysds.test.TestUtils;
-import org.apache.sysds.test.TestConstants.CompressionType;
-import org.apache.sysds.test.TestConstants.MatrixType;
-import org.apache.sysds.test.TestConstants.SparsityType;
-import org.apache.sysds.test.TestConstants.ValueType;
-import org.apache.sysds.test.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.MatrixTypology;
+import org.apache.sysds.test.component.compress.TestConstants.SparsityType;
+import org.apache.sysds.test.component.compress.TestConstants.ValueRange;
+import org.apache.sysds.test.component.compress.TestConstants.ValueType;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 @RunWith(value = Parameterized.class)
-public class ParCompressedMatrixTest extends CompressedTestBase {
-
-	// Input
-	protected double[][] input;
-	protected MatrixBlock mb;
-
-	// Compressed Block
-	protected CompressedMatrixBlock cmb;
+public class ParCompressedMatrixTest extends AbstractCompressedUnaryTests {
 
-	// Compression Result
-	protected MatrixBlock cmbResult;
-
-	// Decompressed Result
-	protected MatrixBlock cmbDeCompressed;
-	protected double[][] deCompressed;
-
-	int k = InfrastructureAnalyzer.getLocalParallelism();
+	private int k = InfrastructureAnalyzer.getLocalParallelism();
 
 	public ParCompressedMatrixTest(SparsityType sparType, ValueType valType, ValueRange valRange,
-		CompressionType compType, MatrixType matrixType, boolean compress, double samplingRatio) {
-		super(sparType, valType, valRange, compType, matrixType, compress, samplingRatio);
-		input = TestUtils.generateTestMatrix(rows, cols, min, max, sparsity, 7);
-		mb = getMatrixBlockInput(input);
-		cmb = new CompressedMatrixBlock(mb);
-		cmb.setSeed(1);
-		cmb.setSamplingRatio(samplingRatio);
-		if(compress) {
-			cmbResult = cmb.compress(k);
-		}
-		cmbDeCompressed = cmb.decompress(k);
-		deCompressed = DataConverter.convertToDoubleMatrix(cmbDeCompressed);
+		CompressionSettings compressionSettings, MatrixTypology matrixTypology) {
+		super(sparType, valType, valRange, compressionSettings, matrixTypology);
 	}
 
 	@Test
 	public void testConstruction() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock)) {
+			if(!(cmb instanceof CompressedMatrixBlock)) {
 				// TODO Compress EVERYTHING!
 				return; // Input was not compressed then just pass test
 				// Assert.assertTrue("Compression Failed \n" + this.toString(), false);
@@ -94,8 +69,9 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 
 	@Test
 	public void testGetValue() {
+
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			for(int i = 0; i < rows; i++)
@@ -114,7 +90,7 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testMatrixMultChain() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector1 = DataConverter
@@ -137,7 +113,7 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 				// compare result with input
 				double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
 				double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
-				TestUtils.compareMatricesBitAvgDistance(d1, d2, cols, 1, 2048, 10);
+				TestUtils.compareMatricesBitAvgDistance(d1, d2, cols, 1, 2048, 32);
 			}
 		}
 		catch(Exception e) {
@@ -149,7 +125,7 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 	@Test
 	public void testTransposeSelfMatrixMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 			// ChainType ctype = ChainType.XtwXv;
 			for(MMTSJType mType : new MMTSJType[] {MMTSJType.LEFT,
@@ -175,9 +151,9 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 	}
 
 	@Test
-	public void testVectorMult() {
+	public void testMatrixVectorMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
 
 			MatrixBlock vector = DataConverter
@@ -202,71 +178,29 @@ public class ParCompressedMatrixTest extends CompressedTestBase {
 		}
 	}
 
-	enum AggType {
-		ROWSUMS, COLSUMS, SUM, ROWSUMSSQ, COLSUMSSQ, SUMSQ, ROWMAXS, COLMAXS, MAX, ROWMINS, COLMINS, MIN,
-	}
-
 	@Test
-	public void testUnaryOperators() {
+	public void testVectorMatrixMult() {
 		try {
-			if(!(cmbResult instanceof CompressedMatrixBlock))
+			if(!(cmb instanceof CompressedMatrixBlock))
 				return; // Input was not compressed then just pass test
-			for(AggType aggType : AggType.values()) {
-				AggregateUnaryOperator auop = null;
-				switch(aggType) {
-					case SUM:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uak+", k);
-						break;
-					case ROWSUMS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uark+", k);
-						break;
-					case COLSUMS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uack+", k);
-						break;
-					case SUMSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uasqk+", k);
-						break;
-					case ROWSUMSSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarsqk+", k);
-						break;
-					case COLSUMSSQ:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacsqk+", k);
-						break;
-					case MAX:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uamax", k);
-						break;
-					case ROWMAXS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarmax", k);
-						break;
-					case COLMAXS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacmax", k);
-						break;
-					case MIN:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uamin", k);
-						break;
-					case ROWMINS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uarmin", k);
-						break;
-					case COLMINS:
-						auop = InstructionUtils.parseBasicAggregateUnaryOperator("uacmin", k);
-						break;
-				}
-				// matrix-vector uncompressed
-				MatrixBlock ret1 = mb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, null, true);
 
-				// matrix-vector compressed
-				MatrixBlock ret2 = cmb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, null, true);
+			MatrixBlock vector = DataConverter
+				.convertToMatrixBlock(TestUtils.generateTestMatrix(1, rows, 1, 1, 1.0, 3));
 
-				// compare result with input
-				double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
-				double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
-				int dim1 = (aggType == AggType.ROWSUMS || aggType == AggType.ROWSUMSSQ || aggType == AggType.ROWMINS ||
-					aggType == AggType.ROWMINS) ? rows : 1;
-				int dim2 = (aggType == AggType.COLSUMS || aggType == AggType.COLSUMSSQ || aggType == AggType.COLMAXS ||
-					aggType == AggType.COLMINS) ? cols : 1;
... 1238 lines suppressed ...