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 2016/07/17 00:23:26 UTC

[1/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 71013e758 -> 16e7b1c88


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/scripts/functions/compress/LinregCG.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/compress/LinregCG.R b/src/test/scripts/functions/compress/LinregCG.R
new file mode 100644
index 0000000..1d69385
--- /dev/null
+++ b/src/test/scripts/functions/compress/LinregCG.R
@@ -0,0 +1,57 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+args <- commandArgs(TRUE)
+options(digits=22)
+library("Matrix")
+
+X = readMM(paste(args[1], "X.mtx", sep=""))
+y = readMM(paste(args[1], "y.mtx", sep=""))
+
+intercept = as.integer(args[2]);
+eps = as.double(args[3]);
+maxiter = as.double(args[4]);
+
+if( intercept == 1 ){
+   ones = matrix(1, nrow(X), 1); 
+   X = cbind(X, ones);
+}
+
+r = -(t(X) %*% y);
+p = -r;
+norm_r2 = sum(r * r);
+w = matrix(0, ncol(X), 1);
+
+i = 0;
+while(i < maxiter) {
+	q = ((t(X) %*% (X %*% p)) + eps  * p);
+	alpha = norm_r2 / ((t(p) %*% q)[1:1]);
+	w = w + alpha * p;
+	old_norm_r2 = norm_r2;
+	r = r + alpha * q;
+	norm_r2 = sum(r * r);
+	beta = norm_r2 / old_norm_r2;
+	p = -r + beta * p;
+	i = i + 1;
+}
+
+writeMM(as(w,"CsparseMatrix"), paste(args[5], "w", sep=""))

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/scripts/functions/compress/LinregCG.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/compress/LinregCG.dml b/src/test/scripts/functions/compress/LinregCG.dml
new file mode 100644
index 0000000..02d0fad
--- /dev/null
+++ b/src/test/scripts/functions/compress/LinregCG.dml
@@ -0,0 +1,56 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+X = read($1);
+y = read($2);
+intercept = $3;
+eps = $4;
+maxiter = $5;
+
+if( intercept == 1 ){
+   ones = matrix(1, nrow(X), 1); 
+   X = append(X, ones);
+}
+
+r = -(t(X) %*% y);
+p = -r;
+norm_r2 = sum(r * r);
+w = matrix(0, rows = ncol(X), cols = 1);
+
+i = 0;
+while(i < maxiter) {
+	q = ((t(X) %*% (X %*% p)) + eps  * p);
+	alpha = norm_r2 / castAsScalar(t(p) %*% q);
+	w = w + alpha * p;
+	old_norm_r2 = norm_r2;
+	r = r + alpha * q;
+	norm_r2 = sum(r * r);
+	beta = norm_r2 / old_norm_r2;
+	p = -r + beta * p;
+	i = i + 1;
+}
+
+write(w, $6);
+
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/scripts/functions/compress/SystemML-config-compress.xml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/compress/SystemML-config-compress.xml b/src/test/scripts/functions/compress/SystemML-config-compress.xml
new file mode 100644
index 0000000..0728ecc
--- /dev/null
+++ b/src/test/scripts/functions/compress/SystemML-config-compress.xml
@@ -0,0 +1,59 @@
+<!--
+ * 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.
+-->
+
+<root>
+   <!-- local fs tmp working directory-->
+   <localtmpdir>/tmp/systemml</localtmpdir>
+
+   <!-- hdfs tmp working directory--> 
+   <scratch>scratch_space</scratch> 
+
+   <!-- compiler optimization level, valid values: 0 | 1 | 2 | 3 | 4, default: 2 -->
+   <optlevel>2</optlevel>  
+
+   <!-- default number of reduce tasks per MR job, default: 2 x number of nodes -->
+   <numreducers>10</numreducers> 
+   
+   <!-- override jvm reuse flag for specific MR jobs, valid values: true | false  -->
+   <jvmreuse>false</jvmreuse> 
+
+   <!-- default block dim for binary block files -->
+   <defaultblocksize>1000</defaultblocksize> 
+
+   <!-- run systemml control program as yarn appmaster, in case of MR1 always falls back to client, please disable for debug mode -->
+   <dml.yarn.appmaster>false</dml.yarn.appmaster>
+
+   <!-- maximum jvm heap size of the dml yarn appmaster in MB, the requested memory is 1.5x this parameter -->
+   <dml.yarn.appmaster.mem>2048</dml.yarn.appmaster.mem>
+
+   <!-- maximum jvm heap size of the map/reduce tasks in MB, the requested memory is 1.5x this parameter, negative values ignored  -->
+   <dml.yarn.mapreduce.mem>2048</dml.yarn.mapreduce.mem>
+
+   <!-- yarn application submission queue, relevant for default capacity scheduler -->
+   <dml.yarn.app.queue>default</dml.yarn.app.queue>
+   
+   <!-- enables multi-threaded matrix multiplications in singlenode control program -->
+   <cp.parallel.matrixmult>true</cp.parallel.matrixmult>
+   
+   <!-- enables multi-threaded read/write of text formats in singlenode control program -->
+   <cp.parallel.textio>true</cp.parallel.textio>
+
+   <!-- enables compressed linear algebra for cp/spark -->
+   <compressed.linalg>true</compressed.linalg>
+</root>

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test_suites/java/org/apache/sysml/test/integration/functions/compress/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/compress/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/compress/ZPackageSuite.java
new file mode 100644
index 0000000..62dc46e
--- /dev/null
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/compress/ZPackageSuite.java
@@ -0,0 +1,56 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+/** Group together the tests in this package into a single suite so that the Maven build
+ *  won't run two of them at once. */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+	BasicCompressionTest.class,
+	BasicMatrixAppendTest.class,
+	BasicMatrixMultChainTest.class,
+	BasicMatrixTransposeSelfMultTest.class,
+	BasicMatrixVectorMultTest.class,
+	BasicScalarOperationsSparseUnsafeTest.class,
+	BasicScalarOperationsTest.class,
+	BasicTransposeSelfLeftMatrixMultTest.class,
+	BasicUnaryAggregateTest.class,
+	BasicVectorMatrixMultTest.class,
+	CompressedLinregCG.class,
+	CompressedSerializationTest.class,
+	LargeCompressionTest.class,
+	LargeMatrixVectorMultTest.class,
+	LargeParMatrixVectorMultTest.class,
+	LargeVectorMatrixMultTest.class,
+	ParMatrixMultChainTest.class,
+	ParMatrixVectorMultTest.class,
+	ParTransposeSelfLeftMatrixMultTest.class,
+	ParUnaryAggregateTest.class,
+	ParVectorMatrixMultTest.class,
+})
+
+
+/** This class is just a holder for the above JUnit annotations. */
+public class ZPackageSuite {
+
+}


[2/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicVectorMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicVectorMatrixMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicVectorMatrixMultTest.java
new file mode 100644
index 0000000..c9b7ec4
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicVectorMatrixMultTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+
+/**
+ * 
+ */
+public class BasicVectorMatrixMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(1, rows, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop);
+			MatrixBlock ret1 = (MatrixBlock)vector.aggregateBinaryOperations(vector, mb, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(vector, cmb, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, 1, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedLinregCG.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedLinregCG.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedLinregCG.java
new file mode 100644
index 0000000..a74f784
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedLinregCG.java
@@ -0,0 +1,151 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import java.util.HashMap;
+
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.lops.LopProperties.ExecType;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class CompressedLinregCG extends AutomatedTestBase 
+{
+	private final static String TEST_NAME1 = "LinregCG";
+	private final static String TEST_DIR = "functions/compress/";
+	private final static String TEST_CONF = "SystemML-config-compress.xml";
+	
+	private final static double eps = 1e-4;
+	
+	private final static int rows = 1468;
+	private final static int cols = 980;
+		
+	private final static double sparsity1 = 0.7; //dense
+	private final static double sparsity2 = 0.1; //sparse
+	
+	private final static int intercept = 0;
+	private final static double epsilon = 0.000000001;
+	private final static double maxiter = 10;
+	
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_DIR, TEST_NAME1, new String[] { "w" })); 
+	}
+
+	@Test
+	public void testGDFOLinregCGDenseCP() {
+		runGDFOTest(TEST_NAME1, false, ExecType.CP);
+	}
+	
+	@Test
+	public void testGDFOLinregCGSparseCP() {
+		runGDFOTest(TEST_NAME1, true, ExecType.CP);
+	}
+	
+	@Test
+	public void testGDFOLinregCGDenseSP() {
+		runGDFOTest(TEST_NAME1, false, ExecType.SPARK);
+	}
+	
+	@Test
+	public void testGDFOLinregCGSparseSP() {
+		runGDFOTest(TEST_NAME1, true, ExecType.SPARK);
+	}
+	
+	/**
+	 * 
+	 * @param sparseM1
+	 * @param sparseM2
+	 * @param instType
+	 */
+	private void runGDFOTest( String testname,boolean sparse, ExecType instType)
+	{
+		//rtplatform for MR
+		RUNTIME_PLATFORM platformOld = rtplatform;
+		switch( instType ){
+			case MR: rtplatform = RUNTIME_PLATFORM.HADOOP; break;
+			case SPARK: rtplatform = RUNTIME_PLATFORM.HYBRID_SPARK; break;
+			default: rtplatform = RUNTIME_PLATFORM.HYBRID; break;
+		}
+	
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		if( rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK )
+			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+		long memOld = InfrastructureAnalyzer.getLocalMaxMemory();
+		
+		try
+		{
+			String TEST_NAME = testname;
+			TestConfiguration config = getTestConfiguration(TEST_NAME);
+			
+			/* This is for running the junit test the new way, i.e., construct the arguments directly */
+			String HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = HOME + TEST_NAME + ".dml";
+			programArgs = new String[]{ "-explain","-stats",
+					                    "-config="+HOME+TEST_CONF,
+					                    "-args", HOME + INPUT_DIR + "X",
+					                             HOME + INPUT_DIR + "y",
+					                             String.valueOf(intercept),
+					                             String.valueOf(epsilon),
+					                             String.valueOf(maxiter),
+					                            HOME + OUTPUT_DIR + "w"};
+			fullRScriptName = HOME + TEST_NAME + ".R";
+			rCmd = "Rscript" + " " + fullRScriptName + " " + 
+			       HOME + INPUT_DIR + " " + 
+			       String.valueOf(intercept) + " " + String.valueOf(epsilon) + " " + 
+			       String.valueOf(maxiter) + " " + HOME + EXPECTED_DIR;
+			
+			loadTestConfiguration(config);
+	
+			//generate actual datasets
+			double[][] X = getRandomMatrix(rows, cols, 1, 1, sparse?sparsity2:sparsity1, 7);
+			writeInputMatrixWithMTD("X", X, true);
+			double[][] y = getRandomMatrix(rows, 1, 0, 10, 1.0, 3);
+			writeInputMatrixWithMTD("y", y, true);
+			
+			if( rtplatform == RUNTIME_PLATFORM.HYBRID_SPARK  )
+				InfrastructureAnalyzer.setLocalMaxMemory(8*1024*1024);
+			
+			runTest(true, false, null, -1); 
+			runRScript(true); 
+			
+			//compare matrices 
+			HashMap<CellIndex, Double> dmlfile = readDMLMatrixFromHDFS("w");
+			HashMap<CellIndex, Double> rfile  = readRMatrixFromFS("w");
+			TestUtils.compareMatrices(dmlfile, rfile, eps, "Stat-DML", "Stat-R");
+		}
+		finally
+		{
+			rtplatform = platformOld;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+			InfrastructureAnalyzer.setLocalMaxMemory(memOld);		
+		}
+	}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedSerializationTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedSerializationTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedSerializationTest.java
new file mode 100644
index 0000000..9405aa8
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/CompressedSerializationTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class CompressedSerializationTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runCompressedSerializationTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runCompressedSerializationTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstantDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runCompressedSerializationTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runCompressedSerializationTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//serialize compressed matrix block
+			ByteArrayOutputStream bos = new ByteArrayOutputStream();
+			DataOutputStream fos = new DataOutputStream(bos);
+			cmb.write(fos);
+			
+			//deserialize compressed matrix block
+			ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+			DataInputStream fis = new DataInputStream(bis);
+			CompressedMatrixBlock cmb2 = new CompressedMatrixBlock();
+			cmb2.readFields(fis);
+			
+			//decompress the compressed matrix block
+			MatrixBlock tmp = cmb2.decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(mb);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(tmp);
+			TestUtils.compareMatrices(d1, d2, rows, cols, 0);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeCompressionTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeCompressionTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeCompressionTest.java
new file mode 100644
index 0000000..4f9101c
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeCompressionTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class LargeCompressionTest extends AutomatedTestBase
+{	
+	private static final int rows = 5*BitmapEncoder.BITMAP_BLOCK_SZ;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runCompressionTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runCompressionTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runCompressionTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//decompress the compressed matrix block
+			MatrixBlock tmp = cmb.decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(mb);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(tmp);
+			TestUtils.compareMatrices(d1, d2, rows, cols, 0);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeMatrixVectorMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeMatrixVectorMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeMatrixVectorMultTest.java
new file mode 100644
index 0000000..d2da1a6
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeMatrixVectorMultTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class LargeMatrixVectorMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 5*BitmapEncoder.BITMAP_BLOCK_SZ;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop);
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateBinaryOperations(mb, vector, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(cmb, vector, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeParMatrixVectorMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeParMatrixVectorMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeParMatrixVectorMultTest.java
new file mode 100644
index 0000000..6cdceee
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeParMatrixVectorMultTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class LargeParMatrixVectorMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 5*BitmapEncoder.BITMAP_BLOCK_SZ;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop,
+					InfrastructureAnalyzer.getLocalParallelism());
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateBinaryOperations(mb, vector, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(cmb, vector, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeVectorMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeVectorMatrixMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeVectorMatrixMultTest.java
new file mode 100644
index 0000000..8335ca4
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/LargeVectorMatrixMultTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class LargeVectorMatrixMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 5*BitmapEncoder.BITMAP_BLOCK_SZ;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(1, rows, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop);
+			MatrixBlock ret1 = (MatrixBlock)vector.aggregateBinaryOperations(vector, mb, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(vector, cmb, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, 1, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixMultChainTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixMultChainTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixMultChainTest.java
new file mode 100644
index 0000000..d87b42a
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixMultChainTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.lops.MapMultChain.ChainType;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class ParMatrixMultChainTest extends AutomatedTestBase
+{	
+	private static final int rows = 2701;
+	private static final int cols = 14;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testEmptyNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseConstDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testEmptyNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testEmptyWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseConstDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testEmptyWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testDenseConstDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseConstDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtwXv, false);
+	}
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixMultChainTest(SparsityType sptype, ValueType vtype, ChainType ctype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector1 = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 0, 1, 1.0, 3));
+			MatrixBlock vector2 = (ctype==ChainType.XtwXv)? DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(rows, 1, 0, 1, 1.0, 3)) : null;
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			int k = InfrastructureAnalyzer.getLocalParallelism();
+			MatrixBlock ret1 = (MatrixBlock)mb.chainMatrixMultOperations(vector1, vector2, new MatrixBlock(), ctype, k);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.chainMatrixMultOperations(vector1, vector2, new MatrixBlock(), ctype, k);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, cols, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixVectorMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixVectorMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixVectorMultTest.java
new file mode 100644
index 0000000..2ec0ab8
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParMatrixVectorMultTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+
+/**
+ * 
+ */
+public class ParMatrixVectorMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop,
+					InfrastructureAnalyzer.getLocalParallelism());
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateBinaryOperations(mb, vector, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(cmb, vector, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/ParTransposeSelfLeftMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/ParTransposeSelfLeftMatrixMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParTransposeSelfLeftMatrixMultTest.java
new file mode 100644
index 0000000..4091315
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParTransposeSelfLeftMatrixMultTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.lops.MMTSJ.MMTSJType;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class ParTransposeSelfLeftMatrixMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runTransposeSelfMatrixMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			int k = InfrastructureAnalyzer.getLocalParallelism();
+			MatrixBlock ret1 = mb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT, k);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = cmb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT, k);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, cols, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/ParUnaryAggregateTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/ParUnaryAggregateTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParUnaryAggregateTest.java
new file mode 100644
index 0000000..7d65418
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParUnaryAggregateTest.java
@@ -0,0 +1,547 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+
+/**
+ * 
+ */
+public class ParUnaryAggregateTest extends AutomatedTestBase
+{	
+	private static final int rows = 2071;
+	private static final int cols1 = 10;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	public enum AggType {
+		ROWSUMS,
+		COLSUMS,
+		SUM,
+		ROWSUMSSQ,
+		COLSUMSSQ,
+		SUMSQ
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testRowSumsDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMS, false);
+	}
+
+	@Test
+	public void testSumDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMSSQ, false);
+	}
+
+	@Test
+	public void testSumSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUMSQ, false);
+	}
+		
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runUnaryAggregateTest(SparsityType sptype, ValueType vtype, AggType aggtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols1, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//prepare unary aggregate operator
+			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;
+			}
+			auop.setNumThreads(InfrastructureAnalyzer.getLocalParallelism());
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed						
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, 1000, null, true);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, 1000, null, true);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			int dim1 = (aggtype == AggType.ROWSUMS)?rows:1;
+			int dim2 = (aggtype == AggType.COLSUMS)?cols1:1;
+			TestUtils.compareMatrices(d1, d2, dim1, dim2, 0.00000000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/ParVectorMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/ParVectorMatrixMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParVectorMatrixMultTest.java
new file mode 100644
index 0000000..bbf3dea
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/ParVectorMatrixMultTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class ParVectorMatrixMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(1, rows, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop,
+					InfrastructureAnalyzer.getLocalParallelism());
+			MatrixBlock ret1 = (MatrixBlock)vector.aggregateBinaryOperations(vector, mb, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(vector, cmb, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, 1, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}



[4/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCoder.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCoder.java b/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCoder.java
new file mode 100644
index 0000000..07d9757
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCoder.java
@@ -0,0 +1,227 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.TreeMap;
+
+import org.apache.sysml.runtime.compress.estim.CompressedSizeEstimator;
+
+public class PlanningCoCoder 
+{
+	//constants for weight computation
+	private final static float GROUPABILITY_THRESHOLD = 0.00064f;
+	private final static boolean USE_BIN_WEIGHT = false;
+	private final static float PARTITION_WEIGHT = 0.05F; //higher values lead to more grouping
+	private final static float PARTITION_SIZE = PARTITION_WEIGHT * GROUPABILITY_THRESHOLD;
+	private final static float BIN_WEIGHT_PARAM = -0.65f; //lower values lead to more grouping
+
+	/**
+	 * 
+	 * @param sizeEstimator
+	 * @param availCols
+	 * @param colsCardinalities
+	 * @param compressedSize
+	 * @param numRows
+	 * @param sparsity
+	 * @return
+	 */
+	public static List<int[]> findCocodesByPartitioning(CompressedSizeEstimator sizeEstimator, List<Integer> availCols, 
+			List<Integer> colsCardinalities,List<Long> compressedSize, int numRows, double sparsity) 
+	{
+		float numRowsWeight = numRows;
+		List<int[]> retGroups = new ArrayList<int[]>();
+		// filtering out non-groupable columns as singleton groups
+		int numCols = availCols.size();
+		List<Integer> groupabaleCols = new ArrayList<Integer>();
+		// weighted of each column is the ratio of its cardinality to the number
+		// of rows scaled by the matrix sparsity
+		List<Float> groupabaleColWeights = new ArrayList<Float>();
+		HashMap<Integer, GroupableColInfo> groupableColsInfo = new HashMap<Integer, GroupableColInfo>();
+		for (int i = 0; i < numCols; i++) {
+			int colIx = availCols.get(i);
+			int cardinality = colsCardinalities.get(i);
+			float weight = ((float) cardinality) / numRowsWeight;
+			if (weight <= GROUPABILITY_THRESHOLD) {
+				groupabaleCols.add(colIx);
+				groupabaleColWeights.add(weight);
+				groupableColsInfo.put(colIx, new GroupableColInfo(weight,
+						compressedSize.get(i)));
+			} else {
+				retGroups.add(new int[] { colIx });
+			}
+		}
+		// bin packing based on PARTITION_WEIGHT and column weights
+		float weight = computeWeightForCoCoding(numRows, sparsity);
+		TreeMap<Float, List<List<Integer>>> bins = new PlanningBinPacker(
+				weight, groupabaleCols, groupabaleColWeights) 
+				.packFirstFit();
+
+		// brute force grouping within each partition
+		for (List<List<Integer>> binList : bins.values()) {
+			for (List<Integer> bin : binList) {
+				// building an array of singleton CoCodingGroup
+				PlanningCoCodingGroup[] singltonGroups = new PlanningCoCodingGroup[bin.size()];
+				int i = 0;
+				GroupableColInfo colInfo;
+				for (Integer col : bin) {
+					colInfo = groupableColsInfo.get(col);
+					singltonGroups[i++] = new PlanningCoCodingGroup(col, colInfo.size,
+							colInfo.cardRatio);
+				}
+				PlanningCoCodingGroup[] outputGroups = findCocodesBruteForce(
+						sizeEstimator, numRowsWeight, singltonGroups);
+				
+				for (PlanningCoCodingGroup grp : outputGroups) {
+					retGroups.add(grp.getColIndices());
+				}
+			}
+		}
+		return retGroups;
+	}
+
+	/**
+	 * 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.
+	 * 
+	 */
+	private static PlanningCoCodingGroup[] findCocodesBruteForce(
+			CompressedSizeEstimator sizeEstimator, float numRowsWeight,
+			PlanningCoCodingGroup[] singltonGroups) 
+	{
+		// Populate a priority queue with all available 2-column cocodings.
+		PriorityQueue<PlanningGroupMergeAction> q = new PriorityQueue<PlanningGroupMergeAction>();
+		for (int leftIx = 0; leftIx < singltonGroups.length; leftIx++) {
+			PlanningCoCodingGroup leftGrp = singltonGroups[leftIx];
+			for (int rightIx = leftIx + 1; rightIx < singltonGroups.length; rightIx++) {
+				PlanningCoCodingGroup rightGrp = singltonGroups[rightIx];
+				// at least one of the two groups should be low-cardinality
+				float cardRatio = leftGrp.getCardinalityRatio() + rightGrp.getCardinalityRatio(); 
+				if ( cardRatio < GROUPABILITY_THRESHOLD) {
+					PlanningGroupMergeAction potentialMerge = new PlanningGroupMergeAction(
+							sizeEstimator, numRowsWeight, leftGrp, rightGrp);
+					if (potentialMerge.getChangeInSize() < 0) {
+						q.add(potentialMerge);
+					}
+				}
+			}
+		}
+		PlanningCoCodingGroup[] colGroups = singltonGroups;
+		
+		// Greedily merge groups until we can no longer reduce the number of
+		// runs by merging groups
+		while (q.size() > 0) {
+			PlanningGroupMergeAction merge = q.poll();
+
+			// The queue can contain merge actions involving column groups that
+			// have already been merged.
+			// Filter those actions out.
+			int leftIx = findInArray(colGroups, merge.getLeftGrp());
+			int rightIx = findInArray(colGroups, merge.getRightGrp());
+			if (leftIx < 0 || rightIx < 0) {
+				// One or more of the groups to be merged has already been made
+				// part of another group.
+				// Drop the merge action.
+			} else {
+				PlanningCoCodingGroup mergedGrp = merge.getMergedGrp();
+
+				PlanningCoCodingGroup[] newColGroups = new PlanningCoCodingGroup[colGroups.length - 1];
+				int targetIx = 0;
+				for (int i = 0; i < colGroups.length; i++) {
+					if (i != leftIx && i != rightIx) {
+						newColGroups[targetIx] = colGroups[i];
+						targetIx++;
+					}
+				}
+
+				// New group goes at the end to (hopefully) speed up future
+				// linear search operations
+				newColGroups[newColGroups.length - 1] = mergedGrp;
+
+				// Consider merging the new group with all the other
+				// pre-existing groups.
+				for (int i = 0; i < newColGroups.length - 1; i++) {
+					PlanningCoCodingGroup newLeftGrp = newColGroups[i];
+					PlanningCoCodingGroup newRightGrp = mergedGrp;
+					if (newLeftGrp.getCardinalityRatio()
+							+ newRightGrp.getCardinalityRatio() < GROUPABILITY_THRESHOLD) {
+						PlanningGroupMergeAction newPotentialMerge = new PlanningGroupMergeAction(
+								sizeEstimator, numRowsWeight, newLeftGrp,
+								newRightGrp);
+						if (newPotentialMerge.getChangeInSize() < 0) {
+							q.add(newPotentialMerge);
+						}
+					}
+				}
+				colGroups = newColGroups;
+			}
+		}
+		return colGroups;
+	}
+
+	/**
+	 * 
+	 * @param numRows
+	 * @param sparsity
+	 * @return
+	 */
+	private static float computeWeightForCoCoding(int numRows, double sparsity)
+	{
+		if( USE_BIN_WEIGHT ) { //new method (non-conclusive)
+			//return (float) Math.pow(numRows*sparsity,BIN_WEIGHT_PARAM);
+			return (float) Math.pow(numRows,BIN_WEIGHT_PARAM);
+		}
+		else {
+			return PARTITION_SIZE;
+		}
+	}
+	
+	/**
+	 * 
+	 * @param arr
+	 * @param val
+	 * @return
+	 */
+	private static int findInArray(Object[] arr, Object val) {
+		for (int i = 0; i < arr.length; i++) {
+			if (arr[i].equals(val)) {
+				return i;
+			}
+		}
+		return -1;
+	}
+	
+	/**
+	 * 
+	 */
+	private static class GroupableColInfo {
+		float cardRatio;
+		long size;
+
+		public GroupableColInfo(float lcardRatio, long lsize) {
+			cardRatio = lcardRatio;
+			size = lsize;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCodingGroup.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCodingGroup.java b/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCodingGroup.java
new file mode 100644
index 0000000..221e4ca
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/PlanningCoCodingGroup.java
@@ -0,0 +1,104 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+
+import org.apache.sysml.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysml.runtime.compress.estim.CompressedSizeInfo;
+
+/** 
+ * Class to represent information about co-coding a group of columns. 
+ * 
+ */
+public class PlanningCoCodingGroup 
+{
+	private int[] _colIndexes;
+	private long _estSize;
+	private float _cardRatio;
+
+	/**
+	 * Constructor for a one-column group; i.e. do not co-code a given column.
+	 * 
+	 */
+	public PlanningCoCodingGroup(int col, long estSize, float cardRatio) {
+		_colIndexes = new int[]{col};
+		_estSize = estSize;
+		_cardRatio = cardRatio;
+	}
+
+	/**
+	 * Constructor for merging two disjoint groups of columns
+	 * 
+	 * @param grp1   first group of columns to merge
+	 * @param grp2   second group to merge
+	 * @param numRowsWeight numRows x sparsity
+	 */
+	public PlanningCoCodingGroup(PlanningCoCodingGroup grp1, PlanningCoCodingGroup grp2,
+			CompressedSizeEstimator bitmapSizeEstimator, float numRowsWeight) 
+	{
+		// merge sorted non-empty arrays
+		_colIndexes = new int[grp1._colIndexes.length + grp2._colIndexes.length];		
+		int grp1Ptr = 0, grp2Ptr = 0;
+		for (int mergedIx = 0; mergedIx < _colIndexes.length; mergedIx++) {
+			if (grp1._colIndexes[grp1Ptr] < grp2._colIndexes[grp2Ptr]) {
+				_colIndexes[mergedIx] = grp1._colIndexes[grp1Ptr++];
+				if (grp1Ptr == grp1._colIndexes.length) {
+					System.arraycopy(grp2._colIndexes, grp2Ptr, _colIndexes,
+							mergedIx + 1, grp2._colIndexes.length - grp2Ptr);
+					break;
+				}
+			} else {
+				_colIndexes[mergedIx] = grp2._colIndexes[grp2Ptr++];
+				if (grp2Ptr == grp2._colIndexes.length) {
+					System.arraycopy(grp1._colIndexes, grp1Ptr, _colIndexes,
+							mergedIx + 1, grp1._colIndexes.length - grp1Ptr);
+					break;
+				}
+			}
+		}
+		
+		// estimating size info
+		CompressedSizeInfo groupSizeInfo = bitmapSizeEstimator
+				.estimateCompressedColGroupSize(_colIndexes);
+		_estSize = groupSizeInfo.getMinSize();
+		_cardRatio = groupSizeInfo.getEstCarinality() / numRowsWeight;
+	}
+
+	public int[] getColIndices() {
+		return _colIndexes;
+	}
+
+	/**
+	 * @return estimated compressed size of the grouped columns
+	 */
+	public long getEstSize() {
+		return _estSize;
+	}
+
+	public float getCardinalityRatio() {
+		return _cardRatio;
+	}
+
+	@Override
+	public String toString() {
+		return Arrays.toString(_colIndexes);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/PlanningGroupMergeAction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/PlanningGroupMergeAction.java b/src/main/java/org/apache/sysml/runtime/compress/PlanningGroupMergeAction.java
new file mode 100644
index 0000000..5e3c6c5
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/PlanningGroupMergeAction.java
@@ -0,0 +1,73 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import org.apache.sysml.runtime.compress.estim.CompressedSizeEstimator;
+
+/**
+ * Internal data structure for tracking potential merges of column groups in
+ * co-coding calculations.
+ * 
+ */
+class PlanningGroupMergeAction implements Comparable<PlanningGroupMergeAction> 
+{
+	private PlanningCoCodingGroup _leftGrp;   //left input
+	private PlanningCoCodingGroup _rightGrp;  //right input
+	private PlanningCoCodingGroup _mergedGrp; //output
+	private long _changeInSize;
+
+	
+	public PlanningGroupMergeAction(CompressedSizeEstimator sizeEstimator,
+			float numRowsWeight, PlanningCoCodingGroup leftGrp, PlanningCoCodingGroup rightGrp) {
+		_leftGrp = leftGrp;
+		_rightGrp = rightGrp;
+		_mergedGrp = new PlanningCoCodingGroup(leftGrp, rightGrp, sizeEstimator, numRowsWeight);
+
+		// Negative size change ==> Decrease in size
+		_changeInSize = _mergedGrp.getEstSize() 
+				- leftGrp.getEstSize() - rightGrp.getEstSize();
+	}
+
+	public int compareTo(PlanningGroupMergeAction o) {
+		// We only sort by the change in size
+		return (int) Math.signum(_changeInSize - o._changeInSize);
+	}
+
+	@Override
+	public String toString() {
+		return String.format("Merge %s and %s", _leftGrp, _rightGrp);
+	}
+
+	public PlanningCoCodingGroup getLeftGrp() {
+		return _leftGrp;
+	}
+
+	public PlanningCoCodingGroup getRightGrp() {
+		return _rightGrp;
+	}
+
+	public PlanningCoCodingGroup getMergedGrp() {
+		return _mergedGrp;
+	}
+
+	public long getChangeInSize() {
+		return _changeInSize;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelection.java b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelection.java
new file mode 100644
index 0000000..a37018f
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelection.java
@@ -0,0 +1,64 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import org.apache.sysml.runtime.compress.utils.DblArray;
+
+/**
+ * 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.
+	 * 
+	 * @return
+	 */
+	public abstract DblArray nextRow();
+
+	/**
+	 * 
+	 * @return
+	 */
+	public int getCurrentRowIndex() {
+		return _lastRow;
+	}
+	
+
+	/**
+	 * Resets the reader to the first row.
+	 */
+	public void reset() {
+		_lastRow = -1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDense.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDense.java b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDense.java
new file mode 100644
index 0000000..d22f39d
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDense.java
@@ -0,0 +1,68 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import org.apache.sysml.runtime.compress.utils.DblArray;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+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);
+		_data = data;
+		reusableArr = new double[colIndices.length];
+		reusableReturn = new DblArray(reusableArr);
+	}
+
+	@Override
+	public DblArray nextRow() {
+		if( _skipZeros) {
+			while ((nonZeroReturn = getNextRow()) != null
+				&& DblArray.isZero(nonZeroReturn)); 
+			return nonZeroReturn;
+		} else {
+			return getNextRow();
+		}
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	private DblArray getNextRow() {
+		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] );
+		}
+		return reusableReturn;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDenseSample.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDenseSample.java b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDenseSample.java
new file mode 100644
index 0000000..06518e4
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionDenseSample.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.sysml.runtime.compress;
+
+import org.apache.sysml.runtime.compress.utils.DblArray;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+/**
+ * 
+ * considers only a subset of row indexes
+ */
+public class ReaderColumnSelectionDenseSample extends ReaderColumnSelection 
+{
+	protected MatrixBlock _data;
+	
+	private int[] _sampleIndexes;
+	private int lastIndex = -1;
+
+	// reusable return
+	private DblArray nonZeroReturn;
+	private DblArray reusableReturn;
+	private double[] reusableArr;
+
+	public ReaderColumnSelectionDenseSample(MatrixBlock data, int[] colIndexes, int[] sampleIndexes, boolean skipZeros) 
+	{
+		super(colIndexes, -1, skipZeros);
+		_data = data;
+		_sampleIndexes = sampleIndexes;
+		reusableArr = new double[colIndexes.length];
+		reusableReturn = new DblArray(reusableArr);
+	}
+
+	@Override
+	public DblArray nextRow() {
+		if (_skipZeros) {
+			while ((nonZeroReturn = getNextRow()) != null
+					&& DblArray.isZero(nonZeroReturn));
+			return nonZeroReturn;
+		} else {
+			return getNextRow();
+		}
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	private DblArray getNextRow() {
+		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]);
+		}
+		return reusableReturn;
+	}
+
+	@Override
+	public int getCurrentRowIndex() {
+		return _sampleIndexes[lastIndex];
+	}
+	
+	@Override
+	public void reset() {
+		lastIndex = -1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionSparse.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionSparse.java b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionSparse.java
new file mode 100644
index 0000000..d2ef5a4
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ReaderColumnSelectionSparse.java
@@ -0,0 +1,115 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+
+import org.apache.sysml.runtime.compress.utils.DblArray;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.data.SparseRow;
+
+/**
+ * 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.
+ */
+public class ReaderColumnSelectionSparse extends ReaderColumnSelection 
+{
+	private final DblArray ZERO_DBL_ARRAY;
+	private DblArray nonZeroReturn;
+
+	// reusable return
+	private DblArray reusableReturn;
+	private double[] reusableArr;
+
+	// 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);
+		ZERO_DBL_ARRAY = new DblArray(new double[colIndexes.length], true);
+		reusableArr = new double[colIndexes.length];
+		reusableReturn = new DblArray(reusableArr);
+		
+		if( !CompressedMatrixBlock.TRANSPOSE_INPUT ){
+			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]);
+		Arrays.fill(sparsePos, 0);
+	}
+
+	@Override
+	public DblArray nextRow() {
+		if(_skipZeros) {
+			while ((nonZeroReturn = getNextRow()) != null
+				&& nonZeroReturn == ZERO_DBL_ARRAY);
+			return nonZeroReturn;
+		} else {
+			return getNextRow();
+		}
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	private DblArray getNextRow() 
+	{
+		if(_lastRow == _numRows-1)
+			return null;
+		_lastRow++;
+		
+		if( !CompressedMatrixBlock.TRANSPOSE_INPUT ){
+			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) )
+			{
+				sparsePos[i]++;
+			}
+		
+		//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 )
+			{
+				reusableArr[i] = sparseCols[i].values()[sparsePos[i]];
+				zeroResult = false;
+			}
+
+		return zeroResult ? ZERO_DBL_ARRAY : reusableReturn;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/UncompressedBitmap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/UncompressedBitmap.java b/src/main/java/org/apache/sysml/runtime/compress/UncompressedBitmap.java
new file mode 100644
index 0000000..971f438
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/UncompressedBitmap.java
@@ -0,0 +1,101 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+
+import org.apache.sysml.runtime.compress.utils.DblArrayIntListHashMap;
+import org.apache.sysml.runtime.compress.utils.DoubleIntListHashMap;
+import org.apache.sysml.runtime.compress.utils.DblArrayIntListHashMap.DArrayIListEntry;
+import org.apache.sysml.runtime.compress.utils.DoubleIntListHashMap.DIListEntry;
+
+/** 
+ * Uncompressed representation of one or more columns in bitmap format. 
+ * 
+ */
+public final class UncompressedBitmap 
+{
+	private int _numCols;
+
+	/** 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 int[][] _offsetsLists;
+
+	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];
+		_offsetsLists = new int[numVals][];
+		int bitmapIx = 0;
+		for( DArrayIListEntry val : distinctVals.extractValues()) {
+			System.arraycopy(val.key.getData(), 0, _values, bitmapIx*numColumns, numColumns);
+			_offsetsLists[bitmapIx++] = val.value.extractValues();
+		}
+		_numCols = numColumns;
+	}
+
+	public UncompressedBitmap( DoubleIntListHashMap distinctVals ) 
+	{
+		// added for one pass bitmap construction
+		// Convert inputs to arrays
+		int numVals = distinctVals.size();
+		_values = new double[numVals];
+		_offsetsLists = new int[numVals][];
+		int bitmapIx = 0;
+		for(DIListEntry val : distinctVals.extractValues()) {
+			_values[bitmapIx] = val.key;
+			_offsetsLists[bitmapIx++] = val.value.extractValues();
+		}
+		_numCols = 1;
+	}
+	
+	public int getNumColumns() {
+		return _numCols;
+	}
+
+	/**
+	 * @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 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;
+	}
+
+	/**
+	 * @param ix   index of a particular distinct value
+	 * @return IMMUTABLE array of the offsets of the rows containing the value
+	 *         with the indicated index
+	 */
+	public int[] getOffsetsList(int ix) {
+		return _offsetsLists[ix];
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimator.java b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimator.java
new file mode 100644
index 0000000..1a1ae55
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimator.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.runtime.compress.estim;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.UncompressedBitmap;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+/**
+ * Base class for all compressed size estimators
+ */
+public abstract class CompressedSizeEstimator 
+{
+	protected MatrixBlock _data;
+
+	public CompressedSizeEstimator(MatrixBlock data) {
+		_data = data;
+	}
+
+	/**
+	 * 
+	 * @param colIndexes
+	 * @return
+	 */
+	public abstract CompressedSizeInfo estimateCompressedColGroupSize(int[] colIndexes);
+
+	/**
+	 * 
+	 * @param ubm
+	 * @return
+	 */
+	public abstract CompressedSizeInfo estimateCompressedColGroupSize(UncompressedBitmap ubm);
+
+	/**
+	 * 
+	 * @param ubm
+	 * @param inclRLE
+	 * @return
+	 */
+	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);
+			numOffs += list.length;
+			numSegs += list[list.length - 1] / BitmapEncoder.BITMAP_BLOCK_SZ + 1;
+			numSingle += (list.length==1) ? 1 : 0;
+			if( inclRLE ) {
+				int lastOff = -2;
+				for (int j = 0; j < list.length; j++) {
+					if (list[j] != lastOff + 1)
+						numRuns++;
+					lastOff = list[j];
+				}
+			}
+		}
+		
+		//construct estimation factors
+		return new SizeEstimationFactors(numVals, numSegs, numOffs, numRuns, numSingle);
+	}
+
+	/**
+	 * Estimates the number of bytes needed to encode this column group
+	 * in RLE encoding format.
+	 * 
+	 * @param numVals
+	 * @param numRuns
+	 * @param numCols
+	 * @return
+	 */
+	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;
+	}
+
+	/**
+	 * Estimates the number of bytes needed to encode this column group 
+	 * in OLE format.
+	 * 
+	 * @param numVals
+	 * @param numOffs
+	 * @param numSeqs
+	 * @param numCols
+	 * @return
+	 */
+	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;
+	}
+	
+	/**
+	 * 
+	 */
+	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;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorExact.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorExact.java b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorExact.java
new file mode 100644
index 0000000..557c518
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorExact.java
@@ -0,0 +1,53 @@
+/*
+ * 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.sysml.runtime.compress.estim;
+
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.UncompressedBitmap;
+import org.apache.sysml.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));
+	}
+
+	@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,
+				getRLESize(fact.numVals, fact.numRuns, ubm.getNumColumns()),
+				getOLESize(fact.numVals, fact.numOffs, fact.numSegs, ubm.getNumColumns()));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorSample.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorSample.java b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorSample.java
new file mode 100644
index 0000000..76a0f06
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeEstimatorSample.java
@@ -0,0 +1,767 @@
+/*
+ * 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.sysml.runtime.compress.estim;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+
+import org.apache.commons.math3.distribution.ChiSquaredDistribution;
+import org.apache.commons.math3.random.RandomDataGenerator;
+import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.runtime.compress.BitmapEncoder;
+import org.apache.sysml.runtime.compress.ReaderColumnSelection;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.compress.ReaderColumnSelectionDense;
+import org.apache.sysml.runtime.compress.ReaderColumnSelectionDenseSample;
+import org.apache.sysml.runtime.compress.ReaderColumnSelectionSparse;
+import org.apache.sysml.runtime.compress.UncompressedBitmap;
+import org.apache.sysml.runtime.compress.utils.DblArray;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+public class CompressedSizeEstimatorSample extends CompressedSizeEstimator 
+{
+	private static final boolean CORRECT_NONZERO_ESTIMATE = false; //TODO enable for production
+	private final static double SHLOSSER_JACKKNIFE_ALPHA = 0.975;
+	public static final float HAAS_AND_STOKES_ALPHA1 = 0.9F; //0.9 recommended in paper
+	public static final float HAAS_AND_STOKES_ALPHA2 = 30F; //30 recommended in paper
+	public static final float HAAS_AND_STOKES_UJ2A_C = 50; //50 recommend in paper
+
+	private int[] _sampleRows = null;
+	private RandomDataGenerator _rng = null;
+	private int _numRows = -1;
+	
+	/**
+	 * 
+	 * @param data
+	 * @param sampleRows
+	 */
+	public CompressedSizeEstimatorSample(MatrixBlock data, int[] sampleRows) {
+		super(data);
+		_sampleRows = sampleRows;
+		_rng = new RandomDataGenerator();
+		_numRows = CompressedMatrixBlock.TRANSPOSE_INPUT ? 
+				_data.getNumColumns() : _data.getNumRows();
+	}
+
+	/**
+	 * 
+	 * @param mb
+	 * @param sampleSize
+	 */
+	public CompressedSizeEstimatorSample(MatrixBlock mb, int sampleSize) {
+		this(mb, null);
+		_sampleRows = getSortedUniformSample(_numRows, sampleSize);
+	}
+
+	/**
+	 * 
+	 * @param sampleRows, assumed to be sorted
+	 */
+	public void setSampleRows(int[] sampleRows) {
+		_sampleRows = sampleRows;
+	}
+
+	/**
+	 * 
+	 * @param sampleSize
+	 */
+	public void resampleRows(int sampleSize) {
+		_sampleRows = getSortedUniformSample(_numRows, sampleSize);
+	}
+
+	@Override
+	public CompressedSizeInfo estimateCompressedColGroupSize(int[] colIndexes) 
+	{
+		//extract statistics from sample
+		UncompressedBitmap ubm = BitmapEncoder.extractBitmapFromSample(
+				colIndexes, _data, _sampleRows);
+		SizeEstimationFactors fact = computeSizeEstimationFactors(ubm, false);
+
+		//estimate number of distinct values 
+		int totalCardinality = getNumDistinctValues(colIndexes);
+		totalCardinality = Math.max(totalCardinality, fact.numVals); //fix anomalies w/ large sample fraction
+		totalCardinality = Math.min(totalCardinality, _numRows); //fix anomalies w/ large sample fraction
+		
+		//estimate unseen values
+		// each unseen is assumed to occur only once (it did not show up in the sample because it is rare)
+		int unseen = Math.max(0, totalCardinality - fact.numVals);
+		int sampleSize = _sampleRows.length;
+		
+		//estimate number of offsets
+		double sparsity = OptimizerUtils.getSparsity(
+				_data.getNumRows(), _data.getNumColumns(), _data.getNonZeros());
+		
+		// expected value given that we don't store the zero values
+		float totalNumOffs = (float) (_numRows * (1 - Math.pow(1 - sparsity,colIndexes.length)));		
+		if( CORRECT_NONZERO_ESTIMATE ) {
+			long numZeros = sampleSize - fact.numOffs;
+			float C = Math.max(1-(float)fact.numSingle/sampleSize, (float)sampleSize/_numRows); 
+			totalNumOffs = _numRows - ((numZeros>0)? (float)_numRows/sampleSize*C*numZeros : 0);
+		}
+		
+		// For a single offset, the number of blocks depends on the value of
+		// that offset. small offsets (first group of rows in the matrix)
+		// require a small number of blocks and large offsets (last group of
+		// rows) require a large number of blocks. The unseen offsets are
+		// distributed over the entire offset range. A reasonable and fast
+		// estimate for the number of blocks is to use the arithmetic mean of
+		// the number of blocks used for the first index (=1) and that of the
+		// last index.
+		int numUnseenSeg = Math.round(unseen
+				* (2.0f * BitmapEncoder.BITMAP_BLOCK_SZ + _numRows) / 2
+				/ BitmapEncoder.BITMAP_BLOCK_SZ);
+		int totalNumSeg = fact.numSegs + numUnseenSeg;
+		int totalNumRuns = getNumRuns(ubm, sampleSize, _numRows) + unseen;
+
+		//construct new size info summary
+		return new CompressedSizeInfo(totalCardinality,
+				getRLESize(totalCardinality, totalNumRuns, colIndexes.length),
+				getOLESize(totalCardinality, totalNumOffs, totalNumSeg, colIndexes.length));
+	}
+
+	@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,
+				getRLESize(fact.numVals, fact.numRuns, ubm.getNumColumns()),
+				getOLESize(fact.numVals, fact.numOffs, fact.numSegs, ubm.getNumColumns()));
+	}
+	
+	/**
+	 * 
+	 * @param colIndexes
+	 * @return
+	 */
+	private int getNumDistinctValues(int[] colIndexes) {
+		return haasAndStokes(colIndexes);
+	}
+
+	/**
+	 * 
+	 * @param sampleUncompressedBitmap
+	 * @param sampleSize
+	 * @param totalNumRows
+	 * @return
+	 */
+	private int getNumRuns(UncompressedBitmap sampleUncompressedBitmap,
+			int sampleSize, int totalNumRows) {
+		int numVals = sampleUncompressedBitmap.getNumValues();
+		// all values in the sample are zeros
+		if (numVals == 0)
+			return 0;
+		float numRuns = 0;
+		for (int vi = 0; vi < numVals; vi++) {
+			int[] offsets = sampleUncompressedBitmap.getOffsetsList(vi);
+			float offsetsRatio = ((float) offsets.length) / sampleSize;
+			float avgAdditionalOffsets = offsetsRatio * totalNumRows
+					/ sampleSize;
+			if (avgAdditionalOffsets < 1) {
+				// Ising-Stevens does not hold
+				// fall-back to using the expected number of offsets as an upper
+				// bound on the number of runs
+				numRuns += ((float) offsets.length) * totalNumRows / sampleSize;
+				continue;
+			}
+			int intervalEnd, intervalSize;
+			float additionalOffsets;
+			// probability of an index being non-offset in current and previous
+			// interval respectively
+			float nonOffsetProb, prevNonOffsetProb = 1;
+			boolean reachedSampleEnd = false;
+			// handling the first interval separately for simplicity
+			int intervalStart = -1;
+			if (_sampleRows[0] == 0) {
+				// empty interval
+				intervalStart = 0;
+			} else {
+				intervalEnd = _sampleRows[0];
+				intervalSize = intervalEnd - intervalStart - 1;
+				// expected value of a multivariate hypergeometric distribution
+				additionalOffsets = offsetsRatio * intervalSize;
+				// expected value of an Ising-Stevens distribution
+				numRuns += (intervalSize - additionalOffsets)
+						* additionalOffsets / intervalSize;
+				intervalStart = intervalEnd;
+				prevNonOffsetProb = (intervalSize - additionalOffsets)
+						/ intervalSize;
+			}
+			// for handling separators
+
+			int withinSepRun = 0;
+			boolean seenNonOffset = false, startedWithOffset = false, endedWithOffset = false;
+			int offsetsPtrs = 0;
+			for (int ix = 1; ix < sampleSize; ix++) {
+				// start of a new separator
+				// intervalStart will always be pointing at the current value
+				// in the separator block
+
+				if (offsetsPtrs < offsets.length
+						&& offsets[offsetsPtrs] == intervalStart) {
+					startedWithOffset = true;
+					offsetsPtrs++;
+					endedWithOffset = true;
+				} else {
+					seenNonOffset = true;
+					endedWithOffset = false;
+				}
+				while (intervalStart + 1 == _sampleRows[ix]) {
+					intervalStart = _sampleRows[ix];
+					if (seenNonOffset) {
+						if (offsetsPtrs < offsets.length
+								&& offsets[offsetsPtrs] == intervalStart) {
+							withinSepRun = 1;
+							offsetsPtrs++;
+							endedWithOffset = true;
+						} else {
+							numRuns += withinSepRun;
+							withinSepRun = 0;
+							endedWithOffset = false;
+						}
+					} else if (offsetsPtrs < offsets.length
+							&& offsets[offsetsPtrs] == intervalStart) {
+						offsetsPtrs++;
+						endedWithOffset = true;
+					} else {
+						seenNonOffset = true;
+						endedWithOffset = false;
+					}
+					//
+					ix++;
+					if (ix == sampleSize) {
+						// end of sample which searching for a start
+						reachedSampleEnd = true;
+						break;
+					}
+				}
+
+				// runs within an interval of unknowns
+				if (reachedSampleEnd)
+					break;
+				intervalEnd = _sampleRows[ix];
+				intervalSize = intervalEnd - intervalStart - 1;
+				// expected value of a multivariate hypergeometric distribution
+				additionalOffsets = offsetsRatio * intervalSize;
+				// expected value of an Ising-Stevens distribution
+				numRuns += (intervalSize - additionalOffsets)
+						* additionalOffsets / intervalSize;
+				nonOffsetProb = (intervalSize - additionalOffsets)
+						/ intervalSize;
+
+				// additional runs resulting from x's on the boundaries of the
+				// separators
+				// endedWithOffset = findInArray(offsets, intervalStart) != -1;
+				if (seenNonOffset) {
+					if (startedWithOffset) {
+						// add p(y in the previous interval)
+						numRuns += prevNonOffsetProb;
+					}
+					if (endedWithOffset) {
+						// add p(y in the current interval)
+						numRuns += nonOffsetProb;
+					}
+				} else {
+					// add p(y in the previous interval and y in the current
+					// interval)
+					numRuns += prevNonOffsetProb * nonOffsetProb;
+				}
+				prevNonOffsetProb = nonOffsetProb;
+				intervalStart = intervalEnd;
+				// reseting separator variables
+				seenNonOffset = startedWithOffset = endedWithOffset = false;
+				withinSepRun = 0;
+
+			}
+			// last possible interval
+			if (intervalStart != totalNumRows - 1) {
+				intervalEnd = totalNumRows;
+				intervalSize = intervalEnd - intervalStart - 1;
+				// expected value of a multivariate hypergeometric distribution
+				additionalOffsets = offsetsRatio * intervalSize;
+				// expected value of an Ising-Stevens distribution
+				numRuns += (intervalSize - additionalOffsets)
+						* additionalOffsets / intervalSize;
+				nonOffsetProb = (intervalSize - additionalOffsets)
+						/ intervalSize;
+			} else {
+				nonOffsetProb = 1;
+			}
+			// additional runs resulting from x's on the boundaries of the
+			// separators
+			endedWithOffset = intervalStart == offsets[offsets.length - 1];
+			if (seenNonOffset) {
+				if (startedWithOffset) {
+					numRuns += prevNonOffsetProb;
+				}
+				if (endedWithOffset) {
+					// add p(y in the current interval)
+					numRuns += nonOffsetProb;
+				}
+			} else {
+				if (endedWithOffset)
+					// add p(y in the previous interval and y in the current
+					// interval)
+					numRuns += prevNonOffsetProb * nonOffsetProb;
+			}
+		}
+		return Math.round(numRuns);
+	}
+
+	/**
+	 * 
+	 * @param colIndexes
+	 * @return
+	 */
+	private int haasAndStokes(int[] colIndexes) {
+		ReaderColumnSelection reader =  new ReaderColumnSelectionDenseSample(_data, 
+				colIndexes, _sampleRows, !CompressedMatrixBlock.MATERIALIZE_ZEROS);
+		return haasAndStokes(_numRows, _sampleRows.length, reader);
+	}
+
+	/**
+	 * TODO remove, just for local debugging.
+	 * 
+	 * @param colIndexes
+	 * @return
+	 */
+	@SuppressWarnings("unused")
+	private int getExactNumDistinctValues(int[] colIndexes) {
+		HashSet<DblArray> distinctVals = new HashSet<DblArray>();
+		ReaderColumnSelection reader = (_data.isInSparseFormat() && CompressedMatrixBlock.TRANSPOSE_INPUT) ? 
+				new ReaderColumnSelectionSparse(_data, colIndexes, !CompressedMatrixBlock.MATERIALIZE_ZEROS) : 
+				new ReaderColumnSelectionDense(_data, colIndexes, !CompressedMatrixBlock.MATERIALIZE_ZEROS);
+		DblArray val = null;
+		while (null != (val = reader.nextRow()))
+			distinctVals.add(val);
+		return distinctVals.size();
+	}
+
+	/**
+	 * Returns a sorted array of n integers, drawn uniformly from the range [0,range).
+	 * 
+	 * @param range
+	 * @param smplSize
+	 * @return
+	 */
+	private int[] getSortedUniformSample(int range, int smplSize) {
+		if (smplSize == 0)
+			return new int[] {};
+		int[] sample = _rng.nextPermutation(range, smplSize);
+		Arrays.sort(sample);
+		return sample;
+	}
+	
+
+	/////////////////////////////////////////////////////
+	// 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
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 *            : a reader for the sampled rows
+	 * @return
+	 */
+	@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
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @return
+	 */
+	@SuppressWarnings("unused")
+	private static int shlosserEstimator(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader) 
+	{
+		return shlosserEstimator(nRows, sampleSize, sampleRowsReader,
+				getValCounts(sampleRowsReader));
+	}
+
+	/**
+	 * 
+	 * @param nRows
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @param valsCount
+	 * @return
+	 */
+	private static int shlosserEstimator(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader,
+			HashMap<DblArray, Integer> valsCount) 
+	{
+		double q = ((double) sampleSize) / nRows;
+		double oneMinusQ = 1 - q;
+
+		int[] freqCounts = getFreqCounts(valsCount);
+
+		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(valsCount.size() + 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
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @return
+	 */
+	@SuppressWarnings("unused")
+	private static int smoothedJackknifeEstimator(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader) 
+	{
+		return smoothedJackknifeEstimator(nRows, sampleSize, sampleRowsReader,
+				getValCounts(sampleRowsReader));
+	}
+
+	/**
+	 * 
+	 * @param nRows
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @param valsCount
+	 * @return
+	 */
+	private static int smoothedJackknifeEstimator(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader,
+			HashMap<DblArray, Integer> valsCount) 
+	{
+		int[] freqCounts = getFreqCounts(valsCount);
+		// all values in the sample are zeros
+		if (freqCounts.length == 0)
+			return 0;
+		// nRows is N and sampleSize is n
+
+		int d = valsCount.size();
+		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 = (int) 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
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @return
+	 */
+	@SuppressWarnings("unused")
+	private static int shlosserJackknifeEstimator(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader) {
+		HashMap<DblArray, Integer> valsCount = getValCounts(sampleRowsReader);
+
+		// uniformity chi-square test
+		double nBar = ((double) sampleSize) / valsCount.size();
+		// test-statistic
+		double u = 0;
+		for (int cnt : valsCount.values()) {
+			u += Math.pow(cnt - nBar, 2);
+		}
+		u /= nBar;
+		if (sampleSize != usedSampleSize)
+			computeCriticalValue(sampleSize);
+		if (u < uniformityCriticalValue) {
+			// uniform
+			return smoothedJackknifeEstimator(nRows, sampleSize,
+					sampleRowsReader, valsCount);
+		} else {
+			return shlosserEstimator(nRows, sampleSize, sampleRowsReader,
+					valsCount);
+		}
+	}
+
+	/*
+	 * 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 double uniformityCriticalValue;
+	private static int usedSampleSize;
+	
+	private static void computeCriticalValue(int sampleSize) {
+		ChiSquaredDistribution chiSqr = new ChiSquaredDistribution(sampleSize - 1);
+		uniformityCriticalValue = chiSqr.inverseCumulativeProbability(SHLOSSER_JACKKNIFE_ALPHA);
+		usedSampleSize = 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
+	 * @param sampleSize
+	 * @param sampleRowsReader
+	 * @return
+	 */
+	private static int haasAndStokes(int nRows, int sampleSize,
+			ReaderColumnSelection sampleRowsReader) 
+	{
+		HashMap<DblArray, Integer> valsCount = getValCounts(sampleRowsReader);
+		// all values in the sample are zeros.
+		if (valsCount.size() == 0)
+			return 1;
+		int[] freqCounts = getFreqCounts(valsCount);
+		float q = ((float) sampleSize) / nRows;
+		float _1MinusQ = 1 - q;
+		// Eq. 11
+		float duj1Fraction = ((float) sampleSize)
+				/ (sampleSize - _1MinusQ * freqCounts[0]);
+		float duj1 = duj1Fraction * valsCount.size();
+		// Eq. 16
+		float gamma = 0;
+		for (int i = 1; i <= freqCounts.length; i++) {
+			gamma += i * (i - 1) * freqCounts[i - 1];
+		}
+		gamma *= duj1 / sampleSize / sampleSize;
+		gamma += duj1 / nRows - 1;
+		gamma = Math.max(gamma, 0);
+		int estimate;
+		
+		if (gamma < HAAS_AND_STOKES_ALPHA1) {
+			// UJ2 - begining of page 1479
+		//	System.out.println("uj2");
+			estimate = (int) (duj1Fraction * (valsCount.size() - freqCounts[0]
+					* _1MinusQ * Math.log(_1MinusQ) * gamma / q));
+		} else if (gamma < HAAS_AND_STOKES_ALPHA2) {
+			// UJ2a - end of page 1998
+			//System.out.println("uj2a");
+			int numRemovedClasses = 0;
+			float updatedNumRows = nRows;
+			int updatedSampleSize = sampleSize;
+
+			for (Integer cnt : valsCount.values()) {
+				if (cnt > HAAS_AND_STOKES_UJ2A_C) {
+					numRemovedClasses++;
+					freqCounts[cnt - 1]--;
+					updatedSampleSize -= cnt;
+					/*
+					 * To avoid solving Eq. 20 numerically for the class size in
+					 * the full population (N_j), the current implementation
+					 * just scales cnt (n_j) by the sampling ratio (q).
+					 * Intuitively, the scaling should be fine since cnt is
+					 * large enough. Also, N_j in Eq. 20 is lower-bounded by cnt
+					 * which is already large enough to make the denominator in
+					 * Eq. 20 very close to 1.
+					 */
+					updatedNumRows -= ((float) cnt) / q;
+				}
+			}
+			if (updatedSampleSize == 0) {
+				// use uJ2a
+				
+				estimate = (int) (duj1Fraction * (valsCount.size() - freqCounts[0]
+						* (_1MinusQ) * Math.log(_1MinusQ) * gamma / q));
+			} else {
+				float updatedQ = ((float) updatedSampleSize) / updatedNumRows;
+				int updatedSampleCardinality = valsCount.size()
+						- numRemovedClasses;
+				float updatedDuj1Fraction = ((float) updatedSampleSize)
+						/ (updatedSampleSize - (1 - updatedQ) * freqCounts[0]);
+				float updatedDuj1 = updatedDuj1Fraction
+						* updatedSampleCardinality;
+				float updatedGamma = 0;
+				for (int i = 1; i <= freqCounts.length; i++) {
+					updatedGamma += i * (i - 1) * freqCounts[i - 1];
+				}
+				updatedGamma *= updatedDuj1 / updatedSampleSize
+						/ updatedSampleSize;
+				updatedGamma += updatedDuj1 / updatedNumRows - 1;
+				updatedGamma = Math.max(updatedGamma, 0);
+
+				estimate = (int) (updatedDuj1Fraction * (updatedSampleCardinality - freqCounts[0]
+						* (1 - updatedQ)
+						* Math.log(1 - updatedQ)
+						* updatedGamma / updatedQ))
+						+ numRemovedClasses;
+			}
+
+		} else {
+			// Sh3 - end of section 3
+			float fraq1Numer = 0;
+			float fraq1Denom = 0;
+			float fraq2Numer = 0;
+			float fraq2Denom = 0;
+			for (int i = 1; i <= freqCounts.length; i++) {
+				fraq1Numer += i * q * q * Math.pow(1 - q * q, i - 1)
+						* freqCounts[i - 1];
+				fraq1Denom += Math.pow(_1MinusQ, i) * (Math.pow(1 + q, i) - 1)
+						* freqCounts[i - 1];
+				fraq2Numer += Math.pow(_1MinusQ, i) * freqCounts[i - 1];
+				fraq2Denom += i * q * Math.pow(_1MinusQ, i - 1)
+						* freqCounts[i - 1];
+			}
+			estimate = (int) (valsCount.size() + freqCounts[0] * fraq1Numer
+					/ fraq1Denom * fraq2Numer * fraq2Numer / fraq2Denom
+					/ fraq2Denom);
+		}
+		return estimate < 1 ? 1 : estimate;
+	}
+
+	/**
+	 * 
+	 * @param sampleRowsReader
+	 * @return
+	 */
+	private static HashMap<DblArray, Integer> getValCounts(
+			ReaderColumnSelection sampleRowsReader) 
+	{
+		HashMap<DblArray, Integer> valsCount = new HashMap<DblArray, Integer>();
+		DblArray val = null;
+		Integer cnt;
+		while (null != (val = sampleRowsReader.nextRow())) {
+			cnt = valsCount.get(val);
+			if (cnt == null)
+				cnt = 0;
+			cnt++;
+			valsCount.put(val, cnt);
+		}
+		return valsCount;
+	}
+
+	/**
+	 * 
+	 * @param valsCount
+	 * @return
+	 */
+	private static int[] getFreqCounts(HashMap<DblArray, Integer> valsCount) 
+	{
+		int maxCount = 0;
+		for (Integer c : valsCount.values()) {
+			if (c > maxCount)
+				maxCount = c;
+		}
+		
+		/*
+		 * freqCounts[i-1] = how many values occured with a frequecy i
+		 */
+		int[] freqCounts = new int[maxCount];
+		for (Integer c : valsCount.values()) {
+			freqCounts[c - 1]++;
+		}
+		return freqCounts;
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeInfo.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeInfo.java b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeInfo.java
new file mode 100644
index 0000000..834483e
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/estim/CompressedSizeInfo.java
@@ -0,0 +1,69 @@
+/*
+ * 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.sysml.runtime.compress.estim;
+
+/**
+ * 
+ * A helper reusable object for maintaining bitmap sizes
+ */
+public class CompressedSizeInfo 
+{
+	private int _estCard = -1;
+	private long _rleSize = -1; 
+	private long _oleSize = -1;
+
+	public CompressedSizeInfo() {
+		
+	}
+
+	public CompressedSizeInfo(int estCard, long rleSize, long oleSize) {
+		_estCard = estCard;
+		_rleSize = rleSize;
+		_oleSize = oleSize;
+	}
+
+	public void setRLESize(long rleSize) {
+		_rleSize = rleSize;
+	}
+	
+	public long getRLESize() {
+		return _rleSize;
+	}
+	
+	public void setOLESize(long oleSize) {
+		_oleSize = oleSize;
+	}
+
+	public long getOLESize() {
+		return _oleSize;
+	}
+
+	public long getMinSize() {
+		return Math.min(_rleSize, _oleSize);
+	}
+
+	public void setEstCardinality(int estCard) {
+		_estCard = estCard;
+	}
+
+	public int getEstCarinality() {
+		return _estCard;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/estim/SizeEstimatorFactory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/estim/SizeEstimatorFactory.java b/src/main/java/org/apache/sysml/runtime/compress/estim/SizeEstimatorFactory.java
new file mode 100644
index 0000000..f857b5b
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/estim/SizeEstimatorFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.sysml.runtime.compress.estim;
+
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+public class SizeEstimatorFactory 
+{
+	public static final float SAMPLING_RATIO = 0.01f; //conservative default
+	
+	/**
+	 * 
+	 * @param data
+	 * @param numRows
+	 * @return
+	 */
+	@SuppressWarnings("unused")
+	public static CompressedSizeEstimator getSizeEstimator(MatrixBlock data, int numRows) {
+		return (SAMPLING_RATIO == 1.0) ?
+				new CompressedSizeEstimatorExact(data):
+				new CompressedSizeEstimatorSample(data, (int) (numRows*SAMPLING_RATIO));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/ConverterUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/ConverterUtils.java b/src/main/java/org/apache/sysml/runtime/compress/utils/ConverterUtils.java
new file mode 100644
index 0000000..e87ac29
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/ConverterUtils.java
@@ -0,0 +1,99 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.sysml.runtime.compress.ColGroup;
+import org.apache.sysml.runtime.compress.ColGroupOLE;
+import org.apache.sysml.runtime.compress.ColGroupRLE;
+import org.apache.sysml.runtime.compress.ColGroupUncompressed;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+
+public class ConverterUtils 
+{
+	/**
+	 * Copy col group instance with deep copy of column indices but
+	 * shallow copy of actual contents;
+	 * 
+	 * @param group
+	 * @return
+	 */
+	public static ColGroup copyColGroup(ColGroup group)
+	{
+		ColGroup ret = null;
+		
+		//deep copy col indices
+		int[] colIndices = Arrays.copyOf(group.getColIndices(), group.getNumCols());
+		
+		//create copy of column group
+		if( group instanceof ColGroupUncompressed ) {
+			ColGroupUncompressed in = (ColGroupUncompressed)group;
+			ret = new ColGroupUncompressed(colIndices, in.getNumRows(), in.getData());
+		}
+		else if( group instanceof ColGroupRLE ) {
+			ColGroupRLE in = (ColGroupRLE)group;
+			ret = new ColGroupRLE(colIndices, in.getNumRows(), in.getValues(), 
+					in.getBitmaps(), in.getBitmapOffsets());
+		}
+		else if( group instanceof ColGroupOLE ) {
+			ColGroupOLE in = (ColGroupOLE) group;
+			ret = new ColGroupOLE(colIndices, in.getNumRows(), in.getValues(), 
+					in.getBitmaps(), in.getBitmapOffsets());
+		}
+		
+		return ret;
+	}
+	
+	/**
+	 * 
+	 * @param vector
+	 * @return
+	 */
+	public static double[] getDenseVector( MatrixBlock vector )
+	{
+		if( vector.isInSparseFormat() )
+			return DataConverter.convertToDoubleVector(vector);
+		else 
+			return vector.getDenseBlock();
+	}
+	
+	/**
+	 * 
+	 * @param group
+	 * @return
+	 */
+	public static MatrixBlock getUncompressedColBlock( ColGroup group )
+	{
+		MatrixBlock ret = null;
+		if( group instanceof ColGroupUncompressed ) {
+			ret = ((ColGroupUncompressed) group).getData();
+		}
+		else {
+			ArrayList<ColGroup> tmpGroup = new ArrayList<ColGroup>(Arrays.asList(group));
+			ColGroupUncompressed decompressedCols = new ColGroupUncompressed(tmpGroup);
+			ret = decompressedCols.getData();
+		}
+		
+		return ret;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/DblArray.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/DblArray.java b/src/main/java/org/apache/sysml/runtime/compress/utils/DblArray.java
new file mode 100644
index 0000000..49c163b
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/DblArray.java
@@ -0,0 +1,91 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import java.util.Arrays;
+
+/**
+ * Helper class used for bitmap extraction.
+ *
+ */
+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);
+	}
+
+	public DblArray(double[] arr, boolean allZeros) {
+		_arr = arr;
+		_zero = allZeros;
+	}
+	
+	public double[] getData() {
+		return _arr;
+	}
+	
+	@Override
+	public int hashCode() {
+		return _zero ? 0 : Arrays.hashCode(_arr);
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		return ( o instanceof DblArray
+			&& _zero == ((DblArray) o)._zero
+			&& Arrays.equals(_arr, ((DblArray) o)._arr) );
+	}
+
+	@Override
+	public String toString() {
+		return Arrays.toString(_arr);
+	}
+
+	/**
+	 * 
+	 * @param ds
+	 * @return
+	 */
+	public static boolean isZero(double[] ds) {
+		for (int i = 0; i < ds.length; i++)
+			if (ds[i] != 0.0)
+				return false;
+		return true;
+	}
+
+	/**
+	 * 
+	 * @param val
+	 * @return
+	 */
+	public static boolean isZero(DblArray val) {
+		return val._zero || isZero(val._arr);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/DblArrayIntListHashMap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/DblArrayIntListHashMap.java b/src/main/java/org/apache/sysml/runtime/compress/utils/DblArrayIntListHashMap.java
new file mode 100644
index 0000000..a5455ab
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/DblArrayIntListHashMap.java
@@ -0,0 +1,179 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import java.util.ArrayList;
+
+/**
+ * This class provides a memory-efficient replacement for
+ * HashMap<DblArray,IntArrayList> for restricted use cases.
+ * 
+ */
+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;
+
+	private DArrayIListEntry[] _data = null;
+	private int _size = -1;
+
+	public DblArrayIntListHashMap() {
+		_data = new DArrayIListEntry[INIT_CAPACITY];
+		_size = 0;
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public int size() {
+		return _size;
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @return
+	 */
+	public IntArrayList get(DblArray key) {
+		// probe for early abort
+		if( _size == 0 )
+			return null;
+
+		// compute entry index position
+		int hash = hash(key);
+		int ix = indexFor(hash, _data.length);
+
+		// find entry
+		for( DArrayIListEntry e = _data[ix]; e != null; e = e.next ) {
+			if( e.key.equals(key) ) {
+				return e.value;
+			}
+		}
+
+		return null;
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @param value
+	 */
+	public void appendValue(DblArray key, IntArrayList value) {
+		// compute entry index position
+		int hash = hash(key);
+		int ix = indexFor(hash, _data.length);
+
+		// add new table entry (constant time)
+		DArrayIListEntry enew = new DArrayIListEntry(key, value);
+		enew.next = _data[ix]; // colliding entries / null
+		_data[ix] = enew;
+		_size++;
+
+		// resize if necessary
+		if( _size >= LOAD_FACTOR * _data.length )
+			resize();
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public ArrayList<DArrayIListEntry> extractValues() {
+		ArrayList<DArrayIListEntry> ret = new ArrayList<DArrayIListEntry>();
+		for( DArrayIListEntry e : _data ) {
+			if( e != null ) {
+				while( e.next != null ) {
+					ret.add(e);
+					e = e.next;
+				}
+				ret.add(e);
+			}
+		}
+
+		return ret;
+	}
+
+	/**
+     * 
+     */
+	private void resize() {
+		// check for integer overflow on resize
+		if( _data.length > Integer.MAX_VALUE / RESIZE_FACTOR )
+			return;
+
+		// resize data array and copy existing contents
+		DArrayIListEntry[] olddata = _data;
+		_data = new DArrayIListEntry[_data.length * RESIZE_FACTOR];
+		_size = 0;
+
+		// rehash all entries
+		for( DArrayIListEntry e : olddata ) {
+			if( e != null ) {
+				while( e.next != null ) {
+					appendValue(e.key, e.value);
+					e = e.next;
+				}
+				appendValue(e.key, e.value);
+			}
+		}
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @return
+	 */
+	private static int hash(DblArray key) {
+		int h = key.hashCode();
+
+		// This function ensures that hashCodes that differ only by
+		// constant multiples at each bit position have a bounded
+		// number of collisions (approximately 8 at default load factor).
+		h ^= (h >>> 20) ^ (h >>> 12);
+		return h ^ (h >>> 7) ^ (h >>> 4);
+	}
+
+	/**
+	 * 
+	 * @param h
+	 * @param length
+	 * @return
+	 */
+	private static int indexFor(int h, int length) {
+		return h & (length - 1);
+	}
+
+	/**
+	 *
+	 */
+	public class DArrayIListEntry {
+		public DblArray key;
+		public IntArrayList value;
+		public DArrayIListEntry next;
+
+		public DArrayIListEntry(DblArray ekey, IntArrayList evalue) {
+			key = ekey;
+			value = evalue;
+			next = null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/DoubleIntListHashMap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/DoubleIntListHashMap.java b/src/main/java/org/apache/sysml/runtime/compress/utils/DoubleIntListHashMap.java
new file mode 100644
index 0000000..5607a3f
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/DoubleIntListHashMap.java
@@ -0,0 +1,181 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import java.util.ArrayList;
+
+/**
+ * This class provides a memory-efficient replacement for
+ * HashMap<Double,IntArrayList> for restricted use cases.
+ * 
+ */
+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;
+
+	private DIListEntry[] _data = null;
+	private int _size = -1;
+
+	public DoubleIntListHashMap() {
+		_data = new DIListEntry[INIT_CAPACITY];
+		_size = 0;
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public int size() {
+		return _size;
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @return
+	 */
+	public IntArrayList get(double key) {
+		// probe for early abort
+		if( _size == 0 )
+			return null;
+
+		// compute entry index position
+		int hash = hash(key);
+		int ix = indexFor(hash, _data.length);
+
+		// find entry
+		for( DIListEntry e = _data[ix]; e != null; e = e.next ) {
+			if( e.key == key ) {
+				return e.value;
+			}
+		}
+
+		return null;
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @param value
+	 */
+	public void appendValue(double key, IntArrayList value) {
+		// compute entry index position
+		int hash = hash(key);
+		int ix = indexFor(hash, _data.length);
+
+		// add new table entry (constant time)
+		DIListEntry enew = new DIListEntry(key, value);
+		enew.next = _data[ix]; // colliding entries / null
+		_data[ix] = enew;
+		_size++;
+
+		// resize if necessary
+		if( _size >= LOAD_FACTOR * _data.length )
+			resize();
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public ArrayList<DIListEntry> extractValues() {
+		ArrayList<DIListEntry> ret = new ArrayList<DIListEntry>();
+		for( DIListEntry e : _data ) {
+			if (e != null) {
+				while( e.next != null ) {
+					ret.add(e);
+					e = e.next;
+				}
+				ret.add(e);
+			}
+		}
+
+		return ret;
+	}
+
+	/**
+     * 
+     */
+	private void resize() {
+		// check for integer overflow on resize
+		if( _data.length > Integer.MAX_VALUE / RESIZE_FACTOR )
+			return;
+
+		// resize data array and copy existing contents
+		DIListEntry[] olddata = _data;
+		_data = new DIListEntry[_data.length * RESIZE_FACTOR];
+		_size = 0;
+
+		// rehash all entries
+		for( DIListEntry e : olddata ) {
+			if( e != null ) {
+				while( e.next != null ) {
+					appendValue(e.key, e.value);
+					e = e.next;
+				}
+				appendValue(e.key, e.value);
+			}
+		}
+	}
+
+	/**
+	 * 
+	 * @param key
+	 * @return
+	 */
+	private static int hash(double key) {
+		// basic double hash code (w/o object creation)
+		long bits = Double.doubleToRawLongBits(key);
+		int h = (int) (bits ^ (bits >>> 32));
+
+		// This function ensures that hashCodes that differ only by
+		// constant multiples at each bit position have a bounded
+		// number of collisions (approximately 8 at default load factor).
+		h ^= (h >>> 20) ^ (h >>> 12);
+		return h ^ (h >>> 7) ^ (h >>> 4);
+	}
+
+	/**
+	 * 
+	 * @param h
+	 * @param length
+	 * @return
+	 */
+	private static int indexFor(int h, int length) {
+		return h & (length - 1);
+	}
+
+	/**
+	 *
+	 */
+	public class DIListEntry {
+		public double key = Double.MAX_VALUE;
+		public IntArrayList value = null;
+		public DIListEntry next = null;
+
+		public DIListEntry(double ekey, IntArrayList evalue) {
+			key = ekey;
+			value = evalue;
+			next = null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java b/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
new file mode 100644
index 0000000..33455a2
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/IntArrayList.java
@@ -0,0 +1,102 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import java.util.Arrays;
+
+/**
+ * This class provides a memory-efficient replacement for ArrayList<Integer> for
+ * restricted use cases.
+ * 
+ */
+public class IntArrayList 
+{
+	private static final int INIT_CAPACITY = 4;
+	private static final int RESIZE_FACTOR = 2;
+
+	private int[] _data = null;
+	private int _size = -1;
+	private int _val0 = -1;
+
+	public IntArrayList() {
+		_data = null;
+		_size = 0;
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public int size() {
+		return _size;
+	}
+
+	/**
+	 * 
+	 * @param value
+	 */
+	public void appendValue(int value) {
+		// embedded value (no array allocation)
+		if( _size == 0 ) {
+			_val0 = value;
+			_size = 1;
+			return;
+		}
+
+		// allocate or resize array if necessary
+		if( _data == null ) {
+			_data = new int[INIT_CAPACITY];
+			_data[0] = _val0;
+		} 
+		else if( _size + 1 >= _data.length ) {
+			resize();
+		}
+
+		// append value
+		_data[_size] = value;
+		_size++;
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public int[] extractValues() {
+		if( _size == 1 )
+			return new int[] { _val0 };
+		else
+			return Arrays.copyOfRange(_data, 0, _size);
+	}
+
+	/**
+	 * 
+	 */
+	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);
+
+		// resize data array and copy existing contents
+		int[] newdata = new int[_data.length * RESIZE_FACTOR];
+		System.arraycopy(_data, 0, newdata, 0, _size);
+		_data = newdata;
+	}
+}


[6/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Posted by mb...@apache.org.
[SYSTEMML-810] New compressed matrix blocks and operations, tests

For details, see https://issues.apache.org/jira/browse/SYSTEMML-449.

Project: http://git-wip-us.apache.org/repos/asf/incubator-systemml/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-systemml/commit/16e7b1c8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-systemml/tree/16e7b1c8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-systemml/diff/16e7b1c8

Branch: refs/heads/master
Commit: 16e7b1c88e45e007d1db229717311fcf70bc6b19
Parents: 71013e7
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Sat Jul 16 00:43:16 2016 -0700
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Sat Jul 16 17:22:50 2016 -0700

----------------------------------------------------------------------
 .../runtime/compress/BitmapDecoderOLE.java      |  127 ++
 .../runtime/compress/BitmapDecoderRLE.java      |  117 ++
 .../sysml/runtime/compress/BitmapEncoder.java   |  392 +++++
 .../apache/sysml/runtime/compress/ColGroup.java |  259 ++++
 .../sysml/runtime/compress/ColGroupBitmap.java  |  498 +++++++
 .../sysml/runtime/compress/ColGroupOLE.java     |  634 +++++++++
 .../sysml/runtime/compress/ColGroupRLE.java     |  617 ++++++++
 .../runtime/compress/ColGroupUncompressed.java  |  360 +++++
 .../runtime/compress/CompressedMatrixBlock.java | 1342 ++++++++++++++++++
 .../runtime/compress/PlanningBinPacker.java     |  191 +++
 .../sysml/runtime/compress/PlanningCoCoder.java |  227 +++
 .../runtime/compress/PlanningCoCodingGroup.java |  104 ++
 .../compress/PlanningGroupMergeAction.java      |   73 +
 .../runtime/compress/ReaderColumnSelection.java |   64 +
 .../compress/ReaderColumnSelectionDense.java    |   68 +
 .../ReaderColumnSelectionDenseSample.java       |   86 ++
 .../compress/ReaderColumnSelectionSparse.java   |  115 ++
 .../runtime/compress/UncompressedBitmap.java    |  101 ++
 .../compress/estim/CompressedSizeEstimator.java |  145 ++
 .../estim/CompressedSizeEstimatorExact.java     |   53 +
 .../estim/CompressedSizeEstimatorSample.java    |  767 ++++++++++
 .../compress/estim/CompressedSizeInfo.java      |   69 +
 .../compress/estim/SizeEstimatorFactory.java    |   40 +
 .../runtime/compress/utils/ConverterUtils.java  |   99 ++
 .../sysml/runtime/compress/utils/DblArray.java  |   91 ++
 .../compress/utils/DblArrayIntListHashMap.java  |  179 +++
 .../compress/utils/DoubleIntListHashMap.java    |  181 +++
 .../runtime/compress/utils/IntArrayList.java    |  102 ++
 .../compress/utils/LinearAlgebraUtils.java      |  383 +++++
 .../runtime/functionobjects/KahanFunction.java  |    8 +
 .../runtime/functionobjects/KahanPlus.java      |    5 +
 .../runtime/functionobjects/KahanPlusSq.java    |   15 +-
 .../runtime/matrix/data/LibMatrixMult.java      |   27 +-
 .../sysml/runtime/matrix/data/MatrixBlock.java  |    2 +-
 .../compress/BasicCompressionTest.java          |  168 +++
 .../compress/BasicMatrixAppendTest.java         |  176 +++
 .../compress/BasicMatrixMultChainTest.java      |  245 ++++
 .../BasicMatrixTransposeSelfMultTest.java       |  172 +++
 .../compress/BasicMatrixVectorMultTest.java     |  180 +++
 .../BasicScalarOperationsSparseUnsafeTest.java  |  177 +++
 .../compress/BasicScalarOperationsTest.java     |  177 +++
 .../BasicTransposeSelfLeftMatrixMultTest.java   |  172 +++
 .../compress/BasicUnaryAggregateTest.java       |  544 +++++++
 .../compress/BasicVectorMatrixMultTest.java     |  180 +++
 .../functions/compress/CompressedLinregCG.java  |  151 ++
 .../compress/CompressedSerializationTest.java   |  185 +++
 .../compress/LargeCompressionTest.java          |  169 +++
 .../compress/LargeMatrixVectorMultTest.java     |  180 +++
 .../compress/LargeParMatrixVectorMultTest.java  |  182 +++
 .../compress/LargeVectorMatrixMultTest.java     |  180 +++
 .../compress/ParMatrixMultChainTest.java        |  247 ++++
 .../compress/ParMatrixVectorMultTest.java       |  182 +++
 .../ParTransposeSelfLeftMatrixMultTest.java     |  174 +++
 .../compress/ParUnaryAggregateTest.java         |  547 +++++++
 .../compress/ParVectorMatrixMultTest.java       |  181 +++
 src/test/scripts/functions/compress/LinregCG.R  |   57 +
 .../scripts/functions/compress/LinregCG.dml     |   56 +
 .../compress/SystemML-config-compress.xml       |   59 +
 .../functions/compress/ZPackageSuite.java       |   56 +
 59 files changed, 12322 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderOLE.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderOLE.java b/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderOLE.java
new file mode 100644
index 0000000..fc6e861
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderOLE.java
@@ -0,0 +1,127 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * General-purpose iterator to decode a compressed OLE bitmap.
+ *  
+ */
+public final class BitmapDecoderOLE implements Iterator<Integer> 
+{
+	// pointer to the compressed bitmap
+	private int _bmOff;
+	private int _bmLen;
+	private char[] _bmPtr;
+
+	// The index of the current block. Block 0 covers bits 1 through 2^16
+	private int _blockIx;
+
+	// The offset where the current block starts within the bitmap.
+	private int _blockStartOffset;
+
+	// The number of offsets in the current block.
+	private int _curBlockSize;
+
+	// The offset <b>in the current block</b> the <b>next</b> element we will
+	// read from the bitmap, or bmPtr.length if we are done.
+	private int _nextBmOffset;
+
+	/**
+	 * Point this object at the beginning of a particular bitmap. After a call
+	 * to this method, the next call to {@link #nextOffset()} will return the
+	 * offset of the first bit in the specified bitmap.
+	 * 
+	 * @param bmPtr
+	 *            pointer to a compressed bitmap
+	 */
+	public BitmapDecoderOLE(char[] bmPtr, int off, int len) {
+		_bmOff = off;
+		_bmLen = len;
+		_bmPtr = bmPtr;
+		_blockIx = 0;
+		_blockStartOffset = 0;
+		_curBlockSize = _bmPtr[_bmOff+_blockStartOffset];
+		if (_curBlockSize < 0) {
+			throw new RuntimeException(String.format(
+					"Negative block size %d at position %d of %s",
+					_curBlockSize, _blockStartOffset, Arrays.toString(bmPtr)));
+		}
+		_nextBmOffset = 0;
+
+		// Advance past any zero-length blocks at the beginning of the array
+		while (_blockStartOffset < _bmLen
+				&& _nextBmOffset >= _curBlockSize) {
+			advanceToNextBlock();
+		}
+	}
+
+	@Override
+	public Integer next() {
+		if( !hasNext() )
+			throw new RuntimeException("No next offset existing.");
+		
+		// Grab the lookahead value Note the +1 in the array indexing; 
+		// the first number in a block is the block size
+		int offsetFromBlockBegin = _bmPtr[_bmOff+_blockStartOffset + 1 + _nextBmOffset];
+		int ret = (_blockIx * BitmapEncoder.BITMAP_BLOCK_SZ)
+				+ offsetFromBlockBegin;
+		_nextBmOffset++;
+
+		// Advance to next non-empty block if we reached the end of the block.
+		while (_blockStartOffset < _bmLen && _nextBmOffset >= _curBlockSize) {
+			advanceToNextBlock();
+		}
+
+		return ret;
+	}
+
+	@Override
+	public boolean hasNext() {
+		return _blockStartOffset < _bmLen;
+	}
+
+	@Override
+	public void remove() {
+		throw new RuntimeException("Not implemented for BitmapDecoderOLE.");
+	}
+
+	/**
+	 * Move forward to the next block. Does not skip empty blocks.
+	 */
+	private void advanceToNextBlock() {
+		_blockStartOffset += (1 + _curBlockSize);
+		_blockIx++;
+		if (_blockStartOffset >= _bmLen) {
+			// Read past last block
+			return;
+		}
+
+		_curBlockSize = _bmPtr[_bmOff+_blockStartOffset];
+		if (_curBlockSize < 0) {
+			throw new RuntimeException(String.format(
+					"Negative block size %d at position %d of %s",
+					_curBlockSize, _blockStartOffset, Arrays.toString(_bmPtr)));
+		}
+		_nextBmOffset = 0;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderRLE.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderRLE.java b/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderRLE.java
new file mode 100644
index 0000000..54f24ae
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/BitmapDecoderRLE.java
@@ -0,0 +1,117 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Iterator;
+
+/**
+ * General-purpose iterator to decode a compressed OLE bitmap.
+ * 
+ */
+public final class BitmapDecoderRLE implements Iterator<Integer>
+{
+	// pointer to the compressed bitmap
+	private int _bmOff;
+	private int _bmLen;
+	private char[] _bmPtr;
+
+	// The offset of the <b>next</b> element we will read from the bitmap, or
+	// bmPtr.length if we are done.
+	private int _nextBmOffset;
+
+	// The offset in the matrix column of the beginning of the current run
+	private int _runStartOffset;
+
+	// The length of the current run
+	private int _curRunLen;
+
+	// The number of bits that we have returned from the current run.
+	private int _runBitsReturned;
+
+	/**
+	 * Point this object at the beginning of a particular bitmap. After a call
+	 * to this method, the next call to {@link #nextOffset()} will return the
+	 * offset of the first bit in the specified bitmap.
+	 * 
+	 * @param bmPtr
+	 *            pointer to a compressed bitmap
+	 */
+	public BitmapDecoderRLE(char[] bmPtr, int off, int len) {
+		_bmOff = off;
+		_bmLen = len;
+		_bmPtr = bmPtr;
+		_nextBmOffset = 0;
+		_runStartOffset = 0;
+		_curRunLen = 0;
+		_runBitsReturned = 0;
+
+		if (0 == _bmLen) {
+			return; //no runs
+		}
+
+		// Advance to the beginning of the first non-empty run.
+		advanceToNextRun();
+	}
+
+	@Override
+	public Integer next() {
+		if( !hasNext() )
+			throw new RuntimeException("No next offset existing.");
+		
+		// Grab the lookahead value
+		int ret = _runStartOffset + _runBitsReturned;
+
+		_runBitsReturned++;
+
+		// Check for end of run
+		if (_runBitsReturned == _curRunLen) {
+			advanceToNextRun();
+		}
+
+		return ret;
+	}
+
+	@Override
+	public boolean hasNext() {
+		return _runBitsReturned < _curRunLen;
+	}
+	
+	@Override
+	public void remove() {
+		throw new RuntimeException("Not implemented for BitmapDecoderRLE.");
+	}
+	
+	/** Move forward to the next non-empty run. */
+	private void advanceToNextRun() {
+		// While loop needed because some runs are of length 0
+		while (_runBitsReturned == _curRunLen && _nextBmOffset < _bmLen) {
+
+			_runBitsReturned = 0;
+
+			// Read the distance to the next run
+			char delta = _bmPtr[_bmOff + _nextBmOffset];
+
+			// Run length is stored in the next element of the array
+			_runStartOffset += delta + _curRunLen;
+			_curRunLen = _bmPtr[_bmOff + _nextBmOffset + 1];
+			_nextBmOffset += 2;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/BitmapEncoder.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/BitmapEncoder.java b/src/main/java/org/apache/sysml/runtime/compress/BitmapEncoder.java
new file mode 100644
index 0000000..8a535e1
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/BitmapEncoder.java
@@ -0,0 +1,392 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.compress.utils.DblArray;
+import org.apache.sysml.runtime.compress.utils.DblArrayIntListHashMap;
+import org.apache.sysml.runtime.compress.utils.DoubleIntListHashMap;
+import org.apache.sysml.runtime.compress.utils.IntArrayList;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.data.SparseBlock;
+
+
+/** 
+ * Static functions for encoding bitmaps in various ways. 
+ * 
+ */
+public class BitmapEncoder 
+{
+	/** Size of the blocks used in a blocked bitmap representation. */
+	public static final int BITMAP_BLOCK_SZ = 65536;
+	
+	/**
+	 * 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
+	 * @return uncompressed bitmap representation of the columns
+	 * @throws DMLRuntimeException 
+	 */
+	public static UncompressedBitmap extractBitmap(int[] colIndices, MatrixBlock rawblock) 
+	{
+		//note: no sparse column selection reader because low potential
+		//single column selection
+		if( colIndices.length==1 ) {
+			return extractBitmap(colIndices[0], rawblock, 
+					!CompressedMatrixBlock.MATERIALIZE_ZEROS);
+		}
+		//multiple column selection	(general case)
+		else { 
+			ReaderColumnSelection reader = null;
+			if( rawblock.isInSparseFormat() && CompressedMatrixBlock.TRANSPOSE_INPUT )
+				reader = new ReaderColumnSelectionSparse(rawblock, colIndices, 
+						!CompressedMatrixBlock.MATERIALIZE_ZEROS);
+			else
+				reader = new ReaderColumnSelectionDense(rawblock, colIndices,
+						!CompressedMatrixBlock.MATERIALIZE_ZEROS); 
+			
+			return extractBitmap(colIndices, rawblock, reader);
+		}
+	}
+
+	/**
+	 * 
+	 * @param colIndices
+	 * @param rawblock
+	 * @param sampleIndexes
+	 * @return
+	 */
+	public static UncompressedBitmap extractBitmapFromSample(int[] colIndices,
+			MatrixBlock rawblock, int[] sampleIndexes) 
+	{
+		//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);
+		}
+		//multiple column selection	(general case)
+		else {			
+			return extractBitmap(colIndices, rawblock,
+					new ReaderColumnSelectionDenseSample(rawblock, colIndices,
+							sampleIndexes, !CompressedMatrixBlock.MATERIALIZE_ZEROS));	
+		}
+	}
+
+	/**
+	 * Encodes the bitmap as a series of run lengths and offsets.
+	 * <p>
+	 * <b>NOTE: This method must be kept in sync with {@link BitmapDecoderRLE}
+	 * !</b>
+	 * 
+	 * @param offsets
+	 *            uncompressed contents of the bitmap, expressed as a list of
+	 *            the offsets of different bits
+	 * @return compressed version of said bitmap
+	 */
+	public static char[] genRLEBitmap(int[] offsets) {
+		if( offsets.length == 0 )
+			return new char[0]; //empty list
+
+		// Use an ArrayList for correctness at the expense of temp space
+		ArrayList<Character> buf = new ArrayList<Character>();
+
+		// 1 + (position of last 1 in the previous run of 1's)
+		// We add 1 because runs may be of length zero.
+		int lastRunEnd = 0;
+
+		// Offset between the end of the previous run of 1's and the first 1 in
+		// the current run. Initialized below.
+		int curRunOff;
+
+		// Length of the most recent run of 1's
+		int curRunLen = 0;
+
+		// Current encoding is as follows:
+		// Negative entry: abs(Entry) encodes the offset to the next lone 1 bit.
+		// Positive entry: Entry encodes offset to next run of 1's. The next
+		// entry in the bitmap holds a run length.
+
+		// Special-case the first run to simplify the loop below.
+		int firstOff = offsets[0];
+
+		// The first run may start more than a short's worth of bits in
+		while (firstOff > Character.MAX_VALUE) {
+			buf.add(Character.MAX_VALUE);
+			buf.add((char) 0);
+			firstOff -= Character.MAX_VALUE;
+			lastRunEnd += Character.MAX_VALUE;
+		}
+
+		// Create the first run with an initial size of 1
+		curRunOff = firstOff;
+		curRunLen = 1;
+
+		// Process the remaining offsets
+		for (int i = 1; i < offsets.length; i++) {
+
+			int absOffset = offsets[i];
+
+			// 1 + (last position in run)
+			int curRunEnd = lastRunEnd + curRunOff + curRunLen;
+
+			if (absOffset > curRunEnd || curRunLen >= Character.MAX_VALUE) {
+				// End of a run, either because we hit a run of 0's or because the 
+				// number of 1's won't fit in 16 bits. Add run to bitmap and start a new one.
+				buf.add((char) curRunOff);
+				buf.add((char) curRunLen);
+
+				lastRunEnd = curRunEnd;
+				curRunOff = absOffset - lastRunEnd;
+
+				while (curRunOff > Character.MAX_VALUE) {
+					// SPECIAL CASE: Offset to next run doesn't fit into 16 bits.
+					// Add zero-length runs until the offset is small enough.
+					buf.add(Character.MAX_VALUE);
+					buf.add((char) 0);
+					lastRunEnd += Character.MAX_VALUE;
+					curRunOff -= Character.MAX_VALUE;
+				}
+				
+				curRunLen = 1;
+			} else {
+				// Middle of a run
+				curRunLen++;
+			}
+		}
+
+		// Close out the last run
+		if (curRunLen >= 1) {
+			buf.add((char) curRunOff);
+			buf.add((char) curRunLen);
+		}
+
+		// Convert wasteful ArrayList to packed array.
+		char[] ret = new char[buf.size()];
+		for (int i = 0; i < buf.size(); i++) {
+			ret[i] = buf.get(i);
+		}
+		return ret;
+	}
+
+	/**
+	 * Encodes the bitmap in blocks of offsets. Within each block, the bits are
+	 * stored as absolute offsets from the start of the block.
+	 * 
+	 * @param offsets
+	 *            uncompressed contents of the bitmap, expressed as a list of
+	 *            the offsets of different bits
+	 * @return compressed version of said bitmap
+	 */
+	public static char[] genOffsetBitmap(int[] offsets) 
+	{
+		int lastOffset = offsets[offsets.length - 1];
+
+		// Build up the blocks
+		int numBlocks = (lastOffset / BITMAP_BLOCK_SZ) + 1;
+		// To simplify the logic, we make two passes.
+		// The first pass divides the offsets by block.
+		int[] blockLengths = new int[numBlocks];
+		Arrays.fill(blockLengths, 0);
+
+		for (int ix = 0; ix < offsets.length; ix++) {
+			int val = offsets[ix];
+			int blockForVal = val / BITMAP_BLOCK_SZ;
+
+			blockLengths[blockForVal]++;
+		}
+
+		// The second pass creates the blocks.
+		int totalSize = numBlocks;
+		for (int block = 0; block < numBlocks; block++) {
+			totalSize += blockLengths[block];
+		}
+		char[] encodedBlocks = new char[totalSize];
+
+		int inputIx = 0;
+		int blockStartIx = 0;
+		for (int block = 0; block < numBlocks; block++) {
+			int blockSz = blockLengths[block];
+
+			// First entry in the block is number of bits
+			encodedBlocks[blockStartIx] = (char) blockSz;
+
+			for (int i = 0; i < blockSz; i++) {
+				encodedBlocks[blockStartIx + i + 1] = (char) 
+						(offsets[inputIx+i] % BITMAP_BLOCK_SZ);
+			}
+
+			inputIx += blockSz;
+			blockStartIx += blockSz + 1;
+		}
+
+		return encodedBlocks;
+	}
+
+	/**
+	 * 
+	 * @param colIndex
+	 * @param rawblock
+	 * @return
+	 * @throws DMLRuntimeException 
+	 */
+	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawblock, boolean skipZeros) 
+	{
+		//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();
+		
+		if( rawblock.isInSparseFormat() //SPARSE 
+			&& CompressedMatrixBlock.TRANSPOSE_INPUT )	
+		{
+			SparseBlock a = rawblock.getSparseBlock();
+			if( a != null && !a.isEmpty(colIndex) ) 
+			{
+				int apos = a.pos(colIndex);
+				int alen = a.size(colIndex);
+				int[] aix = a.indexes(colIndex);
+				double[] avals = a.values(colIndex);
+				
+				IntArrayList lstPtr0 = new IntArrayList(); //for 0 values
+				int last = -1;
+				//iterate over non-zero entries but fill in zeros
+				for( int j=apos; j<apos+alen; j++ ) 
+				{
+					//fill in zero values
+					if( !skipZeros )
+						for( int k=last+1; k<aix[j]; k++ ) 
+							lstPtr0.appendValue(k);
+					//handle non-zero value
+					IntArrayList lstPtr = distinctVals.get(avals[j]);	
+					if( lstPtr == null ) {
+						lstPtr = new IntArrayList();
+						distinctVals.appendValue(avals[j], lstPtr);
+					}
+					lstPtr.appendValue(aix[j]);
+					last = aix[j];
+				}
+				//fill in remaining zero values
+				if( !skipZeros ) {
+					for( int k=last+1; k<m; k++ )
+						lstPtr0.appendValue(k);
+					if( lstPtr0.size()>0 )
+						distinctVals.appendValue(0, lstPtr0);
+				}
+			}
+			else if( !skipZeros ) { //full 0 column 
+				IntArrayList lstPtr = new IntArrayList();
+				for( int i=0; i<m; i++ )
+					lstPtr.appendValue(i);
+				distinctVals.appendValue(0, lstPtr);
+			}
+		}
+		else //GENERAL CASE
+		{
+			for( int i=0; i<m; i++ ) {
+				double val = CompressedMatrixBlock.TRANSPOSE_INPUT ? 
+						rawblock.quickGetValue(colIndex, i):
+						rawblock.quickGetValue(i, colIndex);
+				if( val!=0 || !skipZeros ) {		
+					IntArrayList lstPtr = distinctVals.get(val);	
+					if( lstPtr == null ) {
+						lstPtr = new IntArrayList();
+						distinctVals.appendValue(val, lstPtr);
+					}
+					lstPtr.appendValue(i);
+				}
+			}
+		}
+		
+		return new UncompressedBitmap(distinctVals);
+	}
+	
+	/**
+	 * 
+	 * @param colIndex
+	 * @param rawblock
+	 * @param sampleIndexes
+	 * @return
+	 */
+	private static UncompressedBitmap extractBitmap(int colIndex, MatrixBlock rawblock, int[] sampleIndexes, boolean skipZeros) 
+	{
+		//note: general case only because anyway binary search for small samples
+		
+		//probe map for distinct items (for value or value groups)
+		DoubleIntListHashMap distinctVals = new DoubleIntListHashMap();
+		
+		//scan rows and probe/build distinct items
+		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.quickGetValue(rowIndex, colIndex); 
+			if( val!=0 || !skipZeros ) {					
+				IntArrayList lstPtr = distinctVals.get(val);	
+				if( lstPtr == null ) {
+					lstPtr = new IntArrayList();
+					distinctVals.appendValue(val, lstPtr);
+				}
+				lstPtr.appendValue(i);
+			}
+		}
+
+		return new UncompressedBitmap(distinctVals);
+	}
+	
+	/**
+	 * 
+	 * @param colIndices
+	 * @param rawblock
+	 * @param rowReader
+	 * @return
+	 */
+	private static UncompressedBitmap extractBitmap(int[] colIndices,
+			MatrixBlock rawblock, ReaderColumnSelection rowReader) 
+	{
+		//probe map for distinct items (for value or value groups)
+		DblArrayIntListHashMap distinctVals = new DblArrayIntListHashMap();
+		
+		//scan rows and probe/build distinct items
+		DblArray cellVals = null;
+		while ((cellVals = rowReader.nextRow()) != null) {
+			IntArrayList lstPtr = distinctVals.get(cellVals);
+			if (lstPtr == null) {
+				//create new objects only on demand
+				lstPtr = new IntArrayList();
+				distinctVals.appendValue(new DblArray(cellVals), lstPtr);
+			}
+			lstPtr.appendValue(rowReader.getCurrentRowIndex());
+		}
+		
+		return new UncompressedBitmap(distinctVals, colIndices.length);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ColGroup.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ColGroup.java b/src/main/java/org/apache/sysml/runtime/compress/ColGroup.java
new file mode 100644
index 0000000..647a701
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ColGroup.java
@@ -0,0 +1,259 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.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 
+{
+	private static final long serialVersionUID = 2439785418908671481L;
+
+	public enum CompressionType  {
+		UNCOMPRESSED,   //uncompressed sparse/dense 
+		RLE_BITMAP,     //RLE bitmap
+		OLE_BITMAP;  //OLE bitmap
+	}
+	
+	/**
+	 * Offsets of the columns that make up the column group. Zero-based, and
+	 * relative to the matrix block.
+	 */
+	protected int[] _colIndexes;
+
+	/** Number of rows in the matrix, for use by child classes. */
+	protected int _numRows;
+
+	/** How the elements of the column group are compressed. */
+	private CompressionType _compType;
+
+	
+	/**
+	 * 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
+	 */
+	protected ColGroup(CompressionType type, int[] colIndices, int numRows) {
+		_compType = type;
+		_colIndexes = colIndices;
+		_numRows = numRows;
+	}
+
+	/** Convenience constructor for converting indices to a more compact format. */
+	protected ColGroup(CompressionType type, List<Integer> colIndicesList, int numRows) {
+		_compType = type;
+		_colIndexes = new int[colIndicesList.size()];
+		int i = 0;
+		for (Integer index : colIndicesList)
+			_colIndexes[i++] = index;
+	}
+
+	/**
+	 * @return offsets of the columns in the matrix block that make up the group
+	 */
+	public int[] getColIndices() {
+		return _colIndexes;
+	}
+
+	/**
+	 * @param col
+	 *            an index from 0 to the number of columns in this group - 1
+	 * @return offset of the specified column in the matrix block that make up
+	 *         the group
+	 */
+	public int getColIndex(int colNum) {
+		return _colIndexes[colNum];
+	}
+
+	public int getNumRows() {
+		return _numRows;
+	}
+	
+	/**
+	 * @return number of columns in this column group
+	 */
+	public int getNumCols() {
+		return _colIndexes.length;
+	}
+
+	/**
+	 * @return How the elements of the column group are compressed.
+	 */
+	public CompressionType getCompType() {
+		return _compType;
+	}
+
+	/**
+	 * 
+	 * @param offset
+	 */
+	public void shiftColIndices(int offset)  {
+		for( int i=0; i<_colIndexes.length; i++ )
+			_colIndexes[i] += offset;
+	}
+	
+	/**
+	 * Note: Must be overridden by child classes to account for additional data
+	 * and metadata
+	 * 
+	 * @return an upper bound on the number of bytes used to store this ColGroup
+	 *         in memory.
+	 */
+	public long estimateInMemorySize() {
+		// int numRows (4B) , array reference colIndices (8B) + array object
+		// overhead if exists (32B) + 4B per element, CompressionType compType
+		// (2 booleans 2B + enum overhead 32B + reference to enum 8B)
+		long size = 54;
+		if (_colIndexes == null)
+			return size;
+		else
+			return size + 32 + 4 * _colIndexes.length;
+	}
+
+	/**
+	 * Decompress the contents of this column group into the specified full
+	 * matrix block.
+	 * 
+	 * @param target
+	 *            a matrix block where the columns covered by this column group
+	 *            have not yet been filled in.
+	 */
+	public abstract void decompressToBlock(MatrixBlock target);
+
+	/**
+	 * Decompress the contents of this column group into uncompressed packed
+	 * columns
+	 * 
+	 * @param target
+	 *            a dense matrix block. The block must have enough space to hold
+	 *            the contents of this column group.
+	 * @param colIndexTargets
+	 *            array that maps column indices in the original matrix block to
+	 *            columns of target.
+	 */
+	public abstract void decompressToBlock(MatrixBlock target, int[] colIndexTargets);
+
+	/**
+	 * 
+	 * @param target  dense output vector
+	 * @param colpos  column to decompress, error if larger or equal numCols
+	 */
+	public abstract void decompressToBlock(MatrixBlock target, int colpos);
+
+
+	/**
+	 * Serializes column group to data output.
+	 * 
+	 * @param out
+	 * @throws IOException
+	 */
+	public abstract void write(DataOutput out) 
+		throws IOException;
+	
+	/**
+	 * Deserializes column group from data input.
+	 * 
+	 * @param in
+	 * @throws IOException
+	 */
+	public abstract void readFields(DataInput in) 
+		throws IOException;
+		
+	
+	/**
+	 * Returns the exact serialized size of column group.
+	 * This can be used for example for buffer preallocation.
+	 * 
+	 * @return
+	 */
+	public abstract long getExactSizeOnDisk();
+	
+	/**
+	 * 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)
+	 * @param result
+	 *            accumulator for holding the result
+	 * @throws DMLRuntimeException
+	 *             if the internal SystemML code that performs the
+	 *             multiplication experiences an error
+	 */
+	public abstract void rightMultByVector(MatrixBlock vector,
+			MatrixBlock result, int rl, int ru) throws DMLRuntimeException;
+
+
+	/**
+	 * Multiply the slice of the matrix that this column group represents by a
+	 * row vector on the left (the original column vector is assumed to be
+	 * transposed already i.e. its size now is 1xn).
+	 * 
+	 * @param vector
+	 * @param result
+	 * @throws DMLRuntimeException 
+	 */
+	public abstract void leftMultByRowVector(MatrixBlock vector,
+			MatrixBlock result) throws DMLRuntimeException;
+
+	/**
+	 * Perform the specified scalar operation directly on the compressed column
+	 * group, without decompressing individual cells if possible.
+	 * 
+	 * @param op
+	 *            operation to perform
+	 * @return version of this column group with the operation applied
+	 */
+	public abstract ColGroup scalarOperation(ScalarOperator op)
+			throws DMLRuntimeException;
+
+	/**
+	 * 
+	 * @param op
+	 * @param result
+	 * @throws DMLUnsupportedOperationException
+	 * @throws DMLRuntimeException
+	 */
+	public abstract void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result)
+		throws DMLRuntimeException;
+	
+	/**
+	 * 
+	 * @param rnnz
+	 */
+	protected abstract void countNonZerosPerRow(int[] rnnz);
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ColGroupBitmap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ColGroupBitmap.java b/src/main/java/org/apache/sysml/runtime/compress/ColGroupBitmap.java
new file mode 100644
index 0000000..5d60a49
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ColGroupBitmap.java
@@ -0,0 +1,498 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.compress.utils.LinearAlgebraUtils;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+
+
+/**
+ * Base class for column groups encoded with various types of bitmap encoding.
+ * 
+ * 
+ * NOTES:
+ *  * OLE: separate storage segment length and bitmaps led to a 30% improvement
+ *    but not applied because more difficult to support both data layouts at the
+ *    same time (distributed/local as well as w/ and w/o low-level opt)
+ */
+public abstract class ColGroupBitmap extends ColGroup 
+{
+	private static final long serialVersionUID = -1635828933479403125L;
+	
+	public static final 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)
+	private static final boolean SORT_VALUES_BY_LENGTH = true; 
+	protected static final boolean CREATE_SKIPLIST = true;
+	
+	protected static final int READ_CACHE_BLKSZ = 2 * BitmapEncoder.BITMAP_BLOCK_SZ;
+	protected static final int WRITE_CACHE_BLKSZ = 2 * BitmapEncoder.BITMAP_BLOCK_SZ;
+	
+	/** Distinct values associated with individual bitmaps. */
+	protected double[] _values; //linearized <numcol vals> <numcol vals>
+
+	/** Bitmaps, one per uncompressed value in {@link #values}. */
+	protected int[] _ptr; //bitmap offsets per value
+	protected char[] _data; //linearized bitmaps (variable length)
+	
+	protected int[] _skiplist;
+	
+	public ColGroupBitmap(CompressionType type) {
+		super(type, (int[]) null, -1);
+	}
+	
+	/**
+	 * Main constructor. Stores the headers for the individual bitmaps.
+	 * 
+	 * @param colIndices
+	 *            indices (within the block) of the columns included in this
+	 *            column
+	 * @param numRows
+	 *            total number of rows in the parent block
+	 * @param ubm
+	 *            Uncompressed bitmap representation of the block
+	 */
+	public ColGroupBitmap(CompressionType type, int[] colIndices, int numRows, UncompressedBitmap ubm) 
+	{
+		super(type, colIndices, numRows);
+
+		// Extract and store just the distinct values. The bitmaps themselves go
+		// into the subclasses.
+		final int numCols = ubm.getNumColumns();
+		final int numVals = ubm.getNumValues();
+		
+		_values = new double[numVals*numCols];
+		for (int i=0; i<numVals; i++) {
+			//note: deep copied internally on getValues
+			double[] tmp = ubm.getValues(i);
+			System.arraycopy(tmp, 0, _values, i*numCols, numCols);
+		}
+	}
+
+	/**
+	 * Constructor for subclass methods that need to create shallow copies
+	 * 
+	 * @param colIndices
+	 *            raw column index information
+	 * @param numRows
+	 *            number of rows in the block
+	 * @param values
+	 *            set of distinct values for the block (associated bitmaps are
+	 *            kept in the subclass)
+	 */
+	protected ColGroupBitmap(CompressionType type, int[] colIndices, int numRows, double[] values) {
+		super(type, colIndices, numRows);
+		_values = values;
+	}
+	
+	protected final int len(int k) {
+		return _ptr[k+1] - _ptr[k];
+	}
+
+	/**
+	 * 
+	 * @param numVals
+	 * @param totalLen
+	 * @param lbitmaps
+	 */
+	protected void createCompressedBitmaps(int numVals, int totalLen, char[][] lbitmaps)
+	{
+		// compact bitmaps to linearized representation
+		if( LOW_LEVEL_OPT && SORT_VALUES_BY_LENGTH
+			&& _numRows > BitmapEncoder.BITMAP_BLOCK_SZ ) 
+		{
+			// sort value by num segments in descending order
+			TreeMap<Integer,ArrayList<Integer>> tree = new TreeMap<Integer, ArrayList<Integer>>();
+			for( int i=0; i<numVals; i++ ) {
+				int revlen = totalLen-lbitmaps[i].length;
+				if( !tree.containsKey(revlen) )
+					tree.put(revlen, new ArrayList<Integer>());
+				tree.get(revlen).add(i);
+			}
+			
+			// compact bitmaps to linearized representation
+			_ptr = new int[numVals+1];
+			_data = new char[totalLen];
+			int pos = 0, off = 0;
+			for( Entry<Integer,ArrayList<Integer>> e : tree.entrySet() ) {
+				for( Integer tmpix : e.getValue() ) {
+					int len = lbitmaps[tmpix].length;
+					_ptr[pos] = off;
+					System.arraycopy(lbitmaps[tmpix], 0, _data, off, len);
+					off += len;
+					pos++;
+				}
+			}
+			_ptr[numVals] = totalLen;
+			
+			// reorder values
+			double[] lvalues = new double[_values.length];
+			int off2 = 0; int numCols = _colIndexes.length;
+			for( Entry<Integer,ArrayList<Integer>> e : tree.entrySet() ) {
+				for( Integer tmpix : e.getValue() ) {
+					System.arraycopy(_values, tmpix*numCols, lvalues, off2, numCols);				
+					off2 += numCols;
+				}
+			}			
+			_values = lvalues;
+		}
+		else
+		{
+			// compact bitmaps to linearized representation
+			_ptr = new int[numVals+1];
+			_data = new char[totalLen];
+			for( int i=0, off=0; i<numVals; i++ ) {
+				int len = lbitmaps[i].length;
+				_ptr[i] = off;
+				System.arraycopy(lbitmaps[i], 0, _data, off, len);
+				off += len;
+			}
+			_ptr[numVals] = totalLen;
+		}
+	}
+	
+	@Override
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		
+		// adding the size of values
+		size += 8; //array reference
+		if (_values != null) {
+			size += 32 + _values.length * 8; //values
+		}
+		
+		// adding bitmaps size
+		size += 16; //array references
+		if (_data != null) {
+			size += 32 + _ptr.length * 4; // offsets
+			size += 32 + _data.length * 2;    // bitmaps
+		}
+	
+		return size;
+	}
+
+	//generic decompression for OLE/RLE, to be overwritten for performance
+	@Override
+	public void decompressToBlock(MatrixBlock target) 
+	{
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		int[] colIndices = getColIndices();
+		
+		// Run through the bitmaps for this column group
+		for (int i = 0; i < numVals; i++) {
+			Iterator<Integer> decoder = getDecodeIterator(i);
+			int valOff = i*numCols;
+
+			while (decoder.hasNext()) {
+				int row = decoder.next();
+				for (int colIx = 0; colIx < numCols; colIx++) {
+					target.appendValue(row, colIndices[colIx], _values[valOff+colIx]);
+				}
+			}
+		}
+	}
+
+	//generic decompression for OLE/RLE, to be overwritten for performance
+	@Override
+	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) 
+	{
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		// Run through the bitmaps for this column group
+		for (int i = 0; i < numVals; i++) {
+			Iterator<Integer> decoder = getDecodeIterator(i);
+			int valOff = i*numCols;
+
+			while (decoder.hasNext()) {
+				int row = decoder.next();
+				for (int colIx = 0; colIx < numCols; colIx++) {
+					int origMatrixColIx = getColIndex(colIx);
+					int targetColIx = colIndexTargets[origMatrixColIx];
+					target.quickSetValue(row, targetColIx, _values[valOff+colIx]);
+				}
+			}
+		}
+	}
+	
+	//generic decompression for OLE/RLE, to be overwritten for performance
+	@Override
+	public void decompressToBlock(MatrixBlock target, int colpos) 
+	{
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		// Run through the bitmaps for this column group
+		for (int i = 0; i < numVals; i++) {
+			Iterator<Integer> decoder = getDecodeIterator(i);
+			int valOff = i*numCols;
+
+			while (decoder.hasNext()) {
+				int row = decoder.next();
+				target.quickSetValue(row, 0, _values[valOff+colpos]);
+			}
+		}
+	}
+	
+	/**
+	 * 
+	 * @param bitmapIx
+	 * @return
+	 */
+	protected double sumValues(int bitmapIx)
+	{
+		final int numCols = getNumCols();
+		final int valOff = bitmapIx * numCols;
+		
+		double val = 0.0;
+		for( int i = 0; i < numCols; i++ ) {
+			val += _values[valOff+i];
+		}
+		
+		return val;
+	}
+	
+	protected double sumValues(int bitmapIx, double[] b)
+	{
+		final int numCols = getNumCols();
+		final int valOff = bitmapIx * numCols;
+		
+		double val = 0;
+		for( int i = 0; i < numCols; i++ ) {
+			val += _values[valOff+i] * b[i];
+		}
+		
+		return val;
+	}
+	
+	/**
+	 * 
+	 * @param b
+	 * @param c
+	 */
+	protected void sumAllValues(double[] b, double[] c)
+	{
+		final int numVals = getNumValues();
+		final int numCols = getNumCols();
+		
+		//vectMultiplyAdd over cols instead of dotProduct over vals because
+		//usually more values than columns
+		for( int i=0, off=0; i<numCols; i++, off+=numVals )
+			LinearAlgebraUtils.vectMultiplyAdd(b[i], _values, c, off, 0, numVals);
+	}
+
+	/**
+	 * Method for use by subclasses. Applies a scalar operation to the value
+	 * metadata stored in the superclass.
+	 * 
+	 * @param op
+	 *            scalar operation to perform
+	 * @return transformed copy of value metadata for this column group
+	 * @throws DMLRuntimeException
+	 */
+	protected double[] applyScalarOp(ScalarOperator op)
+			throws DMLRuntimeException 
+	{
+		//scan over linearized values
+		double[] ret = new double[_values.length];
+		for (int i = 0; i < _values.length; i++) {
+			ret[i] = op.executeScalar(_values[i]);
+		}
+
+		return ret;
+	}
+	
+	/**
+	 * 
+	 * @param op
+	 * @param newVal
+	 * @param numCols
+	 * @return
+	 * @throws DMLRuntimeException
+	 */
+	protected double[] applyScalarOp(ScalarOperator op, double newVal, int numCols)
+			throws DMLRuntimeException 
+	{
+		//scan over linearized values
+		double[] ret = new double[_values.length + numCols];
+		for( int i = 0; i < _values.length; i++ ) {
+			ret[i] = op.executeScalar(_values[i]);
+		}
+		
+		//add new value to the end
+		Arrays.fill(ret, _values.length, _values.length+numCols, newVal);
+		
+		return ret;
+	}
+
+	/**
+	 * @return the number of distinct sets of values associated with the bitmaps
+	 *         in this column group
+	 */
+	public int getNumValues() {
+		return _values.length / _colIndexes.length;
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	public double[] getValues() {
+		return _values;
+	}
+	
+	/**
+	 * 
+	 * @return
+	 */
+	public char[] getBitmaps() {
+		return _data;
+	}
+	
+	public int[] getBitmapOffsets() {
+		return _ptr;
+	}
+
+	/**
+	 * @param bmpIx
+	 *            index of a specific compressed bitmap (stored in subclass,
+	 *            index same as {@link #values})
+	 * @return an object for iterating over the row offsets in this bitmap. Only
+	 *         valid until the next call to this method. May be reused across
+	 *         calls.
+	 */
+	public abstract Iterator<Integer> getDecodeIterator(int bmpIx);
+
+	/**
+	 * Utility function of sparse-unsafe operations.
+	 * 
+	 * @param ind
+	 * @return
+	 * @throws DMLRuntimeException
+	 */
+	protected int[] computeOffsets(boolean[] ind)
+		throws DMLRuntimeException 
+	{
+		//determine number of offsets
+		int numOffsets = 0;
+		for( int i=0; i<ind.length; i++ )
+			numOffsets += ind[i] ? 1 : 0;
+		
+		//create offset lists
+		int[] ret = new int[numOffsets];
+		for( int i=0, pos=0; i<ind.length; i++ )
+			if( ind[i] )
+				ret[pos++] = i;
+		
+		return ret;
+	}
+
+	@Override
+	public void readFields(DataInput in) 
+		throws IOException 
+	{
+		_numRows = in.readInt();
+		int numCols = in.readInt();
+		int numVals = in.readInt();
+		
+		//read col indices
+		_colIndexes = new int[ numCols ];
+		for( int i=0; i<numCols; i++ )
+			_colIndexes[i] = in.readInt();
+		
+		//read distinct values
+		_values = new double[numVals*numCols];
+		for( int i=0; i<numVals*numCols; i++ )
+			_values[i] = in.readDouble();
+		
+		//read bitmaps
+		int totalLen = in.readInt();
+		_ptr = new int[numVals+1];
+		_data = new char[totalLen];		
+		for( int i=0, off=0; i<numVals; i++ ) {
+			int len = in.readInt();
+			_ptr[i] = off;
+			for( int j=0; j<len; j++ )
+				_data[off+j] = in.readChar();
+			off += len;
+		}
+		_ptr[numVals] = totalLen;
+	}
+	
+	@Override
+	public void write(DataOutput out) 
+		throws IOException 
+	{
+		int numCols = getNumCols();
+		int numVals = getNumValues();
+		out.writeInt(_numRows);
+		out.writeInt(numCols);
+		out.writeInt(numVals);
+		
+		//write col indices
+		for( int i=0; i<_colIndexes.length; i++ )
+			out.writeInt( _colIndexes[i] );
+		
+		//write distinct values
+		for( int i=0; i<_values.length; i++ )
+			out.writeDouble(_values[i]);
+
+		//write bitmaps (lens and data, offset later recreated)
+		int totalLen = 0;
+		for( int i=0; i<numVals; i++ )
+			totalLen += len(i);
+		out.writeInt(totalLen);	
+		for( int i=0; i<numVals; i++ ) {
+			int len = len(i);
+			int off = _ptr[i];
+			out.writeInt(len);
+			for( int j=0; j<len; j++ )
+				out.writeChar(_data[off+j]);
+		}
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		long ret = 12; //header
+		//col indices
+		ret += 4 * _colIndexes.length; 
+		//distinct values (groups of values)
+		ret += 8 * _values.length;
+		//actual bitmaps
+		ret += 4; //total length
+		for( int i=0; i<getNumValues(); i++ )
+			ret += 4 + 2 * len(i);
+		
+		return ret;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ColGroupOLE.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ColGroupOLE.java b/src/main/java/org/apache/sysml/runtime/compress/ColGroupOLE.java
new file mode 100644
index 0000000..a2b493a
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ColGroupOLE.java
@@ -0,0 +1,634 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.compress.utils.ConverterUtils;
+import org.apache.sysml.runtime.compress.utils.LinearAlgebraUtils;
+import org.apache.sysml.runtime.functionobjects.KahanFunction;
+import org.apache.sysml.runtime.functionobjects.KahanPlus;
+import org.apache.sysml.runtime.functionobjects.KahanPlusSq;
+import org.apache.sysml.runtime.functionobjects.ReduceAll;
+import org.apache.sysml.runtime.functionobjects.ReduceCol;
+import org.apache.sysml.runtime.functionobjects.ReduceRow;
+import org.apache.sysml.runtime.instructions.cp.KahanObject;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.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 ColGroupBitmap 
+{
+	private static final long serialVersionUID = -9157676271360528008L;
+
+	public ColGroupOLE() {
+		super(CompressionType.OLE_BITMAP);
+	}
+	
+	/**
+	 * Main constructor. Constructs and stores the necessary bitmaps.
+	 * 
+	 * @param colIndices
+	 *            indices (within the block) of the columns included in this
+	 *            column
+	 * @param numRows
+	 *            total number of rows in the parent block
+	 * @param ubm
+	 *            Uncompressed bitmap representation of the block
+	 */
+	public ColGroupOLE(int[] colIndices, int numRows, UncompressedBitmap ubm) 
+	{
+		super(CompressionType.OLE_BITMAP, colIndices, numRows, ubm);
+
+		// compress the bitmaps
+		final int numVals = ubm.getNumValues();
+		char[][] lbitmaps = new char[numVals][];
+		int totalLen = 0;
+		for( int i=0; i<numVals; i++ ) {
+			lbitmaps[i] = BitmapEncoder.genOffsetBitmap(ubm.getOffsetsList(i));
+			totalLen += lbitmaps[i].length;
+		}
+		
+		// compact bitmaps to linearized representation
+		createCompressedBitmaps(numVals, totalLen, lbitmaps);
+		
+
+		if( LOW_LEVEL_OPT && CREATE_SKIPLIST
+				&& numRows > 2*BitmapEncoder.BITMAP_BLOCK_SZ )
+		{
+			int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+			_skiplist = new int[numVals];
+			int rl = (getNumRows()/2/blksz)*blksz;
+			for (int k = 0; k < numVals; k++) {
+				int bitmapOff = _ptr[k];
+				int bitmapLen = len(k);
+				int bufIx = 0;
+				for( int i=0; i<rl && bufIx<bitmapLen; i+=blksz ) {
+					bufIx += _data[bitmapOff+bufIx] + 1;
+				}
+				_skiplist[k] = bufIx;
+			}		
+		}
+	}
+
+	/**
+	 * Constructor for internal use.
+	 */
+	public ColGroupOLE(int[] colIndices, int numRows, double[] values, char[] bitmaps, int[] bitmapOffs) {
+		super(CompressionType.OLE_BITMAP, colIndices, numRows, values);
+		_data = bitmaps;
+		_ptr = bitmapOffs;
+	}
+
+	@Override
+	public Iterator<Integer> getDecodeIterator(int bmpIx) {
+		return new BitmapDecoderOLE(_data, _ptr[bmpIx], len(bmpIx));
+	}
+	
+	@Override
+	public void decompressToBlock(MatrixBlock target) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			
+			//cache blocking config and position array
+			int[] apos = new int[numVals];
+					
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				for (int k = 0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);					
+					int bufIx = apos[k];
+					if( bufIx >= bitmapLen ) 
+						continue;
+					int len = _data[bitmapOff+bufIx];
+					int pos = bitmapOff+bufIx+1;
+					for( int i=pos; i<pos+len; i++ )
+						for( int j=0, rix = bi+_data[i]; j<numCols; j++ )
+							if( _values[off+j]!=0 )
+								target.appendValue(rix, _colIndexes[j], _values[off+j]);
+					apos[k] += len + 1;
+				}
+			}		
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target);
+		}
+	}
+
+	@Override
+	public void decompressToBlock(MatrixBlock target, int[] colixTargets) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			
+			//cache blocking config and position array
+			int[] apos = new int[numVals];					
+			int[] cix = new int[numCols];
+			
+			//prepare target col indexes
+			for( int j=0; j<numCols; j++ )
+				cix[j] = colixTargets[_colIndexes[j]];
+			
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				for (int k = 0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);					
+					int bufIx = apos[k];
+					if( bufIx >= bitmapLen ) 
+						continue;
+					int len = _data[bitmapOff+bufIx];
+					int pos = bitmapOff+bufIx+1;
+					for( int i=pos; i<pos+len; i++ )
+						for( int j=0, rix = bi+_data[i]; j<numCols; j++ )
+							if( _values[off+j]!=0 )
+								target.appendValue(rix, cix[j], _values[off+j]);
+					apos[k] += len + 1;
+				}
+			}		
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target, colixTargets);
+		}
+	}
+	
+	@Override
+	public void decompressToBlock(MatrixBlock target, int colpos) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			double[] c = target.getDenseBlock();
+			
+			//cache blocking config and position array
+			int[] apos = new int[numVals];					
+			
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				for (int k = 0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);					
+					int bufIx = apos[k];
+					if( bufIx >= bitmapLen ) 
+						continue;
+					int len = _data[bitmapOff+bufIx];
+					int pos = bitmapOff+bufIx+1;
+					for( int i=pos; i<pos+len; i++ ) {
+						c[bi+_data[i]] = _values[off+colpos];
+					}
+					apos[k] += len + 1;
+				}
+			}	
+			
+			target.recomputeNonZeros();
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target, colpos);
+		}
+	}
+	
+	@Override
+	public ColGroup scalarOperation(ScalarOperator op)
+		throws DMLRuntimeException 
+	{
+		double val0 = op.executeScalar(0);
+		
+		//fast path: sparse-safe operations
+		// Note that bitmaps don't change and are shallow-copied
+		if( op.sparseSafe || val0==0 ) {
+			return new ColGroupOLE(_colIndexes, _numRows, 
+					applyScalarOp(op), _data, _ptr);
+		}
+		
+		//slow path: sparse-unsafe operations (potentially create new bitmap)
+		//note: for efficiency, we currently don't drop values that become 0
+		boolean[] lind = computeZeroIndicatorVector();
+		int[] loff = computeOffsets(lind);
+		if( loff.length==0 ) { //empty offset list: go back to fast path
+			return new ColGroupOLE(_colIndexes, _numRows, 
+					applyScalarOp(op), _data, _ptr);
+		}
+		
+		double[] rvalues = applyScalarOp(op, val0, getNumCols());		
+		char[] lbitmap = BitmapEncoder.genOffsetBitmap(loff);
+		char[] rbitmaps = Arrays.copyOf(_data, _data.length+lbitmap.length);
+		System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
+		int[] rbitmapOffs = Arrays.copyOf(_ptr, _ptr.length+1);
+		rbitmapOffs[rbitmapOffs.length-1] = rbitmaps.length; 
+		
+		return new ColGroupOLE(_colIndexes, _numRows, 
+				rvalues, rbitmaps, rbitmapOffs);
+	}
+
+	@Override
+	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru)
+			throws DMLRuntimeException 
+	{
+		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] c = result.getDenseBlock();
+		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		//prepare reduced rhs w/ relevant values
+		double[] sb = new double[numCols];
+		for (int j = 0; j < numCols; j++) {
+			sb[j] = b[_colIndexes[j]];
+		}
+		
+		if( LOW_LEVEL_OPT && 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)
+			//x=4 leads to a good yet slightly conservative compromise for single-/
+			//multi-threaded and typical number of cores and L3 cache sizes
+			final int blksz2 = ColGroupBitmap.WRITE_CACHE_BLKSZ;
+			
+			//step 1: prepare position and value arrays
+			
+			//current pos / values per OLs
+			int[] apos = new int[numVals];
+			double[] aval = new double[numVals];
+			
+			//skip-scan to beginning for all OLs 
+			if( rl > 0 ) { //rl aligned with blksz		
+				int rskip = (getNumRows()/2/blksz)*blksz;
+				
+				for (int k = 0; k < numVals; k++) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					int start = (rl>=rskip)?rskip:0;
+					int bufIx = (rl>=rskip)?_skiplist[k]:0;
+					for( int i=start; i<rl && bufIx<bitmapLen; i+=blksz ) {
+						bufIx += _data[bitmapOff+bufIx] + 1;
+					}
+					apos[k] = bufIx;
+				}
+			}
+			
+			//pre-aggregate values per OLs
+			for( int k = 0; k < numVals; k++ )
+				aval[k] = sumValues(k, sb);
+					
+			//step 2: cache conscious matrix-vector via horizontal scans 
+			for( int bi=rl; bi<ru; bi+=blksz2 ) 
+			{
+				int bimax = Math.min(bi+blksz2, ru);
+				
+				//horizontal segment scan, incl pos maintenance
+				for (int k = 0; k < numVals; k++) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					double val = aval[k];
+					int bufIx = apos[k];
+					
+					for( int ii=bi; ii<bimax && bufIx<bitmapLen; ii+=blksz ) {
+						//prepare length, start, and end pos
+						int len = _data[bitmapOff+bufIx];
+						int pos = bitmapOff+bufIx+1;
+						
+						//compute partial results
+						//LinearAlgebraUtils.vectAdd(val, c, bitmaps, pos, ii, len);
+						for( int i=pos; i<pos+len; i++ )
+							c[ii + _data[i]] += val;	
+						bufIx += len + 1;
+					}
+
+					apos[k] = bufIx;
+				}
+			}		
+		}
+		else
+		{	
+			//iterate over all values and their bitmaps
+			for (int k = 0; k < numVals; k++) 
+			{
+				//prepare value-to-add for entire value bitmap
+				int bitmapOff = _ptr[k];
+				int bitmapLen = len(k);
+				double val = sumValues(k, sb);
+				
+				//iterate over bitmap blocks and add values
+				if (val != 0) {
+					int offsetBase = 0;
+					int bufIx = 0;
+					int curBlckLen;
+					
+					//scan to beginning offset if necessary 
+					if( rl > 0 ){
+						for (; bufIx<bitmapLen & offsetBase<rl; bufIx += curBlckLen + 1, offsetBase += blksz) {
+							curBlckLen = _data[bitmapOff+bufIx];
+						}	
+					}
+					
+					//compute partial results
+					for (; bufIx<bitmapLen & offsetBase<ru; bufIx += curBlckLen + 1, offsetBase += blksz) {
+						curBlckLen = _data[bitmapOff+bufIx];
+						for (int blckIx = 1; blckIx <= curBlckLen; blckIx++) {
+							c[offsetBase + _data[bitmapOff+bufIx + blckIx]] += val;
+						}
+					}
+				}
+			}
+		}
+	}
+
+	@Override
+	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result)
+		throws DMLRuntimeException 
+	{
+		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] c = result.getDenseBlock();
+		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 )
+		{
+			//cache blocking config (see matrix-vector mult for explanation)
+			final int blksz2 = ColGroupBitmap.READ_CACHE_BLKSZ;
+			
+			//step 1: prepare position and value arrays
+			
+			//current pos per OLs / output values
+			int[] apos = new int[numVals];
+			double[] cvals = new double[numVals];
+			
+			//step 2: cache conscious matrix-vector via horizontal scans 
+			for( int ai=0; ai<n; ai+=blksz2 ) 
+			{
+				int aimax = Math.min(ai+blksz2, n);
+				
+				//horizontal segment scan, incl pos maintenance
+				for (int k = 0; k < numVals; k++) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					int bufIx = apos[k];
+					double vsum = 0;	
+					
+					for( int ii=ai; ii<aimax && bufIx<bitmapLen; ii+=blksz ) {
+						//prepare length, start, and end pos
+						int len = _data[bitmapOff+bufIx];
+						int pos = bitmapOff+bufIx+1;
+						
+						//iterate over bitmap blocks and compute partial results (a[i]*1)
+						vsum += LinearAlgebraUtils.vectSum(a, _data, ii, pos, len);
+						bufIx += len + 1;
+					}
+
+					apos[k] = bufIx;
+					cvals[k] += vsum;
+				}
+			}
+			
+			//step 3: scale partial results by values and write to global output
+			for (int k = 0, valOff=0; k < numVals; k++, valOff+=numCols)
+				for( int j = 0; j < numCols; j++ )
+					c[ _colIndexes[j] ] += cvals[k] * _values[valOff+j];		
+		}
+		else
+		{
+			//iterate over all values and their bitmaps
+			for (int k=0, valOff=0; k<numVals; k++, valOff+=numCols) 
+			{
+				int bitmapOff = _ptr[k];
+				int bitmapLen = len(k);
+				
+				//iterate over bitmap blocks and add partial results
+				double vsum = 0;
+				for (int bix=0, off=0; bix < bitmapLen; bix+=_data[bitmapOff+bix]+1, off+=blksz)
+					vsum += LinearAlgebraUtils.vectSum(a, _data, off, bitmapOff+bix+1, _data[bitmapOff+bix]);
+				
+				//scale partial results by values and write results
+				for( int j = 0; j < numCols; j++ )
+					c[ _colIndexes[j] ] += vsum * _values[ valOff+j ];
+			}
+		}
+	}
+
+	@Override
+	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result) 
+		throws DMLRuntimeException 
+	{
+		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);
+		else if( op.indexFn instanceof ReduceRow )
+			computeColSums(result, kplus);
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeSum(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(result.quickGetValue(0, 0), result.quickGetValue(0, 1));
+		
+		//iterate over all values and their bitmaps
+		final int numVals = getNumValues();
+		final int numCols = getNumCols();
+		
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) 
+		{
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			int valOff = bitmapIx * numCols;
+			
+			//iterate over bitmap blocks and count partial lengths
+			int count = 0;
+			for (int bix=0; bix < bitmapLen; bix+=_data[bitmapOff+bix]+1)
+				count += _data[bitmapOff+bix];
+			
+			//scale counts by all values
+			for( int j = 0; j < numCols; j++ )
+				kplus.execute3(kbuff, _values[ valOff+j ], count);
+		}
+		
+		result.quickSetValue(0, 0, kbuff._sum);
+		result.quickSetValue(0, 1, kbuff._correction);
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeRowSums(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(0, 0);
+	
+		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+		final int numVals = getNumValues();
+		
+		//iterate over all values and their bitmaps
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) 
+		{
+			//prepare value-to-add for entire value bitmap
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			double val = sumValues(bitmapIx);
+			
+			//iterate over bitmap blocks and add values
+			if (val != 0) {
+				int offsetBase = 0;
+				int bufIx = 0;
+				int curBlckLen;
+				for (; bufIx < bitmapLen; bufIx += curBlckLen + 1, offsetBase += blksz) {
+					curBlckLen = _data[bitmapOff+bufIx];
+					for (int blckIx = 1; blckIx <= curBlckLen; blckIx++) {
+						int rix = offsetBase + _data[bitmapOff+bufIx + blckIx];
+						kbuff.set(result.quickGetValue(rix, 0), result.quickGetValue(rix, 1));
+						kplus.execute2(kbuff, val);
+						result.quickSetValue(rix, 0, kbuff._sum);
+						result.quickSetValue(rix, 1, kbuff._correction);
+					}
+				}
+			}
+		}
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeColSums(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(0, 0);
+		
+		//iterate over all values and their bitmaps
+		final int numVals = getNumValues();
+		final int numCols = getNumCols();
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) 
+		{
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			int valOff = bitmapIx * numCols;
+			
+			//iterate over bitmap blocks and count partial lengths
+			int count = 0;
+			for (int bix=0; bix < bitmapLen; bix+=_data[bitmapOff+bix]+1)
+				count += _data[bitmapOff+bix];
+			
+			//scale counts by all values
+			for( int j = 0; j < numCols; j++ ) {
+				kbuff.set(result.quickGetValue(0, _colIndexes[j]),result.quickGetValue(1, _colIndexes[j]));
+				kplus.execute3(kbuff, _values[ valOff+j ], count);
+				result.quickSetValue(0, _colIndexes[j], kbuff._sum);
+				result.quickSetValue(1, _colIndexes[j], kbuff._correction);
+			}
+		}
+	}
+	
+	/**
+	 * Utility function of sparse-unsafe operations.
+	 * 
+	 * @return
+	 * @throws DMLRuntimeException
+	 */
+	private boolean[] computeZeroIndicatorVector()
+		throws DMLRuntimeException 
+	{
+		boolean[] ret = new boolean[_numRows];
+		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+		final int numVals = getNumValues();
+		
+		//initialize everything with zero
+		Arrays.fill(ret, true);
+		
+		//iterate over all values and their bitmaps
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) 
+		{
+			//prepare value-to-add for entire value bitmap
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			
+			//iterate over bitmap blocks and add values
+			int offsetBase = 0;
+			int bufIx = 0;
+			int curBlckLen;
+			for (; bufIx < bitmapLen; bufIx += curBlckLen + 1, offsetBase += blksz) {
+				curBlckLen = _data[bitmapOff+bufIx];
+				for (int blckIx = 1; blckIx <= curBlckLen; blckIx++) {
+					ret[offsetBase + _data[bitmapOff+bufIx + blckIx]] &= false;
+				}
+			}
+		}
+		
+		return ret;
+	}
+	
+	@Override
+	protected void countNonZerosPerRow(int[] rnnz)
+	{
+		final int blksz = BitmapEncoder.BITMAP_BLOCK_SZ;
+		final int numVals = getNumValues();
+		final int numCols = getNumCols();
+		
+		//iterate over all values and their bitmaps
+		for (int k = 0; k < numVals; k++) 
+		{
+			//prepare value-to-add for entire value bitmap
+			int bitmapOff = _ptr[k];
+			int bitmapLen = len(k);
+			
+			//iterate over bitmap blocks and add values
+			int offsetBase = 0;
+			int curBlckLen;
+			for (int bufIx=0; bufIx<bitmapLen; bufIx+=curBlckLen+1, offsetBase+=blksz) {
+				curBlckLen = _data[bitmapOff+bufIx];
+				for (int blckIx = 1; blckIx <= curBlckLen; blckIx++) {
+					rnnz[offsetBase + _data[bitmapOff+bufIx + blckIx]] += numCols;
+				}
+			}
+		}
+	}
+}


[5/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ColGroupRLE.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ColGroupRLE.java b/src/main/java/org/apache/sysml/runtime/compress/ColGroupRLE.java
new file mode 100644
index 0000000..39bc162
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ColGroupRLE.java
@@ -0,0 +1,617 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.compress.utils.ConverterUtils;
+import org.apache.sysml.runtime.compress.utils.LinearAlgebraUtils;
+import org.apache.sysml.runtime.functionobjects.KahanFunction;
+import org.apache.sysml.runtime.functionobjects.KahanPlus;
+import org.apache.sysml.runtime.functionobjects.KahanPlusSq;
+import org.apache.sysml.runtime.functionobjects.ReduceAll;
+import org.apache.sysml.runtime.functionobjects.ReduceCol;
+import org.apache.sysml.runtime.functionobjects.ReduceRow;
+import org.apache.sysml.runtime.instructions.cp.KahanObject;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+
+
+/** A group of columns compressed with a single run-length encoded bitmap. */
+public class ColGroupRLE extends ColGroupBitmap 
+{
+	private static final long serialVersionUID = 7450232907594748177L;
+
+	public ColGroupRLE() {
+		super(CompressionType.RLE_BITMAP);
+	}
+	
+	/**
+	 * Main constructor. Constructs and stores the necessary bitmaps.
+	 * 
+	 * @param colIndices
+	 *            indices (within the block) of the columns included in this
+	 *            column
+	 * @param numRows
+	 *            total number of rows in the parent block
+	 * @param ubm
+	 *            Uncompressed bitmap representation of the block
+	 */
+	public ColGroupRLE(int[] colIndices, int numRows, UncompressedBitmap ubm) 
+	{
+		super(CompressionType.RLE_BITMAP, colIndices, numRows, ubm);
+		
+		// compress the bitmaps
+		final int numVals = ubm.getNumValues();
+		char[][] lbitmaps = new char[numVals][];
+		int totalLen = 0;
+		for( int i=0; i<numVals; i++ ) {
+			lbitmaps[i] = BitmapEncoder.genRLEBitmap(ubm.getOffsetsList(i));
+			totalLen += lbitmaps[i].length;
+		}
+		
+		// compact bitmaps to linearized representation
+		createCompressedBitmaps(numVals, totalLen, lbitmaps);
+	}
+
+	/**
+	 * Constructor for internal use.
+	 */
+	public ColGroupRLE(int[] colIndices, int numRows, double[] values, char[] bitmaps, int[] bitmapOffs) {
+		super(CompressionType.RLE_BITMAP, colIndices, numRows, values);
+		_data = bitmaps;
+		_ptr = bitmapOffs;
+	}
+
+	@Override
+	public Iterator<Integer> getDecodeIterator(int bmpIx) {
+		return new BitmapDecoderRLE(_data, _ptr[bmpIx], len(bmpIx)); 
+	}
+	
+	@Override
+	public void decompressToBlock(MatrixBlock target) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = 128 * 1024;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			
+			//position and start offset arrays
+			int[] apos = new int[numVals];
+			int[] astart = new int[numVals];
+			
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				int bimax = Math.min(bi+blksz, n);					
+				for (int k=0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					int bufIx = apos[k];
+					int start = astart[k];
+					for( ; bufIx<bitmapLen & start<bimax; bufIx+=2) {
+						start += _data[bitmapOff + bufIx];
+						int len = _data[bitmapOff + bufIx+1];
+						for( int i=start; i<start+len; i++ )
+							for( int j=0; j<numCols; j++ )
+								if( _values[off+j]!=0 )
+									target.appendValue(i, _colIndexes[j], _values[off+j]);
+						start += len;
+					}
+					apos[k] = bufIx;	
+					astart[k] = start;
+				}
+			}
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target);
+		}
+	}
+
+	@Override
+	public void decompressToBlock(MatrixBlock target, int[] colixTargets) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = 128 * 1024;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			
+			//position and start offset arrays
+			int[] apos = new int[numVals];
+			int[] astart = new int[numVals];
+			int[] cix = new int[numCols];
+			
+			//prepare target col indexes
+			for( int j=0; j<numCols; j++ )
+				cix[j] = colixTargets[_colIndexes[j]];
+			
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				int bimax = Math.min(bi+blksz, n);					
+				for (int k=0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					int bufIx = apos[k];
+					if( bufIx >= bitmapLen ) 
+						continue;
+					int start = astart[k];
+					for( ; bufIx<bitmapLen & start<bimax; bufIx+=2) {
+						start += _data[bitmapOff + bufIx];
+						int len = _data[bitmapOff + bufIx+1];
+						for( int i=start; i<start+len; i++ )
+							for( int j=0; j<numCols; j++ )
+								if( _values[off+j]!=0 )
+									target.appendValue(i, cix[j], _values[off+j]);
+						start += len;
+					}
+					apos[k] = bufIx;	
+					astart[k] = start;
+				}
+			}
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target, colixTargets);
+		}
+	}
+
+	@Override
+	public void decompressToBlock(MatrixBlock target, int colpos) 
+	{
+		if( LOW_LEVEL_OPT && getNumValues() > 1 )
+		{
+			final int blksz = 128 * 1024;
+			final int numCols = getNumCols();
+			final int numVals = getNumValues();
+			final int n = getNumRows();
+			double[] c = target.getDenseBlock();
+			
+			//position and start offset arrays
+			int[] apos = new int[numVals];
+			int[] astart = new int[numVals];
+			
+			//cache conscious append via horizontal scans 
+			for( int bi=0; bi<n; bi+=blksz ) {
+				int bimax = Math.min(bi+blksz, n);					
+				for (int k=0, off=0; k < numVals; k++, off+=numCols) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);
+					int bufIx = apos[k];
+					if( bufIx >= bitmapLen ) 
+						continue;
+					int start = astart[k];
+					for( ; bufIx<bitmapLen & start<bimax; bufIx+=2) {
+						start += _data[bitmapOff + bufIx];
+						int len = _data[bitmapOff + bufIx+1];
+						for( int i=start; i<start+len; i++ )
+							c[i] = _values[off+colpos];
+						start += len;
+					}
+					apos[k] = bufIx;	
+					astart[k] = start;
+				}
+			}
+			
+			target.recomputeNonZeros();
+		}
+		else
+		{
+			//call generic decompression with decoder
+			super.decompressToBlock(target, colpos);
+		}
+	}
+	
+	@Override
+	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru)
+			throws DMLRuntimeException 
+	{
+		double[] b = ConverterUtils.getDenseVector(vector);
+		double[] c = result.getDenseBlock();
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		//prepare reduced rhs w/ relevant values
+		double[] sb = new double[numCols];
+		for (int j = 0; j < numCols; j++) {
+			sb[j] = b[_colIndexes[j]];
+		}
+		
+		if( LOW_LEVEL_OPT && 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
+			final int blksz = ColGroupBitmap.WRITE_CACHE_BLKSZ; 
+			
+			//step 1: prepare position and value arrays
+			
+			//current pos / values per RLE list
+			int[] apos = new int[numVals];
+			int[] astart = new int[numVals];
+			double[] aval = new double[numVals];
+			
+			//skip-scan to beginning for all OLs 
+			if( rl > 0 ) { //rl aligned with blksz	
+				for (int k = 0; k < numVals; k++) {
+					int boff = _ptr[k];
+					int blen = len(k);
+					int bix = 0;
+					int start = 0;
+					while( bix<blen ) {	
+						int lstart = _data[boff + bix]; //start
+						int llen = _data[boff + bix + 1]; //len
+						if( start+lstart+llen >= rl )
+							break;
+						start += lstart + llen;
+						bix += 2;
+					}
+					apos[k] = bix;
+					astart[k] = start;
+				}
+			}
+			
+			//pre-aggregate values per OLs
+			for( int k = 0; k < numVals; k++ )
+				aval[k] = sumValues(k, sb);
+					
+			//step 2: cache conscious matrix-vector via horizontal scans 
+			for( int bi=rl; bi<ru; bi+=blksz ) 
+			{
+				int bimax = Math.min(bi+blksz, ru);
+					
+				//horizontal segment scan, incl pos maintenance
+				for (int k = 0; k < numVals; k++) {
+					int boff = _ptr[k];
+					int blen = len(k);
+					double val = aval[k];
+					int bix = apos[k];
+					int start = astart[k];
+					
+					//compute partial results, not aligned
+					while( bix<blen ) {
+						int lstart = _data[boff + bix];
+						int llen = _data[boff + bix + 1];
+						LinearAlgebraUtils.vectAdd(val, c, Math.max(bi, start+lstart), 
+								Math.min(start+lstart+llen,bimax) - Math.max(bi, start+lstart));
+						if(start+lstart+llen >= bimax)
+							break;
+						start += lstart + llen;
+						bix += 2;
+					}
+					
+					apos[k] = bix;	
+					astart[k] = start;
+				}
+			}
+		}
+		else
+		{
+			for (int k = 0; k < numVals; k++) {
+				int boff = _ptr[k];
+				int blen = len(k);
+				double val = sumValues(k, sb);
+				int bix = 0;
+				int start = 0;
+				
+				//scan to beginning offset if necessary 
+				if( rl > 0 ) { //rl aligned with blksz	
+					while( bix<blen ) {	
+						int lstart = _data[boff + bix]; //start
+						int llen = _data[boff + bix + 1]; //len
+						if( start+lstart+llen >= rl )
+							break;
+						start += lstart + llen;
+						bix += 2;
+					}
+				}
+				
+				//compute partial results, not aligned
+				while( bix<blen ) {
+					int lstart = _data[boff + bix];
+					int llen = _data[boff + bix + 1];
+					LinearAlgebraUtils.vectAdd(val, c, Math.max(rl, start+lstart), 
+							Math.min(start+lstart+llen,ru) - Math.max(rl, start+lstart));
+					if(start+lstart+llen >= ru)
+						break;
+					start += lstart + llen;
+					bix += 2;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result)
+			throws DMLRuntimeException 
+	{		
+		double[] a = ConverterUtils.getDenseVector(vector);
+		double[] c = result.getDenseBlock();
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		final int n = getNumRows();
+		
+		if( LOW_LEVEL_OPT && numVals > 1 
+			&& _numRows > BitmapEncoder.BITMAP_BLOCK_SZ ) 
+		{
+			final int blksz = ColGroupBitmap.READ_CACHE_BLKSZ; 
+			
+			//step 1: prepare position and value arrays
+			
+			//current pos per OLs / output values
+			int[] apos = new int[numVals];
+			int[] astart = new int[numVals];
+			double[] cvals = new double[numVals];
+			
+			//step 2: cache conscious matrix-vector via horizontal scans 
+			for( int ai=0; ai<n; ai+=blksz ) 
+			{
+				int aimax = Math.min(ai+blksz, n);
+				
+				//horizontal scan, incl pos maintenance
+				for (int k = 0; k < numVals; k++) {
+					int bitmapOff = _ptr[k];
+					int bitmapLen = len(k);						
+					int bufIx = apos[k];
+					int start = astart[k];
+					
+					//compute partial results, not aligned
+					while( bufIx<bitmapLen & start<aimax ) {
+						start += _data[bitmapOff + bufIx];
+						int len = _data[bitmapOff + bufIx+1];
+						cvals[k] += LinearAlgebraUtils.vectSum(a, start, len);
+						start += len;
+						bufIx += 2;
+					}
+					
+					apos[k] = bufIx;	
+					astart[k] = start;
+				}
+			}
+			
+			//step 3: scale partial results by values and write to global output
+			for (int k = 0, valOff=0; k < numVals; k++, valOff+=numCols)
+				for( int j = 0; j < numCols; j++ )
+					c[ _colIndexes[j] ] += cvals[k] * _values[valOff+j];
+			
+		}
+		else
+		{
+			//iterate over all values and their bitmaps
+			for (int bitmapIx=0, valOff=0; bitmapIx<numVals; bitmapIx++, valOff+=numCols) 
+			{	
+				int bitmapOff = _ptr[bitmapIx];
+				int bitmapLen = len(bitmapIx);
+				
+				double vsum = 0;
+				int curRunEnd = 0;
+				for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+					int curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+					int curRunLen = _data[bitmapOff+bufIx + 1];
+					vsum += LinearAlgebraUtils.vectSum(a, curRunStartOff, curRunLen);
+					curRunEnd = curRunStartOff + curRunLen;
+				}
+				
+				//scale partial results by values and write results
+				for( int j = 0; j < numCols; j++ )
+					c[ _colIndexes[j] ] += vsum * _values[ valOff+j ];
+			}
+		}
+	}
+
+	@Override
+	public ColGroup scalarOperation(ScalarOperator op)
+			throws DMLRuntimeException 
+	{
+		double val0 = op.executeScalar(0);
+		
+		//fast path: sparse-safe operations
+		// Note that bitmaps don't change and are shallow-copied
+		if( op.sparseSafe || val0==0 ) {
+			return new ColGroupRLE(_colIndexes, _numRows, 
+					applyScalarOp(op), _data, _ptr);
+		}
+		
+		//slow path: sparse-unsafe operations (potentially create new bitmap)
+		//note: for efficiency, we currently don't drop values that become 0
+		boolean[] lind = computeZeroIndicatorVector();
+		int[] loff = computeOffsets(lind);
+		if( loff.length==0 ) { //empty offset list: go back to fast path
+			return new ColGroupRLE(_colIndexes, _numRows, 
+					applyScalarOp(op), _data, _ptr);
+		}
+		
+		double[] rvalues = applyScalarOp(op, val0, getNumCols());		
+		char[] lbitmap = BitmapEncoder.genRLEBitmap(loff);
+		char[] rbitmaps = Arrays.copyOf(_data, _data.length+lbitmap.length);
+		System.arraycopy(lbitmap, 0, rbitmaps, _data.length, lbitmap.length);
+		int[] rbitmapOffs = Arrays.copyOf(_ptr, _ptr.length+1);
+		rbitmapOffs[rbitmapOffs.length-1] = rbitmaps.length; 
+		
+		return new ColGroupRLE(_colIndexes, _numRows, 
+				rvalues, rbitmaps, rbitmapOffs);
+	}
+	
+	@Override
+	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock result) 
+		throws DMLRuntimeException 
+	{
+		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);
+		else if( op.indexFn instanceof ReduceRow )
+			computeColSums(result, kplus);
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeSum(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(result.quickGetValue(0, 0), result.quickGetValue(0, 1));
+		
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) {
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			int valOff = bitmapIx * numCols;
+			int curRunEnd = 0;
+			int count = 0;
+			for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+				int curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+				curRunEnd = curRunStartOff + _data[bitmapOff+bufIx + 1];
+				count += curRunEnd-curRunStartOff;
+			}
+			
+			//scale counts by all values
+			for( int j = 0; j < numCols; j++ )
+				kplus.execute3(kbuff, _values[ valOff+j ], count);
+		}
+		
+		result.quickSetValue(0, 0, kbuff._sum);
+		result.quickSetValue(0, 1, kbuff._correction);
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeRowSums(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(0, 0);
+		final int numVals = getNumValues();
+		
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) {
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			double val = sumValues(bitmapIx);
+					
+			if (val != 0.0) {
+				int curRunStartOff = 0;
+				int curRunEnd = 0;
+				for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+					curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+					curRunEnd = curRunStartOff + _data[bitmapOff+bufIx + 1];
+					for (int rix = curRunStartOff; rix < curRunEnd; rix++) {
+						kbuff.set(result.quickGetValue(rix, 0), result.quickGetValue(rix, 1));
+						kplus.execute2(kbuff, val);
+						result.quickSetValue(rix, 0, kbuff._sum);
+						result.quickSetValue(rix, 1, kbuff._correction);
+					}
+				}
+			}
+		}
+	}
+	
+	/**
+	 * 
+	 * @param result
+	 */
+	private void computeColSums(MatrixBlock result, KahanFunction kplus)
+	{
+		KahanObject kbuff = new KahanObject(0, 0);
+		
+		final int numCols = getNumCols();
+		final int numVals = getNumValues();
+		
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) {
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			int valOff = bitmapIx * numCols;
+			int curRunEnd = 0;
+			int count = 0;
+			for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+				int curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+				curRunEnd = curRunStartOff + _data[bitmapOff+bufIx + 1];
+				count += curRunEnd-curRunStartOff;
+			}
+			
+			//scale counts by all values
+			for( int j = 0; j < numCols; j++ ) {
+				kbuff.set(result.quickGetValue(0, _colIndexes[j]),result.quickGetValue(1, _colIndexes[j]));
+				kplus.execute3(kbuff, _values[ valOff+j ], count);
+				result.quickSetValue(0, _colIndexes[j], kbuff._sum);
+				result.quickSetValue(1, _colIndexes[j], kbuff._correction);
+			}
+		}
+	}
+	
+	public boolean[] computeZeroIndicatorVector()
+		throws DMLRuntimeException 
+	{	
+		boolean[] ret = new boolean[_numRows];
+		final int numVals = getNumValues();
+
+		//initialize everything with zero
+		Arrays.fill(ret, true);
+		
+		for (int bitmapIx = 0; bitmapIx < numVals; bitmapIx++) {
+			int bitmapOff = _ptr[bitmapIx];
+			int bitmapLen = len(bitmapIx);
+			
+			int curRunStartOff = 0;
+			int curRunEnd = 0;
+			for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+				curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+				curRunEnd = curRunStartOff + _data[bitmapOff+bufIx + 1];
+				Arrays.fill(ret, curRunStartOff, curRunEnd, false);
+			}
+		}
+		
+		return ret;
+	}
+	
+	@Override
+	protected void countNonZerosPerRow(int[] rnnz)
+	{
+		final int numVals = getNumValues();
+		final int numCols = getNumCols();
+		
+		for (int k = 0; k < numVals; k++) {
+			int bitmapOff = _ptr[k];
+			int bitmapLen = len(k);
+			
+			int curRunStartOff = 0;
+			int curRunEnd = 0;
+			for (int bufIx = 0; bufIx < bitmapLen; bufIx += 2) {
+				curRunStartOff = curRunEnd + _data[bitmapOff+bufIx];
+				curRunEnd = curRunStartOff + _data[bitmapOff+bufIx + 1];
+				for( int i=curRunStartOff; i<curRunEnd; i++ )
+					rnnz[i] += numCols;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/ColGroupUncompressed.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/ColGroupUncompressed.java b/src/main/java/org/apache/sysml/runtime/compress/ColGroupUncompressed.java
new file mode 100644
index 0000000..d9b75a1
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/ColGroupUncompressed.java
@@ -0,0 +1,360 @@
+/*
+ * 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.sysml.runtime.compress;
+
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.matrix.data.LibMatrixAgg;
+import org.apache.sysml.runtime.matrix.data.LibMatrixMult;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysml.runtime.util.SortUtils;
+
+
+/**
+ * Column group type for columns that are stored as dense arrays of doubles.
+ * Uses a MatrixBlock internally to store the column contents.
+ * 
+ */
+public class ColGroupUncompressed extends ColGroup 
+{
+	private static final long serialVersionUID = 4870546053280378891L;
+
+	/**
+	 * We store the contents of the columns as a MatrixBlock to take advantage
+	 * of high-performance routines available for this data structure.
+	 */
+	private MatrixBlock _data;
+
+	public ColGroupUncompressed() {
+		super(CompressionType.UNCOMPRESSED, (int[])null, -1);
+	}
+	
+	/**
+	 * Main 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
+	 * @throws DMLRuntimeException
+	 */
+	@SuppressWarnings("unused")
+	public ColGroupUncompressed(List<Integer> colIndicesList, MatrixBlock rawblock) 
+		throws DMLRuntimeException 
+	{
+		super(CompressionType.UNCOMPRESSED, colIndicesList, 
+				CompressedMatrixBlock.TRANSPOSE_INPUT ? 
+				rawblock.getNumColumns() : rawblock.getNumRows());
+
+		//prepare meta data
+		int numRows = CompressedMatrixBlock.TRANSPOSE_INPUT ? 
+				rawblock.getNumColumns() : rawblock.getNumRows();
+		
+		// Create a matrix with just the requested rows of the original block
+		_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))
+			return;
+		//special cases full block
+		if( !CompressedMatrixBlock.TRANSPOSE_INPUT &&
+			_data.getNumColumns() == rawblock.getNumColumns() ){
+			_data.copy(rawblock);
+			return;
+		}
+		
+		// dense implementation for dense and sparse matrices to avoid linear search
+		int m = numRows;
+		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]);	
+				_data.appendValue(i, j, val);
+			}
+		}
+		_data.examSparsity();
+	}
+
+	/**
+	 * Constructor for creating temporary decompressed versions of one or more
+	 * compressed column groups.
+	 * 
+	 * @param groupsToDecompress
+	 *            compressed columns to subsume. Must contain at least one
+	 *            element.
+	 */
+	public ColGroupUncompressed(ArrayList<ColGroup> groupsToDecompress) 
+	{
+		super(CompressionType.UNCOMPRESSED, 
+				mergeColIndices(groupsToDecompress),
+				groupsToDecompress.get(0)._numRows);
+
+		// Invert the list of column indices
+		int maxColIndex = _colIndexes[_colIndexes.length - 1];
+		int[] colIndicesInverted = new int[maxColIndex + 1];
+		for (int i = 0; i < _colIndexes.length; i++) {
+			colIndicesInverted[_colIndexes[i]] = i;
+		}
+
+		// Create the buffer that holds the uncompressed data, packed together
+		_data = new MatrixBlock(_numRows, _colIndexes.length, false);
+
+		for (ColGroup colGroup : groupsToDecompress) {
+			colGroup.decompressToBlock(_data, colIndicesInverted);
+		}
+	}
+
+	/**
+	 * Constructor for internal use. Used when a method needs to build an
+	 * instance of this class from scratch.
+	 * 
+	 * @param colIndices
+	 *            column mapping for this column group
+	 * @param numRows
+	 *            number of rows in the column, for passing to the superclass
+	 * @param colContents
+	 *            uncompressed cell values
+	 */
+	public ColGroupUncompressed(int[] colIndices, int numRows, MatrixBlock data) 
+	{
+		super(CompressionType.UNCOMPRESSED, colIndices, numRows);
+		_data = data;
+	}
+
+
+	/**
+	 * Access for superclass
+	 * 
+	 * @return direct pointer to the internal representation of the columns
+	 */
+	public MatrixBlock getData() {
+		return _data;
+	}
+	
+	/**
+	 * Subroutine of constructor.
+	 * 
+	 * @param groupsToDecompress
+	 *            input to the constructor that decompresses into a temporary
+	 *            UncompressedColGroup
+	 * @return a merged set of column indices across all those groups
+	 */
+	private static int[] mergeColIndices(ArrayList<ColGroup> groupsToDecompress) 
+	{
+		// Pass 1: Determine number of columns
+		int sz = 0;
+		for (ColGroup colGroup : groupsToDecompress) {
+			sz += colGroup.getNumCols();
+		}
+
+		// Pass 2: Copy column offsets out
+		int[] ret = new int[sz];
+		int pos = 0;
+		for (ColGroup colGroup : groupsToDecompress) {
+			int[] tmp = colGroup.getColIndices();
+			System.arraycopy(tmp, 0, ret, pos, tmp.length);
+			pos += tmp.length;
+		}
+
+		// Pass 3: Sort and return the list of columns
+		Arrays.sort(ret);
+		return ret;
+	}
+
+	@Override
+	public long estimateInMemorySize() {
+		long size = super.estimateInMemorySize();
+		// adding the size of colContents
+		return size + 8 + _data.estimateSizeInMemory();
+	}
+
+	@Override
+	public void decompressToBlock(MatrixBlock target) {
+		//empty block, nothing to add to output
+		if( _data.isEmptyBlock(false) )
+			return;		
+		for (int row = 0; row < _data.getNumRows(); row++) {
+			for (int colIx = 0; colIx < _colIndexes.length; colIx++) {
+				int col = _colIndexes[colIx];
+				double cellVal = _data.quickGetValue(row, colIx);
+				target.quickSetValue(row, col, cellVal);
+			}
+		}
+	}
+
+	@Override
+	public void decompressToBlock(MatrixBlock target, int[] colIndexTargets) {
+		//empty block, nothing to add to output
+		if( _data.isEmptyBlock(false) ) {
+			return;		
+		}
+		// Run through the rows, putting values into the appropriate locations
+		for (int row = 0; row < _data.getNumRows(); row++) {
+			for (int colIx = 0; colIx < _data.getNumColumns(); colIx++) {
+				int origMatrixColIx = getColIndex(colIx);
+				int col = colIndexTargets[origMatrixColIx];
+				double cellVal = _data.quickGetValue(row, colIx);
+				target.quickSetValue(row, col, cellVal);
+			}
+		}	
+	}
+	
+	@Override
+	public void decompressToBlock(MatrixBlock target, int colpos) {
+		//empty block, nothing to add to output
+		if( _data.isEmptyBlock(false) ) {
+			return;		
+		}
+		// Run through the rows, putting values into the appropriate locations
+		for (int row = 0; row < _data.getNumRows(); row++) {
+			double cellVal = _data.quickGetValue(row, colpos);
+			target.quickSetValue(row, 0, cellVal);
+		}	
+	}
+
+	@Override
+	public void rightMultByVector(MatrixBlock vector, MatrixBlock result, int rl, int ru)
+			throws DMLRuntimeException 
+	{
+		// Pull out the relevant rows of the vector
+		int clen = _colIndexes.length;
+		
+		MatrixBlock shortVector = new MatrixBlock(clen, 1, false);
+		shortVector.allocateDenseBlock();
+		double[] b = shortVector.getDenseBlock();
+		for (int colIx = 0; colIx < clen; colIx++)
+			b[colIx] = vector.quickGetValue(_colIndexes[colIx], 0);
+		shortVector.recomputeNonZeros();
+		
+		// Multiply the selected columns by the appropriate parts of the vector
+		LibMatrixMult.matrixMult(_data, shortVector, result, rl, ru);	
+	}
+	
+	@Override
+	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result)
+			throws DMLRuntimeException 
+	{
+		MatrixBlock pret = new MatrixBlock(1, _colIndexes.length, false);
+		LibMatrixMult.matrixMult(vector, _data, pret);
+		
+		// copying partialResult to the proper indices of the result
+		if( !pret.isEmptyBlock(false) ) {
+			double[] rsltArr = result.getDenseBlock();
+			for (int colIx = 0; colIx < _colIndexes.length; colIx++)
+				rsltArr[_colIndexes[colIx]] = pret.quickGetValue(0, colIx);
+			result.recomputeNonZeros();
+		}
+	}
+	
+	public void leftMultByRowVector(MatrixBlock vector, MatrixBlock result, int k)
+			throws DMLRuntimeException 
+	{
+		MatrixBlock pret = new MatrixBlock(1, _colIndexes.length, false);
+		LibMatrixMult.matrixMult(vector, _data, pret, k);
+		
+		// copying partialResult to the proper indices of the result
+		if( !pret.isEmptyBlock(false) ) {
+			double[] rsltArr = result.getDenseBlock();
+			for (int colIx = 0; colIx < _colIndexes.length; colIx++)
+				rsltArr[_colIndexes[colIx]] = pret.quickGetValue(0, colIx);
+			result.recomputeNonZeros();
+		}
+	}
+
+	@Override
+	public ColGroup scalarOperation(ScalarOperator op)
+			throws DMLRuntimeException 
+	{
+		//execute scalar operations
+		MatrixBlock retContent = (MatrixBlock) _data
+				.scalarOperations(op, new MatrixBlock());
+
+		//construct new uncompressed column group
+		return new ColGroupUncompressed(getColIndices(), _data.getNumRows(), retContent);
+	}
+	
+	@Override
+	public void unaryAggregateOperations(AggregateUnaryOperator op, MatrixBlock ret)
+		throws DMLRuntimeException 
+	{
+		//execute unary aggregate operations
+		LibMatrixAgg.aggregateUnaryMatrix(_data, ret, op);
+	}
+
+	@Override
+	public void readFields(DataInput in)
+		throws IOException 
+	{
+		//read col contents (w/ meta data)
+		_data = new MatrixBlock();
+		_data.readFields(in);		
+		_numRows = _data.getNumRows();
+		
+		//read col indices
+		int numCols = _data.getNumColumns();
+		_colIndexes = new int[ numCols ];
+		for( int i=0; i<numCols; i++ )
+			_colIndexes[i] = in.readInt();
+	}
+
+	@Override
+	public void write(DataOutput out) 
+		throws IOException 
+	{
+		//write col contents first (w/ meta data)
+		_data.write(out);
+		
+		//write col indices
+		int len = _data.getNumColumns();
+		for( int i=0; i<len; i++ )
+			out.writeInt( _colIndexes[i] );
+	}
+
+	@Override
+	public long getExactSizeOnDisk() {
+		return _data.getExactSizeOnDisk()
+			   + 4 * _data.getNumColumns();
+	}
+	
+	@Override
+	protected void countNonZerosPerRow(int[] rnnz)
+	{
+		for( int i=0; i<_data.getNumRows(); i++ )
+			rnnz[i] += _data.recomputeNonZeros(i, i, 0, _data.getNumColumns()-1);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/CompressedMatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/CompressedMatrixBlock.java b/src/main/java/org/apache/sysml/runtime/compress/CompressedMatrixBlock.java
new file mode 100644
index 0000000..6ac26c6
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/CompressedMatrixBlock.java
@@ -0,0 +1,1342 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.Externalizable;
+import java.io.IOException;
+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.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+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.sysml.hops.OptimizerUtils;
+import org.apache.sysml.lops.MMTSJ.MMTSJType;
+import org.apache.sysml.lops.MapMultChain.ChainType;
+import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.compress.ColGroup.CompressionType;
+import org.apache.sysml.runtime.compress.estim.CompressedSizeEstimator;
+import org.apache.sysml.runtime.compress.estim.CompressedSizeInfo;
+import org.apache.sysml.runtime.compress.estim.SizeEstimatorFactory;
+import org.apache.sysml.runtime.compress.utils.ConverterUtils;
+import org.apache.sysml.runtime.compress.utils.LinearAlgebraUtils;
+import org.apache.sysml.runtime.controlprogram.parfor.stat.Timing;
+import org.apache.sysml.runtime.functionobjects.KahanPlus;
+import org.apache.sysml.runtime.functionobjects.KahanPlusSq;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.ReduceRow;
+import org.apache.sysml.runtime.instructions.cp.KahanObject;
+import org.apache.sysml.runtime.matrix.data.LibMatrixBincell;
+import org.apache.sysml.runtime.matrix.data.LibMatrixReorg;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixIndexes;
+import org.apache.sysml.runtime.matrix.data.MatrixValue;
+import org.apache.sysml.runtime.matrix.data.SparseBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+
+/**
+ * Experimental version of MatrixBlock that allows a compressed internal
+ * representation.
+ */
+public class CompressedMatrixBlock extends MatrixBlock implements Externalizable
+{
+	private static final long serialVersionUID = 7319972089143154057L;
+	
+	//internal configuration
+	public static final int MAX_NUMBER_COCODING_COLUMNS = 1000;
+	public static final double MIN_COMPRESSION_RATIO = 2.0;
+	public static final double MIN_RLE_RATIO = 1.0; // Minimum additional compression (non-RLE size / RLE size) before we switch to run-length encoding.
+	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;
+	private static final boolean LDEBUG = false; //local debug flag
+	
+	private static final Log LOG = LogFactory.getLog(CompressedMatrixBlock.class.getName());
+	
+	static{
+		// for internal debugging only
+		if( LDEBUG ) {
+			Logger.getLogger("org.apache.sysml.runtime.compress")
+				  .setLevel((Level) Level.DEBUG);
+		}	
+	}
+	
+	protected ArrayList<ColGroup> _colGroups = null;
+	protected CompressionStatistics _stats = null;
+	
+	public CompressedMatrixBlock() {
+		super(-1, -1, true);
+	}
+	
+	/**
+	 * 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 CompressedMatrixBlock(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.
+	 */
+	public CompressedMatrixBlock(MatrixBlock mb) {
+		super(mb.getNumRows(), mb.getNumColumns(), mb.isInSparseFormat());
+		
+		//shallow copy (deep copy on compression, prevents unnecessary copy) 
+		if( isInSparseFormat() )
+			sparseBlock = mb.getSparseBlock();
+		else
+			denseBlock = mb.getDenseBlock();
+		nonZeros = mb.getNonZeros();
+	}
+
+	/**
+	 * 
+	 * @return the column groups constructed by the compression process.
+	 * 
+	 */
+	public ArrayList<ColGroup> getColGroups() {
+		return _colGroups;
+	}
+
+	/**
+	 * @return true if this block is in compressed form; false if the block has
+	 *         not yet been compressed
+	 */
+	public boolean isCompressed() {
+		return (_colGroups != null);
+	}
+	
+	/**
+	 * 
+	 * @return
+	 */
+	public boolean isSingleUncompressedGroup(){
+		return (_colGroups!=null && _colGroups.size()==1 
+				&& _colGroups.get(0) instanceof ColGroupUncompressed);
+	}
+
+	private void allocateColGroupList() {
+		_colGroups = new ArrayList<ColGroup>();
+	}
+	
+	@Override
+	public boolean isEmptyBlock(boolean safe)  {
+		if( !isCompressed() )
+			return super.isEmptyBlock(safe);		
+		return (_colGroups == null || getNonZeros()==0);
+	}
+	
+	/**
+	 * Compress the contents of this matrix block. After compression, the
+	 * uncompressed data is discarded. Attempts to update this block after
+	 * calling this method currently result in INCORRECT BEHAVIOR, something
+	 * which should be fixed if we move ahead with this compression strategy.
+	 * 
+	 * +per column sparsity
+	 */
+	public void compress() 
+		throws DMLRuntimeException 
+	{
+		//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 potentionls are made based on a
+		// subset of the rows. For large datasets, 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 = this;
+		if( TRANSPOSE_INPUT )
+			rawblock = LibMatrixReorg.transpose(rawblock, new MatrixBlock(numCols, numRows, sparse));
+		else
+			rawblock = new MatrixBlock(this);
+		
+		//construct sample-based size estimator
+		CompressedSizeEstimator bitmapSizeEstimator = 
+				SizeEstimatorFactory.getSizeEstimator(rawblock, numRows);
+
+		//allocate list of column groups
+		allocateColGroupList();
+
+		// The current implementation of this method is written for correctness,
+		// not for performance or for minimal use of temporary space.
+
+		// We start with a full set of columns.
+		HashSet<Integer> remainingCols = new HashSet<Integer>();
+		for (int i = 0; i < numCols; i++) {
+			remainingCols.add(i);
+		}
+
+		// PHASE 1: Classify columns by compression type
+		// We start by determining which columns are amenable to bitmap
+		// compression
+
+		// It is correct to use the dense size as the uncompressed size
+		// FIXME not numRows but nnz / col otherwise too aggressive overestimation
+		// of uncompressed size and hence overestimation of compression potential
+		double uncompressedColumnSize = 8 * numRows;
+
+		// information about the bitmap amenable columns
+		List<Integer> bitmapCols = new ArrayList<Integer>();
+		List<Integer> uncompressedCols = new ArrayList<Integer>();
+		List<Integer> colsCardinalities = new ArrayList<Integer>();
+		List<Long> compressedSizes = new ArrayList<Long>();
+		HashMap<Integer, Double> compressionRatios = new HashMap<Integer, Double>();
+		
+		// Minimum ratio (size of uncompressed / size of compressed) that we
+		// will accept when encoding a field with a bitmap.
+		for (int col = 0; col < numCols; col++) 
+		{
+			CompressedSizeInfo compressedSizeInfo = bitmapSizeEstimator
+					.estimateCompressedColGroupSize(new int[] { col });
+			long compressedSize = compressedSizeInfo.getMinSize();
+			double compRatio = uncompressedColumnSize / compressedSize;
+			
+			//FIXME: compression ratio should be checked against 1 instead of min compression
+			//ratio; I think this threshold was only required because we overestimated the 
+			//the uncompressed column size with n\alpha instead of z\alpha
+			if (compRatio >= MIN_COMPRESSION_RATIO) {
+				bitmapCols.add(col);
+				compressionRatios.put(col, compRatio);
+				colsCardinalities.add(compressedSizeInfo.getEstCarinality());
+				compressedSizes.add(compressedSize);
+			}
+			else
+				uncompressedCols.add(col);
+		}
+
+		_stats.timePhase1 = time.stop();
+		if( LOG.isDebugEnabled() )
+			LOG.debug("compression phase 1: "+_stats.timePhase1);
+		
+		// Filters for additional types of compression should be inserted here.
+
+		// PHASE 2: Grouping columns
+		// Divide the bitmap columns into column groups.
+		List<int[]> bitmapColGrps = null;
+		if (bitmapCols.size() > MAX_NUMBER_COCODING_COLUMNS) {
+			// Too many columns to compute co-coding groups with current methods.
+			// Generate singleton groups.
+			bitmapColGrps = new ArrayList<int[]>(bitmapCols.size());
+			for (int col : bitmapCols) {
+				bitmapColGrps.add(new int[] { col });
+			}
+		} 
+		else {
+			bitmapColGrps = PlanningCoCoder.findCocodesByPartitioning(
+					bitmapSizeEstimator, bitmapCols, colsCardinalities,
+					compressedSizes, numRows, isInSparseFormat() ? 
+					OptimizerUtils.getSparsity(numRows, numCols, getNonZeros()): 1);
+		}
+
+		_stats.timePhase2 = time.stop();
+		if( LOG.isDebugEnabled() )
+			LOG.debug("compression phase 2: "+_stats.timePhase2);
+		
+		if( INVESTIGATE_ESTIMATES ) {
+			double est = 0;
+			for( int[] groupIndices : bitmapColGrps )
+				est += bitmapSizeEstimator.estimateCompressedColGroupSize(groupIndices).getMinSize();
+			est += uncompressedCols.size() * uncompressedColumnSize;
+			_stats.estSize = est;
+		}
+		
+		// PHASE 3: Compress and correct sample-based decisions
+		
+		for (int[] groupIndices : bitmapColGrps) 
+		{
+			int[] allGroupIndices = null;
+			int allColsCount = groupIndices.length;
+			CompressedSizeInfo bitmapSizeInfo;
+			// The compression type is decided based on a full bitmap since it
+			// will be reused for the actual compression step.
+			UncompressedBitmap ubm;
+			PriorityQueue<CompressedColumn> compRatioPQ = null;
+			boolean skipGroup = false;
+			while (true) 
+			{
+				ubm = BitmapEncoder.extractBitmap(groupIndices, rawblock); 
+				bitmapSizeInfo = bitmapSizeEstimator
+						.estimateCompressedColGroupSize(ubm);
+				double compRatio = uncompressedColumnSize * groupIndices.length
+						/ bitmapSizeInfo.getMinSize();
+				if (compRatio >= MIN_COMPRESSION_RATIO) {
+					// we have a good group
+					for( Integer col : groupIndices )
+						remainingCols.remove(col);
+					break;
+				} else {
+					// modify the group
+					if (compRatioPQ == null) {
+						// first modification
+						allGroupIndices = Arrays.copyOf(groupIndices, groupIndices.length);
+						compRatioPQ = new PriorityQueue<CompressedMatrixBlock.CompressedColumn>();
+						for (int i = 0; i < groupIndices.length; i++)
+							compRatioPQ.add(new CompressedColumn(i,
+									compressionRatios.get(groupIndices[i])));
+					}
+
+					// index in allGroupIndices
+					int removeIx = compRatioPQ.poll().colIx;
+					allGroupIndices[removeIx] = -1;
+					allColsCount--;
+					if (allColsCount == 0) {
+						skipGroup = true;
+						break;
+					}
+					groupIndices = new int[allColsCount];
+					// copying the values that do not equal -1
+					int ix = 0;
+					for (int col : allGroupIndices) {
+						if (col != -1) {
+							groupIndices[ix++] = col;
+						}
+					}
+
+				}
+			}
+
+			if (skipGroup)
+				continue;
+			long rleNumBytes = bitmapSizeInfo.getRLESize();
+			long offsetNumBytes = bitmapSizeInfo.getOLESize();
+			double rleRatio = (double) offsetNumBytes / (double) rleNumBytes;
+
+			if (rleRatio > MIN_RLE_RATIO) {
+				ColGroupRLE compressedGroup = new ColGroupRLE(groupIndices,
+						numRows, ubm);
+				_colGroups.add(compressedGroup);
+			} 
+			else {
+				ColGroupOLE compressedGroup = new ColGroupOLE(
+						groupIndices, numRows, ubm);
+				_colGroups.add(compressedGroup);
+			}
+
+		}
+		
+		_stats.timePhase3 = time.stop();
+		if( LOG.isDebugEnabled() )
+			LOG.debug("compression phase 3: "+_stats.timePhase3);
+		
+		// Phase 4: Cleanup
+		// The remaining columns are stored uncompressed as one big column group
+		if (remainingCols.size() > 0) {
+			ArrayList<Integer> list = new ArrayList<Integer>(remainingCols);
+			ColGroupUncompressed ucgroup = new ColGroupUncompressed(list, rawblock);
+			_colGroups.add(ucgroup);
+		}
+
+		//final cleanup (discard uncompressed block)
+		rawblock.cleanupBlock(true, true);
+		this.cleanupBlock(true, true);
+		
+		_stats.timePhase4 = time.stop();
+		if( LOG.isDebugEnabled() )
+			LOG.debug("compression phase 4: "+_stats.timePhase4);
+	}
+
+	/**
+	 * @return a new uncompressed matrix block containing the contents of this
+	 *         block
+	 */
+	public MatrixBlock decompress() throws DMLRuntimeException 
+	{
+		//early abort for not yet compressed blocks
+		if( !isCompressed() )
+			return new MatrixBlock(this); 
+		
+		//preallocation sparse rows to avoid repeated reallocations		
+		MatrixBlock ret = new MatrixBlock(getNumRows(), getNumColumns(), isInSparseFormat(), getNonZeros());
+		if( ret.isInSparseFormat() ) {
+			int[] rnnz = new int[rlen];
+			for (ColGroup grp : _colGroups)
+				grp.countNonZerosPerRow(rnnz);
+			ret.allocateSparseRowsBlock();
+			SparseBlock rows = ret.getSparseBlock();
+			for( int i=0; i<rlen; i++ )
+				rows.allocate(i, rnnz[i]);
+		}
+		
+		//core decompression (append if sparse)
+		for (ColGroup grp : _colGroups)
+			grp.decompressToBlock(ret);
+		
+		//post-processing (for append in decompress)
+		if( isInSparseFormat() )
+			ret.sortSparseRows();
+
+		return ret;
+	}
+
+	public CompressionStatistics getCompressionStatistics(){
+		return _stats;
+	}
+
+	/**
+	 * 
+	 * @return an upper bound on the memory used to store this compressed block
+	 *         considering class overhead.
+	 */
+	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();
+		for (ColGroup grp : _colGroups)
+			total += grp.estimateInMemorySize();
+		return total;
+	}
+
+	private class CompressedColumn implements Comparable<CompressedColumn> {
+		int colIx;
+		double compRatio;
+
+		public CompressedColumn(int colIx, double compRatio) {
+			this.colIx = colIx;
+			this.compRatio = compRatio;
+		}
+
+		@Override
+		public int compareTo(CompressedColumn o) {
+			return (int) Math.signum(compRatio - o.compRatio);
+		}
+	}
+	
+	public static class CompressionStatistics {
+		public double timePhase1 = -1;
+		public double timePhase2 = -1;
+		public double timePhase3 = -1;
+		public double timePhase4 = -1;
+		public double estSize = -1;
+		
+		public CompressionStatistics() {
+			//do nothing
+		}
+		
+		public CompressionStatistics(double t1, double t2, double t3, double t4){
+			timePhase1 = t1;
+			timePhase2 = t2;
+			timePhase3 = t3;
+			timePhase4 = t4;
+		}
+	} 
+
+	
+	//////////////////////////////////////////
+	// Serialization / Deserialization
+
+	@Override
+	public long getExactSizeOnDisk() 
+	{
+		//header information
+		long ret = 12;
+		
+		for( ColGroup grp : _colGroups ) {
+			ret += 1; //type info
+			ret += grp.getExactSizeOnDisk();
+		}
+		
+		return ret;
+	}
+	
+	@Override
+	public void readFields(DataInput in) 
+		throws IOException 
+	{
+		boolean compressed = in.readBoolean();
+		
+		//deserialize uncompressed block
+		if( !compressed ) {
+			super.readFields(in);
+			return;
+		}
+		
+		//deserialize compressed block
+		rlen = in.readInt();
+		clen = in.readInt();
+		nonZeros = in.readLong();
+		int ncolGroups = in.readInt();
+		
+		_colGroups = new ArrayList<ColGroup>(ncolGroups);
+		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;
+			}
+			
+			//deserialize and add column group
+			grp.readFields(in);
+			_colGroups.add(grp);
+		}
+	}
+	
+	@Override
+	public void write(DataOutput out) 
+		throws IOException 
+	{
+		out.writeBoolean( isCompressed() );
+		
+		//serialize uncompressed block
+		if( !isCompressed() ) {
+			super.write(out);
+			return;
+		}
+		
+		//serialize compressed matrix block
+		out.writeInt(rlen);
+		out.writeInt(clen);
+		out.writeLong(nonZeros);
+		out.writeInt(_colGroups.size());
+		
+		for( ColGroup grp : _colGroups ) {
+			out.writeByte( grp.getCompType().ordinal() );
+			grp.write(out); //delegate serialization
+		}
+	}
+	
+	
+	/**
+	 * Redirects the default java serialization via externalizable to our default 
+	 * hadoop writable serialization for efficient broadcast/rdd deserialization. 
+	 * 
+	 * @param is
+	 * @throws IOException
+	 */
+	@Override
+	public void readExternal(ObjectInput is) 
+		throws IOException
+	{
+		readFields(is);
+	}
+	
+	/**
+	 * Redirects the default java serialization via externalizable to our default 
+	 * hadoop writable serialization for efficient broadcast/rdd serialization. 
+	 * 
+	 * @param is
+	 * @throws IOException
+	 */
+	@Override
+	public void writeExternal(ObjectOutput os) 
+		throws IOException
+	{
+		write(os);	
+	}
+	
+	
+	//////////////////////////////////////////
+	// Operations (overwrite existing ops for seamless integration)
+
+	@Override
+	public MatrixValue scalarOperations(ScalarOperator sop, MatrixValue result) 
+		throws DMLRuntimeException
+	{
+		//call uncompressed matrix scalar if necessary
+		if( !isCompressed() ) {
+			return super.scalarOperations(sop, result);
+		}
+		
+		//allocate the output matrix block
+		CompressedMatrixBlock ret = null;
+		if( result==null || !(result instanceof CompressedMatrixBlock) )
+			ret = new CompressedMatrixBlock(getNumRows(), getNumColumns(), sparse);
+		else {
+			ret = (CompressedMatrixBlock) result;
+			ret.reset(rlen, clen);
+		}
+		
+		// Apply the operation recursively to each of the column groups.
+		// Most implementations will only modify metadata.
+		ArrayList<ColGroup> newColGroups = new ArrayList<ColGroup>();
+		for (ColGroup grp : _colGroups) {
+			newColGroups.add(grp.scalarOperation(sop));
+		}
+		ret._colGroups = newColGroups;
+		ret.setNonZeros(rlen*clen);
+		
+		return ret;
+	}
+
+	@Override
+	public MatrixBlock appendOperations(MatrixBlock that, MatrixBlock ret) 
+		throws DMLRuntimeException
+	{
+		//call uncompressed matrix append if necessary
+		if( !isCompressed() ) {
+			if( that instanceof CompressedMatrixBlock )
+				that = ((CompressedMatrixBlock) that).decompress();
+			return super.appendOperations(that, ret);
+		}
+		
+		final int m = rlen;
+		final int n = clen+that.getNumColumns();
+		final long nnz = nonZeros+that.getNonZeros();		
+		
+		//init result matrix 
+		CompressedMatrixBlock ret2 = null;
+		if( ret == null || !(ret instanceof CompressedMatrixBlock) ) {
+			ret2 = new CompressedMatrixBlock(m, n, isInSparseFormat());
+		}
+		else {
+			ret2 = (CompressedMatrixBlock) ret;
+			ret2.reset(m, n);
+		}
+			
+		//shallow copy of lhs column groups
+		ret2.allocateColGroupList();
+		ret2._colGroups.addAll(_colGroups);
+		
+		//copy of rhs column groups w/ col index shifting
+		if( !(that instanceof CompressedMatrixBlock) ) {
+			that = new CompressedMatrixBlock(that);
+			((CompressedMatrixBlock)that).compress();
+		}
+		ArrayList<ColGroup> inColGroups = ((CompressedMatrixBlock) that)._colGroups;
+		for( ColGroup group : inColGroups ) {
+			ColGroup tmp = ConverterUtils.copyColGroup(group);
+			tmp.shiftColIndices(clen);
+			ret2._colGroups.add(tmp);
+		}
+		
+		//meta data maintenance
+		ret2.setNonZeros(nnz);		
+		return ret2;
+	}
+	
+	@Override
+	public MatrixBlock chainMatrixMultOperations(MatrixBlock v, MatrixBlock w, MatrixBlock out, ChainType ctype) 
+		throws DMLRuntimeException 
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ) {
+			return super.chainMatrixMultOperations(v, w, out, ctype);
+		}
+		
+		//single-threaded mmchain of single uncompressed colgroup
+		if( isSingleUncompressedGroup() ){
+			return ((ColGroupUncompressed)_colGroups.get(0))
+				.getData().chainMatrixMultOperations(v, w, out, ctype);
+		}
+		
+		//Timing time = new Timing(true);
+		
+		//prepare result
+		if( out != null )
+			out.reset(clen, 1, false);
+		else 
+			out = new MatrixBlock(clen, 1, false);
+		
+		//empty block handling
+		if( isEmptyBlock(false) ) 
+			return out;
+			
+		//compute matrix mult
+		MatrixBlock tmp = new MatrixBlock(rlen, 1, false);
+		rightMultByVector(v, tmp);
+		if( ctype == ChainType.XtwXv ) {
+			BinaryOperator bop = new BinaryOperator(Multiply.getMultiplyFnObject());
+			LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
+		}
+		leftMultByVectorTranspose(_colGroups, tmp, out, true);
+		
+		//System.out.println("Compressed MMChain in "+time.stop());
+		
+		return out;
+	}
+
+	@Override
+	public MatrixBlock chainMatrixMultOperations(MatrixBlock v, MatrixBlock w, MatrixBlock out, ChainType ctype, int k) 
+		throws DMLRuntimeException 
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ){
+			return super.chainMatrixMultOperations(v, w, out, ctype, k);
+		}
+		
+		//multi-threaded mmchain of single uncompressed colgroup
+		if( isSingleUncompressedGroup() ){
+			return ((ColGroupUncompressed)_colGroups.get(0))
+				.getData().chainMatrixMultOperations(v, w, out, ctype, k);
+		}
+
+		Timing time = LOG.isDebugEnabled() ? new Timing(true) : null;
+		
+		//prepare result
+		if( out != null )
+			out.reset(clen, 1, false);
+		else 
+			out = new MatrixBlock(clen, 1, false);
+		
+		//empty block handling
+		if( isEmptyBlock(false) ) 
+			return out;
+		
+		//compute matrix mult
+		MatrixBlock tmp = new MatrixBlock(rlen, 1, false);
+		rightMultByVector(v, tmp, k);
+		if( ctype == ChainType.XtwXv ) {
+			BinaryOperator bop = new BinaryOperator(Multiply.getMultiplyFnObject());
+			LibMatrixBincell.bincellOpInPlace(tmp, w, bop);
+		}
+		leftMultByVectorTranspose(_colGroups, tmp, out, true, k);
+		
+		if( LOG.isDebugEnabled() )
+			LOG.debug("Compressed MMChain k="+k+" in "+time.stop());
+		
+		return out;
+	}
+	
+	@Override
+	public MatrixValue aggregateBinaryOperations(MatrixValue mv1, MatrixValue mv2, MatrixValue result, AggregateBinaryOperator op)
+			throws DMLRuntimeException 
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ) {
+			return super.aggregateBinaryOperations(mv1, mv2, result, op);
+		}
+	
+		//multi-threaded mm of single uncompressed colgroup
+		if( isSingleUncompressedGroup() ){
+			MatrixBlock tmp = ((ColGroupUncompressed)_colGroups.get(0)).getData();
+			return tmp.aggregateBinaryOperations(this==mv1?tmp:mv1, this==mv2?tmp:mv2, result, op);
+		}
+		
+		Timing time = LOG.isDebugEnabled() ? new Timing(true) : null;
+		
+		//setup meta data (dimensions, sparsity)
+		int rl = mv1.getNumRows();
+		int cl = mv2.getNumColumns();
+		
+		//create output matrix block
+		MatrixBlock ret = (MatrixBlock) result;
+		if( ret==null )
+			ret = new MatrixBlock(rl, cl, false, rl*cl);
+		else
+			ret.reset(rl, cl, false, rl*cl);
+		
+		//compute matrix mult
+		if( mv1.getNumRows()>1 && mv2.getNumColumns()==1 ) { //MV right
+			CompressedMatrixBlock cmb = (CompressedMatrixBlock)mv1;
+			MatrixBlock mb = (MatrixBlock) mv2;
+			if( op.getNumThreads()>1 )
+				cmb.rightMultByVector(mb, ret, op.getNumThreads());
+			else
+				cmb.rightMultByVector(mb, ret);
+		}
+		else if( mv1.getNumRows()==1 && mv2.getNumColumns()>1 ) { //MV left
+			MatrixBlock mb = (MatrixBlock) mv1;
+			if( op.getNumThreads()>1 )
+				leftMultByVectorTranspose(_colGroups, mb, ret, false, op.getNumThreads());
+			else
+				leftMultByVectorTranspose(_colGroups, mb, ret, false);
+		}
+		else {
+			//NOTE: we could decompress and invoke super.aggregateBinary but for now
+			//we want to have an eager fail if this happens
+			throw new DMLRuntimeException("Unsupported matrix-matrix multiplication over compressed matrix block.");
+		}
+		
+		if( LOG.isDebugEnabled() )
+			LOG.debug("Compressed MM in "+time.stop());
+		
+		return ret;
+	}
+	
+	@Override
+	public MatrixValue aggregateUnaryOperations(AggregateUnaryOperator op, MatrixValue result, 
+			int blockingFactorRow, int blockingFactorCol, MatrixIndexes indexesIn, boolean inCP) 
+		throws DMLRuntimeException
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ) {
+			return super.aggregateUnaryOperations(op, result, blockingFactorRow, blockingFactorCol, indexesIn, inCP);
+		}
+		
+		//check for supported operations
+		if( !(op.aggOp.increOp.fn instanceof KahanPlus || op.aggOp.increOp.fn instanceof KahanPlusSq) ){
+			throw new DMLRuntimeException("Unary aggregates other than sums not supported yet.");
+		}
+		
+		//prepare output dimensions
+		CellIndex tempCellIndex = new CellIndex(-1,-1);
+		op.indexFn.computeDimension(rlen, clen, tempCellIndex);
+		if(op.aggOp.correctionExists) {
+			switch(op.aggOp.correctionLocation)
+			{
+				case LASTROW: tempCellIndex.row++;  break;
+				case LASTCOLUMN: tempCellIndex.column++; break;
+				case LASTTWOROWS: tempCellIndex.row+=2; break;
+				case LASTTWOCOLUMNS: tempCellIndex.column+=2; break;
+				default:
+					throw new DMLRuntimeException("unrecognized correctionLocation: "+op.aggOp.correctionLocation);	
+			}
+		}
+		
+		//prepare output
+		if(result==null)
+			result=new MatrixBlock(tempCellIndex.row, tempCellIndex.column, false);
+		else
+			result.reset(tempCellIndex.row, tempCellIndex.column, false);
+		
+		MatrixBlock ret = (MatrixBlock) result;
+		
+		//core unary aggregate
+		if(    op.getNumThreads() > 1 
+			&& getExactSizeOnDisk() > MIN_PAR_AGG_THRESHOLD ) 
+		{
+			// initialize and allocate the result
+			ret.allocateDenseBlock();
+			
+			//multi-threaded execution of all groups 
+			ArrayList<ColGroup>[] grpParts = createStaticTaskPartitioning(op.getNumThreads(), false);
+			ColGroupUncompressed uc = getUncompressedColGroup();
+			try {
+				//compute all compressed column groups
+				ExecutorService pool = Executors.newFixedThreadPool( op.getNumThreads() );
+				ArrayList<UnaryAggregateTask> tasks = new ArrayList<UnaryAggregateTask>();
+				for( ArrayList<ColGroup> grp : grpParts )
+					tasks.add(new UnaryAggregateTask(grp, ret, op));
+				pool.invokeAll(tasks);	
+				pool.shutdown();
+				//compute uncompressed column group in parallel (otherwise bottleneck)
+				if( uc != null )
+					 ret = (MatrixBlock)uc.getData().aggregateUnaryOperations(op, ret, blockingFactorRow, blockingFactorCol, indexesIn, false);					
+				//aggregate partial results
+				if( !(op.indexFn instanceof ReduceRow) ){
+					KahanObject kbuff = new KahanObject(0,0);
+					KahanPlus kplus = KahanPlus.getKahanPlusFnObject();
+					for( int i=0; i<ret.getNumRows(); i++ ) {
+						kbuff.set(ret.quickGetValue(i, 0), ret.quickGetValue(i, 0));
+						for( UnaryAggregateTask task : tasks )
+							kplus.execute2(kbuff, task.getResult().quickGetValue(i, 0));
+						ret.quickSetValue(i, 0, kbuff._sum);
+						ret.quickSetValue(i, 1, kbuff._correction);
+					}
+				}		
+			}
+			catch(Exception ex) {
+				throw new DMLRuntimeException(ex);
+			}
+		}
+		else {
+			for (ColGroup grp : _colGroups) {
+				grp.unaryAggregateOperations(op, ret);
+			}
+		}
+		
+		//drop correction if necessary
+		if(op.aggOp.correctionExists && inCP)
+			ret.dropLastRowsOrColums(op.aggOp.correctionLocation);
+	
+		//post-processing
+		ret.recomputeNonZeros();
+		
+		return ret;
+	}
+	
+	@Override
+	public MatrixBlock transposeSelfMatrixMultOperations(MatrixBlock out, MMTSJType tstype) 
+		throws DMLRuntimeException 
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ) {
+			return super.transposeSelfMatrixMultOperations(out, tstype);
+		}
+				
+		//single-threaded tsmm of single uncompressed colgroup
+		if( isSingleUncompressedGroup() ){
+			return ((ColGroupUncompressed)_colGroups.get(0))
+				.getData().transposeSelfMatrixMultOperations(out, tstype);
+		}
+
+		Timing time = LOG.isDebugEnabled() ? new Timing(true) : null;
+
+		//check for transpose type
+		if( tstype != MMTSJType.LEFT ) //right not supported yet
+			throw new DMLRuntimeException("Invalid MMTSJ type '"+tstype.toString()+"'.");
+		
+		//create output matrix block
+		if( out == null )
+			out = new MatrixBlock(clen, clen, false);
+		else
+			out.reset(clen, clen, false);
+		out.allocateDenseBlock();
+		
+		if( !isEmptyBlock(false) ) {
+			//compute matrix mult
+			leftMultByTransposeSelf(_colGroups, out, 0, _colGroups.size());
+			
+			// post-processing
+			out.recomputeNonZeros();
+		}
+		
+		if( LOG.isDebugEnabled() )
+			LOG.debug("Compressed TSMM in "+time.stop());
+		
+		return out;
+	}
+
+	
+	@Override
+	public MatrixBlock transposeSelfMatrixMultOperations(MatrixBlock out, MMTSJType tstype, int k) 
+		throws DMLRuntimeException 
+	{
+		//call uncompressed matrix mult if necessary
+		if( !isCompressed() ){
+			return super.transposeSelfMatrixMultOperations(out, tstype, k);
+		}
+		
+		//multi-threaded tsmm of single uncompressed colgroup
+		if( isSingleUncompressedGroup() ){
+			return ((ColGroupUncompressed)_colGroups.get(0))
+				.getData().transposeSelfMatrixMultOperations(out, tstype, k);
+		}
+		
+		Timing time = LOG.isDebugEnabled() ? new Timing(true) : null;
+		
+		//check for transpose type
+		if( tstype != MMTSJType.LEFT ) //right not supported yet
+			throw new DMLRuntimeException("Invalid MMTSJ type '"+tstype.toString()+"'.");
+		
+		//create output matrix block
+		if( out == null )
+			out = new MatrixBlock(clen, clen, false);
+		else
+			out.reset(clen, clen, false);
+		out.allocateDenseBlock();
+		
+		if( !isEmptyBlock(false) ) {
+			//compute matrix mult
+			try {
+				ExecutorService pool = Executors.newFixedThreadPool( k );
+				ArrayList<MatrixMultTransposeTask> tasks = new ArrayList<MatrixMultTransposeTask>();
+				int blklen = (int)(Math.ceil((double)clen/(2*k)));
+				for( int i=0; i<2*k & i*blklen<clen; i++ )
+					tasks.add(new MatrixMultTransposeTask(_colGroups, out, i*blklen, Math.min((i+1)*blklen, clen)));
+				List<Future<Object>> ret = pool.invokeAll(tasks);
+				for( Future<Object> tret : ret )
+					tret.get(); //check for errors
+				pool.shutdown();
+			}
+			catch(Exception ex) {
+				throw new DMLRuntimeException(ex);
+			}
+			
+			// post-processing
+			out.recomputeNonZeros();
+		}
+		
+		if( LOG.isDebugEnabled() )
+			LOG.debug("Compressed TSMM k="+k+" in "+time.stop());
+		
+		return out;
+	}
+
+	
+	/**
+	 * Multiply this matrix block by a column vector on the right.
+	 * 
+	 * @param vector
+	 *            right-hand operand of the multiplication
+	 * @param result
+	 *            buffer to hold the result; must have the appropriate size
+	 *            already
+	 */
+	private void rightMultByVector(MatrixBlock vector, MatrixBlock result)
+		throws DMLRuntimeException 
+	{
+		// initialize and allocate the result
+		result.allocateDenseBlock();
+
+		// delegate matrix-vector operation to each column group
+		for( ColGroup grp : _colGroups )
+			if( grp instanceof ColGroupUncompressed ) //overwrites output
+				grp.rightMultByVector(vector, result, 0, result.getNumRows());
+		for( ColGroup grp : _colGroups )
+			if( !(grp instanceof ColGroupUncompressed) ) //adds to output
+				grp.rightMultByVector(vector, result, 0, result.getNumRows());
+		
+		// post-processing
+		result.recomputeNonZeros();
+	}
+
+	/**
+	 * Multi-threaded version of rightMultByVector.
+	 * 
+	 * @param vector
+	 * @param result
+	 * @param k
+	 * @throws DMLRuntimeException
+	 */
+	private void rightMultByVector(MatrixBlock vector, MatrixBlock result, int k)
+		throws DMLRuntimeException 
+	{
+		// initialize and allocate the result
+		result.allocateDenseBlock();
+
+		//multi-threaded execution of all groups
+		try {
+			ExecutorService pool = Executors.newFixedThreadPool( k );
+			int rlen = getNumRows();
+			int seqsz = BitmapEncoder.BITMAP_BLOCK_SZ;
+			int blklen = (int)(Math.ceil((double)rlen/k));
+			blklen += (blklen%seqsz != 0)?seqsz-blklen%seqsz:0;
+			ArrayList<RightMatrixMultTask> tasks = new ArrayList<RightMatrixMultTask>();
+			for( int i=0; i<k & i*blklen<getNumRows(); i++ )
+				tasks.add(new RightMatrixMultTask(_colGroups, vector, result, i*blklen, Math.min((i+1)*blklen,rlen)));
+			pool.invokeAll(tasks);	
+			pool.shutdown();
+		}
+		catch(Exception ex) {
+			throw new DMLRuntimeException(ex);
+		}
+		
+		// post-processing
+		result.recomputeNonZeros();
+	}
+	
+	/**
+	 * Multiply this matrix block by the transpose of a column vector (i.e.
+	 * t(v)%*%X)
+	 * 
+	 * @param vector
+	 *            left-hand operand of the multiplication
+	 * @param result
+	 *            buffer to hold the result; must have the appropriate size
+	 *            already
+	 */
+	private static void leftMultByVectorTranspose(List<ColGroup> colGroups, MatrixBlock vector, MatrixBlock result, boolean doTranspose) 
+		throws DMLRuntimeException 
+	{
+		//transpose vector if required
+		MatrixBlock rowVector = vector;
+		if (doTranspose) {
+			rowVector = new MatrixBlock(1, vector.getNumRows(), false);
+			LibMatrixReorg.transpose(vector, rowVector);
+		}
+		
+		// initialize and allocate the result
+		result.reset();
+		result.allocateDenseBlock();
+		
+		// delegate matrix-vector operation to each column group
+		for (ColGroup grp : colGroups) {			
+			grp.leftMultByRowVector(rowVector, result);
+		}
+
+		// post-processing
+		result.recomputeNonZeros();
+	}
+	
+	/**
+	 * Multi-thread version of leftMultByVectorTranspose.
+	 * 
+	 * @param vector
+	 * @param result
+	 * @param doTranspose
+	 * @param k
+	 * @throws DMLRuntimeException
+	 */
+	private static void leftMultByVectorTranspose(List<ColGroup> colGroups,MatrixBlock vector, MatrixBlock result, boolean doTranspose, int k) 
+		throws DMLRuntimeException 
+	{
+		int kuc = Math.max(1, k - colGroups.size() + 1);
+		
+		//transpose vector if required
+		MatrixBlock rowVector = vector;
+		if (doTranspose) {
+			rowVector = new MatrixBlock(1, vector.getNumRows(), false);
+			LibMatrixReorg.transpose(vector, rowVector);
+		}
+		
+		// initialize and allocate the result
+		result.reset();
+		result.allocateDenseBlock();
+
+		//multi-threaded execution
+		try {
+			ExecutorService pool = Executors.newFixedThreadPool( Math.min(colGroups.size(), k) );
+			ArrayList<LeftMatrixMultTask> tasks = new ArrayList<LeftMatrixMultTask>();
+			for( ColGroup grp : colGroups )
+				tasks.add(new LeftMatrixMultTask(grp, rowVector, result, kuc));
+			pool.invokeAll(tasks);	
+			pool.shutdown();
+		}
+		catch(Exception ex) {
+			throw new DMLRuntimeException(ex);
+		}
+
+		// post-processing
+		result.recomputeNonZeros();
+	}
+
+	/**
+	 * 
+	 * @param result
+	 * @throws DMLRuntimeException
+	 */
+	private static void leftMultByTransposeSelf(ArrayList<ColGroup> groups, MatrixBlock result, int cl, int cu)
+		throws DMLRuntimeException 
+	{
+		final int numRows = groups.get(0).getNumRows();
+		final int numGroups = groups.size();		
+		
+		//preallocated dense matrix block
+		MatrixBlock lhs = new MatrixBlock(numRows, 1, false);
+		lhs.allocateDenseBlock();
+		
+		//approach: for each colgroup, extract uncompressed columns one at-a-time
+		//vector-matrix multiplies against remaining col groups
+		for( int i=cl; i<cu; i++ ) 
+		{
+			//get current group and relevant col groups
+			ColGroup group = groups.get(i);	
+			int[] ixgroup = group.getColIndices();
+			List<ColGroup> tmpList = groups.subList(i, numGroups);
+			
+			//for all uncompressed lhs columns vectors
+			for( int j=0; j<ixgroup.length; j++ ) {
+				//decompress single column
+				lhs.reset(numRows, 1, false);				
+				group.decompressToBlock(lhs, j);
+				
+				if( !lhs.isEmptyBlock(false) ) {
+					//compute vector-matrix partial result
+					MatrixBlock tmpret = new MatrixBlock(1,result.getNumColumns(),false);
+					leftMultByVectorTranspose(tmpList, lhs, tmpret, true);								
+					
+					//write partial results (disjoint non-zeros)
+					LinearAlgebraUtils.copyNonZerosToRowCol(result, tmpret, ixgroup[j]);	
+				}
+			}
+		}
+	}
+	
+	/**
+	 * 
+	 * @param k
+	 * @return
+	 */
+	@SuppressWarnings("unchecked")
+	private ArrayList<ColGroup>[] createStaticTaskPartitioning(int k, boolean inclUncompressed)
+	{
+		// special case: single uncompressed col group
+		if( _colGroups.size()==1 && _colGroups.get(0) instanceof ColGroupUncompressed ){
+			return new ArrayList[0];
+		}
+		
+		// initialize round robin col group distribution
+		// (static task partitioning to reduce mem requirements/final agg)
+		int numTasks = Math.min(k, _colGroups.size());
+		ArrayList<ColGroup>[] grpParts = new ArrayList[numTasks];
+		int pos = 0;
+		for( ColGroup grp : _colGroups ){
+			if( grpParts[pos]==null )
+				grpParts[pos] = new ArrayList<ColGroup>();
+			if( inclUncompressed || !(grp instanceof ColGroupUncompressed) ) {
+				grpParts[pos].add(grp);
+				pos = (pos==numTasks-1) ? 0 : pos+1;
+			}
+		}
+		
+		return grpParts;
+	}
+	
+	/**
+	 * 
+	 * @return
+	 */
+	private ColGroupUncompressed getUncompressedColGroup()
+	{
+		for( ColGroup grp : _colGroups )
+			if( grp instanceof ColGroupUncompressed ) 
+				return (ColGroupUncompressed)grp;
+		
+		return null;
+	}
+	
+	/**
+	 * 
+	 */
+	private static class LeftMatrixMultTask implements Callable<Object> 
+	{
+		private ColGroup _group = null;
+		private MatrixBlock _vect = null;
+		private MatrixBlock _ret = null;
+		private int _kuc = 1;
+		
+		protected LeftMatrixMultTask( ColGroup group, MatrixBlock vect, MatrixBlock ret, int kuc)  {
+			_group = group;
+			_vect = vect;
+			_ret = ret;
+			_kuc = kuc;
+		}
+		
+		@Override
+		public Object call() throws DMLRuntimeException 
+		{
+			// delegate matrix-vector operation to each column group
+			if( _group instanceof ColGroupUncompressed && _kuc >1 && ColGroupBitmap.LOW_LEVEL_OPT )
+				((ColGroupUncompressed)_group).leftMultByRowVector(_vect, _ret, _kuc);
+			else
+				_group.leftMultByRowVector(_vect, _ret);
+			return null;
+		}
+	}
+	
+	/**
+	 * 
+	 */
+	private static class RightMatrixMultTask implements Callable<Object> 
+	{
+		private ArrayList<ColGroup> _groups = null;
+		private MatrixBlock _vect = null;
+		private MatrixBlock _ret = null;
+		private int _rl = -1;
+		private int _ru = -1;
+		
+		protected RightMatrixMultTask( ArrayList<ColGroup> groups, MatrixBlock vect, MatrixBlock ret, int rl, int ru)  {
+			_groups = groups;
+			_vect = vect;
+			_ret = ret;
+			_rl = rl;
+			_ru = ru;
+		}
+		
+		@Override
+		public Object call() throws DMLRuntimeException 
+		{
+			// delegate vector-matrix operation to each column group
+			for( ColGroup grp : _groups )
+				if( grp instanceof ColGroupUncompressed ) //overwrites output
+					grp.rightMultByVector(_vect, _ret, _rl, _ru);
+			for( ColGroup grp : _groups )
+				if( !(grp instanceof ColGroupUncompressed) ) //adds to output
+					grp.rightMultByVector(_vect, _ret, _rl, _ru);
+			return null;
+		}
+	}
+	
+	private static class MatrixMultTransposeTask implements Callable<Object> 
+	{
+		private ArrayList<ColGroup> _groups = null;
+		private MatrixBlock _ret = null;
+		private int _cl = -1;
+		private int _cu = -1;
+		
+		protected MatrixMultTransposeTask(ArrayList<ColGroup> groups, MatrixBlock ret, int cl, int cu)  {
+			_groups = groups;
+			_ret = ret;
+			_cl = cl;
+			_cu = cu;
+		}
+		
+		@Override
+		public Object call() throws DMLRuntimeException {
+			leftMultByTransposeSelf(_groups, _ret, _cl, _cu);
+			return null;
+		}
+	}
+	
+	private static class UnaryAggregateTask implements Callable<Object> 
+	{
+		private ArrayList<ColGroup> _groups = null;
+		private MatrixBlock _ret = null;
+		private AggregateUnaryOperator _op = null;
+		
+		protected UnaryAggregateTask( ArrayList<ColGroup> groups, MatrixBlock ret, AggregateUnaryOperator op)  {
+			_groups = groups;
+			_op = op;
+			
+			if( !(_op.indexFn instanceof ReduceRow) ) { //sum/rowSums
+				_ret = new MatrixBlock(ret.getNumRows(), ret.getNumColumns(), false);
+				_ret.allocateDenseBlock();
+			}
+			else { //colSums
+				_ret = ret;
+			}
+		}
+		
+		@Override
+		public Object call() throws DMLRuntimeException 
+		{
+			// delegate vector-matrix operation to each column group
+			for( ColGroup grp : _groups )
+				grp.unaryAggregateOperations(_op, _ret);
+			return null;
+		}
+		
+		public MatrixBlock getResult(){
+			return _ret;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/PlanningBinPacker.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/PlanningBinPacker.java b/src/main/java/org/apache/sysml/runtime/compress/PlanningBinPacker.java
new file mode 100644
index 0000000..a76223c
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/PlanningBinPacker.java
@@ -0,0 +1,191 @@
+/*
+ * 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.sysml.runtime.compress;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.commons.math3.random.RandomDataGenerator;
+
+/**
+ * Used for the finding columns to co-code
+ * 
+ */
+public class PlanningBinPacker 
+{
+	private final float _binWeight;
+	private final List<Integer> _items;
+	private final List<Float> _itemWeights;
+
+	public PlanningBinPacker(float binWeight, List<Integer> items, List<Float> itemWeights) {
+		_binWeight = binWeight;
+		_items = items;
+		_itemWeights = itemWeights;
+	}
+
+	/**
+	 * NOTE: upper bound is 17/10 OPT
+	 * 
+	 * @return key: available space, value: list of the bins that have that free space
+	 */
+	public TreeMap<Float, List<List<Integer>>> packFirstFit() {
+		return packFirstFit(_items, _itemWeights);
+	}
+
+	/**
+	 * shuffling the items to make some potential for having bins of different
+	 * sizes when consecutive columns are of close cardinalities
+	 * 
+	 * @return key: available space, value: list of the bins that have that free
+	 *         space
+	 */
+	public TreeMap<Float, List<List<Integer>>> packFirstFitShuffled() {
+		RandomDataGenerator rnd = new RandomDataGenerator();
+		int[] permutation = rnd.nextPermutation(_items.size(), _items.size());
+		List<Integer> shuffledItems = new ArrayList<Integer>(_items.size());
+		List<Float> shuffledWeights = new ArrayList<Float>(_items.size());
+		for (int ix : permutation) {
+			shuffledItems.add(_items.get(ix));
+			shuffledWeights.add(_itemWeights.get(ix));
+		}
+
+		return packFirstFit(shuffledItems, shuffledWeights);
+	}
+
+	/**
+	 * 
+	 * @param items
+	 * @param itemWeights
+	 * @return
+	 */
+	private TreeMap<Float, List<List<Integer>>> packFirstFit(List<Integer> items, List<Float> itemWeights) 
+	{
+		// when searching for a bin, the first bin in the list is used
+		TreeMap<Float, List<List<Integer>>> bins = new TreeMap<Float, List<List<Integer>>>();
+		// first bin
+		bins.put(_binWeight, createBinList());
+		int numItems = items.size();
+		for (int i = 0; i < numItems; i++) {
+			float itemWeight = itemWeights.get(i);
+			Map.Entry<Float, List<List<Integer>>> entry = bins
+					.ceilingEntry(itemWeight);
+			if (entry == null) {
+				// new bin
+				float newBinWeight = _binWeight - itemWeight;
+				List<List<Integer>> binList = bins.get(newBinWeight);
+				if (binList == null) {
+					bins.put(newBinWeight, createBinList(items.get(i)));
+				} else {
+					List<Integer> newBin = new ArrayList<Integer>();
+					newBin.add(items.get(i));
+					binList.add(newBin);
+				}
+			} else {
+				// add to the first bin in the list
+				List<Integer> assignedBin = entry.getValue().remove(0);
+				assignedBin.add(items.get(i));
+				if (entry.getValue().size() == 0)
+					bins.remove(entry.getKey());
+				float newBinWeight = entry.getKey() - itemWeight;
+				List<List<Integer>> newBinsList = bins.get(newBinWeight);
+				if (newBinsList == null) {
+					// new bin
+					bins.put(newBinWeight, createBinList(assignedBin));
+				} else {
+					newBinsList.add(assignedBin);
+				}
+			}
+		}
+		return bins;
+	}
+
+	/**
+	 * NOTE: upper bound is 11/9 OPT + 6/9 (~1.22 OPT)
+	 * 
+	 * @return
+	 */
+	public TreeMap<Float, List<List<Integer>>> packFirstFitDescending() {
+		// sort items descending based on their weights
+		Integer[] indexes = new Integer[_items.size()];
+		for (int i = 0; i < indexes.length; i++)
+			indexes[i] = i;
+		Arrays.sort(indexes, new Comparator<Integer>() {
+
+			@Override
+			public int compare(Integer o1, Integer o2) {
+				return _itemWeights.get(o1).compareTo(_itemWeights.get(o2));
+			}
+		});
+		List<Integer> sortedItems = new ArrayList<Integer>();
+		List<Float> sortedItemWeights = new ArrayList<Float>();
+		for (int i = indexes.length - 1; i >= 0; i--) {
+			sortedItems.add(_items.get(i));
+			sortedItemWeights.add(_itemWeights.get(i));
+		}
+		return packFirstFit(sortedItems, sortedItemWeights);
+	}
+
+	/**
+	 * NOTE: upper bound is 71/60 OPT + 6/9 (~1.18 OPT)
+	 * 
+	 * @return
+	 */
+	public TreeMap<Float, List<List<Integer>>> packModifiedFirstFitDescending() {
+		throw new UnsupportedOperationException("Not implemented yet!");
+	}
+
+	/**
+	 * 
+	 * @return
+	 */
+	private List<List<Integer>> createBinList() {
+		List<List<Integer>> binList = new ArrayList<List<Integer>>();
+		binList.add(new ArrayList<Integer>());
+		return binList;
+	}
+
+	/**
+	 * 
+	 * @param item
+	 * @return
+	 */
+	private List<List<Integer>> createBinList(int item) {
+		List<List<Integer>> binList = new ArrayList<List<Integer>>();
+		List<Integer> bin = new ArrayList<Integer>();
+		binList.add(bin);
+		bin.add(item);
+		return binList;
+	}
+
+	/**
+	 * 
+	 * @param bin
+	 * @return
+	 */
+	private List<List<Integer>> createBinList(List<Integer> bin) {
+		List<List<Integer>> binList = new ArrayList<List<Integer>>();
+		binList.add(bin);
+		return binList;
+	}
+}


[3/6] incubator-systemml git commit: [SYSTEMML-810] New compressed matrix blocks and operations, tests

Posted by mb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/compress/utils/LinearAlgebraUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/compress/utils/LinearAlgebraUtils.java b/src/main/java/org/apache/sysml/runtime/compress/utils/LinearAlgebraUtils.java
new file mode 100644
index 0000000..c14e3bf
--- /dev/null
+++ b/src/main/java/org/apache/sysml/runtime/compress/utils/LinearAlgebraUtils.java
@@ -0,0 +1,383 @@
+/*
+ * 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.sysml.runtime.compress.utils;
+
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+
+/**
+ * Various low-level primitives for compressed matrix blocks, some of which
+ * were copied from LibMatrixMult.
+ * 
+ */
+public class LinearAlgebraUtils {
+
+	/**
+	 * 
+	 * @param a
+	 * @param b
+	 * @param len
+	 * @return
+	 */
+	public static double dotProduct(double[] a, double[] b, final int len) 
+	{
+		double val = 0;
+		final int bn = len % 8;
+
+		// compute rest
+		for (int i = 0; i < bn; i++)
+			val += a[i] * b[i];
+
+		// unrolled 8-block (for better instruction-level parallelism)
+		for (int i = bn; i < len; i += 8) {
+			// read 64B cachelines of a and b
+			// compute cval' = sum(a * b) + cval
+			val += a[i + 0] * b[i + 0] 
+				 + a[i + 1] * b[i + 1] 
+				 + a[i + 2] * b[i + 2] 
+				 + a[i + 3] * b[i + 3] 
+				 + a[i + 4] * b[i + 4]
+				 + a[i + 5] * b[i + 5] 
+				 + a[i + 6] * b[i + 6] 
+				 + a[i + 7] * b[i + 7];
+		}
+
+		// scalar result
+		return val;
+	}
+	
+	/**
+	 * 
+	 * @param a
+	 * @param b
+	 * @param ai
+	 * @param bi
+	 * @param len
+	 * @return
+	 */
+	public static double dotProduct( double[] a, double[] b, int ai, int bi, final int len )
+	{
+		double val = 0;
+		final int bn = len%8;
+				
+		//compute rest
+		for( int i = 0; i < bn; i++, ai++, bi++ )
+			val += a[ ai ] * b[ bi ];
+		
+		//unrolled 8-block (for better instruction-level parallelism)
+		for( int i = bn; i < len; i+=8, ai+=8, bi+=8 )
+		{
+			//read 64B cachelines of a and b
+			//compute cval' = sum(a * b) + cval
+			val += a[ ai+0 ] * b[ bi+0 ]
+			     + a[ ai+1 ] * b[ bi+1 ]
+			     + a[ ai+2 ] * b[ bi+2 ]
+			     + a[ ai+3 ] * b[ bi+3 ]
+			     + a[ ai+4 ] * b[ bi+4 ]
+			     + a[ ai+5 ] * b[ bi+5 ]
+			     + a[ ai+6 ] * b[ bi+6 ]
+			     + a[ ai+7 ] * b[ bi+7 ];
+		}
+		
+		//scalar result
+		return val; 
+	}
+	
+	/**
+	 * 
+	 * @param a
+	 * @param c
+	 * @param ai
+	 * @param ci
+	 * @param len
+	 */
+	public static void vectAdd( double[] a, double[] c, int ai, int ci, final int len )
+	{
+		final int bn = len%8;
+		
+		//rest, not aligned to 8-blocks
+		for( int j = 0; j < bn; j++, ai++, ci++)
+			c[ ci ] += a[ ai ];
+		
+		//unrolled 8-block  (for better instruction-level parallelism)
+		for( int j = bn; j < len; j+=8, ai+=8, ci+=8) 
+		{
+			//read 64B cachelines of a and c
+			//compute c' = c * a
+			//write back 64B cacheline of c = c'
+			c[ ci+0 ] += a[ ai+0 ];
+			c[ ci+1 ] += a[ ai+1 ];
+			c[ ci+2 ] += a[ ai+2 ];
+			c[ ci+3 ] += a[ ai+3 ];
+			c[ ci+4 ] += a[ ai+4 ];
+			c[ ci+5 ] += a[ ai+5 ];
+			c[ ci+6 ] += a[ ai+6 ];
+			c[ ci+7 ] += a[ ai+7 ];
+		}
+	}
+	
+	/**
+	 * 
+	 * @param aval
+	 * @param b
+	 * @param c
+	 * @param bix
+	 * @param ci
+	 * @param 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;
+		}
+	}
+	
+	/**
+	 * 
+	 * @param aval
+	 * @param c
+	 * @param ci
+	 * @param len
+	 */
+	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;
+		}
+	}
+	
+	/**
+	 * 
+	 * @param aval
+	 * @param b
+	 * @param c
+	 * @param bix
+	 * @param bi
+	 * @param ci
+	 * @param len
+	 */
+	public static void vectMultiplyAdd( final double aval, double[] b, double[] c, int[] bix, final int bi, final int ci, final int len )
+	{
+		final int bn = (len-bi)%8;
+		
+		//rest, not aligned to 8-blocks
+		for( int j = bi; j < bi+bn; j++ )
+			c[ ci + bix[j] ] += aval * b[ j ];
+		
+		//unrolled 8-block (for better instruction-level parallelism)
+		for( int j = bi+bn; j < len; j+=8 )
+		{
+			c[ ci+bix[j+0] ] += aval * b[ j+0 ];
+			c[ ci+bix[j+1] ] += aval * b[ j+1 ];
+			c[ ci+bix[j+2] ] += aval * b[ j+2 ];
+			c[ ci+bix[j+3] ] += aval * b[ j+3 ];
+			c[ ci+bix[j+4] ] += aval * b[ j+4 ];
+			c[ ci+bix[j+5] ] += aval * b[ j+5 ];
+			c[ ci+bix[j+6] ] += aval * b[ j+6 ];
+			c[ ci+bix[j+7] ] += aval * b[ j+7 ];
+		}
+	}
+	
+	/**
+	 * 
+	 * @param aval
+	 * @param b
+	 * @param c
+	 * @param bi
+	 * @param ci
+	 * @param len
+	 */
+	public static void vectMultiplyAdd( final double aval, double[] b, double[] c, int bi, int ci, final int len )
+	{
+		final int bn = len%8;
+		
+		//rest, not aligned to 8-blocks
+		for( int j = 0; j < bn; j++, bi++, ci++)
+			c[ ci ] += aval * b[ bi ];
+		
+		//unrolled 8-block  (for better instruction-level parallelism)
+		for( int j = bn; j < len; j+=8, bi+=8, ci+=8) 
+		{
+			//read 64B cachelines of b and c
+			//compute c' = aval * b + c
+			//write back 64B cacheline of c = c'
+			c[ ci+0 ] += aval * b[ bi+0 ];
+			c[ ci+1 ] += aval * b[ bi+1 ];
+			c[ ci+2 ] += aval * b[ bi+2 ];
+			c[ ci+3 ] += aval * b[ bi+3 ];
+			c[ ci+4 ] += aval * b[ bi+4 ];
+			c[ ci+5 ] += aval * b[ bi+5 ];
+			c[ ci+6 ] += aval * b[ bi+6 ];
+			c[ ci+7 ] += aval * b[ bi+7 ];
+		}
+	}
+
+	/**
+	 * 
+	 * @param a
+	 * @param aix
+	 * @param ai
+	 * @param ai2
+	 * @param len
+	 * @return
+	 */
+	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] ];
+		}
+		
+		return val;
+	}
+	
+	/**
+	 * 
+	 * @param a
+	 * @param ai
+	 * @param len
+	 * @return
+	 */
+	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 = 0; j < bn; j++, ai++ )
+			val += a[ ai ];
+		
+		//unrolled 8-block (for better instruction-level parallelism)
+		for( int j = bn; j < len; j+=8, ai+=8 )
+		{
+			val += a[ ai+0 ]
+			     + a[ ai+1 ]
+			     + a[ ai+2 ]
+			     + a[ ai+3 ]
+			     + a[ ai+4 ]
+			     + a[ ai+5 ]
+			     + a[ ai+6 ]
+			     + a[ ai+7 ];
+		}
+		
+		return val;
+	}
+	
+	/**
+	 * 
+	 * @param ret
+	 */
+	public static void copyUpperToLowerTriangle( MatrixBlock ret )
+	{
+		double[] c = ret.getDenseBlock();
+		final int m = ret.getNumRows();
+		final int n = ret.getNumColumns();
+		
+		//copy symmetric values
+		for( int i=0, uix=0; i<m; i++, uix+=n )
+			for( int j=i+1, lix=j*n+i; j<n; j++, lix+=n )
+				c[ lix ] = c[ uix+j ];
+	}
+	
+	/**
+	 * 
+	 * @param ret
+	 * @param tmp
+	 * @param ix
+	 */
+	public static void copyNonZerosToRowCol( MatrixBlock ret, MatrixBlock tmp, int ix )
+	{
+		for(int i=0; i<tmp.getNumColumns(); i++) {
+			double val = tmp.quickGetValue(0, i);
+			if( val != 0 ) {
+				ret.setValueDenseUnsafe(ix, i, val);
+				ret.setValueDenseUnsafe(i, ix, val);
+			}
+		}
+	}
+	
+	/**
+	 * 
+	 * @param a
+	 * @param x
+	 * @return the index of the closest element in a to the value x
+	 */
+	public static int getClosestK(int[] a, int x) {
+
+		int low = 0;
+		int high = a.length - 1;
+
+		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) {
+				low = mid + 1;
+			} else {
+				high = mid;
+			}
+		}
+		return high;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/functionobjects/KahanFunction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanFunction.java b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanFunction.java
index 071d75a..a85d0c0 100644
--- a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanFunction.java
+++ b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanFunction.java
@@ -42,4 +42,12 @@ public abstract class KahanFunction extends ValueFunction implements Serializabl
      * @param in The current term to be added.
      */
     public abstract void execute2(KahanObject kObj, double in);
+    
+    /**
+     * 
+     * @param kObj
+     * @param in
+     * @param count
+     */
+    public abstract void execute3(KahanObject kObj, double in, int count);
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlus.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlus.java b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlus.java
index cce4e26..77e2617 100644
--- a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlus.java
+++ b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlus.java
@@ -109,4 +109,9 @@ public class KahanPlus extends KahanFunction implements Serializable
 		double sum = in1._sum + correction;
 		in1.set(sum, correction-(sum-in1._sum)); //prevent eager JIT opt 	
 	}
+	
+	@Override
+	public void execute3(KahanObject in1, double in2, int count) {
+		execute2(in1, in2*count);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlusSq.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlusSq.java b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlusSq.java
index cc910e8..164a43b 100644
--- a/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlusSq.java
+++ b/src/main/java/org/apache/sysml/runtime/functionobjects/KahanPlusSq.java
@@ -88,16 +88,13 @@ public class KahanPlusSq extends KahanFunction implements Serializable {
         return kObj;
     }
 
-    /**
-     * Square the given term, then add to the existing sum using
-     * the Kahan summation algorithm.
-     *
-     * @param kObj A KahanObject containing the current sum and
-     *             correction factor for the Kahan summation
-     *             algorithm.
-     * @param in The current term to be squared and added.
-     */
+    @Override
     public void execute2(KahanObject kObj, double in) {
         kplus.execute2(kObj, in * in);
     }
+    
+    @Override
+    public void execute3(KahanObject kObj, double in, int count) {
+        kplus.execute3(kObj, in * in, count);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
index e054dd8..27438d9 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/LibMatrixMult.java
@@ -95,6 +95,21 @@ public class LibMatrixMult
 	public static void matrixMult(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret) 
 		throws DMLRuntimeException
 	{	
+		matrixMult(m1, m2, ret, 0, m1.rlen);
+	}
+	
+	/**
+	 * 
+	 * @param m1
+	 * @param m2
+	 * @param ret
+	 * @param rl
+	 * @param ru
+	 * @throws DMLRuntimeException
+	 */
+	public static void matrixMult(MatrixBlock m1, MatrixBlock m2, MatrixBlock ret, int rl, int ru) 
+		throws DMLRuntimeException
+	{	
 		//check inputs / outputs
 		if( m1.isEmptyBlock(false) || m2.isEmptyBlock(false) ) {
 			ret.examSparsity(); //turn empty dense into sparse
@@ -112,20 +127,20 @@ public class LibMatrixMult
 		
 		//prepare row-upper for special cases of vector-matrix
 		boolean pm2 = checkParMatrixMultRightInputRows(m1, m2, Integer.MAX_VALUE);
-		int ru = pm2 ? m2.rlen : m1.rlen; 
+		int ru2 = (pm2 && ru==m1.rlen) ? m2.rlen : ru; 
 		int cu = m2.clen;
 		
 		//core matrix mult computation
 		if( m1.isUltraSparse() || m2.isUltraSparse() )
-			matrixMultUltraSparse(m1, m2, ret, 0, ru);
+			matrixMultUltraSparse(m1, m2, ret, 0, ru2);
 		else if(!m1.sparse && !m2.sparse)
-			matrixMultDenseDense(m1, m2, ret, tm2, pm2, 0, ru, 0, cu);
+			matrixMultDenseDense(m1, m2, ret, tm2, pm2, 0, ru2, 0, cu);
 		else if(m1.sparse && m2.sparse)
-			matrixMultSparseSparse(m1, m2, ret, pm2, 0, ru);
+			matrixMultSparseSparse(m1, m2, ret, pm2, 0, ru2);
 		else if(m1.sparse)
-			matrixMultSparseDense(m1, m2, ret, pm2, 0, ru);
+			matrixMultSparseDense(m1, m2, ret, pm2, 0, ru2);
 		else
-			matrixMultDenseSparse(m1, m2, ret, pm2, 0, ru);
+			matrixMultDenseSparse(m1, m2, ret, pm2, 0, ru2);
 		
 		//post-processing: nnz/representation
 		if( !ret.sparse )

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
index cfff1f8..720aed1 100644
--- a/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/matrix/data/MatrixBlock.java
@@ -1282,7 +1282,7 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	 * @param cu 	column upper index, 0-based, inclusive
 	 * @return
 	 */
-	protected long recomputeNonZeros(int rl, int ru, int cl, int cu)
+	public long recomputeNonZeros(int rl, int ru, int cl, int cu)
 	{
 		if( sparse && sparseBlock!=null ) //SPARSE (max long)
 		{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicCompressionTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicCompressionTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicCompressionTest.java
new file mode 100644
index 0000000..2ec2f61
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicCompressionTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicCompressionTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runCompressionTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runCompressionTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runCompressionTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runCompressionTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runCompressionTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//decompress the compressed matrix block
+			MatrixBlock tmp = cmb.decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(mb);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(tmp);
+			TestUtils.compareMatrices(d1, d2, rows, cols, 0);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixAppendTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixAppendTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixAppendTest.java
new file mode 100644
index 0000000..93324b3
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixAppendTest.java
@@ -0,0 +1,176 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicMatrixAppendTest extends AutomatedTestBase
+{	
+	private static final int rows = 2071;
+	private static final int cols1 = 10;
+	private static final int cols2 = 1;	
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixAppendTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixAppendTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixAppendTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixAppendTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixAppendTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols1, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(rows, cols2, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			MatrixBlock ret1 = (MatrixBlock)mb.appendOperations(vector, new MatrixBlock());
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = cmb.appendOperations(vector, new MatrixBlock());
+			if( compress )
+				ret2 = ((CompressedMatrixBlock)ret2).decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, cols1+cols2, 0);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixMultChainTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixMultChainTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixMultChainTest.java
new file mode 100644
index 0000000..8f17f91
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixMultChainTest.java
@@ -0,0 +1,245 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.lops.MapMultChain.ChainType;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicMatrixMultChainTest extends AutomatedTestBase
+{	
+	private static final int rows = 2701;
+	private static final int cols = 14;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testEmptyNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testSparseConstDataNoWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtXv, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testEmptyNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtXv, false);
+	}
+	
+	@Test
+	public void testDenseRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testEmptyWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testSparseConstDataWeightsCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtwXv, true);
+	}
+	
+	@Test
+	public void testDenseRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testEmptyWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.EMPTY, ValueType.RAND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.RAND_ROUND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.RAND_ROUND, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testDenseConstDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.DENSE, ValueType.CONST, ChainType.XtwXv, false);
+	}
+	
+	@Test
+	public void testSparseConstDataWeightsNoCompression() {
+		runMatrixMultChainTest(SparsityType.SPARSE, ValueType.CONST, ChainType.XtwXv, false);
+	}
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixMultChainTest(SparsityType sptype, ValueType vtype, ChainType ctype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector1 = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 0, 1, 1.0, 3));
+			MatrixBlock vector2 = (ctype==ChainType.XtwXv)? DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(rows, 1, 0, 1, 1.0, 3)) : null;
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			MatrixBlock ret1 = (MatrixBlock)mb.chainMatrixMultOperations(vector1, vector2, new MatrixBlock(), ctype);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.chainMatrixMultOperations(vector1, vector2, new MatrixBlock(), ctype);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, cols, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixTransposeSelfMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixTransposeSelfMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixTransposeSelfMultTest.java
new file mode 100644
index 0000000..ff2a103
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixTransposeSelfMultTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.lops.MMTSJ.MMTSJType;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicMatrixTransposeSelfMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runTransposeSelfMatrixMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			MatrixBlock ret1 = mb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = cmb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, cols, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixVectorMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixVectorMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixVectorMultTest.java
new file mode 100644
index 0000000..29b467d
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicMatrixVectorMultTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateBinaryOperator;
+import org.apache.sysml.runtime.matrix.operators.AggregateOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+
+/**
+ * 
+ */
+public class BasicMatrixVectorMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runMatrixVectorMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runMatrixVectorMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runMatrixVectorMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			MatrixBlock vector = DataConverter.convertToMatrixBlock(
+					TestUtils.generateTestMatrix(cols, 1, 1, 1, 1.0, 3));
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			AggregateOperator aop = new AggregateOperator(0, Plus.getPlusFnObject());
+			AggregateBinaryOperator abop = new AggregateBinaryOperator(Multiply.getMultiplyFnObject(), aop);
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateBinaryOperations(mb, vector, new MatrixBlock(), abop);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateBinaryOperations(cmb, vector, new MatrixBlock(), abop);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, 1, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsSparseUnsafeTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsSparseUnsafeTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsSparseUnsafeTest.java
new file mode 100644
index 0000000..55497a6
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsSparseUnsafeTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Plus;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.RightScalarOperator;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicScalarOperationsSparseUnsafeTest extends AutomatedTestBase
+{	
+	private static final int rows = 1321;
+	private static final int cols = 37;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runScalarOperationsTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runScalarOperationsTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runScalarOperationsTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-scalar uncompressed
+			ScalarOperator sop = new RightScalarOperator(Plus.getPlusFnObject(), 7);
+			MatrixBlock ret1 = (MatrixBlock)mb.scalarOperations(sop, new MatrixBlock());
+			
+			//matrix-scalar compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.scalarOperations(sop, new MatrixBlock());
+			if( compress )
+				ret2 = ((CompressedMatrixBlock)ret2).decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsTest.java
new file mode 100644
index 0000000..ec708a7
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicScalarOperationsTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.functionobjects.Multiply;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.RightScalarOperator;
+import org.apache.sysml.runtime.matrix.operators.ScalarOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicScalarOperationsTest extends AutomatedTestBase
+{	
+	private static final int rows = 1321;
+	private static final int cols = 37;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runScalarOperationsTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runScalarOperationsTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runScalarOperationsTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runScalarOperationsTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runScalarOperationsTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-scalar uncompressed
+			ScalarOperator sop = new RightScalarOperator(Multiply.getMultiplyFnObject(), 7);
+			MatrixBlock ret1 = (MatrixBlock)mb.scalarOperations(sop, new MatrixBlock());
+			
+			//matrix-scalar compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.scalarOperations(sop, new MatrixBlock());
+			if( compress )
+				ret2 = ((CompressedMatrixBlock)ret2).decompress();
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, rows, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicTransposeSelfLeftMatrixMultTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicTransposeSelfLeftMatrixMultTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicTransposeSelfLeftMatrixMultTest.java
new file mode 100644
index 0000000..dfbe453
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicTransposeSelfLeftMatrixMultTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.lops.MMTSJ.MMTSJType;
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicTransposeSelfLeftMatrixMultTest extends AutomatedTestBase
+{	
+	private static final int rows = 1023;
+	private static final int cols = 20;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testDenseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testSparseRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testEmptyCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, true);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, true);
+	}
+	
+	@Test
+	public void testDenseConstantDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testSparseConstDataCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, true);
+	}
+	
+	@Test
+	public void testDenseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testSparseRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testEmptyNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.EMPTY, ValueType.RAND, false);
+	}
+	
+	@Test
+	public void testDenseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testSparseRoundRandDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.RAND_ROUND, false);
+	}
+	
+	@Test
+	public void testDenseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.DENSE, ValueType.CONST, false);
+	}
+	
+	@Test
+	public void testSparseConstDataNoCompression() {
+		runTransposeSelfMatrixMultTest(SparsityType.SPARSE, ValueType.CONST, false);
+	}
+	
+
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runTransposeSelfMatrixMultTest(SparsityType sptype, ValueType vtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed
+			MatrixBlock ret1 = mb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = cmb.transposeSelfMatrixMultOperations(new MatrixBlock(), MMTSJType.LEFT);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			TestUtils.compareMatrices(d1, d2, cols, cols, 0.0000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/16e7b1c8/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicUnaryAggregateTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicUnaryAggregateTest.java b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicUnaryAggregateTest.java
new file mode 100644
index 0000000..960008d
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/compress/BasicUnaryAggregateTest.java
@@ -0,0 +1,544 @@
+/*
+ * 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.sysml.test.integration.functions.compress;
+
+import org.apache.sysml.runtime.compress.CompressedMatrixBlock;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.operators.AggregateUnaryOperator;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class BasicUnaryAggregateTest extends AutomatedTestBase
+{	
+	private static final int rows = 2071;
+	private static final int cols1 = 10;
+	private static final double sparsity1 = 0.9;
+	private static final double sparsity2 = 0.1;
+	private static final double sparsity3 = 0.0;
+	
+	public enum SparsityType {
+		DENSE,
+		SPARSE,
+		EMPTY,
+	}
+	
+	public enum ValueType {
+		RAND,
+		RAND_ROUND,
+		CONST,
+	}
+	
+	public enum AggType {
+		ROWSUMS,
+		COLSUMS,
+		SUM,
+		ROWSUMSSQ,
+		COLSUMSSQ,
+		SUMSQ
+	}
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testRowSumsDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMS, true);
+	}
+	
+	@Test
+	public void testRowSumsDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testRowSumsSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMS, true);
+	}
+	
+	@Test
+	public void testColSumsDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMS, false);
+	}
+	
+	@Test
+	public void testColSumsSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMS, false);
+	}
+
+	@Test
+	public void testSumDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUM, true);
+	}
+	
+	@Test
+	public void testSumDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testSumSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUM, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMSSQ, true);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testRowSumsSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.ROWSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMSSQ, true);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.COLSUMSSQ, false);
+	}
+	
+	@Test
+	public void testColSumsSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.COLSUMSSQ, false);
+	}
+
+	@Test
+	public void testSumSqDenseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqEmptyCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseRoundRandDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseConstantDataCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqSparseConstDataCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUMSQ, true);
+	}
+	
+	@Test
+	public void testSumSqDenseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqEmptyNoCompression() {
+		runUnaryAggregateTest(SparsityType.EMPTY, ValueType.RAND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqDenseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.RAND_ROUND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseRoundRandDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.RAND_ROUND, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqDenseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.DENSE, ValueType.CONST, AggType.SUMSQ, false);
+	}
+	
+	@Test
+	public void testSumSqSparseConstDataNoCompression() {
+		runUnaryAggregateTest(SparsityType.SPARSE, ValueType.CONST, AggType.SUMSQ, false);
+	}
+	
+	/**
+	 * 
+	 * @param mb
+	 */
+	private void runUnaryAggregateTest(SparsityType sptype, ValueType vtype, AggType aggtype, boolean compress)
+	{
+		try
+		{
+			//prepare sparsity for input data
+			double sparsity = -1;
+			switch( sptype ){
+				case DENSE: sparsity = sparsity1; break;
+				case SPARSE: sparsity = sparsity2; break;
+				case EMPTY: sparsity = sparsity3; break;
+			}
+			
+			//generate input data
+			double min = (vtype==ValueType.CONST)? 10 : -10;
+			double[][] input = TestUtils.generateTestMatrix(rows, cols1, min, 10, sparsity, 7);
+			if( vtype==ValueType.RAND_ROUND )
+				input = TestUtils.round(input);
+			MatrixBlock mb = DataConverter.convertToMatrixBlock(input);
+			
+			//prepare unary aggregate operator
+			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;
+			}
+			
+			//compress given matrix block
+			CompressedMatrixBlock cmb = new CompressedMatrixBlock(mb);
+			if( compress )
+				cmb.compress();
+			
+			//matrix-vector uncompressed						
+			MatrixBlock ret1 = (MatrixBlock)mb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, 1000, null, true);
+			
+			//matrix-vector compressed
+			MatrixBlock ret2 = (MatrixBlock)cmb.aggregateUnaryOperations(auop, new MatrixBlock(), 1000, 1000, null, true);
+			
+			//compare result with input
+			double[][] d1 = DataConverter.convertToDoubleMatrix(ret1);
+			double[][] d2 = DataConverter.convertToDoubleMatrix(ret2);
+			int dim1 = (aggtype == AggType.ROWSUMS)?rows:1;
+			int dim2 = (aggtype == AggType.COLSUMS)?cols1:1;
+			TestUtils.compareMatrices(d1, d2, dim1, dim2, 0.00000000001);
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+}