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 2018/08/09 06:15:47 UTC

systemml git commit: [SYSTEMML-2479] Fix MNC estimator (nnz, op, plus), incl various tests

Repository: systemml
Updated Branches:
  refs/heads/master 9ae0a9167 -> b7f569bd0


[SYSTEMML-2479] Fix MNC estimator (nnz, op, plus), incl various tests

Closes #823.


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

Branch: refs/heads/master
Commit: b7f569bd001c0799430857e4d00af69da93c691d
Parents: 9ae0a91
Author: Johanna Sommer <jo...@mail-sommer.com>
Authored: Wed Aug 8 23:13:19 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Wed Aug 8 23:13:20 2018 -0700

----------------------------------------------------------------------
 .../sysml/hops/estim/EstimatorDensityMap.java   |   2 +-
 .../hops/estim/EstimatorMatrixHistogram.java    |  27 +-
 .../hops/rewrite/RewriteGPUSpecificOps.java     |   1 +
 .../integration/functions/estim/OpBindTest.java | 159 ++++++++++++
 .../functions/estim/OpElemWTest.java            | 161 ++++++++++++
 .../integration/functions/estim/OpSingle.java   | 252 +++++++++++++++++++
 .../functions/estim/ZPackageSuite.java          |   3 +
 7 files changed, 593 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java b/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
index d752fe7..66c5826 100644
--- a/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
+++ b/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
@@ -83,7 +83,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 		DensityMap m1Map = new DensityMap(m1, _b);
 		DensityMap m2Map = (m1 == m2) ? //self product
 			m1Map : new DensityMap(m2, _b);
-		DensityMap outMap = estimIntern(m1Map, m2Map, OpCode.MM);
+		DensityMap outMap = estimIntern(m1Map, m2Map, op);
 		return OptimizerUtils.getSparsity( //aggregate output histogram
 			outMap.getNumRowsOrig(), outMap.getNumColumnsOrig(), outMap.getNonZeros());
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/main/java/org/apache/sysml/hops/estim/EstimatorMatrixHistogram.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/estim/EstimatorMatrixHistogram.java b/src/main/java/org/apache/sysml/hops/estim/EstimatorMatrixHistogram.java
index 637c7f3..a299c45 100644
--- a/src/main/java/org/apache/sysml/hops/estim/EstimatorMatrixHistogram.java
+++ b/src/main/java/org/apache/sysml/hops/estim/EstimatorMatrixHistogram.java
@@ -97,23 +97,28 @@ public class EstimatorMatrixHistogram extends SparsityEstimator
 	
 	private double estimIntern(MatrixHistogram h1, MatrixHistogram h2, OpCode op) {
 		double msize = (double)h1.getRows()*h1.getCols();
-		
 		switch (op) {
 			case MM:
 				return estimInternMM(h1, h2);
-			case MULT:
+			case MULT: {
 				final long N1 = h1.getNonZeros();
 				final long N2 = h2.getNonZeros();
 				final long scale = IntStream.range(0, h1.getCols())
 					.mapToLong(j -> (long)h1.cNnz[j] * h2.cNnz[j]).sum();
-				return IntStream.range(0, h1.getRows()).mapToLong(
-					i -> (long)h1.rNnz[i] * h2.rNnz[i] * scale / N1 / N2).sum() / msize;
-			case PLUS:
-				return Math.min(
-					IntStream.range(0, h1.getRows()).mapToDouble(i -> h1.rNnz[i]/msize 
-						+ h2.rNnz[i]/msize - h1.rNnz[i]/msize * h2.rNnz[i]/msize).sum(),
-					IntStream.range(0, h1.getCols()).mapToDouble(i -> h1.cNnz[i]/msize 
-						+ h2.cNnz[i]/msize - h1.cNnz[i]/msize * h2.cNnz[i]/msize).sum());
+				return IntStream.range(0, h1.getRows())
+					.mapToLong(i -> (long)h1.rNnz[i] * h2.rNnz[i] * scale / N1 / N2) //collisions
+					.sum() / msize;
+			}
+			case PLUS: {
+				final long N1 = h1.getNonZeros();
+				final long N2 = h2.getNonZeros();
+				final long scale = IntStream.range(0, h1.getCols())
+					.mapToLong(j -> (long)h1.cNnz[j] * h2.cNnz[j]).sum();
+				return IntStream.range(0, h1.getRows())
+					.mapToLong(i -> (long)h1.rNnz[i] + h2.rNnz[i] //all minus collisions
+						- (long)h1.rNnz[i] * h2.rNnz[i] * scale / N1 / N2)
+					.sum() / msize;
+			}
 			case EQZERO:
 				return OptimizerUtils.getSparsity(h1.getRows(), h1.getCols(),
 					(long)h1.getRows() * h1.getCols() - h1.getNonZeros());
@@ -301,7 +306,7 @@ public class EstimatorMatrixHistogram extends SparsityEstimator
 		public long getNonZeros() {
 			return getRows() < getCols() ?
 				IntStream.range(0, getRows()).mapToLong(i-> rNnz[i]).sum() :
-				IntStream.range(0, getRows()).mapToLong(i-> cNnz[i]).sum();
+				IntStream.range(0, getCols()).mapToLong(i-> cNnz[i]).sum();
 		}
 		
 		public static MatrixHistogram deriveOutputHistogram(MatrixHistogram h1, MatrixHistogram h2, double spOut, OpCode op) {

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
index 1ee23f8..2a1699d 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteGPUSpecificOps.java
@@ -566,6 +566,7 @@ public class RewriteGPUSpecificOps extends HopRewriteRule {
 	 * @param pos position
 	 * @return a new FunctionOp or hi
 	 */
+	@SuppressWarnings("unused")
 	private static Hop batchNormTrain(ArrayList<Hop> roots, Hop parent, Hop hi, int pos) 
 	{		
 		// norm = bias_multiply(bias_add(X, -mean), 1/sqrt(var+eps))

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/test/java/org/apache/sysml/test/integration/functions/estim/OpBindTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/estim/OpBindTest.java b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpBindTest.java
new file mode 100644
index 0000000..a8f9e49
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpBindTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.estim;
+
+import org.junit.Test;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysml.hops.estim.EstimatorBasicAvg;
+import org.apache.sysml.hops.estim.EstimatorBasicWorst;
+import org.apache.sysml.hops.estim.EstimatorMatrixHistogram;
+import org.apache.sysml.hops.estim.SparsityEstimator;
+import org.apache.sysml.hops.estim.SparsityEstimator.OpCode;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+
+/**
+ * this is the basic operation check for all estimators with single operations
+ */
+public class OpBindTest extends AutomatedTestBase 
+{
+	private final static int m = 600;
+	private final static int k = 300;
+	private final static int n = 100;
+	private final static double[] sparsity = new double[]{0.2, 0.4};
+//	private final static OpCode mult = OpCode.MULT;
+//	private final static OpCode plus = OpCode.PLUS;
+	private final static OpCode rbind = OpCode.RBIND;
+	private final static OpCode cbind = OpCode.CBIND;
+//	private final static OpCode eqzero = OpCode.EQZERO;
+//	private final static OpCode diag = OpCode.DIAG;
+//	private final static OpCode neqzero = OpCode.NEQZERO;
+//	private final static OpCode trans = OpCode.TRANS;
+//	private final static OpCode reshape = OpCode.RESHAPE;
+
+	@Override
+	public void setUp() {
+		//do  nothing
+	}
+	
+	//Average Case
+	@Test
+	public void testAvgRbind() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, n, sparsity, rbind);
+	}
+	
+	@Test
+	public void testAvgCbind() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, n, sparsity, cbind);
+	}
+	
+	//Worst Case
+	@Test
+	public void testWorstRbind() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, n, sparsity, rbind);
+	}
+	
+	@Test
+	public void testWorstCbind() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, n, sparsity, cbind);
+	}
+	
+	//DensityMap
+	/*@Test
+	public void testDMCaserbind() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, n, sparsity, rbind);
+	}
+	
+	@Test
+	public void testDMCasecbind() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, n, sparsity, cbind);
+	}*/
+	
+	//MNC
+	@Test
+	public void testMNCRbind() {
+		runSparsityEstimateTest(new EstimatorMatrixHistogram(), m, k, n, sparsity, rbind);
+	}
+		
+	@Test
+	public void testMNCCbind() {
+		runSparsityEstimateTest(new EstimatorMatrixHistogram(), m, k, n, sparsity, cbind);
+	}
+
+	//Bitset
+	/*@Test
+	public void testBitsetCaserbind() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, sparsity, rbind);
+	}
+		
+	@Test
+	public void testBitsetCasecbind() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, sparsity, cbind);
+	}
+		
+	//Layered Graph
+	@Test
+	public void testLGCaserbind() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, n, sparsity, rbind);
+	}
+			
+	@Test
+	public void testLGCasecbind() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, n, sparsity, cbind);
+	}
+		
+	//Sample
+	@Test
+	public void testSampleCaserbind() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, n, sparsity, rbind);
+	}
+			
+	@Test
+	public void testSampleCasecbind() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, n, sparsity, cbind);
+	}*/
+	
+	
+	private void runSparsityEstimateTest(SparsityEstimator estim, int m, int k, int n, double[] sp, OpCode op) {
+		MatrixBlock m1;
+		MatrixBlock m2;
+		MatrixBlock m3 = new MatrixBlock();
+		double est = 0;
+		switch(op) {
+			case RBIND:
+				m1 = MatrixBlock.randOperations(m, k, sp[0], 1, 1, "uniform", 3);
+				m2 = MatrixBlock.randOperations(n, k, sp[1], 1, 1, "uniform", 3);
+				m1.append(m2, m3, false);
+				est = estim.estim(m1, m2, op);
+				break;
+			case CBIND:
+				m1 = MatrixBlock.randOperations(m, k, sp[0], 1, 1, "uniform", 3);
+				m2 = MatrixBlock.randOperations(m, n, sp[1], 1, 1, "uniform", 3);
+				m1.append(m2, m3);
+				est = estim.estim(m1, m2, op);
+				break;
+			default:
+				throw new NotImplementedException();
+		}
+		//compare estimated and real sparsity
+		TestUtils.compareScalars(est, m3.getSparsity(), (estim instanceof EstimatorBasicWorst) ? 5e-1 : 1e-2);
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/test/java/org/apache/sysml/test/integration/functions/estim/OpElemWTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/estim/OpElemWTest.java b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpElemWTest.java
new file mode 100644
index 0000000..29cd607
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpElemWTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.estim;
+
+import org.junit.Test;
+import org.apache.sysml.runtime.matrix.operators.BinaryOperator;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.sysml.hops.estim.EstimatorBasicAvg;
+import org.apache.sysml.hops.estim.EstimatorBasicWorst;
+import org.apache.sysml.hops.estim.EstimatorDensityMap;
+import org.apache.sysml.hops.estim.EstimatorMatrixHistogram;
+import org.apache.sysml.hops.estim.SparsityEstimator;
+import org.apache.sysml.hops.estim.SparsityEstimator.OpCode;
+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.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+
+/**
+ * this is the basic operation check for all estimators with single operations
+ */
+public class OpElemWTest extends AutomatedTestBase 
+{
+	//TODO experiment with m>2n for MNC (currently suboptimal accuracy)
+	private final static int m = 600;
+	private final static int n = 700;
+	private final static double[] sparsity = new double[]{0.1, 0.04};
+	private final static OpCode mult = OpCode.MULT;
+	private final static OpCode plus = OpCode.PLUS;
+//	private final static OpCode rbind = OpCode.RBIND;
+//	private final static OpCode cbind = OpCode.CBIND;
+//	private final static OpCode eqzero = OpCode.EQZERO;
+//	private final static OpCode diag = OpCode.DIAG;
+//	private final static OpCode neqzero = OpCode.NEQZERO;
+//	private final static OpCode trans = OpCode.TRANS;
+//	private final static OpCode reshape = OpCode.RESHAPE;
+
+	@Override
+	public void setUp() {
+		//do  nothing
+	}
+	//Average Case
+	@Test
+	public void testAvgMult() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, n, sparsity, mult);
+	}
+	
+	@Test
+	public void testAvgPlus() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, n, sparsity, plus);
+	}
+	
+	//Worst Case
+	@Test
+	public void testWorstMult() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, n, sparsity, mult);
+	}
+	
+	@Test
+	public void testWorstPlus() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, n, sparsity, plus);
+	}
+	
+	//DensityMap
+	@Test
+	public void testDMMult() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, n, sparsity, mult);
+	}
+	
+	@Test
+	public void testDMPlus() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, n, sparsity, plus);
+	}
+	
+	//MNC
+	@Test
+	public void testMNCMult() {
+		runSparsityEstimateTest(new EstimatorMatrixHistogram(), m, n, sparsity, mult);
+	}
+	
+	@Test
+	public void testMNCPlus() {
+		runSparsityEstimateTest(new EstimatorMatrixHistogram(), m, n, sparsity, plus);
+	}
+	
+	//Bitset
+	/*@Test
+	public void testBitsetCasemult() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, sparsity, mult);
+	}
+	
+	@Test
+	public void testBitsetCaseplus() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, sparsity, plus);
+	}
+	
+	//Layered Graph
+	@Test
+	public void testLGCasemult() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, n, sparsity, mult);
+	}
+		
+	@Test
+	public void testLGCaseplus() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, n, sparsity, plus);
+	}
+	
+	//Sample
+	@Test
+	public void testSampleCasemult() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, n, sparsity, mult);
+	}
+		
+	@Test
+	public void testSampleCaseplus() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, n, sparsity, plus);
+	}*/
+	
+	
+	private void runSparsityEstimateTest(SparsityEstimator estim, int m, int n, double[] sp, OpCode op) {
+		MatrixBlock m1 = MatrixBlock.randOperations(m, n, sp[0], 1, 1, "uniform", 3);
+		MatrixBlock m2 = MatrixBlock.randOperations(m, n, sp[1], 1, 1, "uniform", 3);
+		MatrixBlock m3 = new MatrixBlock();
+		BinaryOperator bOp;
+		double est = 0;
+		switch(op) {
+			case MULT:
+				bOp = new BinaryOperator(Multiply.getMultiplyFnObject());
+				m1.binaryOperations(bOp, m2, m3);
+				est = estim.estim(m1, m2, op);
+				break;
+			case PLUS:
+				bOp = new BinaryOperator(Plus.getPlusFnObject());
+				m1.binaryOperations(bOp, m2, m3);
+				est = estim.estim(m1, m2, op);
+				break;
+			default:
+				throw new NotImplementedException();
+		}
+		//compare estimated and real sparsity
+		TestUtils.compareScalars(est, m3.getSparsity(), (estim instanceof EstimatorBasicWorst) ? 5e-1 : 1e-3);
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/test/java/org/apache/sysml/test/integration/functions/estim/OpSingle.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/estim/OpSingle.java b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpSingle.java
new file mode 100644
index 0000000..1756a8d
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/estim/OpSingle.java
@@ -0,0 +1,252 @@
+/*
+ * 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.estim;
+
+import org.junit.Test;
+import org.apache.sysml.hops.estim.EstimatorBasicAvg;
+import org.apache.sysml.hops.estim.EstimatorBasicWorst;
+import org.apache.sysml.hops.estim.EstimatorBitsetMM;
+import org.apache.sysml.hops.estim.EstimatorDensityMap;
+import org.apache.sysml.hops.estim.EstimatorLayeredGraph;
+import org.apache.sysml.hops.estim.EstimatorSample;
+import org.apache.sysml.hops.estim.SparsityEstimator;
+import org.apache.sysml.hops.estim.SparsityEstimator.OpCode;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+
+/**
+ * this is the basic operation check for all estimators with single operations
+ */
+public class OpSingle extends AutomatedTestBase 
+{
+	private final static int m = 600;
+	private final static int k = 300;
+	private final static double sparsity = 0.2;
+//	private final static OpCode mult = OpCode.MULT;
+//	private final static OpCode plus = OpCode.PLUS;
+//	private final static OpCode rbind = OpCode.RBIND;
+//	private final static OpCode cbind = OpCode.CBIND;
+	private final static OpCode eqzero = OpCode.EQZERO;
+	private final static OpCode diag = OpCode.DIAG;
+	private final static OpCode neqzero = OpCode.NEQZERO;
+	private final static OpCode trans = OpCode.TRANS;
+	private final static OpCode reshape = OpCode.RESHAPE;
+
+	@Override
+	public void setUp() {
+		//do  nothing
+	}
+	
+	//Average Case
+	@Test
+	public void testAvgCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testAvgCasediag() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testAvgCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testAvgCasetrans() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testAvgCasereshape() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, sparsity, reshape);
+	}
+	
+	//Worst Case
+	@Test
+	public void testWCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testWCasediag() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testWCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testWCasetrans() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testWCasereshape() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, sparsity, reshape);
+	} 
+	
+	//DensityMap
+	@Test
+	public void testDMCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testDMCasediag() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testDMCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testDMCasetrans() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, trans);
+	}
+		
+	@Test
+	public void testDMCasereshape() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, reshape);
+	}
+	
+	//MNC
+	@Test
+	public void testMNCCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testMNCCasediag() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testMNCCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testMNCCasetrans() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testMNCCasereshape() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, sparsity, reshape);
+	}
+	
+	//Bitset
+	@Test
+	public void testBitsetCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testBitsetCasediag() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testBitsetCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testBitsetCasetrans() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testBitsetCasereshape() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, sparsity, reshape);
+	}
+	
+	//Layered Graph
+	@Test
+	public void testLGCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testLGCasediag() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testLGCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testLGCasetans() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testLGCasereshape() {
+		runSparsityEstimateTest(new EstimatorLayeredGraph(), m, k, sparsity, reshape);
+	}
+	
+	//Sample
+	@Test
+	public void testSampleCaseeqzero() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, sparsity, eqzero);
+	}
+	
+	@Test
+	public void testSampleCasediag() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, sparsity, diag);
+	}
+	
+	@Test
+	public void testSampleCaseneqzero() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, sparsity, neqzero);
+	}
+	
+	@Test
+	public void testSampleCasetrans() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, sparsity, trans);
+	}
+	
+	@Test
+	public void testSampleCasereshape() {
+		runSparsityEstimateTest(new EstimatorSample(), m, k, sparsity, reshape);
+	}
+	
+	private void runSparsityEstimateTest(SparsityEstimator estim, int m, int k, double sp, OpCode op) {
+//		MatrixBlock m1 = MatrixBlock.randOperations(m, k, sp, 1, 1, "uniform", 3);
+//		MatrixBlock m2 = null;
+//		double est = 0;
+//		switch(op) {
+//			case EQZERO:
+//			case DIAG:
+//			case NEQZERO:
+//			case TRANS:
+//			case RESHAPE:
+//		}
+//		//compare estimated and real sparsity
+//		TestUtils.compareScalars(est, m2.getSparsity(), (estim instanceof EstimatorBasicWorst) ? 5e-1 : 1e-2);
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/b7f569bd/src/test_suites/java/org/apache/sysml/test/integration/functions/estim/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/estim/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/estim/ZPackageSuite.java
index 2842905..2e9a6d6 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/estim/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/estim/ZPackageSuite.java
@@ -26,10 +26,13 @@ import org.junit.runners.Suite;
  *  won't run two of them at once. */
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
+	OpBindTest.class,
+	OpElemWTest.class,
 	OuterProductTest.class,
 	SelfProductTest.class,
 	SquaredProductChainTest.class,
 	SquaredProductTest.class,
+	//OpSingle.class
 })