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 2017/02/16 20:22:49 UTC

[1/4] incubator-systemml git commit: [SYSTEMML-1273] Performance spark right indexing (aligned, w/o agg)

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 80ab57bda -> 066a8213e


[SYSTEMML-1273] Performance spark right indexing (aligned, w/o agg)

This patch adds a special case for spark right indexing without
aggregation, where a single block is either directly passed through to
the output or sliced to a single output block. The main application is
aligned slicing of entire row batches. The change also includes specific
tests for this special case.
 

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

Branch: refs/heads/master
Commit: f893f6a2deaa13e5f4e4a7e601e71abde3aea55c
Parents: 80ab57b
Author: Matthias Boehm <mb...@gmail.com>
Authored: Wed Feb 15 22:57:50 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Feb 16 12:13:45 2017 -0800

----------------------------------------------------------------------
 .../spark/MatrixIndexingSPInstruction.java      |  74 ++++++++--
 .../indexing/IndexRangeBlockAlignmentTest.java  |  86 ++++++++++++
 .../indexing/RowBatchRightIndexingTest.java     | 136 +++++++++++++++++++
 .../functions/indexing/RowBatchIndexingTest.R   |  35 +++++
 .../functions/indexing/RowBatchIndexingTest.dml |  32 +++++
 .../functions/indexing/ZPackageSuite.java       |   6 +-
 6 files changed, 354 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
index 5b1012f..9d58718 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/MatrixIndexingSPInstruction.java
@@ -152,21 +152,22 @@ public class MatrixIndexingSPInstruction  extends IndexingSPInstruction
 						(int)mcOut.getCols(), mcOut.getRowsPerBlock(), mcOut.getColsPerBlock(), -1);
 				sec.setMatrixOutput(output.getName(), mbout);
 			}
-			else {
-				//rdd output for general case
+			else { //rdd output for general case
 				JavaPairRDD<MatrixIndexes,MatrixBlock> out = null;
+				
 				if( isPartitioningPreservingRightIndexing(mcIn, ixrange) ) {
 					out = in1.mapPartitionsToPair(
 							new SliceBlockPartitionFunction(ixrange, mcOut), true);
 				}
+				else if( _aggType == SparkAggType.NONE
+					|| OptimizerUtils.isIndexingRangeBlockAligned(ixrange, mcIn) ) {
+					out = in1.filter(new IsBlockInRange(rl, ru, cl, cu, mcOut))
+				             .mapToPair(new SliceSingleBlock(ixrange, mcOut));
+				}
 				else {
 					out = in1.filter(new IsBlockInRange(rl, ru, cl, cu, mcOut))
-				             .flatMapToPair(new SliceBlock(ixrange, mcOut));
-					
-					//aggregation if required 
-					boolean aligned = OptimizerUtils.isIndexingRangeBlockAligned(ixrange, mcIn);
-					if( _aggType != SparkAggType.NONE && !aligned ) 
-						out = RDDAggregateUtils.mergeByKey(out);
+				             .flatMapToPair(new SliceMultipleBlocks(ixrange, mcOut));
+					out = RDDAggregateUtils.mergeByKey(out);
 				}
 					
 				//put output RDD handle into symbol table
@@ -405,15 +406,62 @@ public class MatrixIndexingSPInstruction  extends IndexingSPInstruction
 		}
 	}
 
-	private static class SliceBlock implements PairFlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>, MatrixIndexes, MatrixBlock> 
+	private static class SliceSingleBlock implements PairFunction<Tuple2<MatrixIndexes,MatrixBlock>, MatrixIndexes, MatrixBlock> 
+	{
+		private static final long serialVersionUID = -6724027136506200924L;
+		
+		private final IndexRange _ixrange;
+		private final int _brlen; 
+		private final int _bclen;
+		
+		public SliceSingleBlock(IndexRange ixrange, MatrixCharacteristics mcOut) {
+			_ixrange = ixrange;
+			_brlen = mcOut.getRowsPerBlock();
+			_bclen = mcOut.getColsPerBlock();
+		}
+
+		@Override
+		public Tuple2<MatrixIndexes, MatrixBlock> call(Tuple2<MatrixIndexes, MatrixBlock> kv) 
+			throws Exception 
+		{	
+			//get inputs (guaranteed to fall into indexing range)
+			MatrixIndexes ix = kv._1();
+			MatrixBlock block = kv._2();
+			
+			//compute local index range 
+			long grix = UtilFunctions.computeCellIndex(ix.getRowIndex(), _brlen, 0);
+			long gcix = UtilFunctions.computeCellIndex(ix.getColumnIndex(), _bclen, 0);
+			int lrl = (int)((_ixrange.rowStart<grix) ? 0 : _ixrange.rowStart - grix);
+			int lcl = (int)((_ixrange.colStart<gcix) ? 0 : _ixrange.colStart - gcix);
+			int lru = (int)Math.min(block.getNumRows()-1, _ixrange.rowEnd - grix);
+			int lcu = (int)Math.min(block.getNumColumns()-1, _ixrange.colEnd - gcix);
+			
+			//compute output index
+			MatrixIndexes ixOut = new MatrixIndexes(
+				ix.getRowIndex() - (_ixrange.rowStart-1)/_brlen, 
+				ix.getColumnIndex() - (_ixrange.colStart-1)/_bclen);
+			
+			//create return matrix block (via shallow copy or slice)
+			if( lrl == 0 && lru == block.getNumRows()-1
+				&& lcl == 0 && lcu == block.getNumColumns()-1 ) {
+				return new Tuple2<MatrixIndexes, MatrixBlock>(ixOut, block);
+			}
+			else {
+				return new Tuple2<MatrixIndexes, MatrixBlock>(ixOut, 
+					block.sliceOperations(lrl, lru, lcl, lcu, new MatrixBlock()));
+			}
+		}		
+	}
+	
+	private static class SliceMultipleBlocks implements PairFlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>, MatrixIndexes, MatrixBlock> 
 	{
 		private static final long serialVersionUID = 5733886476413136826L;
 		
-		private IndexRange _ixrange;
-		private int _brlen; 
-		private int _bclen;
+		private final IndexRange _ixrange;
+		private final int _brlen; 
+		private final int _bclen;
 		
-		public SliceBlock(IndexRange ixrange, MatrixCharacteristics mcOut) {
+		public SliceMultipleBlocks(IndexRange ixrange, MatrixCharacteristics mcOut) {
 			_ixrange = ixrange;
 			_brlen = mcOut.getRowsPerBlock();
 			_bclen = mcOut.getColsPerBlock();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/test/java/org/apache/sysml/test/integration/functions/indexing/IndexRangeBlockAlignmentTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/indexing/IndexRangeBlockAlignmentTest.java b/src/test/java/org/apache/sysml/test/integration/functions/indexing/IndexRangeBlockAlignmentTest.java
new file mode 100644
index 0000000..b14ae99
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/indexing/IndexRangeBlockAlignmentTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.functions.indexing;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.sysml.hops.OptimizerUtils;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+
+
+public class IndexRangeBlockAlignmentTest extends AutomatedTestBase
+{
+	private static final int BRLEN = 1000;
+	private static final int BCLEN = 1000;
+	
+	@Override
+	public void setUp() {
+		
+	}
+	
+	@Test
+	public void testRowBlockFirstColumn() {
+		Assert.assertEquals(new Boolean(true), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2001, 4000, 1, 1736, BRLEN, BCLEN));
+	}
+	
+	@Test
+	public void testRowBlockColBlock() {
+		Assert.assertEquals(new Boolean(true), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2001, 4000, 7001, 9000, BRLEN, BCLEN));
+	}
+
+	@Test
+	public void testSingleRowBlockFirstColumn() {
+		Assert.assertEquals(new Boolean(true), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2500, 2600, 1, 1736, BRLEN, BCLEN));
+	}
+	
+	@Test
+	public void testSingleRowBlockColBlock() {
+		Assert.assertEquals(new Boolean(true), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2500, 2600, 7001, 9000, BRLEN, BCLEN));
+	}
+	
+	@Test
+	public void testRowBlockFirstColumnNeg() {
+		Assert.assertEquals(new Boolean(false), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2501, 4500, 1, 1736, BRLEN, BCLEN));
+	}
+	
+	@Test
+	public void testRowBlockColBlockNeg() {
+		Assert.assertEquals(new Boolean(false), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2501, 4500, 7001, 9000, BRLEN, BCLEN));
+	}
+
+	@Test
+	public void testSingleRowBlockFirstColumnNeg() {
+		Assert.assertEquals(new Boolean(false), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2500, 3001, 1, 1736, BRLEN, BCLEN));
+	}
+	
+	@Test
+	public void testSingleRowBlockColBlockNeg() {
+		Assert.assertEquals(new Boolean(false), 
+			OptimizerUtils.isIndexingRangeBlockAligned(2500, 3001, 7001, 9000, BRLEN, BCLEN));
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/test/java/org/apache/sysml/test/integration/functions/indexing/RowBatchRightIndexingTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/indexing/RowBatchRightIndexingTest.java b/src/test/java/org/apache/sysml/test/integration/functions/indexing/RowBatchRightIndexingTest.java
new file mode 100644
index 0000000..a9a967e
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/indexing/RowBatchRightIndexingTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.indexing;
+
+import java.util.HashMap;
+import org.junit.Test;
+
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.lops.LopProperties.ExecType;
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.apache.sysml.test.utils.TestUtils;
+
+public class RowBatchRightIndexingTest extends AutomatedTestBase
+{	
+	private final static String TEST_NAME = "RowBatchIndexingTest";
+	private final static String TEST_DIR = "functions/indexing/";
+	private final static String TEST_CLASS_DIR = TEST_DIR + RowBatchRightIndexingTest.class.getSimpleName() + "/";
+	
+	private final static double epsilon=0.0000000001;
+	private final static int rows = 1500; //multiple of 500
+	private final static int cols = 1050;
+	
+	private final static double sparsity1 = 0.5;
+	private final static double sparsity2 = 0.01;
+	
+	
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_NAME, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] {"B"}));
+	}
+	
+	@Test
+	public void testRightIndexingDenseCP() {
+		runRightIndexingTest(ExecType.CP, false);
+	}
+	
+	@Test
+	public void testRightIndexingDenseSP() {
+		runRightIndexingTest(ExecType.SPARK, false);
+	}
+	
+	@Test
+	public void testRightIndexingDenseMR() {
+		runRightIndexingTest(ExecType.MR, false);
+	}
+	
+	@Test
+	public void testRightIndexingSparseCP() {
+		runRightIndexingTest(ExecType.CP, true);
+	}
+	
+	@Test
+	public void testRightIndexingSparseSP() {
+		runRightIndexingTest(ExecType.SPARK, true);
+	}
+	
+	@Test
+	public void testRightIndexingSparseMR() {
+		runRightIndexingTest(ExecType.MR, true);
+	}
+	
+	/**
+	 * 
+	 * @param et
+	 * @param sparse
+	 */
+	public void runRightIndexingTest( ExecType et, boolean sparse ) 
+	{
+		RUNTIME_PLATFORM platformOld = rtplatform;
+		switch( et ){
+			case MR: rtplatform = RUNTIME_PLATFORM.HADOOP; break;
+			case SPARK: rtplatform = RUNTIME_PLATFORM.SPARK; break;
+			default: rtplatform = RUNTIME_PLATFORM.HYBRID; break;
+		}	
+		
+		boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+		if( rtplatform == RUNTIME_PLATFORM.SPARK )
+			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+		
+		try
+		{
+		    TestConfiguration config = getTestConfiguration(TEST_NAME);
+			loadTestConfiguration(config);
+
+		    double sparsity = sparse ? sparsity2 : sparsity1;
+		    
+	        config.addVariable("rows", rows);
+	        config.addVariable("cols", cols);
+	        
+	        String RI_HOME = SCRIPT_DIR + TEST_DIR;
+			fullDMLScriptName = RI_HOME + TEST_NAME + ".dml";
+			programArgs = new String[]{"-args",  input("A"), output("B") };
+			
+			fullRScriptName = RI_HOME + TEST_NAME + ".R";
+			rCmd = "Rscript" + " " + fullRScriptName + " " + 
+				inputDir() + " " + expectedDir();
+	
+			double[][] A = getRandomMatrix(rows, cols, 0, 1, sparsity, 23);
+	        writeInputMatrixWithMTD("A", A, true);
+	        
+	        //run tests
+	        runTest(true, false, null, -1);
+			runRScript(true);
+			
+			//compare output aggregate
+			HashMap<CellIndex, Double> dmlfile = readDMLMatrixFromHDFS("B");
+			HashMap<CellIndex, Double> rfile = readRMatrixFromFS("B");
+			TestUtils.compareMatrices(dmlfile, rfile, epsilon, "DML", "R");
+		}
+		finally {
+			rtplatform = platformOld;
+			DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/test/scripts/functions/indexing/RowBatchIndexingTest.R
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RowBatchIndexingTest.R b/src/test/scripts/functions/indexing/RowBatchIndexingTest.R
new file mode 100644
index 0000000..99334cf
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RowBatchIndexingTest.R
@@ -0,0 +1,35 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+
+
+args <- commandArgs(TRUE)
+options(digits=22)
+library("Matrix")
+
+A = as.matrix(readMM(paste(args[1], "A.mtx", sep="")))
+s = 0;
+for( i in 1:(nrow(A)/500) ) {
+  Xi = A[((i-1)*500+1):(i*500),]
+  s = s + sum(Xi);
+}
+
+B = as.matrix(s);
+writeMM(as(B,"CsparseMatrix"), paste(args[2], "B", sep=""))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/test/scripts/functions/indexing/RowBatchIndexingTest.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RowBatchIndexingTest.dml b/src/test/scripts/functions/indexing/RowBatchIndexingTest.dml
new file mode 100644
index 0000000..164de84
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RowBatchIndexingTest.dml
@@ -0,0 +1,32 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+
+A = read($1)
+s = 0;
+for( i in 1:nrow(A)/500 ) {
+  Xi = A[((i-1)*500+1):(i*500),]
+  if(1==1){}
+  s = s + sum(Xi);
+}
+
+B = as.matrix(s);
+write(B, $2)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/f893f6a2/src/test_suites/java/org/apache/sysml/test/integration/functions/indexing/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/indexing/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/indexing/ZPackageSuite.java
index 76e4832..ea67d1a 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/indexing/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/indexing/ZPackageSuite.java
@@ -26,15 +26,17 @@ import org.junit.runners.Suite;
  *  won't run two of them at once. */
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
+	IndexRangeBlockAlignmentTest.class,
+	Jdk7IssueRightIndexingTest.class,
 	LeftIndexingScalarTest.class,
 	LeftIndexingSparseDenseTest.class,
 	LeftIndexingSparseSparseTest.class,
 	LeftIndexingTest.class,
 	LeftIndexingUpdateInPlaceTest.class,
+	PyDMLImplicitSlicingBounds.class,
 	RightIndexingMatrixTest.class,
 	RightIndexingVectorTest.class,
-	
-	Jdk7IssueRightIndexingTest.class,
+	RowBatchRightIndexingTest.class,
 	UnboundedScalarRightIndexingTest.class,
 })
 


[3/4] incubator-systemml git commit: [SYSTEMML-1274] Improved nnz maintenance on spark rdd write

Posted by mb...@apache.org.
[SYSTEMML-1274] Improved nnz maintenance on spark rdd write

We now consistently piggyback any nnz maintenance on write operations in
order to avoid unnecessary RDD computation. Furthermore, this change
also removes the utils primitive to compute the nnz in isolation in
order to prevent reintroducing such inefficiencies.


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

Branch: refs/heads/master
Commit: 73afc2c19fe34caf08ec2c63bdbfb0b42aab881f
Parents: ee7591c
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Feb 16 12:12:57 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Feb 16 12:13:48 2017 -0800

----------------------------------------------------------------------
 .../context/SparkExecutionContext.java          |  9 ++++---
 .../instructions/spark/WriteSPInstruction.java  | 13 ++++++----
 .../instructions/spark/utils/SparkUtils.java    | 25 --------------------
 3 files changed, 15 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/73afc2c1/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java b/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
index 66fab1e..77bcc8d 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/context/SparkExecutionContext.java
@@ -35,6 +35,7 @@ import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.broadcast.Broadcast;
 import org.apache.spark.storage.RDDInfo;
 import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.util.LongAccumulator;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.MLContextProxy;
 import org.apache.sysml.conf.ConfigurationManager;
@@ -55,6 +56,7 @@ import org.apache.sysml.runtime.instructions.spark.data.LineageObject;
 import org.apache.sysml.runtime.instructions.spark.data.PartitionedBlock;
 import org.apache.sysml.runtime.instructions.spark.data.PartitionedBroadcast;
 import org.apache.sysml.runtime.instructions.spark.data.RDDObject;
+import org.apache.sysml.runtime.instructions.spark.functions.ComputeBinaryBlockNnzFunction;
 import org.apache.sysml.runtime.instructions.spark.functions.CopyBinaryCellFunction;
 import org.apache.sysml.runtime.instructions.spark.functions.CopyFrameBlockPairFunction;
 import org.apache.sysml.runtime.instructions.spark.functions.CopyTextInputFunction;
@@ -966,8 +968,9 @@ public class SparkExecutionContext extends ExecutionContext
 	{
 		JavaPairRDD<MatrixIndexes,MatrixBlock> lrdd = (JavaPairRDD<MatrixIndexes, MatrixBlock>) rdd.getRDD();
 		
-		//recompute nnz 
-		long nnz = SparkUtils.computeNNZFromBlocks(lrdd);
+		//piggyback nnz maintenance on write
+		LongAccumulator aNnz = getSparkContextStatic().sc().longAccumulator("nnz");
+		lrdd = lrdd.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
 		
 		//save file is an action which also triggers nnz maintenance
 		lrdd.saveAsHadoopFile(path, 
@@ -976,7 +979,7 @@ public class SparkExecutionContext extends ExecutionContext
 				oinfo.outputFormatClass);
 		
 		//return nnz aggregate of all blocks
-		return nnz;
+		return aNnz.value();
 	}
 
 	@SuppressWarnings("unchecked")

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/73afc2c1/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
index 3387770..c30c85f 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/WriteSPInstruction.java
@@ -39,7 +39,6 @@ import org.apache.sysml.runtime.instructions.spark.functions.ComputeBinaryBlockN
 import org.apache.sysml.runtime.instructions.spark.utils.FrameRDDConverterUtils;
 import org.apache.sysml.runtime.instructions.spark.utils.FrameRDDConverterUtils.LongFrameToLongWritableFrameFunction;
 import org.apache.sysml.runtime.instructions.spark.utils.RDDConverterUtils;
-import org.apache.sysml.runtime.instructions.spark.utils.SparkUtils;
 import org.apache.sysml.runtime.matrix.MatrixCharacteristics;
 import org.apache.sysml.runtime.matrix.data.CSVFileFormatProperties;
 import org.apache.sysml.runtime.matrix.data.FileFormatProperties;
@@ -179,9 +178,12 @@ public class WriteSPInstruction extends SPInstruction
 		if(    oi == OutputInfo.MatrixMarketOutputInfo
 			|| oi == OutputInfo.TextCellOutputInfo     ) 
 		{
-			//recompute nnz if necessary (required for header if matrix market)
-			if ( isInputMatrixBlock && !mc.nnzKnown() )
-				mc.setNonZeros( SparkUtils.computeNNZFromBlocks(in1) );
+			//piggyback nnz maintenance on write
+			LongAccumulator aNnz = null;
+			if ( isInputMatrixBlock && !mc.nnzKnown() ) {
+				aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
+				in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
+			}
 			
 			JavaRDD<String> header = null;				
 			if( oi == OutputInfo.MatrixMarketOutputInfo  ) {
@@ -199,6 +201,9 @@ public class WriteSPInstruction extends SPInstruction
 				customSaveTextFile(header.union(ijv), fname, true);
 			else
 				customSaveTextFile(ijv, fname, false);
+			
+			if ( isInputMatrixBlock && !mc.nnzKnown() )
+				mc.setNonZeros( aNnz.value() );
 		}
 		else if( oi == OutputInfo.CSVOutputInfo ) 
 		{

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/73afc2c1/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/SparkUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/SparkUtils.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/SparkUtils.java
index d53f3cf..d27e37a 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/SparkUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/SparkUtils.java
@@ -249,29 +249,4 @@ public class SparkUtils
 					arg0.getNonZeros() + arg1.getNonZeros() ); //sum
 		}	
 	}
-
-	/**
-	 * Utility to compute number of non-zeros from the given RDD of MatrixBlocks
-	 * 
-	 * @param rdd matrix as {@code JavaPairRDD<MatrixIndexes, MatrixBlock>}
-	 * @return number of non-zeros
-	 */
-	public static long computeNNZFromBlocks(JavaPairRDD<MatrixIndexes, MatrixBlock> rdd) {
-		long nnz = rdd.values().aggregate(	0L, 
-						new Function2<Long,MatrixBlock,Long>() {
-							private static final long serialVersionUID = 4907645080949985267L;
-							@Override
-							public Long call(Long v1, MatrixBlock v2) throws Exception {
-								return (v1 + v2.getNonZeros());
-							} 
-						}, 
-						new Function2<Long,Long,Long>() {
-							private static final long serialVersionUID = 333028431986883739L;
-							@Override
-							public Long call(Long v1, Long v2) throws Exception {
-								return v1+v2;
-							}
-						} );
-		return nnz;
-	}
 }


[4/4] incubator-systemml git commit: [SYSTEMML-1257] Fix hops construction relational expressions w/o target

Posted by mb...@apache.org.
[SYSTEMML-1257] Fix hops construction relational expressions w/o target


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

Branch: refs/heads/master
Commit: 066a8213ebe6a67aee69dbe6e7e039f3efc21e67
Parents: 73afc2c
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Feb 16 12:13:10 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Feb 16 12:13:50 2017 -0800

----------------------------------------------------------------------
 src/main/java/org/apache/sysml/parser/DMLTranslator.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/066a8213/src/main/java/org/apache/sysml/parser/DMLTranslator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLTranslator.java b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
index 1d92e25..e3533b7 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -1795,8 +1795,9 @@ public class DMLTranslator
 			target = createTarget(source);
 			if(left.getDataType() == DataType.MATRIX || right.getDataType() == DataType.MATRIX) {
 				// Added to support matrix relational comparison
+				// (we support only matrices of value type double)
 				target.setDataType(DataType.MATRIX);
-				target.setValueType(ValueType.BOOLEAN);
+				target.setValueType(ValueType.DOUBLE);
 			}
 			else {
 				// Added to support scalar relational comparison


[2/4] incubator-systemml git commit: [SYSTEMML-1275] Remove workaround flags disable_sparse/disable_caching

Posted by mb...@apache.org.
[SYSTEMML-1275] Remove workaround flags disable_sparse/disable_caching

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

Branch: refs/heads/master
Commit: ee7591cb243af5fb5a2cd45796f71f2936e3d2ef
Parents: f893f6a
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Feb 16 12:12:42 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Feb 16 12:13:47 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/sysml/api/DMLScript.java    | 19 -------------
 .../java/org/apache/sysml/hops/AggBinaryOp.java |  2 +-
 .../org/apache/sysml/hops/OptimizerUtils.java   |  2 +-
 .../hops/cost/CostEstimatorStaticRuntime.java   |  2 +-
 .../parfor/opt/OptimizerRuleBased.java          |  2 +-
 .../spark/utils/RDDConverterUtils.java          |  2 +-
 .../sysml/runtime/matrix/data/MatrixBlock.java  | 29 ++++----------------
 .../applications/dml/ApplyTransformDMLTest.java | 15 ----------
 .../applications/dml/ArimaDMLTest.java          | 14 ----------
 .../applications/dml/CsplineCGDMLTest.java      | 15 ----------
 .../applications/dml/CsplineDSDMLTest.java      | 16 -----------
 .../applications/dml/GLMDMLTest.java            | 14 ----------
 .../applications/dml/GNMFDMLTest.java           | 14 ----------
 .../applications/dml/HITSDMLTest.java           | 17 +-----------
 .../applications/dml/ID3DMLTest.java            | 14 ----------
 .../applications/dml/L2SVMDMLTest.java          | 14 ----------
 .../applications/dml/LinearLogRegDMLTest.java   | 14 ----------
 .../dml/LinearRegressionDMLTest.java            | 15 ----------
 .../dml/MDABivariateStatsDMLTest.java           | 15 ----------
 .../applications/dml/MultiClassSVMDMLTest.java  | 15 ----------
 .../applications/dml/NaiveBayesDMLTest.java     | 14 ----------
 .../dml/NaiveBayesParforDMLTest.java            | 14 ----------
 .../applications/dml/PageRankDMLTest.java       | 14 ----------
 .../applications/dml/WelchTDMLTest.java         | 14 ----------
 24 files changed, 12 insertions(+), 294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/api/DMLScript.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/DMLScript.java b/src/main/java/org/apache/sysml/api/DMLScript.java
index e0e0cb2..83d0f5b 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -116,20 +116,6 @@ public class DMLScript
 	public static boolean USE_ACCELERATOR = false;
 	public static boolean FORCE_ACCELERATOR = false;
 	
-	// ------------------------------------------------------------------------
-	// We have identified two performance bugs that frequently occurs in deep learning script.
-	//
-	// First, we repeatedly perform unnecessary conversion to sparse format.
-	// Also, the operations such as matrix multiplication (including BLAS and CuBLAS) are 
-	// optimized for dense.
-	//
-	// Second, even with large memory budget, we sometimes spend almost 20-30% time in caching.
-	// These two bugs are tracked in SYSTEMML-1140 and should be addressed before SystemML 1.0 release.
-	// We are keeping this flag for performance debugging until SYSTEMML-1140is resolved.
-	public static boolean DISABLE_SPARSE = false;
-	public static boolean DISABLE_CACHING = false;
-	// ------------------------------------------------------------------------
-	
 	// flag that indicates whether or not to suppress any prints to stdout
 	public static boolean _suppressPrint2Stdout = false;
 	
@@ -674,11 +660,6 @@ public class DMLScript
 		{  
 			initHadoopExecution( dmlconf );
 			
-			if(DISABLE_CACHING) {
-				//disable caching globally 
-				CacheableData.disableCaching();
-			}
-			
 			//run execute (w/ exception handling to ensure proper shutdown)
 			ec = ExecutionContextFactory.createContext(rtprog);
 			rtprog.execute( ec );  

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/hops/AggBinaryOp.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/AggBinaryOp.java b/src/main/java/org/apache/sysml/hops/AggBinaryOp.java
index fe8a30d..73dd8a4 100644
--- a/src/main/java/org/apache/sysml/hops/AggBinaryOp.java
+++ b/src/main/java/org/apache/sysml/hops/AggBinaryOp.java
@@ -358,7 +358,7 @@ public class AggBinaryOp extends Hop implements MultiThreadedHop
 		
 		//account for potential final dense-sparse transformation (worst-case sparse representation)
 		if( dim2 >= 2 ) //vectors always dense
-			ret = OptimizerUtils.estimateSizeExactSparsity(dim1, dim2, MatrixBlock.getSparsityTurnPoint());
+			ret = OptimizerUtils.estimateSizeExactSparsity(dim1, dim2, MatrixBlock.SPARSITY_TURN_POINT);
 		
 		return ret;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/OptimizerUtils.java b/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
index a7525a5..86b7968 100644
--- a/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
+++ b/src/main/java/org/apache/sysml/hops/OptimizerUtils.java
@@ -459,7 +459,7 @@ public class OptimizerUtils
 
 	public static boolean checkSparseBlockCSRConversion( MatrixCharacteristics mcIn ) {
 		return Checkpoint.CHECKPOINT_SPARSE_CSR
-			&& OptimizerUtils.getSparsity(mcIn) < MatrixBlock.getSparsityTurnPoint();
+			&& OptimizerUtils.getSparsity(mcIn) < MatrixBlock.SPARSITY_TURN_POINT;
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java b/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
index 3d46198..f8cd476 100644
--- a/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
+++ b/src/main/java/org/apache/sysml/hops/cost/CostEstimatorStaticRuntime.java
@@ -981,7 +981,7 @@ public class CostEstimatorStaticRuntime extends CostEstimator
 								if( d3s==1.0 )
 									return d3m * d3n * nflopRand + d3m * d3n * 8; //DENSE gen (incl allocate)    
 								else 
-									return (d3s>=MatrixBlock.getSparsityTurnPoint())? 
+									return (d3s>=MatrixBlock.SPARSITY_TURN_POINT)? 
 										    2 * d3m * d3n * nflopRand + d3m * d3n * 8: //DENSE gen (incl allocate)    
 									        3 * d3m * d3n * d3s * nflopRand + d3m * d3n * d3s * 24; //SPARSE gen (incl allocate)
 							}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
index 98fa664..66065bf 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
@@ -1590,7 +1590,7 @@ public class OptimizerRuleBased extends Optimizer
 			
 			if(    dat !=null && dat instanceof MatrixObject 
 				&& moDpf == PDataPartitionFormat.COLUMN_WISE	
-				&& ((MatrixObject)dat).getSparsity()<= MatrixBlock.getSparsityTurnPoint()  //check for sparse matrix
+				&& ((MatrixObject)dat).getSparsity()<=MatrixBlock.SPARSITY_TURN_POINT  //check for sparse matrix
 				&& rIsTransposeSafePartition(pn, moVarname) ) //tranpose-safe
 			{
 				pfpb.setTransposeSparseColumnVector( true );

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
index 4d95b95..6b6c61d 100644
--- a/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
+++ b/src/main/java/org/apache/sysml/runtime/instructions/spark/utils/RDDConverterUtils.java
@@ -305,7 +305,7 @@ public class RDDConverterUtils
 				mc.getNumRowBlocks(), mc.getColsPerBlock(), Math.ceil((double)mc.getNonZeros()/mc.getRows()));
 		double partsize = Math.ceil(datasize/in.partitions().size());
 		double blksz = Math.min(mc.getRows(), mc.getRowsPerBlock());
-		return partsize/rowsize/blksz < MatrixBlock.getSparsityTurnPoint();
+		return partsize/rowsize/blksz < MatrixBlock.SPARSITY_TURN_POINT;
 	}
 
 	private static int countNnz(Object vect, boolean isVector, int off) {

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/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 58e2034..fb4f196 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
@@ -33,7 +33,6 @@ import java.util.Iterator;
 
 import org.apache.commons.math3.random.Well1024a;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.hops.Hop.OpOp2;
 import org.apache.sysml.hops.OptimizerUtils;
@@ -92,9 +91,9 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	private static final long serialVersionUID = 7319972089143154056L;
 	
 	//sparsity nnz threshold, based on practical experiments on space consumption and performance
-	private static final double SPARSITY_TURN_POINT = 0.4;
+	public static final double SPARSITY_TURN_POINT = 0.4;
 	//sparsity threshold for ultra-sparse matrix operations (40nnz in a 1kx1k block)
-	private static final double ULTRA_SPARSITY_TURN_POINT = 0.00004; 
+	public static final double ULTRA_SPARSITY_TURN_POINT = 0.00004; 
 	//default sparse block type: modified compressed sparse rows, for efficient incremental construction
 	public static final SparseBlock.Type DEFAULT_SPARSEBLOCK = SparseBlock.Type.MCSR;
 	//default sparse block type for update in place: compressed sparse rows, to prevent serialization
@@ -184,18 +183,6 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	// Initialization methods
 	// (reset, init, allocate, etc)
 	
-	public static double getSparsityTurnPoint() {
-		if(DMLScript.DISABLE_SPARSE)
-			return 1e-6;
-		return SPARSITY_TURN_POINT;
-	}
-	
-	public static double getUltraSparsityTurnPoint() {
-		if(DMLScript.DISABLE_SPARSE)
-			return 1e-9;
-		return ULTRA_SPARSITY_TURN_POINT;
-	}
-	
 	@Override
 	public void reset() {
 		reset(rlen, clen, sparse, -1, 0);
@@ -904,7 +891,7 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	{
 		double sp = ((double)nonZeros/rlen)/clen;
 		//check for sparse representation in order to account for vectors in dense
-		return sparse && sp< getUltraSparsityTurnPoint() && nonZeros<40;
+		return sparse && sp<ULTRA_SPARSITY_TURN_POINT && nonZeros<40;
 	}
 
 	/**
@@ -1006,14 +993,10 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	 * @return true if matrix block shold be in sparse format in memory
 	 */
 	public static boolean evalSparseFormatInMemory( final long nrows, final long ncols, final long nnz )
-	{		
-//		// Extremely low getSparsityTurnPoint should disable sparse in most cases
-//		if(DMLScript.DISABLE_SPARSE)
-//			return false;
-		
+	{				
 		//evaluate sparsity threshold
 		double lsparsity = (double)nnz/nrows/ncols;
-		boolean lsparse = (lsparsity < getSparsityTurnPoint());
+		boolean lsparse = (lsparsity < SPARSITY_TURN_POINT);
 		
 		//compare size of sparse and dense representation in order to prevent
 		//that the sparse size exceed the dense size since we use the dense size
@@ -1038,7 +1021,7 @@ public class MatrixBlock extends MatrixValue implements CacheBlock, Externalizab
 	{
 		//evaluate sparsity threshold
 		double lsparsity = ((double)nnz/nrows)/ncols;
-		boolean lsparse = (lsparsity < getSparsityTurnPoint());
+		boolean lsparse = (lsparsity < SPARSITY_TURN_POINT);
 		
 		double sizeUltraSparse = estimateSizeUltraSparseOnDisk( nrows, ncols, nnz );
 		double sizeSparse = estimateSizeSparseOnDisk(nrows, ncols, nnz);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/ApplyTransformDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/ApplyTransformDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/ApplyTransformDMLTest.java
index e419844..240eb6f 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/ApplyTransformDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/ApplyTransformDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.ApplyTransformTest;
 
 @RunWith(value = Parameterized.class)
@@ -38,18 +37,4 @@ public class ApplyTransformDMLTest extends ApplyTransformTest {
 	public void testApplyTransformDml() {
 		testApplyTransform(ScriptType.DML);
 	}
-	
-	@Test
-	public void testApplyTransformDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testApplyTransform(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/ArimaDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/ArimaDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/ArimaDMLTest.java
index 86fc575..17ef153 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/ArimaDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/ArimaDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.ArimaTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class ArimaDMLTest extends ArimaTest {
 	public void testArimaDml() {
 		testArima(ScriptType.DML);
 	}
-
-	@Test
-	public void testArimaDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testArima(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineCGDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineCGDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineCGDMLTest.java
index 7b9c466..5a61764 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineCGDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineCGDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.CsplineCGTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,18 +36,4 @@ public class CsplineCGDMLTest extends CsplineCGTest {
 	public void testCsplineCGDml() {
 		testCsplineCG(ScriptType.DML);
 	}
-	
-	@Test
-	public void testCsplineCGDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testCsplineCG(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineDSDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineDSDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineDSDMLTest.java
index f99440e..ddb03a8 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineDSDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/CsplineDSDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.CsplineDSTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,19 +36,4 @@ public class CsplineDSDMLTest extends CsplineDSTest {
 	public void testCsplineDSDml() {
 		testCsplineDS(ScriptType.DML);
 	}
-	
-	
-	@Test
-	public void testCsplineDSDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testCsplineDS(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/GLMDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/GLMDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/GLMDMLTest.java
index 97c0faa..71e16f5 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/GLMDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/GLMDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.GLMTest;
 
 @RunWith(value = Parameterized.class)
@@ -40,17 +39,4 @@ public class GLMDMLTest extends GLMTest {
 	public void testGLMDml() {
 		testGLM(ScriptType.DML);
 	}
-
-	@Test
-	public void testGLMDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testGLM(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/GNMFDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/GNMFDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/GNMFDMLTest.java
index 56e10d5..933b892 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/GNMFDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/GNMFDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.GNMFTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class GNMFDMLTest extends GNMFTest {
 	public void testGNMFDml() {
 		testGNMF(ScriptType.DML);
 	}
-	
-	@Test
-	public void testGNMFDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testGNMF(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/HITSDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/HITSDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/HITSDMLTest.java
index 535582a..52c8630 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/HITSDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/HITSDMLTest.java
@@ -20,7 +20,6 @@
 package org.apache.sysml.test.integration.applications.dml;
 
 import org.junit.Test;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.HITSTest;
 
 public class HITSDMLTest extends HITSTest {
@@ -33,19 +32,5 @@ public class HITSDMLTest extends HITSTest {
 	@Test
 	public void testHitsDml() {
 		testHits(ScriptType.DML);
-	}
-
-	@Test
-	public void testHitsDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testHits(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-	
+	}	
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/ID3DMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/ID3DMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/ID3DMLTest.java
index 012e678..676e0e3 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/ID3DMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/ID3DMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.ID3Test;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class ID3DMLTest extends ID3Test {
 	public void testID3Dml() {
 		testID3(ScriptType.DML);
 	}
-
-	@Test
-	public void testID3DmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testID3(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/L2SVMDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/L2SVMDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/L2SVMDMLTest.java
index cc69b77..23249e7 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/L2SVMDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/L2SVMDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.L2SVMTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class L2SVMDMLTest extends L2SVMTest {
 	public void testL2SVMDml() {
 		testL2SVM(ScriptType.DML);
 	}
-
-	@Test
-	public void testL2SVMDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testL2SVM(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearLogRegDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearLogRegDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearLogRegDMLTest.java
index e19d856..4691e25 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearLogRegDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearLogRegDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.LinearLogRegTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class LinearLogRegDMLTest extends LinearLogRegTest {
 	public void testLinearLogRegDml() {
 		testLinearLogReg(ScriptType.DML);
 	}
-
-	@Test
-	public void testLinearLogRegDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testLinearLogReg(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearRegressionDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearRegressionDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearRegressionDMLTest.java
index d18f98a..c9dc307 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearRegressionDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/LinearRegressionDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.LinearRegressionTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,18 +36,4 @@ public class LinearRegressionDMLTest extends LinearRegressionTest {
 	public void testLinearRegressionDml() {
 		testLinearRegression(ScriptType.DML);
 	}
-	
-	@Test
-	public void testLinearRegressionDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testLinearRegression(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/MDABivariateStatsDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/MDABivariateStatsDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/MDABivariateStatsDMLTest.java
index 249508f..0564929 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/MDABivariateStatsDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/MDABivariateStatsDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.MDABivariateStatsTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,18 +36,4 @@ public class MDABivariateStatsDMLTest extends MDABivariateStatsTest {
 	public void testMDABivariateStatsDml() {
 		testMDABivariateStats(ScriptType.DML);
 	}
-	
-	@Test
-	public void testMDABivariateStatsDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testMDABivariateStats(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/MultiClassSVMDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/MultiClassSVMDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/MultiClassSVMDMLTest.java
index c3654dc..7791ef6 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/MultiClassSVMDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/MultiClassSVMDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.MultiClassSVMTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,18 +36,4 @@ public class MultiClassSVMDMLTest extends MultiClassSVMTest {
 	public void testMultiClassSVMDml() {
 		testMultiClassSVM(ScriptType.DML);
 	}
-	
-	@Test
-	public void testMultiClassSVMDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testMultiClassSVM(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesDMLTest.java
index 8fd18d4..6f7a253 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.NaiveBayesTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class NaiveBayesDMLTest extends NaiveBayesTest {
 	public void testNaiveBayesDml() {
 		testNaiveBayes(ScriptType.DML);
 	}
-
-	@Test
-	public void testNaiveBayesDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testNaiveBayes(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
index 926bb3d..330e86a 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/NaiveBayesParforDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.NaiveBayesParforTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class NaiveBayesParforDMLTest extends NaiveBayesParforTest {
 	public void testNaiveBayesDml() {
 		testNaiveBayes(ScriptType.DML);
 	}
-
-	@Test
-	public void testNaiveBayesDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testNaiveBayes(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/PageRankDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/PageRankDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/PageRankDMLTest.java
index de493b6..b78a081 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/PageRankDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/PageRankDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.PageRankTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class PageRankDMLTest extends PageRankTest {
 	public void testPageRankDml() {
 		testPageRank(ScriptType.DML);
 	}
-
-	@Test
-	public void testPageRankDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testPageRank(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/ee7591cb/src/test/java/org/apache/sysml/test/integration/applications/dml/WelchTDMLTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/applications/dml/WelchTDMLTest.java b/src/test/java/org/apache/sysml/test/integration/applications/dml/WelchTDMLTest.java
index 98befa7..0fba2c2 100644
--- a/src/test/java/org/apache/sysml/test/integration/applications/dml/WelchTDMLTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/applications/dml/WelchTDMLTest.java
@@ -22,7 +22,6 @@ package org.apache.sysml.test.integration.applications.dml;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
-import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.test.integration.applications.WelchTTest;
 
 @RunWith(value = Parameterized.class)
@@ -37,17 +36,4 @@ public class WelchTDMLTest extends WelchTTest {
 	public void testWelchTTestDml() {
 		testWelchTTest(ScriptType.DML);
 	}
-
-	@Test
-	public void testWelchTTestDmlDisableSparseNCaching() {
-		try {
-			DMLScript.DISABLE_SPARSE = true;
-			DMLScript.DISABLE_CACHING = true;
-			testWelchTTest(ScriptType.DML);
-		}
-		finally {
-			DMLScript.DISABLE_SPARSE = false;
-			DMLScript.DISABLE_CACHING = false;
-		}
-	}
 }