You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by mb...@apache.org on 2016/09/19 21:44:09 UTC

[1/2] incubator-systemml git commit: [SYSTEMML-869] Fix implicit cleanup of mlcontext input variables, tests

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 062ebc67a -> 3d523b851


[SYSTEMML-869] Fix implicit cleanup of mlcontext input variables, tests

Depending on the script control structure (e.g., multiple statement
blocks), passed intermediates from one script to another script might be
cleaned up as they appear to be no longer used. However, this leads to
destroyed intermediates that a user might still want to pass to other
scripts. Hence, we now disable the cleanup of any mlcontext input
variables as this should be in the hand of the user. 

Furthermore, this patch also includes some minor cleanups of the
monitoring api.

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

Branch: refs/heads/master
Commit: 0d9e7b2a05e5ce82f8176e58222ca73465d57e68
Parents: 062ebc6
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Mon Sep 19 13:57:33 2016 -0700
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Mon Sep 19 13:57:33 2016 -0700

----------------------------------------------------------------------
 .../org/apache/sysml/api/mlcontext/Script.java  |  15 ++-
 .../api/monitoring/InstructionComparator.java   |  10 +-
 .../apache/sysml/api/monitoring/Location.java   |  40 ++----
 .../mlcontext/MLContextMultipleScriptsTest.java | 134 +++++++++++++++++++
 .../apache/sysml/api/mlcontext/MultiScript1.dml |   2 +
 .../apache/sysml/api/mlcontext/MultiScript2.dml |   8 ++
 .../sysml/api/mlcontext/MultiScript2b.dml       |   9 ++
 .../apache/sysml/api/mlcontext/MultiScript3.dml |   2 +
 .../sysml/api/mlcontext/MultiScript3b.dml       |   4 +
 9 files changed, 181 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/main/java/org/apache/sysml/api/mlcontext/Script.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/mlcontext/Script.java b/src/main/java/org/apache/sysml/api/mlcontext/Script.java
index 17a3996..aac2fc2 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/Script.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/Script.java
@@ -29,8 +29,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.sysml.runtime.controlprogram.LocalVariableMap;
-import org.apache.sysml.runtime.controlprogram.caching.FrameObject;
-import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
+import org.apache.sysml.runtime.controlprogram.caching.CacheableData;
 import org.apache.sysml.runtime.instructions.cp.Data;
 
 import scala.Tuple2;
@@ -332,15 +331,19 @@ public class Script {
 				inputParameters = new LinkedHashMap<String, Object>();
 			}
 			inputParameters.put(name, value);
-		} else {
+		} 
+		else {
 			Data data = MLContextUtil.convertInputType(name, value, metadata);
 			if (data != null) {
+				//store input variable name and data
 				symbolTable.put(name, data);
 				inputVariables.add(name);
-				if (data instanceof MatrixObject || data instanceof FrameObject) {
-					if (metadata != null) {
+				
+				//store matrix/frame meta data and disable variable cleanup
+				if( data instanceof CacheableData ) {
+					if( metadata != null )
 						inputMetadata.put(name, metadata);
-					}
+					((CacheableData<?>)data).enableCleanup(false);
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/main/java/org/apache/sysml/api/monitoring/InstructionComparator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/monitoring/InstructionComparator.java b/src/main/java/org/apache/sysml/api/monitoring/InstructionComparator.java
index 05aa081..485ada4 100644
--- a/src/main/java/org/apache/sysml/api/monitoring/InstructionComparator.java
+++ b/src/main/java/org/apache/sysml/api/monitoring/InstructionComparator.java
@@ -27,14 +27,10 @@ public class InstructionComparator implements Comparator<String>{
 	public InstructionComparator(HashMap<String, Long> instructionCreationTime) {
 		this.instructionCreationTime = instructionCreationTime;
 	}
+	
 	@Override
 	public int compare(String o1, String o2) {
-		try {
-			return instructionCreationTime.get(o1).compareTo(instructionCreationTime.get(o2));
-		}
-		catch(Exception e) {
-			return -1;
-		}
+		return instructionCreationTime.get(o1)
+			.compareTo(instructionCreationTime.get(o2));
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/main/java/org/apache/sysml/api/monitoring/Location.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/monitoring/Location.java b/src/main/java/org/apache/sysml/api/monitoring/Location.java
index ffb6867..ddc9749 100644
--- a/src/main/java/org/apache/sysml/api/monitoring/Location.java
+++ b/src/main/java/org/apache/sysml/api/monitoring/Location.java
@@ -32,15 +32,11 @@ public class Location implements Comparable<Location> {
 	
 	@Override
 	public boolean equals(Object other) {
-		if(other instanceof Location) {
-			Location loc = (Location) other;
-			return loc.beginLine == beginLine && loc.endLine == endLine && loc.beginCol == beginCol && loc.endCol == endCol;
-		}
-		return false;
-	}
-	
-	private int compare(int v1, int v2) {
-		return new Integer(v1).compareTo(v2);
+		if(!( other instanceof Location ))
+			return false;
+		Location loc = (Location) other;
+		return loc.beginLine == beginLine && loc.endLine == endLine 
+				&& loc.beginCol == beginCol && loc.endCol == endCol;
 	}
 	
 	public String toString() {
@@ -49,27 +45,11 @@ public class Location implements Comparable<Location> {
 
 	@Override
 	public int compareTo(Location loc) {
-		if(loc.beginLine == beginLine && loc.endLine == endLine && loc.beginCol == beginCol && loc.endCol == endCol)
-			return 0;
+		int ret1 = Integer.compare(loc.beginLine, beginLine);
+		int ret2 = Integer.compare(loc.endLine, endLine);
+		int ret3 = Integer.compare(loc.beginCol, beginCol);
+		int ret4 = Integer.compare(loc.endCol, endCol);
 		
-		int retVal = compare(beginLine, loc.beginLine);
-		if(retVal != 0) { 
-			return retVal;
-		}
-		else { 
-			retVal = compare(beginCol, loc.beginCol);
-			if(retVal != 0) { 
-				return retVal;
-			}
-			else { 
-				retVal = compare(endLine, loc.endLine);
-				if(retVal != 0) { 
-					return retVal;
-				}
-				else {
-					return compare(endCol, loc.endCol);
-				}
-			}
-		}
+		return (ret1 != 0) ? ret1 : (ret2 != 0) ? ret2 : (ret3 != 0) ? ret3 : ret4;   
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java b/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
new file mode 100644
index 0000000..7f2dfaf
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.mlcontext;
+
+import static org.apache.sysml.api.mlcontext.ScriptFactory.dmlFromFile;
+
+import java.io.File;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
+import org.apache.sysml.api.mlcontext.MLContext;
+import org.apache.sysml.api.mlcontext.MLContext.ExplainLevel;
+import org.apache.sysml.api.mlcontext.Matrix;
+import org.apache.sysml.api.mlcontext.Script;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.After;
+import org.junit.Test;
+
+
+public class MLContextMultipleScriptsTest extends AutomatedTestBase 
+{
+	private final static String TEST_DIR = "org/apache/sysml/api/mlcontext";
+	private final static String TEST_NAME = "MLContextMultiScript";
+
+	private final static int rows = 1123;
+	private final static int cols = 789;
+	
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		addTestConfiguration(TEST_DIR, TEST_NAME);
+		getAndLoadTestConfiguration(TEST_NAME);
+	}
+
+	@Test
+	public void testMLContextMultipleScriptsCP() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.SINGLE_NODE, false);
+	}
+	
+	@Test
+	public void testMLContextMultipleScriptsHybrid() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.HYBRID_SPARK, false);
+	}
+	
+	@Test
+	public void testMLContextMultipleScriptsSpark() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.SPARK, false);
+	}
+	
+	@Test
+	public void testMLContextMultipleScriptsWithReadCP() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.SINGLE_NODE, true);
+	}
+	
+	@Test
+	public void testMLContextMultipleScriptsWithReadHybrid() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.HYBRID_SPARK, true);
+	}
+	
+	@Test
+	public void testMLContextMultipleScriptsWithReadSpark() {
+		runMLContextTestMultipleScript(RUNTIME_PLATFORM.SPARK, true);
+	}
+
+	/**
+	 * 
+	 * @param platform
+	 */
+	private void runMLContextTestMultipleScript(RUNTIME_PLATFORM platform, boolean wRead) 
+	{
+		RUNTIME_PLATFORM oldplatform = DMLScript.rtplatform;
+		DMLScript.rtplatform = platform;
+		
+		//create mlcontext
+		SparkConf conf = new SparkConf().setAppName("MLContextFrameTest").setMaster("local");
+		JavaSparkContext sc = new JavaSparkContext(conf);
+		MLContext ml = new MLContext(sc);
+		ml.setExplain(true);
+		ml.setExplainLevel(ExplainLevel.RUNTIME);
+		
+		String dml1 = baseDirectory + File.separator + "MultiScript1.dml";
+		String dml2 = baseDirectory + File.separator + (wRead?"MultiScript2b.dml":"MultiScript2.dml");
+		String dml3 = baseDirectory + File.separator + (wRead?"MultiScript3b.dml":"MultiScript3.dml");
+		
+		try
+		{
+			//run script 1
+			Script script1 = dmlFromFile(dml1).in("$rows", rows).in("$cols", cols).out("X");
+			Matrix X = ml.execute(script1).getMatrix("X");
+			
+			Script script2 = dmlFromFile(dml2).in("X", X).out("Y");
+			Matrix Y = ml.execute(script2).getMatrix("Y");
+			
+			Script script3 = dmlFromFile(dml3).in("X", X).in("Y",Y).out("z");
+			String z = ml.execute(script3).getString("z");
+			
+			System.out.println(z);
+		}
+		finally {
+			DMLScript.rtplatform = oldplatform;
+			
+			// stop spark context to allow single jvm tests (otherwise the
+			// next test that tries to create a SparkContext would fail)
+			sc.stop();
+			// clear status mlcontext and spark exec context
+			ml.close();
+		}
+	}
+
+	@After
+	public void tearDown() {
+		super.tearDown();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript1.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript1.dml b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript1.dml
new file mode 100644
index 0000000..8aeedb6
--- /dev/null
+++ b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript1.dml
@@ -0,0 +1,2 @@
+X = rand(rows=$rows, cols=$cols);
+write(X, "out/X", format="binary");
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2.dml b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2.dml
new file mode 100644
index 0000000..15d5e42
--- /dev/null
+++ b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2.dml
@@ -0,0 +1,8 @@
+Y = X^2 + X;
+
+#cut to test cleanup X
+if( sum(X)>1 ){
+   Y = Y + sum(X);
+}
+
+write(Y, "out/Y", format="binary");
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2b.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2b.dml b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2b.dml
new file mode 100644
index 0000000..db016d4
--- /dev/null
+++ b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript2b.dml
@@ -0,0 +1,9 @@
+X = read("out/X", format="binary");
+Y = X^2 + X;
+
+#cut to test cleanup X
+if( sum(X)>1 ){
+   Y = Y + sum(X);
+}
+
+write(Y, "out/Y", format="binary");
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3.dml b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3.dml
new file mode 100644
index 0000000..d9f0232
--- /dev/null
+++ b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3.dml
@@ -0,0 +1,2 @@
+z = sum(X + Y);
+write(z, "out/z");

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/0d9e7b2a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3b.dml
----------------------------------------------------------------------
diff --git a/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3b.dml b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3b.dml
new file mode 100644
index 0000000..4178fa7
--- /dev/null
+++ b/src/test/scripts/org/apache/sysml/api/mlcontext/MultiScript3b.dml
@@ -0,0 +1,4 @@
+X = read("out/X", format="binary");
+Y = read("out/Y", format="binary");
+z = sum(X + Y);
+write(z, "out/z");


[2/2] incubator-systemml git commit: [SYSTEMML-944] Fix mlcontext default explain type (runtime, not hops)

Posted by mb...@apache.org.
[SYSTEMML-944] Fix mlcontext default explain type (runtime, not hops) 

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

Branch: refs/heads/master
Commit: 3d523b851313986c047b2be008de57333caf44c9
Parents: 0d9e7b2
Author: Matthias Boehm <mb...@us.ibm.com>
Authored: Mon Sep 19 14:30:20 2016 -0700
Committer: Matthias Boehm <mb...@us.ibm.com>
Committed: Mon Sep 19 14:30:20 2016 -0700

----------------------------------------------------------------------
 .../sysml/api/mlcontext/ScriptExecutor.java       | 18 ++++++------------
 .../mlcontext/MLContextMultipleScriptsTest.java   |  2 --
 2 files changed, 6 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/3d523b85/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
index 2973ed2..17bae7a 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
@@ -200,19 +200,13 @@ public class ScriptExecutor {
 	protected void showExplanation() {
 		if (explain) {
 			try {
-				if (explainLevel == null) {
-					System.out.println(Explain.explain(dmlProgram));
-				} else {
-					ExplainType explainType = explainLevel.getExplainType();
-					System.out.println(Explain.explain(dmlProgram, runtimeProgram, explainType));
-				}
-			} catch (HopsException e) {
-				throw new MLContextException("Exception occurred while explaining dml program", e);
-			} catch (DMLRuntimeException e) {
+				ExplainType explainType = (explainLevel != null) ? 
+						explainLevel.getExplainType() : ExplainType.RUNTIME;
+				System.out.println(Explain.explain(dmlProgram, runtimeProgram, explainType));
+			} 
+			catch (Exception e) {
 				throw new MLContextException("Exception occurred while explaining dml program", e);
-			} catch (LanguageException e) {
-				throw new MLContextException("Exception occurred while explaining dml program", e);
-			}
+			} 
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/3d523b85/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java b/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
index 7f2dfaf..e0e784c 100644
--- a/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/mlcontext/MLContextMultipleScriptsTest.java
@@ -28,7 +28,6 @@ import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.DMLScript.RUNTIME_PLATFORM;
 import org.apache.sysml.api.mlcontext.MLContext;
-import org.apache.sysml.api.mlcontext.MLContext.ExplainLevel;
 import org.apache.sysml.api.mlcontext.Matrix;
 import org.apache.sysml.api.mlcontext.Script;
 import org.apache.sysml.test.integration.AutomatedTestBase;
@@ -96,7 +95,6 @@ public class MLContextMultipleScriptsTest extends AutomatedTestBase
 		JavaSparkContext sc = new JavaSparkContext(conf);
 		MLContext ml = new MLContext(sc);
 		ml.setExplain(true);
-		ml.setExplainLevel(ExplainLevel.RUNTIME);
 		
 		String dml1 = baseDirectory + File.separator + "MultiScript1.dml";
 		String dml2 = baseDirectory + File.separator + (wRead?"MultiScript2b.dml":"MultiScript2.dml");