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:27 UTC

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

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);
+		}
+	}
+}