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

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