You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by re...@apache.org on 2017/11/03 18:02:52 UTC

[50/50] [abbrv] systemml git commit: [SYSTEMML-1984] Robustness JMLC prepared scripts (outputs and configs)

[SYSTEMML-1984] Robustness JMLC prepared scripts (outputs and configs)

This patch fixes the robustness of JMLC prepared scripts for deployments
where prepared scripts are not necessarily created and executed by the
same thread. Accordingly, we no longer maintain output variables (for
dynamic recompilation) and compiler configurations in a thread-local
manner, but attach the output variables to the symbol table as well as
set and cleanup thread-local compiler configurations individually at
connection level and on prepared script execute.


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

Branch: refs/heads/master
Commit: 0d4672207185ac183d22e157171a58c5f38a5bb0
Parents: e1f5866
Author: Matthias Boehm <mb...@gmail.com>
Authored: Thu Nov 2 20:02:22 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Thu Nov 2 20:02:22 2017 -0700

----------------------------------------------------------------------
 .../org/apache/sysml/api/jmlc/Connection.java   | 36 ++++++-------
 .../org/apache/sysml/api/jmlc/JMLCProxy.java    | 55 --------------------
 .../apache/sysml/api/jmlc/PreparedScript.java   | 39 ++++++++++----
 .../apache/sysml/api/mlcontext/MLContext.java   | 11 ----
 .../sysml/api/mlcontext/ScriptExecutor.java     |  8 +--
 .../apache/sysml/hops/recompile/Recompiler.java | 13 ++---
 .../controlprogram/LocalVariableMap.java        | 25 ++++++---
 .../org/apache/sysml/utils/MLContextProxy.java  | 11 +---
 .../functions/jmlc/MulticlassSVMScoreTest.java  | 41 +++++++++------
 9 files changed, 102 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/api/jmlc/Connection.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/jmlc/Connection.java b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
index f933396..4caa86c 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/Connection.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
@@ -95,9 +95,10 @@ import org.apache.wink.json4j.JSONObject;
  * </ul>
  */
 public class Connection implements Closeable
-{		
-	private DMLConfig _dmlconf = null;
-
+{
+	private final DMLConfig _dmlconf;
+	private final CompilerConfig _cconf;
+	
 	/**
 	 * Connection constructor, the starting point for any other JMLC API calls.
 	 * 
@@ -122,14 +123,17 @@ public class Connection implements Closeable
 		cconf.set(ConfigType.ALLOW_INDIVIDUAL_SB_SPECIFIC_OPS, false);
 		cconf.set(ConfigType.ALLOW_CSE_PERSISTENT_READS, false);
 		cconf.set(ConfigType.CODEGEN_ENABLED, false);
-		ConfigurationManager.setLocalConfig(cconf);
+		_cconf = cconf;
 		
 		//disable caching globally 
 		CacheableData.disableCaching();
 		
-		//create thread-local default configuration
+		//create default configuration
 		_dmlconf = new DMLConfig();
+		
+		//set thread-local configurations for compilation
 		ConfigurationManager.setLocalConfig(_dmlconf);
+		ConfigurationManager.setLocalConfig(_cconf);
 	}
 	
 	/**
@@ -143,10 +147,12 @@ public class Connection implements Closeable
 		this();
 		
 		//set optional compiler configurations in current config
-		CompilerConfig cconf = ConfigurationManager.getCompilerConfig();
 		for( ConfigType configType : configs )
-			cconf.set(configType, true);
-		ConfigurationManager.setLocalConfig(cconf);
+			_cconf.set(configType, true);
+		
+		//set thread-local configurations for compilation
+		ConfigurationManager.setLocalConfig(_dmlconf);
+		ConfigurationManager.setLocalConfig(_cconf);
 	}
 	
 	/**
@@ -202,7 +208,7 @@ public class Connection implements Closeable
 			
 			//language validate
 			DMLTranslator dmlt = new DMLTranslator(prog);
-			dmlt.liveVariableAnalysis(prog);			
+			dmlt.liveVariableAnalysis(prog);
 			dmlt.validateParseTree(prog);
 			
 			//hop construct/rewrite
@@ -220,10 +226,6 @@ public class Connection implements Closeable
 			
 			//final cleanup runtime prog
 			JMLCUtils.cleanupRuntimeProgram(rtprog, outputs);
-			
-			//activate thread-local proxy for dynamic recompilation
-			if( ConfigurationManager.isDynamicRecompilation() )
-				JMLCProxy.setActive(outputs);
 		}
 		catch(ParseException pe) {
 			// don't chain ParseException (for cleaner error output)
@@ -232,9 +234,9 @@ public class Connection implements Closeable
 		catch(Exception ex) {
 			throw new DMLException(ex);
 		}
-			
+		
 		//return newly create precompiled script 
-		return new PreparedScript(rtprog, inputs, outputs);
+		return new PreparedScript(rtprog, inputs, outputs, _dmlconf, _cconf);
 	}
 	
 	/**
@@ -243,10 +245,8 @@ public class Connection implements Closeable
 	 */
 	@Override
 	public void close() {
-		//clear thread-local dml / compiler configs
+		//clear thread-local configurations
 		ConfigurationManager.clearLocalConfigs();
-		if( ConfigurationManager.isDynamicRecompilation() )
-			JMLCProxy.setActive(null);
 		if( ConfigurationManager.isCodegenEnabled() )
 			SpoofCompiler.cleanupCodeGenerator();
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/api/jmlc/JMLCProxy.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/jmlc/JMLCProxy.java b/src/main/java/org/apache/sysml/api/jmlc/JMLCProxy.java
deleted file mode 100644
index 400380c..0000000
--- a/src/main/java/org/apache/sysml/api/jmlc/JMLCProxy.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.api.jmlc;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-
-import org.apache.sysml.runtime.instructions.Instruction;
-
-/**
- * This proxy provides thread-local access to output variables per connection
- * in order to enable dynamic recompilation in JMLC.
- */
-public class JMLCProxy
-{
-	private static ThreadLocal<HashSet<String>> _outputs = new ThreadLocal<HashSet<String>>() {
-		@Override 
-		protected HashSet<String> initialValue() { 
-			return null;
-		}
-	};
-	
-	public static void setActive(String[] output) {
-		if( output != null )
-			_outputs.set(new HashSet<>(Arrays.asList(output)));
-		else
-			_outputs.remove();
-	}
-
-	public static boolean isActive() {
-		return (_outputs.get() != null);
-	}
-
-	public static ArrayList<Instruction> performCleanupAfterRecompilation(ArrayList<Instruction> tmp) {
-		return JMLCUtils.cleanupRuntimeInstructions(tmp, _outputs.get());
-	}
-}

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/api/jmlc/PreparedScript.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/jmlc/PreparedScript.java b/src/main/java/org/apache/sysml/api/jmlc/PreparedScript.java
index 6cd041c..f14d3a1 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/PreparedScript.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/PreparedScript.java
@@ -29,6 +29,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.sysml.api.DMLException;
 import org.apache.sysml.conf.CompilerConfig;
 import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.hops.ipa.FunctionCallGraph;
@@ -64,13 +65,15 @@ public class PreparedScript
 	private static final Log LOG = LogFactory.getLog(PreparedScript.class.getName());
 	
 	//input/output specification
-	private HashSet<String> _inVarnames = null;
-	private HashSet<String> _outVarnames = null;
-	private HashMap<String,Data> _inVarReuse = null;
+	private final HashSet<String> _inVarnames;
+	private final HashSet<String> _outVarnames;
+	private final HashMap<String,Data> _inVarReuse;
 	
 	//internal state (reused)
-	private Program _prog = null;
-	private LocalVariableMap _vars = null; 
+	private final Program _prog;
+	private final LocalVariableMap _vars;
+	private final DMLConfig _dmlconf;
+	private final CompilerConfig _cconf;
 	
 	/**
 	 * Meant to be invoked only from Connection.
@@ -78,8 +81,10 @@ public class PreparedScript
 	 * @param prog the DML/PyDML program
 	 * @param inputs input variables to register
 	 * @param outputs output variables to register
+	 * @param dmlconf dml configuration 
+	 * @param cconf compiler configuration
 	 */
-	protected PreparedScript( Program prog, String[] inputs, String[] outputs ) 
+	protected PreparedScript( Program prog, String[] inputs, String[] outputs, DMLConfig dmlconf, CompilerConfig cconf ) 
 	{
 		_prog = prog;
 		_vars = new LocalVariableMap();
@@ -90,6 +95,14 @@ public class PreparedScript
 		_outVarnames = new HashSet<>();
 		Collections.addAll(_outVarnames, outputs);
 		_inVarReuse = new HashMap<>();
+		
+		//attach registered outputs (for dynamic recompile)
+		_vars.setRegisteredOutputs(_outVarnames);
+		
+		//keep dml and compiler configuration to be set as thread-local config
+		//on execute, which allows different threads creating/executing the script
+		_dmlconf = dmlconf;
+		_cconf = cconf;
 	}
 	
 	/**
@@ -386,11 +399,15 @@ public class PreparedScript
 		//add reused variables
 		_vars.putAll(_inVarReuse);
 		
+		//set thread-local configurations
+		ConfigurationManager.setLocalConfig(_dmlconf);
+		ConfigurationManager.setLocalConfig(_cconf);
+		
 		//create and populate execution context
-		ExecutionContext ec = ExecutionContextFactory.createContext(_vars, _prog);	
+		ExecutionContext ec = ExecutionContextFactory.createContext(_vars, _prog);
 		
-		//core execute runtime program	
-		_prog.execute(ec);  
+		//core execute runtime program
+		_prog.execute(ec);
 		
 		//cleanup unnecessary outputs
 		_vars.removeAllNotIn(_outVarnames);
@@ -402,6 +419,10 @@ public class PreparedScript
 			if( tmpVar != null )
 				rvars.addResult(ovar, tmpVar);
 		}
+		
+		//clear thread-local configurations
+		ConfigurationManager.clearLocalConfigs();
+		
 		return rvars;
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/api/mlcontext/MLContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/mlcontext/MLContext.java b/src/main/java/org/apache/sysml/api/mlcontext/MLContext.java
index 762db8f..6fe66ff 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/MLContext.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/MLContext.java
@@ -19,7 +19,6 @@
 
 package org.apache.sysml.api.mlcontext;
 
-import java.util.ArrayList;
 import java.util.Date;
 import java.util.Set;
 
@@ -28,7 +27,6 @@ import org.apache.spark.SparkContext;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.SparkSession;
 import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.jmlc.JMLCUtils;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.parser.DataExpression;
@@ -39,7 +37,6 @@ import org.apache.sysml.runtime.DMLRuntimeException;
 import org.apache.sysml.runtime.controlprogram.LocalVariableMap;
 import org.apache.sysml.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
-import org.apache.sysml.runtime.instructions.Instruction;
 import org.apache.sysml.runtime.instructions.cp.Data;
 import org.apache.sysml.runtime.instructions.cp.ScalarObject;
 import org.apache.sysml.runtime.matrix.MatrixFormatMetaData;
@@ -590,14 +587,6 @@ public class MLContext {
 			}
 			throw new MLContextException("getMatrixObject not set for parameter: " + parameterName);
 		}
-
-		public ArrayList<Instruction> performCleanupAfterRecompilation(ArrayList<Instruction> instructions) {
-			if (executionScript == null || executionScript.getOutputVariables() == null)
-				return instructions;
-
-			Set<String> outputVariableNames = executionScript.getOutputVariables();
-			return JMLCUtils.cleanupRuntimeInstructions(instructions, outputVariableNames.toArray(new String[0]));
-		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/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 f4f8803..aa2364d 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.api.mlcontext;
 
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
@@ -230,10 +231,11 @@ public class ScriptExecutor {
 	protected void createAndInitializeExecutionContext() {
 		executionContext = ExecutionContextFactory.createContext(runtimeProgram);
 		LocalVariableMap symbolTable = script.getSymbolTable();
-		if (symbolTable != null) {
+		if (symbolTable != null)
 			executionContext.setVariables(symbolTable);
-		}
-
+		//attach registered outputs (for dynamic recompile)
+		executionContext.getVariables().setRegisteredOutputs(
+			new HashSet<String>(script.getOutputVariables()));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
index 463e41b..3c1f6c9 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.wink.json4j.JSONObject;
 import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.api.jmlc.JMLCProxy;
+import org.apache.sysml.api.jmlc.JMLCUtils;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.hops.DataGenOp;
@@ -105,7 +105,6 @@ import org.apache.sysml.runtime.util.UtilFunctions;
 import org.apache.sysml.utils.Explain;
 import org.apache.sysml.utils.Explain.ExplainType;
 import org.apache.sysml.utils.JSONHelper;
-import org.apache.sysml.utils.MLContextProxy;
 
 /**
  * Dynamic recompilation of hop dags to runtime instructions, which includes the 
@@ -252,19 +251,17 @@ public class Recompiler
 			newInst = ProgramConverter.createDeepCopyInstructionSet(newInst, tid, -1, null, null, null, false, false);
 		
 		// remove writes if called through mlcontext or jmlc 
-		if( MLContextProxy.isActive() )
-			newInst = MLContextProxy.performCleanupAfterRecompilation(newInst);
-		else if( JMLCProxy.isActive() )
-			newInst = JMLCProxy.performCleanupAfterRecompilation(newInst);
+		if( vars.getRegisteredOutputs() != null )
+			newInst = JMLCUtils.cleanupRuntimeInstructions(newInst, vars.getRegisteredOutputs());
 		
 		// explain recompiled hops / instructions
 		if( DMLScript.EXPLAIN == ExplainType.RECOMPILE_HOPS ){
 			LOG.info("EXPLAIN RECOMPILE \nGENERIC (lines "+sb.getBeginLine()+"-"+sb.getEndLine()+"):\n" + 
-		    Explain.explainHops(hops, 1));
+			Explain.explainHops(hops, 1));
 		}
 		if( DMLScript.EXPLAIN == ExplainType.RECOMPILE_RUNTIME ){
 			LOG.info("EXPLAIN RECOMPILE \nGENERIC (lines "+sb.getBeginLine()+"-"+sb.getEndLine()+"):\n" + 
-		    Explain.explain(newInst, 1));
+			Explain.explain(newInst, 1));
 		}
 	
 		return newInst;

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/runtime/controlprogram/LocalVariableMap.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/LocalVariableMap.java b/src/main/java/org/apache/sysml/runtime/controlprogram/LocalVariableMap.java
index 0743d39..7ebe1a0 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/LocalVariableMap.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/LocalVariableMap.java
@@ -20,6 +20,7 @@
 package org.apache.sysml.runtime.controlprogram;
 
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -37,13 +38,17 @@ import org.apache.sysml.runtime.instructions.cp.Data;
  */
 public class LocalVariableMap implements Cloneable
 {
-	private static String eol = System.getProperty ("line.separator");
-	private static String ELEMENT_DELIM = org.apache.sysml.runtime.controlprogram.parfor.ProgramConverter.ELEMENT_DELIM;
-	private static IDSequence _seq = new IDSequence();
+	private static final String eol = System.getProperty ("line.separator");
+	private static final String ELEMENT_DELIM = ProgramConverter.ELEMENT_DELIM;
+	private static final IDSequence _seq = new IDSequence();
 	
-	private HashMap <String, Data> localMap = null;
+	//variable map data and id
+	private final HashMap<String, Data> localMap;
 	private final long localID;
 	
+	//optional set of registered outputs
+	private HashSet<String> outputs = null;
+	
 	public LocalVariableMap() {
 		localMap = new HashMap<>();
 		localID = _seq.getNextID();
@@ -104,6 +109,14 @@ public class LocalVariableMap implements Cloneable
 	public boolean hasReferences( Data d ) {
 		return localMap.containsValue(d);
 	}
+	
+	public void setRegisteredOutputs(HashSet<String> outputs) {
+		this.outputs = outputs;
+	}
+	
+	public HashSet<String> getRegisteredOutputs() {
+		return outputs;
+	}
 
 	public String serialize() throws DMLRuntimeException {
 		StringBuilder sb = new StringBuilder();
@@ -115,7 +128,7 @@ public class LocalVariableMap implements Cloneable
 				.serializeDataObject(e.getKey(), e.getValue()));
 			count++;
 		}
-		return sb.toString();		
+		return sb.toString();
 	}
 
 	public static LocalVariableMap deserialize(String varStr) 
@@ -128,7 +141,7 @@ public class LocalVariableMap implements Cloneable
 			Object[] tmp2 = ProgramConverter.parseDataObject (tmp);
 			vars.put((String) tmp2 [0], (Data) tmp2 [1]);
 		}
-		return vars;		
+		return vars;
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/main/java/org/apache/sysml/utils/MLContextProxy.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/MLContextProxy.java b/src/main/java/org/apache/sysml/utils/MLContextProxy.java
index 825d42a..d67abac 100644
--- a/src/main/java/org/apache/sysml/utils/MLContextProxy.java
+++ b/src/main/java/org/apache/sysml/utils/MLContextProxy.java
@@ -19,13 +19,10 @@
 
 package org.apache.sysml.utils;
 
-import java.util.ArrayList;
-
 import org.apache.sysml.api.mlcontext.MLContext;
 import org.apache.sysml.api.mlcontext.MLContextException;
 import org.apache.sysml.parser.Expression;
 import org.apache.sysml.parser.LanguageException;
-import org.apache.sysml.runtime.instructions.Instruction;
 
 /**
  * The purpose of this proxy is to shield systemml internals from direct access to MLContext
@@ -46,12 +43,7 @@ public class MLContextProxy
 	public static boolean isActive() {
 		return _active;
 	}
-
-	public static ArrayList<Instruction> performCleanupAfterRecompilation(ArrayList<Instruction> tmp)
-	{
-		return MLContext.getActiveMLContext().getInternalProxy().performCleanupAfterRecompilation(tmp);
-	}
-
+	
 	public static void setAppropriateVarsForRead(Expression source, String targetname)
 		throws LanguageException
 	{
@@ -69,5 +61,4 @@ public class MLContextProxy
 		throw new MLContextException("No MLContext object is currently active. Have you created one? "
 				+ "Hint: in Scala, 'val ml = new MLContext(sc)'", true);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/0d467220/src/test/java/org/apache/sysml/test/integration/functions/jmlc/MulticlassSVMScoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/jmlc/MulticlassSVMScoreTest.java b/src/test/java/org/apache/sysml/test/integration/functions/jmlc/MulticlassSVMScoreTest.java
index 5304688..9a0602a 100644
--- a/src/test/java/org/apache/sysml/test/integration/functions/jmlc/MulticlassSVMScoreTest.java
+++ b/src/test/java/org/apache/sysml/test/integration/functions/jmlc/MulticlassSVMScoreTest.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import org.apache.sysml.api.jmlc.Connection;
 import org.apache.sysml.api.jmlc.PreparedScript;
 import org.apache.sysml.api.jmlc.ResultVariables;
+import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.runtime.controlprogram.parfor.stat.Timing;
 import org.apache.sysml.runtime.matrix.data.InputInfo;
 import org.apache.sysml.runtime.matrix.data.MatrixBlock;
@@ -48,7 +49,7 @@ public class MulticlassSVMScoreTest extends AutomatedTestBase
 	private final static int rows = 107;
 	private final static int cols = 46; //fixed
 	
-	private final static int nRuns = 10;
+	private final static int nRuns = 5;
 	
 	private final static double sparsity1 = 0.7;
 	private final static double sparsity2 = 0.1;
@@ -59,25 +60,28 @@ public class MulticlassSVMScoreTest extends AutomatedTestBase
 	{
 		addTestConfiguration(TEST_NAME, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] { "predicted_y" }) ); 
 	}
-
 	
 	@Test
-	public void testJMLCMulticlassScoreDense() 
-		throws IOException
-	{
-		//should apply diag_mm rewrite
-		runJMLCMulticlassTest(false);
+	public void testJMLCMulticlassScoreDense() throws IOException {
+		runJMLCMulticlassTest(false, false);
 	}
 	
 	@Test
-	public void testJMLCMulticlassScoreSparse() 
-		throws IOException
-	{
-		//should apply diag_mm rewrite
-		runJMLCMulticlassTest(true);
+	public void testJMLCMulticlassScoreSparse() throws IOException {
+		runJMLCMulticlassTest(true, false);
+	}
+	
+	@Test
+	public void testJMLCMulticlassScoreDenseFlags() throws IOException {
+		runJMLCMulticlassTest(false, true);
+	}
+	
+	@Test
+	public void testJMLCMulticlassScoreSparseFlags() throws IOException {
+		runJMLCMulticlassTest(true, true);
 	}
 
-	private void runJMLCMulticlassTest( boolean sparse ) 
+	private void runJMLCMulticlassTest( boolean sparse, boolean flags ) 
 		throws IOException
 	{	
 		TestConfiguration config = getTestConfiguration(TEST_NAME);
@@ -87,7 +91,7 @@ public class MulticlassSVMScoreTest extends AutomatedTestBase
 		ArrayList<double[][]> Xset = generateInputs(nRuns, rows, cols, sparse?sparsity2:sparsity1); 
 		
 		//run DML via JMLC
-		ArrayList<double[][]> Yset = execDMLScriptviaJMLC( Xset );
+		ArrayList<double[][]> Yset = execDMLScriptviaJMLC( Xset, flags );
 		
 		//run R and compare results to DML result
 		String HOME = SCRIPT_DIR + TEST_DIR;
@@ -117,7 +121,7 @@ public class MulticlassSVMScoreTest extends AutomatedTestBase
 		}
 	}
 
-	private static ArrayList<double[][]> execDMLScriptviaJMLC( ArrayList<double[][]> X) 
+	private static ArrayList<double[][]> execDMLScriptviaJMLC(ArrayList<double[][]> X, boolean flags) 
 		throws IOException
 	{
 		Timing time = new Timing(true);
@@ -125,8 +129,11 @@ public class MulticlassSVMScoreTest extends AutomatedTestBase
 		ArrayList<double[][]> ret = new ArrayList<double[][]>();
 		
 		//establish connection to SystemML
-		Connection conn = new Connection();
-				
+		Connection conn = !flags ? new Connection():
+			new Connection(ConfigType.PARALLEL_CP_MATRIX_OPERATIONS,
+				ConfigType.PARALLEL_LOCAL_OR_REMOTE_PARFOR,
+				ConfigType.ALLOW_DYN_RECOMPILATION);
+		
 		try
 		{
 			// For now, JMLC pipeline only allows dml