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/05/02 03:54:21 UTC

[1/3] systemml git commit: [SYSTEMML-2294] Fix robustness constrained parfor optimizer (parameters)

Repository: systemml
Updated Branches:
  refs/heads/master 45a93396a -> aa253eb7e


[SYSTEMML-2294] Fix robustness constrained parfor optimizer (parameters)

This patch fixes a subtle bug of parameter handling for the CONSTRAINED
parfor optimizer (which optimizes around user-provided forced
parameters). Although all parameters are supposed to be
case-insensitive, when used as "constrained" for example all unspecified
parameters where replaced with defaults which left the constrained
optimizer with no room for decisions. 


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

Branch: refs/heads/master
Commit: b65f8ba3349305bac48f0e498b891e9989a2fa5c
Parents: 45a9339
Author: Matthias Boehm <mb...@gmail.com>
Authored: Tue May 1 17:28:22 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Tue May 1 17:28:22 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/parser/ParForStatementBlock.java    | 7 ++++---
 .../controlprogram/parfor/opt/OptimizerConstrained.java       | 4 ++--
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/b65f8ba3/src/main/java/org/apache/sysml/parser/ParForStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ParForStatementBlock.java b/src/main/java/org/apache/sysml/parser/ParForStatementBlock.java
index dee1384..1de4bf3 100644
--- a/src/main/java/org/apache/sysml/parser/ParForStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/ParForStatementBlock.java
@@ -205,7 +205,8 @@ public class ParForStatementBlock extends ForStatementBlock
 			
 			//set defaults for all non-specified values
 			//(except if CONSTRAINT optimizer, in order to distinguish specified parameters)
-			boolean constrained = (params.containsKey( OPT_MODE ) && params.get( OPT_MODE ).equals(POptMode.CONSTRAINED.toString()));
+			boolean constrained = (params.containsKey( OPT_MODE ) 
+				&& params.get( OPT_MODE ).equalsIgnoreCase(POptMode.CONSTRAINED.name()));
 			for( String key : _paramNames )
 				if( !params.containsKey(key) )
 				{
@@ -214,7 +215,7 @@ public class ParForStatementBlock extends ForStatementBlock
 					}
 					//special treatment for degree of parallelism
 					else if( key.equals(PAR) && params.containsKey(EXEC_MODE)
-							&& params.get(EXEC_MODE).equals(PExecMode.REMOTE_MR.toString()))
+							&& params.get(EXEC_MODE).equalsIgnoreCase(PExecMode.REMOTE_MR.name()))
 					{
 						int maxPMap = InfrastructureAnalyzer.getRemoteParallelMapTasks();
 						//correction max number of reducers on yarn clusters
@@ -223,7 +224,7 @@ public class ParForStatementBlock extends ForStatementBlock
 						params.put(key, String.valueOf(maxPMap));
 					}
 					else if( key.equals(PAR) && params.containsKey(EXEC_MODE)
-							&& params.get(EXEC_MODE).equals(PExecMode.REMOTE_MR_DP.toString()) )
+							&& params.get(EXEC_MODE).equalsIgnoreCase(PExecMode.REMOTE_MR_DP.name()) )
 					{
 						int maxPRed = InfrastructureAnalyzer.getRemoteParallelReduceTasks();
 						//correction max number of reducers on yarn clusters

http://git-wip-us.apache.org/repos/asf/systemml/blob/b65f8ba3/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerConstrained.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerConstrained.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerConstrained.java
index f523cdc..3864004 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerConstrained.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerConstrained.java
@@ -312,7 +312,7 @@ public class OptimizerConstrained extends OptimizerRuleBased
 	protected void rewriteSetTaskPartitioner(OptNode pn, boolean flagNested, boolean flagLIX)
 	{
 		// constraint awareness
-		if( !pn.getParam(ParamType.TASK_PARTITIONER).equals(PTaskPartitioner.UNSPECIFIED.toString()) )
+		if( !pn.getParam(ParamType.TASK_PARTITIONER).equals(PTaskPartitioner.UNSPECIFIED.name()) )
 		{
 			ParForProgramBlock pfpb = (ParForProgramBlock) OptTreeConverter
 				.getAbstractPlanMapping().getMappedProg(pn.getID())[1];
@@ -342,7 +342,7 @@ public class OptimizerConstrained extends OptimizerRuleBased
 	@Override
 	protected void rewriteSetResultMerge( OptNode n, LocalVariableMap vars, boolean inLocal ) {
 		// constraint awareness
-		if( !n.getParam(ParamType.RESULT_MERGE).equals(PResultMerge.UNSPECIFIED.toString()) )
+		if( !n.getParam(ParamType.RESULT_MERGE).equals(PResultMerge.UNSPECIFIED.name()) )
 		{
 			ParForProgramBlock pfpb = (ParForProgramBlock) OptTreeConverter
 				    .getAbstractPlanMapping().getMappedProg(n.getID())[1];


[2/3] systemml git commit: [SYSTEMML-2295] New sparsity estimator based on bitsets (exact est)

Posted by mb...@apache.org.
[SYSTEMML-2295] New sparsity estimator based on bitsets (exact est)

This patch adds a naive yet not too uncommon sparsity estimator based on
boolean matrix multiplication, which allows to get the exact non-zero
pattern of the output (e.g., for sparse output preallocation). Compared
to the double precision matrix multiply this bitset multiply uses
implicit ANDs and bitwise ORs of entire rhs input and output rows.


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

Branch: refs/heads/master
Commit: e21f840eda9eef8a61f37717ac0b8240e4e27bb4
Parents: b65f8ba
Author: Matthias Boehm <mb...@gmail.com>
Authored: Tue May 1 20:13:02 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Tue May 1 20:13:02 2018 -0700

----------------------------------------------------------------------
 .../sysml/hops/estim/EstimatorBitsetMM.java     | 149 +++++++++++++++++++
 1 file changed, 149 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/e21f840e/src/main/java/org/apache/sysml/hops/estim/EstimatorBitsetMM.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/estim/EstimatorBitsetMM.java b/src/main/java/org/apache/sysml/hops/estim/EstimatorBitsetMM.java
new file mode 100644
index 0000000..c90fbfc
--- /dev/null
+++ b/src/main/java/org/apache/sysml/hops/estim/EstimatorBitsetMM.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.hops.estim;
+
+import java.util.BitSet;
+
+import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
+import org.apache.sysml.runtime.matrix.data.DenseBlock;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.matrix.data.SparseBlock;
+
+/**
+ * This estimator implements naive but rather common approach of boolean matrix
+ * multiplies which allows to infer the exact non-zero structure and thus is also
+ * useful for sparse result preallocation.
+ * 
+ */
+public class EstimatorBitsetMM extends SparsityEstimator
+{
+	@Override
+	public double estim(MMNode root) {
+		//recursive density map computation of non-leaf nodes
+		if( !root.getLeft().isLeaf() )
+			estim(root.getLeft()); //obtain synopsis
+		if( !root.getRight().isLeaf() )
+			estim(root.getLeft()); //obtain synopsis
+		BitsetMatrix m1Map = !root.getLeft().isLeaf() ?
+			(BitsetMatrix)root.getLeft().getSynopsis() : new BitsetMatrix(root.getLeft().getData());
+		BitsetMatrix m2Map = !root.getRight().isLeaf() ?
+			(BitsetMatrix)root.getLeft().getSynopsis() : new BitsetMatrix(root.getLeft().getData());
+		
+		//estimate output density map and sparsity via boolean matrix mult
+		BitsetMatrix outMap = m1Map.matMult(m2Map);
+		root.setSynopsis(outMap); //memoize boolean matrix
+		return OptimizerUtils.getSparsity(
+			outMap.getNumRows(), outMap.getNumColumns(), outMap.getNonZeros());
+	}
+
+	@Override
+	public double estim(MatrixBlock m1, MatrixBlock m2) {
+		BitsetMatrix m1Map = new BitsetMatrix(m1);
+		BitsetMatrix m2Map = new BitsetMatrix(m2);
+		BitsetMatrix outMap = m1Map.matMult(m2Map);
+		return OptimizerUtils.getSparsity( //aggregate output histogram
+			outMap.getNumRows(), outMap.getNumColumns(), outMap.getNonZeros());
+	}
+
+	@Override
+	public double estim(MatrixCharacteristics mc1, MatrixCharacteristics mc2) {
+		LOG.warn("Meta-data-only estimates not supported in EstimatorBitsetMM, falling back to EstimatorBasicAvg.");
+		return new EstimatorBasicAvg().estim(mc1, mc2);
+	}
+	
+	private static class BitsetMatrix {
+		private final int _rlen;
+		private final int _clen;
+		private long _nonZeros;
+		private BitSet[] _data;
+		
+		public BitsetMatrix(int rlen, int clen) {
+			_rlen = rlen;
+			_clen = clen;
+			_data = new BitSet[_rlen];
+			for(int i=0; i<_rlen; i++)
+				_data[i] = new BitSet(_clen);
+			_nonZeros = 0;
+		}
+		
+		public BitsetMatrix(MatrixBlock in) {
+			this(in.getNumRows(), in.getNumColumns());
+			init(in);
+		}
+		
+		public int getNumRows() {
+			return _rlen;
+		}
+		
+		public int getNumColumns() {
+			return _clen;
+		}
+		
+		public long getNonZeros() {
+			return _nonZeros;
+		}
+		
+		private void init(MatrixBlock in) {
+			if( in.isInSparseFormat() ) {
+				SparseBlock sblock = in.getSparseBlock();
+				for(int i=0; i<in.getNumRows(); i++) {
+					if(sblock.isEmpty(i)) continue;
+					BitSet lbs = _data[i];
+					int alen = sblock.size(i);
+					int apos = sblock.pos(i);
+					int[] aix = sblock.indexes(i);
+					for(int k=apos; k<apos+alen; k++)
+						lbs.set(aix[k]);
+				}
+			}
+			else {
+				DenseBlock dblock = in.getDenseBlock();
+				for(int i=0; i<in.getNumRows(); i++) {
+					BitSet lbs = _data[i];
+					double[] avals = dblock.values(i);
+					int aix = dblock.pos(i);
+					for(int j=0; j<in.getNumColumns(); j++)
+						if( avals[aix+j] != 0 )
+							lbs.set(j);
+				}
+			}
+			_nonZeros = in.getNonZeros();
+		}
+		
+		public BitsetMatrix matMult(BitsetMatrix m2) {
+			final int m = this._rlen;
+			final int cd = this._clen;
+			final int n = m2._clen;
+			//matrix multiply with IKJ schedule and pure OR ops in inner loop
+			BitsetMatrix out = new BitsetMatrix(m, n);
+			for(int i=0; i<m; i++) {
+				BitSet a = this._data[i], c = out._data[i];
+				for(int k=0; k<cd; k++) {
+					if( a.get(k) )
+						c.or(m2._data[k]);
+				}
+				//maintain nnz 
+				out._nonZeros += c.cardinality();
+			}
+			return out;
+		}
+	}
+}


[3/3] systemml git commit: [SYSTEMML-2288] Fix density-map sparsity estimator, more tests

Posted by mb...@apache.org.
[SYSTEMML-2288] Fix density-map sparsity estimator, more tests

This patch fixes a correctness issue of the sparsity estimator based on
density maps along with new tests for squared uniformly distributed
matrices (for all estimators).

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

Branch: refs/heads/master
Commit: aa253eb7eaf272175fbfe7e0e7b1a586cb18b68c
Parents: e21f840
Author: Matthias Boehm <mb...@gmail.com>
Authored: Tue May 1 20:55:05 2018 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Tue May 1 20:55:05 2018 -0700

----------------------------------------------------------------------
 .../sysml/hops/estim/EstimatorDensityMap.java   |  17 +--
 .../functions/estim/OuterProductTest.java       |  11 ++
 .../functions/estim/SquaredProductTest.java     | 117 +++++++++++++++++++
 3 files changed, 137 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/aa253eb7/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 5e8d880..1883d59 100644
--- a/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
+++ b/src/main/java/org/apache/sysml/hops/estim/EstimatorDensityMap.java
@@ -65,7 +65,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 		
 		//estimate output density map and sparsity
 		MatrixBlock outMap = estimIntern(m1Map, m2Map,
-			true, root.getRows(), root.getCols());
+			true, root.getRows(), root.getLeft().getCols(), root.getCols());
 		root.setSynopsis(outMap); //memoize density map
 		return OptimizerUtils.getSparsity( //aggregate output histogram
 			root.getRows(), root.getCols(), (long)outMap.sum());
@@ -76,7 +76,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 		MatrixBlock m1Map = computeDensityMap(m1);
 		MatrixBlock m2Map = computeDensityMap(m2);
 		MatrixBlock outMap = estimIntern(m1Map, m2Map,
-			true, m1.getNumRows(), m2.getNumColumns());
+			true, m1.getNumRows(), m1.getNumColumns(), m2.getNumColumns());
 		return OptimizerUtils.getSparsity( //aggregate output histogram
 			m1.getNumRows(), m2.getNumColumns(), (long)outMap.sum());
 	}
@@ -135,11 +135,12 @@ public class EstimatorDensityMap extends SparsityEstimator
 	 * @param m1Map density map left-hand-side operand
 	 * @param m2Map density map right-hand-side operand
 	 * @param retNnz return number of non-zeros instead of sparsity per cell
-	 * @param rlen number of rows of output matrix, required for returning nnz
-	 * @param clen number of columns of output matrix, required for returning nnz
+	 * @param mOrig number of rows of output matrix, required for returning nnz
+	 * @param cdOrig common dimension of original matrix multiply
+	 * @param nOrig number of columns of output matrix, required for returning nnz
 	 * @return density map
 	 */
-	private MatrixBlock estimIntern(MatrixBlock m1Map, MatrixBlock m2Map, boolean retNnz, int rlen, int clen) {
+	private MatrixBlock estimIntern(MatrixBlock m1Map, MatrixBlock m2Map, boolean retNnz, int mOrig, int cdOrig, int nOrig) {
 		final int m = m1Map.getNumRows();
 		final int cd = m1Map.getNumColumns();
 		final int n = m2Map.getNumColumns();
@@ -151,7 +152,7 @@ public class EstimatorDensityMap extends SparsityEstimator
 		DenseBlock c = out.allocateBlock().getDenseBlock();
 		for(int i=0; i<m; i++) {
 			for(int k=0; k<cd; k++) {
-				int lbk = UtilFunctions.computeBlockSize(cd, k+1, _b);
+				int lbk = UtilFunctions.computeBlockSize(cdOrig, k+1, _b);
 				double sp1 = m1Map.quickGetValue(i, k);
 				for(int j=0; j<n; j++) {
 					double sp2 = m2Map.quickGetValue(k, j);
@@ -164,9 +165,9 @@ public class EstimatorDensityMap extends SparsityEstimator
 			}
 			//scale to non-zeros instead of sparsity if needed
 			if( retNnz ) {
-				int lbm = UtilFunctions.computeBlockSize(rlen, i+1, _b);
+				int lbm = UtilFunctions.computeBlockSize(mOrig, i+1, _b);
 				for( int j=0; j<n; j++ ) {
-					int lbn = UtilFunctions.computeBlockSize(clen, j+1, _b);
+					int lbn = UtilFunctions.computeBlockSize(nOrig, j+1, _b);
 					c.set(i, j, c.get(i, j) * lbm * lbn);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/systemml/blob/aa253eb7/src/test/java/org/apache/sysml/test/integration/functions/estim/OuterProductTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/estim/OuterProductTest.java b/src/test/java/org/apache/sysml/test/integration/functions/estim/OuterProductTest.java
index 0392c7b..90f316c 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/estim/OuterProductTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/estim/OuterProductTest.java
@@ -22,6 +22,7 @@ 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.SparsityEstimator;
 import org.apache.sysml.runtime.instructions.InstructionUtils;
@@ -86,6 +87,16 @@ public class OuterProductTest extends AutomatedTestBase
 		runSparsityEstimateTest(new EstimatorDensityMap(7), m, k, n, case2);
 	}
 	
+	@Test
+	public void testBitsetMatrixCase1() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, case1);
+	}
+	
+	@Test
+	public void testBitsetMatrixCase2() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, case2);
+	}
+	
 	private void runSparsityEstimateTest(SparsityEstimator estim, int m, int k, int n, double[] sp) {
 		MatrixBlock m1 = MatrixBlock.randOperations(m, k, sp[0], 1, 1, "uniform", 3);
 		MatrixBlock m2 = MatrixBlock.randOperations(k, n, sp[1], 1, 1, "uniform", 3);

http://git-wip-us.apache.org/repos/asf/systemml/blob/aa253eb7/src/test/java/org/apache/sysml/test/integration/functions/estim/SquaredProductTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/estim/SquaredProductTest.java b/src/test/java/org/apache/sysml/test/integration/functions/estim/SquaredProductTest.java
new file mode 100644
index 0000000..99b1b3b
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/estim/SquaredProductTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.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.SparsityEstimator;
+import org.apache.sysml.runtime.instructions.InstructionUtils;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+
+/**
+ * This is a basic sanity check for all estimator, which need
+ * to compute a reasonable estimate for uniform data.
+ */
+public class SquaredProductTest extends AutomatedTestBase 
+{
+	private final static int m = 1000;
+	private final static int k = 1000;
+	private final static int n = 1000;
+	private final static double[] case1 = new double[]{0.0001, 0.00007};
+	private final static double[] case2 = new double[]{0.0006, 0.00007};
+
+	private final static double eps1 = 0.05;
+	private final static double eps2 = 1e-4;
+	private final static double eps3 = 0;
+	
+	
+	@Override
+	public void setUp() {
+		//do  nothing
+	}
+	
+	@Test
+	public void testBasicAvgCase1() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, n, case1);
+	}
+	
+	@Test
+	public void testBasicAvgCase2() {
+		runSparsityEstimateTest(new EstimatorBasicAvg(), m, k, n, case2);
+	}
+	
+	@Test
+	public void testBasicWorstCase1() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, n, case1);
+	}
+	
+	@Test
+	public void testBasicWorstCase2() {
+		runSparsityEstimateTest(new EstimatorBasicWorst(), m, k, n, case2);
+	}
+	
+	@Test
+	public void testDensityMapCase1() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, n, case1);
+	}
+	
+	@Test
+	public void testDensityMapCase2() {
+		runSparsityEstimateTest(new EstimatorDensityMap(), m, k, n, case2);
+	}
+	
+	@Test
+	public void testDensityMap7Case1() {
+		runSparsityEstimateTest(new EstimatorDensityMap(7), m, k, n, case1);
+	}
+	
+	@Test
+	public void testDensityMap7Case2() {
+		runSparsityEstimateTest(new EstimatorDensityMap(7), m, k, n, case2);
+	}
+	
+	@Test
+	public void testBitsetMatrixCase1() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, case1);
+	}
+	
+	@Test
+	public void testBitsetMatrixCase2() {
+		runSparsityEstimateTest(new EstimatorBitsetMM(), m, k, n, case2);
+	}
+	
+	private void runSparsityEstimateTest(SparsityEstimator estim, int m, int k, int n, double[] sp) {
+		MatrixBlock m1 = MatrixBlock.randOperations(m, k, sp[0], 1, 1, "uniform", 3);
+		MatrixBlock m2 = MatrixBlock.randOperations(k, n, sp[1], 1, 1, "uniform", 3);
+		MatrixBlock m3 = m1.aggregateBinaryOperations(m1, m2, 
+			new MatrixBlock(), InstructionUtils.getMatMultOperator(1));
+		
+		//compare estimated and real sparsity
+		double est = estim.estim(m1, m2);
+		TestUtils.compareScalars(est, m3.getSparsity(),
+			(estim instanceof EstimatorBitsetMM) ? eps3 : //exact
+			(estim instanceof EstimatorBasicWorst) ? eps1 : eps2);
+	}
+}