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/07/22 20:52:29 UTC

[5/5] systemml git commit: [SYSTEMML-1506] Support codegen through all APIs, incl jmlc/mlcontext

[SYSTEMML-1506] Support codegen through all APIs, incl jmlc/mlcontext

This patch modifies the codegen compiler integration to make it
applicable through all APIs (command line, mlcontext, jmlc, debug)
without modifying the individual replicated compilation chains.
Furthermore, this patch also cleans up the dml program data structure
and the generation of runtime programs.


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

Branch: refs/heads/master
Commit: 856230c56535742d824aab681fceea1567c26567
Parents: 0fee3f6
Author: Matthias Boehm <mb...@gmail.com>
Authored: Sat Jul 22 13:40:45 2017 -0700
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Sat Jul 22 13:53:18 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/api/DMLScript.java    |  25 +-
 .../apache/sysml/api/ScriptExecutorUtils.java   |   5 +-
 .../org/apache/sysml/api/jmlc/Connection.java   |   6 +-
 .../sysml/api/mlcontext/ScriptExecutor.java     |   6 +-
 .../org/apache/sysml/conf/CompilerConfig.java   |   6 +-
 .../apache/sysml/conf/ConfigurationManager.java |   5 +
 .../apache/sysml/hops/recompile/Recompiler.java |   9 +-
 .../org/apache/sysml/parser/DMLProgram.java     | 597 +------------------
 .../org/apache/sysml/parser/DMLTranslator.java  | 406 ++++++++++++-
 .../parfor/opt/ProgramRecompiler.java           |   3 +-
 .../java/org/apache/sysml/utils/Statistics.java |   3 +-
 .../functions/codegen/APICodegenTest.java       | 115 ++++
 .../functions/codegen/ZPackageSuite.java        |   1 +
 13 files changed, 549 insertions(+), 638 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/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 2674af4..f428aa2 100644
--- a/src/main/java/org/apache/sysml/api/DMLScript.java
+++ b/src/main/java/org/apache/sysml/api/DMLScript.java
@@ -65,9 +65,6 @@ import org.apache.sysml.debug.DMLDebuggerProgramInfo;
 import org.apache.sysml.hops.HopsException;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.hops.OptimizerUtils.OptimizationLevel;
-import org.apache.sysml.hops.codegen.SpoofCompiler;
-import org.apache.sysml.hops.codegen.SpoofCompiler.IntegrationType;
-import org.apache.sysml.hops.codegen.SpoofCompiler.PlanCachePolicy;
 import org.apache.sysml.hops.globalopt.GlobalOptimizerWrapper;
 import org.apache.sysml.lops.Lop;
 import org.apache.sysml.lops.LopsException;
@@ -690,16 +687,6 @@ public class DMLScript
 	
 		//Step 5: rewrite HOP DAGs (incl IPA and memory estimates)
 		dmlt.rewriteHopsDAG(prog);
-
-		//Step 5.1: Generate code for the rewritten Hop dags 
-		if( dmlconf.getBooleanValue(DMLConfig.CODEGEN) ){
-			SpoofCompiler.PLAN_CACHE_POLICY = PlanCachePolicy.get(
-					dmlconf.getBooleanValue(DMLConfig.CODEGEN_PLANCACHE),
-					dmlconf.getIntValue(DMLConfig.CODEGEN_LITERALS)==2);
-			SpoofCompiler.setExecTypeSpecificJavaCompiler();
-			if( SpoofCompiler.INTEGRATION==IntegrationType.HOPS )
-				dmlt.codgenHopsDAG(prog);
-		}
 		
 		//Step 6: construct lops (incl exec type and op selection)
 		dmlt.constructLops(prog);
@@ -710,14 +697,8 @@ public class DMLScript
 			dmlt.resetLopsDAGVisitStatus(prog);
 		}
 		
-		//Step 7: generate runtime program
-		Program rtprog = prog.getRuntimeProgram(dmlconf);
-
-		//Step 7.1: Generate code for the rewritten Hop dags w/o modify
-		if( dmlconf.getBooleanValue(DMLConfig.CODEGEN) 
-			&& SpoofCompiler.INTEGRATION==IntegrationType.RUNTIME ){
-			dmlt.codgenHopsDAG(rtprog);
-		}
+		//Step 7: generate runtime program, incl codegen
+		Program rtprog = dmlt.getRuntimeProgram(prog, dmlconf);
 		
 		//Step 8: [optional global data flow optimization]
 		if(OptimizerUtils.isOptLevel(OptimizationLevel.O4_GLOBAL_TIME_MEMORY) ) 
@@ -807,7 +788,7 @@ public class DMLScript
 		dmlt.constructLops(prog);
 	
 		//Step 6: generate runtime program
-		dbprog.rtprog = prog.getRuntimeProgram(conf);
+		dbprog.rtprog = dmlt.getRuntimeProgram(prog, conf);
 		
 		try {
 			//set execution environment

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
index b094c91..ebbcc21 100644
--- a/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
+++ b/src/main/java/org/apache/sysml/api/ScriptExecutorUtils.java
@@ -22,6 +22,7 @@ package org.apache.sysml.api;
 import java.util.List;
 
 import org.apache.sysml.api.mlcontext.ScriptExecutor;
+import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.codegen.SpoofCompiler;
 import org.apache.sysml.runtime.DMLRuntimeException;
@@ -98,9 +99,9 @@ public class ScriptExecutorUtils {
 				ec.getGPUContexts().forEach(gCtx -> gCtx.clearTemporaryMemory());
 				GPUContextPool.freeAllGPUContexts();
 			}
-			if (dmlconf.getBooleanValue(DMLConfig.CODEGEN))
+			if( ConfigurationManager.isCodegenEnabled() )
 				SpoofCompiler.cleanupCodeGenerator();
-
+			
 			// display statistics (incl caching stats if enabled)
 			Statistics.stopRunTimer();
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/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 be440c8..1993ed4 100644
--- a/src/main/java/org/apache/sysml/api/jmlc/Connection.java
+++ b/src/main/java/org/apache/sysml/api/jmlc/Connection.java
@@ -42,6 +42,7 @@ import org.apache.sysml.conf.CompilerConfig;
 import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.conf.DMLConfig;
+import org.apache.sysml.hops.codegen.SpoofCompiler;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.hops.rewrite.RewriteRemovePersistentReadWrite;
 import org.apache.sysml.parser.DMLProgram;
@@ -122,6 +123,7 @@ public class Connection implements Closeable
 		cconf.set(ConfigType.ALLOW_DYN_RECOMPILATION, false);
 		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);
 		
 		//disable caching globally 
@@ -216,7 +218,7 @@ public class Connection implements Closeable
 			
 			//lop construct and runtime prog generation
 			dmlt.constructLops(prog);
-			rtprog = prog.getRuntimeProgram(_dmlconf);
+			rtprog = dmlt.getRuntimeProgram(prog, _dmlconf);
 			
 			//final cleanup runtime prog
 			JMLCUtils.cleanupRuntimeProgram(rtprog, outputs);
@@ -247,6 +249,8 @@ public class Connection implements Closeable
 		ConfigurationManager.clearLocalConfigs();
 		if( ConfigurationManager.isDynamicRecompilation() )
 			JMLCProxy.setActive(null);
+		if( ConfigurationManager.isCodegenEnabled() )
+			SpoofCompiler.cleanupCodeGenerator();
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/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 6d19166..1a5d0bb 100644
--- a/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
+++ b/src/main/java/org/apache/sysml/api/mlcontext/ScriptExecutor.java
@@ -169,7 +169,7 @@ public class ScriptExecutor {
 	protected void rewriteHops() {
 		try {
 			dmlTranslator.rewriteHopsDAG(dmlProgram);
-		} catch (LanguageException | HopsException | ParseException e) {
+		} catch (LanguageException | HopsException | ParseException | DMLRuntimeException e) {
 			throw new MLContextException("Exception occurred while rewriting HOPS (high-level operators)", e);
 		}
 	}
@@ -210,8 +210,8 @@ public class ScriptExecutor {
 	 */
 	protected void generateRuntimeProgram() {
 		try {
-			runtimeProgram = dmlProgram.getRuntimeProgram(config);
-		} catch (LanguageException | DMLRuntimeException | LopsException | IOException e) {
+			runtimeProgram = dmlTranslator.getRuntimeProgram(dmlProgram, config);
+		} catch (LanguageException | DMLRuntimeException | LopsException | IOException | HopsException e) {
 			throw new MLContextException("Exception occurred while generating runtime program", e);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/conf/CompilerConfig.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/conf/CompilerConfig.java b/src/main/java/org/apache/sysml/conf/CompilerConfig.java
index fa473f0..4fc7765 100644
--- a/src/main/java/org/apache/sysml/conf/CompilerConfig.java
+++ b/src/main/java/org/apache/sysml/conf/CompilerConfig.java
@@ -71,7 +71,10 @@ public class CompilerConfig
 		//data on mlcontext (local) /jmlc (global); ignore unknowns on jmlc
 		IGNORE_READ_WRITE_METADATA, // global skip meta data reads
 		REJECT_READ_WRITE_UNKNOWNS, // ignore missing meta data	
-		MLCONTEXT // execution via new MLContext
+		MLCONTEXT, // execution via new MLContext
+		
+		//code generation enabled 
+		CODEGEN_ENABLED;
 	}
 	
 	//default flags (exposed for testing purposes only)
@@ -98,6 +101,7 @@ public class CompilerConfig
 		_bmap.put(ConfigType.IGNORE_READ_WRITE_METADATA, false);
 		_bmap.put(ConfigType.REJECT_READ_WRITE_UNKNOWNS, true);
 		_bmap.put(ConfigType.MLCONTEXT, false);
+		_bmap.put(ConfigType.CODEGEN_ENABLED, false);
 		
 		_imap = new HashMap<CompilerConfig.ConfigType, Integer>();
 		_imap.put(ConfigType.BLOCK_SIZE, OptimizerUtils.DEFAULT_BLOCKSIZE);

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/conf/ConfigurationManager.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/conf/ConfigurationManager.java b/src/main/java/org/apache/sysml/conf/ConfigurationManager.java
index 7a8c70b..903466a 100644
--- a/src/main/java/org/apache/sysml/conf/ConfigurationManager.java
+++ b/src/main/java/org/apache/sysml/conf/ConfigurationManager.java
@@ -178,6 +178,11 @@ public class ConfigurationManager
 		return getCompilerConfigFlag(ConfigType.PARALLEL_LOCAL_OR_REMOTE_PARFOR);
 	}
 	
+	public static boolean isCodegenEnabled() {
+		return getDMLConfig().getBooleanValue(DMLConfig.CODEGEN)
+			|| getCompilerConfigFlag(ConfigType.CODEGEN_ENABLED);
+	}
+	
 	
 	///////////////////////////////////////
 	// Thread-local classes

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/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 04d521b7..091164d 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
@@ -34,7 +34,6 @@ import org.apache.wink.json4j.JSONObject;
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.api.jmlc.JMLCProxy;
 import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.hops.DataGenOp;
 import org.apache.sysml.hops.DataOp;
@@ -222,8 +221,8 @@ public class Recompiler
 			memo.extract(hops, status);
 			
 			// codegen if enabled
-			if( ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.CODEGEN) 
-					&& SpoofCompiler.RECOMPILE_CODEGEN ) {
+			if( ConfigurationManager.isCodegenEnabled()
+				&& SpoofCompiler.RECOMPILE_CODEGEN ) {
 				Hop.resetVisitStatus(hops);
 				hops = SpoofCompiler.optimize(hops, 
 					(status==null || !status.isInitialCodegen()));
@@ -337,8 +336,8 @@ public class Recompiler
 			hops.refreshMemEstimates(memo); 		
 			
 			// codegen if enabled
-			if( ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.CODEGEN) 
-					&& SpoofCompiler.RECOMPILE_CODEGEN ) {
+			if( ConfigurationManager.isCodegenEnabled()
+				&& SpoofCompiler.RECOMPILE_CODEGEN ) {
 				hops.resetVisitStatus();
 				hops = SpoofCompiler.optimize(hops,
 					(status==null || !status.isInitialCodegen()));

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/parser/DMLProgram.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/DMLProgram.java b/src/main/java/org/apache/sysml/parser/DMLProgram.java
index 15d46c7..82ca7a6 100644
--- a/src/main/java/org/apache/sysml/parser/DMLProgram.java
+++ b/src/main/java/org/apache/sysml/parser/DMLProgram.java
@@ -19,39 +19,17 @@
 
 package org.apache.sysml.parser;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import org.apache.sysml.api.DMLScript;
-import org.apache.sysml.conf.DMLConfig;
-import org.apache.sysml.lops.LopProperties;
-import org.apache.sysml.lops.Lop;
-import org.apache.sysml.lops.LopsException;
-import org.apache.sysml.lops.compile.Dag;
-import org.apache.sysml.parser.Expression.DataType;
-import org.apache.sysml.runtime.DMLRuntimeException;
-import org.apache.sysml.runtime.controlprogram.ExternalFunctionProgramBlock;
-import org.apache.sysml.runtime.controlprogram.ExternalFunctionProgramBlockCP;
-import org.apache.sysml.runtime.controlprogram.ForProgramBlock;
-import org.apache.sysml.runtime.controlprogram.FunctionProgramBlock;
-import org.apache.sysml.runtime.controlprogram.IfProgramBlock;
-import org.apache.sysml.runtime.controlprogram.ParForProgramBlock;
 import org.apache.sysml.runtime.controlprogram.Program;
-import org.apache.sysml.runtime.controlprogram.ProgramBlock;
-import org.apache.sysml.runtime.controlprogram.WhileProgramBlock;
-import org.apache.sysml.runtime.controlprogram.parfor.ProgramConverter;
-import org.apache.sysml.runtime.instructions.CPInstructionParser;
-import org.apache.sysml.runtime.instructions.Instruction;
-import org.apache.sysml.runtime.instructions.cp.VariableCPInstruction;
 
 
 public class DMLProgram 
 {
-	
 	private ArrayList<StatementBlock> _blocks;
 	private HashMap<String, FunctionStatementBlock> _functionBlocks;
 	private HashMap<String,DMLProgram> _namespaces;
@@ -183,582 +161,11 @@ public class DMLProgram
 		return sb.toString();
 	}
 	
-	
-	public Program getRuntimeProgram(DMLConfig config) throws IOException, LanguageException, DMLRuntimeException, LopsException {
-		
-		// constructor resets the set of registered functions
-		Program rtprog = new Program();
-		
-		// for all namespaces, translate function statement blocks into function program blocks
-		for (String namespace : _namespaces.keySet()){
-		
-			for (String fname : getFunctionStatementBlocks(namespace).keySet()){
-				// add program block to program
-				FunctionStatementBlock fsb = getFunctionStatementBlocks(namespace).get(fname);
-				FunctionProgramBlock rtpb = (FunctionProgramBlock)createRuntimeProgramBlock(rtprog, fsb, config);
-				rtprog.addFunctionProgramBlock(namespace, fname, rtpb);
-				rtpb.setRecompileOnce( fsb.isRecompileOnce() );
-			}
-		}
-		
-		// for each top-level block
-		for (StatementBlock sb : _blocks) {
-		
-			// add program block to program
-			ProgramBlock rtpb = createRuntimeProgramBlock(rtprog, sb, config);
-			rtprog.addProgramBlock(rtpb);
-		}
-		
-		
-		return rtprog ;
-	}
-	
-	public ProgramBlock createRuntimeProgramBlock(Program prog, StatementBlock sb, DMLConfig config) 
-		throws IOException, LopsException, DMLRuntimeException 
-	{
-		Dag<Lop> dag = null; 
-		Dag<Lop> pred_dag = null;
-
-		ArrayList<Instruction> instruct;
-		ArrayList<Instruction> pred_instruct = null;
-		
-		ProgramBlock retPB = null;
-		
-		// process While Statement - add runtime program blocks to program
-		if (sb instanceof WhileStatementBlock){
-		
-			// create DAG for loop predicates
-			pred_dag = new Dag<Lop>();
-			((WhileStatementBlock) sb).get_predicateLops().addToDag(pred_dag);
-			
-			// create instructions for loop predicates
-			pred_instruct = new ArrayList<Instruction>();
-			ArrayList<Instruction> pInst = pred_dag.getJobs(null, config);
-			for (Instruction i : pInst ) {
-				pred_instruct.add(i);
-			}
-			
-			// create while program block
-			WhileProgramBlock rtpb = new WhileProgramBlock(prog, pred_instruct);
-			
-			if (rtpb.getPredicateResultVar() == null) {
-				// e.g case : WHILE(continue)
-				if ( ((WhileStatementBlock) sb).get_predicateLops().getExecLocation() == LopProperties.ExecLocation.Data ) {
-					String resultVar = ((WhileStatementBlock) sb).get_predicateLops().getOutputParameters().getLabel();
-					rtpb.setPredicateResultVar( resultVar );
-				}
-				else {
-					LOG.error(sb.printBlockErrorLocation() + "Error in translating the WHILE predicate."); 
-					throw new LopsException(sb.printBlockErrorLocation() + "Error in translating the WHILE predicate."); 
-			
-				}
-			}			
-			//// process the body of the while statement block ////
-			
-			WhileStatementBlock wsb = (WhileStatementBlock)sb;
-			if (wsb.getNumStatements() > 1){
-				LOG.error(wsb.printBlockErrorLocation() + "WhileStatementBlock should only have 1 statement");
-				throw new LopsException(wsb.printBlockErrorLocation() + "WhileStatementBlock should only have 1 statement");
-			}
-			WhileStatement wstmt = (WhileStatement)wsb.getStatement(0);
-			for (StatementBlock sblock : wstmt.getBody()){
-				
-				// process the body
-				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
-				rtpb.addProgramBlock(childBlock);
-			}
-			
-			// check there are actually Lops in to process (loop stmt body will not have any)
-			if (wsb.getLops() != null && !wsb.getLops().isEmpty() ){
-				LOG.error(wsb.printBlockErrorLocation() + "WhileStatementBlock should have no Lops");
-				throw new LopsException(wsb.printBlockErrorLocation() + "WhileStatementBlock should have no Lops");
-			}
-			
-			
-			retPB = rtpb;
-			
-			//post processing for generating missing instructions
-			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
-			
-			// add statement block
-			retPB.setStatementBlock(sb);
-			
-			// add location information
-			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
-		}
-		
-		// process If Statement - add runtime program blocks to program
-		else if (sb instanceof IfStatementBlock){
-		
-			// create DAG for loop predicates
-			pred_dag = new Dag<Lop>();
-			((IfStatementBlock) sb).get_predicateLops().addToDag(pred_dag);
-			
-			// create instructions for loop predicates
-			pred_instruct = new ArrayList<Instruction>();
-			ArrayList<Instruction> pInst = pred_dag.getJobs(null, config);
-			for (Instruction i : pInst ) {
-				pred_instruct.add(i);
-			}
-			
-			// create if program block
-			IfProgramBlock rtpb = new IfProgramBlock(prog, pred_instruct);
-			
-			if (rtpb.getPredicateResultVar() == null ) {
-				// e.g case : If(continue)
-				if ( ((IfStatementBlock) sb).get_predicateLops().getExecLocation() == LopProperties.ExecLocation.Data ) {
-					String resultVar = ((IfStatementBlock) sb).get_predicateLops().getOutputParameters().getLabel();
-					rtpb.setPredicateResultVar( resultVar );
-				}
-				else {
-					LOG.error(sb.printBlockErrorLocation() + "Error in translating the IF predicate."); 
-					throw new LopsException(sb.printBlockErrorLocation() + "Error in translating the IF predicate."); 
-				}
-			}
-			
-			// process the body of the if statement block
-			IfStatementBlock isb = (IfStatementBlock)sb;
-			if (isb.getNumStatements() > 1){
-				LOG.error(isb.printBlockErrorLocation() + "IfStatementBlock should have only 1 statement");
-				throw new LopsException(isb.printBlockErrorLocation() + "IfStatementBlock should have only 1 statement");
-			}
-			IfStatement istmt = (IfStatement)isb.getStatement(0);
-			
-			// process the if body
-			for (StatementBlock sblock : istmt.getIfBody()){
-				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
-				rtpb.addProgramBlockIfBody(childBlock);
-			}
-			
-			// process the else body
-			for (StatementBlock sblock : istmt.getElseBody()){
-				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
-				rtpb.addProgramBlockElseBody(childBlock); 
-			}
-			
-			// check there are actually Lops in to process (loop stmt body will not have any)
-			if (isb.getLops() != null && !isb.getLops().isEmpty() ){
-				LOG.error(isb.printBlockErrorLocation() + "IfStatementBlock should have no Lops");
-				throw new LopsException(isb.printBlockErrorLocation() + "IfStatementBlock should have no Lops");
-			}
-			
-			retPB = rtpb;
-			
-			//post processing for generating missing instructions
-			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
-			
-			// add statement block
-			retPB.setStatementBlock(sb);
-			
-			// add location information
-			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
-		}
-		
-		// process For Statement - add runtime program blocks to program
-		// NOTE: applies to ForStatementBlock and ParForStatementBlock
-		else if (sb instanceof ForStatementBlock) 
-		{ 
-			ForStatementBlock fsb = (ForStatementBlock) sb;
-			
-			// create DAGs for loop predicates 
-			Dag<Lop> fromDag = new Dag<Lop>();
-			Dag<Lop> toDag = new Dag<Lop>();
-			Dag<Lop> incrementDag = new Dag<Lop>();
-			if( fsb.getFromHops()!=null )
-				fsb.getFromLops().addToDag(fromDag);
-			if( fsb.getToHops()!=null )
-				fsb.getToLops().addToDag(toDag);		
-			if( fsb.getIncrementHops()!=null )
-				fsb.getIncrementLops().addToDag(incrementDag);		
-				
-			// create instructions for loop predicates			
-			ArrayList<Instruction> fromInstructions = fromDag.getJobs(null, config);
-			ArrayList<Instruction> toInstructions = toDag.getJobs(null, config);
-			ArrayList<Instruction> incrementInstructions = incrementDag.getJobs(null, config);		
-
-			// create for program block
-			String sbName = null;
-			ForProgramBlock rtpb = null;
-			IterablePredicate iterPred = fsb.getIterPredicate();
-			String [] iterPredData= IterablePredicate.createIterablePredicateVariables(iterPred.getIterVar().getName(),
-					                                                                   fsb.getFromLops(), fsb.getToLops(), fsb.getIncrementLops()); 
-			
-			if( sb instanceof ParForStatementBlock )
-			{
-				sbName = "ParForStatementBlock";
-				rtpb = new ParForProgramBlock(prog, iterPredData,iterPred.getParForParams());
-				ParForProgramBlock pfrtpb = (ParForProgramBlock)rtpb;
-				pfrtpb.setResultVariables( ((ParForStatementBlock)sb).getResultVariables() );
-				pfrtpb.setStatementBlock((ParForStatementBlock)sb); //used for optimization and creating unscoped variables
-			}
-			else //ForStatementBlock
-			{
-				sbName = "ForStatementBlock";
-				rtpb = new ForProgramBlock(prog, iterPredData);
-			}
-			 
-			rtpb.setFromInstructions(      fromInstructions      );
-			rtpb.setToInstructions(        toInstructions        );
-			rtpb.setIncrementInstructions( incrementInstructions );
-			
-			rtpb.setIterablePredicateVars( iterPredData );
-			
-			// process the body of the for statement block
-			if (fsb.getNumStatements() > 1){
-				LOG.error(fsb.printBlockErrorLocation() + " "  + sbName + " should have 1 statement" );
-				throw new LopsException(fsb.printBlockErrorLocation() + " "  + sbName + " should have 1 statement" );
-			}
-			ForStatement fs = (ForStatement)fsb.getStatement(0);
-			for (StatementBlock sblock : fs.getBody()){
-				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
-				rtpb.addProgramBlock(childBlock); 
-			}
-		
-			// check there are actually Lops in to process (loop stmt body will not have any)
-			if (fsb.getLops() != null && !fsb.getLops().isEmpty()){
-				LOG.error(fsb.printBlockErrorLocation() + sbName + " should have no Lops" );
-				throw new LopsException(fsb.printBlockErrorLocation() + sbName + " should have no Lops" );
-			}
-			
-			retPB = rtpb;
-			
-			//post processing for generating missing instructions
-			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
-			
-			// add statement block
-			retPB.setStatementBlock(sb);
-			
-			// add location information
-			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
-		}
-		
-		// process function statement block - add runtime program blocks to program
-		else if (sb instanceof FunctionStatementBlock){
-			
-			FunctionStatementBlock fsb = (FunctionStatementBlock)sb;
-			if (fsb.getNumStatements() > 1){
-				LOG.error(fsb.printBlockErrorLocation() + "FunctionStatementBlock should only have 1 statement");
-				throw new LopsException(fsb.printBlockErrorLocation() + "FunctionStatementBlock should only have 1 statement");
-			}
-			FunctionStatement fstmt = (FunctionStatement)fsb.getStatement(0);
-			FunctionProgramBlock rtpb = null;
-			
-			if (fstmt instanceof ExternalFunctionStatement) {
-				 // create external function program block
-				
-				String execType = ((ExternalFunctionStatement) fstmt)
-                				    .getOtherParams().get(ExternalFunctionStatement.EXEC_TYPE);
-				boolean isCP = (execType.equals(ExternalFunctionStatement.IN_MEMORY)) ? true : false;
-				
-				String scratchSpaceLoc = null;
-				try {
-					scratchSpaceLoc = config.getTextValue(DMLConfig.SCRATCH_SPACE);
-				} catch (Exception e){
-					LOG.error(fsb.printBlockErrorLocation() + "could not retrieve parameter " + DMLConfig.SCRATCH_SPACE + " from DMLConfig");
-				}				
-				StringBuilder buff = new StringBuilder();
-				buff.append(scratchSpaceLoc);
-				buff.append(Lop.FILE_SEPARATOR);
-				buff.append(Lop.PROCESS_PREFIX);
-				buff.append(DMLScript.getUUID());
-				buff.append(Lop.FILE_SEPARATOR);
-				buff.append(ProgramConverter.CP_ROOT_THREAD_ID);
-				buff.append(Lop.FILE_SEPARATOR);
-				buff.append("PackageSupport");
-				buff.append(Lop.FILE_SEPARATOR);
-				String basedir =  buff.toString();
-				
-				if( isCP )
-				{
-					
-					rtpb = new ExternalFunctionProgramBlockCP(prog, 
-									fstmt.getInputParams(), fstmt.getOutputParams(), 
-									((ExternalFunctionStatement) fstmt).getOtherParams(),
-									basedir );					
-				}
-				else
-				{
-					rtpb = new ExternalFunctionProgramBlock(prog, 
-									fstmt.getInputParams(), fstmt.getOutputParams(), 
-									((ExternalFunctionStatement) fstmt).getOtherParams(),
-									basedir);
-				}
-				
-				if (!fstmt.getBody().isEmpty()){
-					LOG.error(fstmt.printErrorLocation() + "ExternalFunctionStatementBlock should have no statement blocks in body");
-					throw new LopsException(fstmt.printErrorLocation() + "ExternalFunctionStatementBlock should have no statement blocks in body");
-				}
-			}
-			else 
-			{
-				// create function program block
-				rtpb = new FunctionProgramBlock(prog, fstmt.getInputParams(), fstmt.getOutputParams());
-				
-				// process the function statement body
-				for (StatementBlock sblock : fstmt.getBody()){	
-					// process the body
-					ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
-					rtpb.addProgramBlock(childBlock);
-				}
-			}
-			
-			// check there are actually Lops in to process (loop stmt body will not have any)
-			if (fsb.getLops() != null && !fsb.getLops().isEmpty()){
-				LOG.error(fsb.printBlockErrorLocation() + "FunctionStatementBlock should have no Lops");
-				throw new LopsException(fsb.printBlockErrorLocation() + "FunctionStatementBlock should have no Lops");
-			}
-			
-			retPB = rtpb;
-			
-			// add location information
-			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
-		}
-		else {
-	
-			// handle general case
-			ProgramBlock rtpb = new ProgramBlock(prog);
-		
-			// DAGs for Lops
-			dag = new Dag<Lop>();
-
-			// check there are actually Lops in to process (loop stmt body will not have any)
-			if (sb.getLops() != null && !sb.getLops().isEmpty()){
-			
-				for (Lop l : sb.getLops()) {
-					l.addToDag(dag);
-				}
-				
-				// Instructions for Lobs DAGs
-				instruct = dag.getJobs(sb, config);
-				rtpb.addInstructions(instruct);
-			}
-			
-			/*// TODO: check with Doug
-			// add instruction for a function call
-			if (sb.getFunctionCallInst() != null){
-				rtpb.addInstruction(sb.getFunctionCallInst());
-			}*/
-			
-			retPB = rtpb;
-			
-			//post processing for generating missing instructions
-			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
-			
-			// add statement block
-			retPB.setStatementBlock(sb);
-			
-			// add location information
-			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
-		}
-		
-		return retPB;
-	}	
-	
-	/**
-	 * Post processing of each created program block in order to adhere to livein/liveout
-	 * (currently needed for cleanup (especially for caching) of intermediate results if the last datasink 
-	 * is an external function because instructions of external functions are created outside hops/lops,
-	 * e.g., X=..., Y=fun(X) and X is not used afterwards )
-	 * 
-	 * NOTES: 
-	 * (1) Rule1: checking livein and liveout is sufficient because the last external function is in its own
-	 * programblock anyway.
-	 * (2) as we cannot efficiently distinguish if the problematic var is created by an external function
-	 * or some other instruction, we generate RMVAR instructions although for vars created by non-CP
-	 * external functions RMFILEVAR instructions are required. However, all remaining files in scratch_space
-	 * are cleaned after execution anyway.
-	 * (3) As an alternative to doing rule 2, we could also check for existing objects in createvar and function invocation
-	 * (or generic at program block level) and remove objects of previous iterations accordingly (but objects of last iteration
-	 * would still require separate cleanup).
-	 * 
-	 * TODO: MB: external function invocations should become hops/lops as well (see instruction gen in DMLTranslator), 
-	 * (currently not possible at Hops/Lops level due the requirement of multiple outputs for functions) 
-	 * TODO: MB: we should in general always leverage livein/liveout during hops/lops generation.
-	 * TODO: MB: verify and correct can be removed once everything is integrated in hops/lops generation
-	 * 
-	 * @param in
-	 * @param out
-	 * @param pb
-	 * @return
-	 * @throws DMLRuntimeException 
-	 */
-	@SuppressWarnings("unused")
-	private ProgramBlock verifyAndCorrectProgramBlock(VariableSet in, VariableSet out, VariableSet kill, ProgramBlock pb) 
-		throws DMLRuntimeException
-	{	
-		//RULE 1: if in IN and not in OUT, then there should be an rmvar or rmfilevar inst
-		//(currently required for specific cases of external functions)
-		for( String varName : in.getVariableNames() )
-			if( !out.containsVariable(varName) ) 
-			{
-				DataType dt = in.getVariable(varName).getDataType();
-				if( !(dt==DataType.MATRIX || dt==DataType.UNKNOWN) )
-					continue; //skip rm instructions for non-matrix objects
-				
-				boolean foundRMInst = rContainsRMInstruction(pb, varName);
-				
-				if( !foundRMInst )
-				{
-					//create RMVAR instruction and put it into the programblock
-					Instruction inst = createCleanupInstruction(varName);
-					
-					inst.setLocation(in.getVariable(varName));
-					
-					addCleanupInstruction(pb, inst);
-
-					LOG.trace("Adding instruction (r1) "+inst.toString());
-				}		
-			}
-
-		//RULE 2: if in KILL and not in IN and not in OUT, then there should be an rmvar or rmfilevar inst
-		//(currently required for specific cases of nested loops)
-		for( String varName : kill.getVariableNames() )
-			if( (!in.containsVariable(varName)) && (!out.containsVariable(varName)) ) 
-			{
-				DataType dt = kill.getVariable(varName).getDataType();
-				if( !(dt==DataType.MATRIX || dt==DataType.UNKNOWN) )
-					continue; //skip rm instructions for non-matrix objects
-				
-				boolean foundRMInst = rContainsRMInstruction(pb, varName);
-				
-				if( !foundRMInst )
-				{
-					//create RMVAR instruction and put it into the programblock
-					Instruction inst = createCleanupInstruction(varName);
-					
-					inst.setLocation(kill.getVariable(varName));
-					
-					//System.out.println("add rvar rule2 "+inst.toString());
-					addCleanupInstruction(pb, inst);
-					
-					LOG.trace("Adding instruction (r2) "+inst.toString());
-				}		
-			}
-		
-		return pb;
-	}
-	
-	private Instruction createCleanupInstruction(String varName) 
-		throws DMLRuntimeException
-	{
-		//(example "CP+Lops.OPERAND_DELIMITOR+rmvar+Lops.OPERAND_DELIMITOR+Var7")
-		StringBuilder sb = new StringBuilder();
-		sb.append("CP");
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append("rmvar");
-		sb.append(Lop.OPERAND_DELIMITOR);
-		sb.append(varName);
-		String str = sb.toString();
-		Instruction inst = CPInstructionParser.parseSingleInstruction( str );
-		
-		return inst;
-	}
-	
-	/**
-	 * Determines if the given program block includes a RMVAR or RMFILEVAR
-	 * instruction for the given varName.
-	 * 
-	 * @param pb program block
-	 * @param varName variable name
-	 * @return true if program block contains remove instruction for variable
-	 */
-	private boolean rContainsRMInstruction(ProgramBlock pb, String varName)
-	{	
-		if (pb instanceof WhileProgramBlock)
-		{
-			WhileProgramBlock tmp = (WhileProgramBlock)pb;	
-			for( ProgramBlock c : tmp.getChildBlocks() )
-				if( rContainsRMInstruction(c, varName) )
-					return true;
-		}
-		else if (pb instanceof IfProgramBlock)
-		{
-			IfProgramBlock tmp = (IfProgramBlock)pb;	
-			for( ProgramBlock c : tmp.getChildBlocksIfBody() )
-				if( rContainsRMInstruction(c, varName) )
-					return true;
-			for( ProgramBlock c : tmp.getChildBlocksElseBody() )
-				if( rContainsRMInstruction(c, varName) )
-					return true;
-		}
-		else if (pb instanceof ForProgramBlock) //includes ParFORProgramBlock
-		{ 
-			ForProgramBlock tmp = (ForProgramBlock)pb;	
-			for( ProgramBlock c : tmp.getChildBlocks() )
-				if( rContainsRMInstruction(c, varName) )
-					return true;
-		}		
-		else if (  pb instanceof FunctionProgramBlock ) //includes ExternalFunctionProgramBlock and ExternalFunctionProgramBlockCP)
-		{
-			//do nothing
-		}
-		else 
-		{
-			for( Instruction inst : pb.getInstructions() )
-				if( inst instanceof VariableCPInstruction 
-					&& ((VariableCPInstruction) inst).isRemoveVariable(varName) )
-					return true;
-		}
-		
-		return false;
-	}
-	
-	/**
-	 * Adds the generated cleanup RMVAR instruction to the given program block.
-	 * In case of generic (last-level) programblocks it is added to the end of 
-	 * the list of instructions, while for complex program blocks it is added to
-	 * the end of the list of exit instructions.
-	 * 
-	 * @param pb program block
-	 * @param inst instruction
-	 * @throws DMLRuntimeException if DMLRuntimeException occurs
-	 */
-	private void addCleanupInstruction( ProgramBlock pb, Instruction inst ) 
-		throws DMLRuntimeException
-	{
-		//System.out.println("Adding rm var instructions: "+inst.toString());
-		
-		if (pb instanceof WhileProgramBlock)
-		{
-			WhileProgramBlock wpb = (WhileProgramBlock)pb;
-			ArrayList<ProgramBlock> childs = wpb.getChildBlocks();
-			if( !childs.get(childs.size()-1).getInstructions().isEmpty() ) //generic last level pb
-				childs.get(childs.size()-1).addInstruction(inst);
-			else{
-				ProgramBlock pbNew = new ProgramBlock(pb.getProgram());
-				pbNew.addInstruction(inst);
-				childs.add(pbNew); 
-			}
-		}
-		else if (pb instanceof ForProgramBlock) //includes ParFORProgramBlock
-		{
-			ForProgramBlock wpb = (ForProgramBlock)pb;
-			ArrayList<ProgramBlock> childs = wpb.getChildBlocks();
-			if( !childs.get(childs.size()-1).getInstructions().isEmpty() ) //generic last level pb
-				childs.get(childs.size()-1).addInstruction(inst);
-			else{
-				ProgramBlock pbNew = new ProgramBlock(pb.getProgram());
-				pbNew.addInstruction(inst);
-				childs.add(pbNew); 
-			}
-		}
-		else if (pb instanceof IfProgramBlock)
-			((IfProgramBlock)pb).addExitInstruction(inst);
-		else if (   pb instanceof FunctionProgramBlock )  //includes ExternalFunctionProgramBlock and ExternalFunctionProgramBlockCP)
-			; //do nothing
-		else 
-		{
-			pb.addInstruction(inst); //add inst at end of pb	
-		}
-	}
-	
-	public static String constructFunctionKey(String fnamespace, String fname)
-	{
+	public static String constructFunctionKey(String fnamespace, String fname) {
 		return fnamespace + Program.KEY_DELIM + fname;
 	}
 	
-	public static String[] splitFunctionKey(String fkey)
-	{
+	public static String[] splitFunctionKey(String fkey) {
 		return fkey.split(Program.KEY_DELIM);
 	}
 }

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/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 61f7217..3708bb6 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -27,7 +27,9 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.ConfigurationManager;
+import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.AggBinaryOp;
 import org.apache.sysml.hops.AggUnaryOp;
 import org.apache.sysml.hops.BinaryOp;
@@ -58,12 +60,16 @@ import org.apache.sysml.hops.ReorgOp;
 import org.apache.sysml.hops.TernaryOp;
 import org.apache.sysml.hops.UnaryOp;
 import org.apache.sysml.hops.codegen.SpoofCompiler;
+import org.apache.sysml.hops.codegen.SpoofCompiler.IntegrationType;
+import org.apache.sysml.hops.codegen.SpoofCompiler.PlanCachePolicy;
 import org.apache.sysml.hops.ipa.InterProceduralAnalysis;
 import org.apache.sysml.hops.recompile.Recompiler;
 import org.apache.sysml.hops.rewrite.HopRewriteUtils;
 import org.apache.sysml.hops.rewrite.ProgramRewriter;
 import org.apache.sysml.lops.Lop;
+import org.apache.sysml.lops.LopProperties;
 import org.apache.sysml.lops.LopsException;
+import org.apache.sysml.lops.compile.Dag;
 import org.apache.sysml.parser.Expression.BuiltinFunctionOp;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.FormatType;
@@ -71,7 +77,17 @@ import org.apache.sysml.parser.Expression.ParameterizedBuiltinFunctionOp;
 import org.apache.sysml.parser.Expression.ValueType;
 import org.apache.sysml.parser.PrintStatement.PRINTTYPE;
 import org.apache.sysml.runtime.DMLRuntimeException;
+import org.apache.sysml.runtime.controlprogram.ExternalFunctionProgramBlock;
+import org.apache.sysml.runtime.controlprogram.ExternalFunctionProgramBlockCP;
+import org.apache.sysml.runtime.controlprogram.ForProgramBlock;
+import org.apache.sysml.runtime.controlprogram.FunctionProgramBlock;
+import org.apache.sysml.runtime.controlprogram.IfProgramBlock;
+import org.apache.sysml.runtime.controlprogram.ParForProgramBlock;
 import org.apache.sysml.runtime.controlprogram.Program;
+import org.apache.sysml.runtime.controlprogram.ProgramBlock;
+import org.apache.sysml.runtime.controlprogram.WhileProgramBlock;
+import org.apache.sysml.runtime.controlprogram.parfor.ProgramConverter;
+import org.apache.sysml.runtime.instructions.Instruction;
 
 
 public class DMLTranslator 
@@ -256,7 +272,7 @@ public class DMLTranslator
 	}
 
 	public void rewriteHopsDAG(DMLProgram dmlp) 
-		throws ParseException, LanguageException, HopsException 
+		throws ParseException, LanguageException, HopsException, DMLRuntimeException 
 	{
 		//apply hop rewrites (static rewrites)
 		ProgramRewriter rewriter = new ProgramRewriter(true, false);
@@ -275,10 +291,21 @@ public class DMLTranslator
 		rewriter2.rewriteProgramHopDAGs(dmlp);
 		resetHopsDAGVisitStatus(dmlp);
 		
-		// Compute memory estimates for all the hops. These estimates are used
-		// subsequently in various optimizations, e.g. CP vs. MR scheduling and parfor.
+		//compute memory estimates for all the hops. These estimates are used
+		//subsequently in various optimizations, e.g. CP vs. MR scheduling and parfor.
 		refreshMemEstimates(dmlp);
 		resetHopsDAGVisitStatus(dmlp);
+		
+		//enhance HOP DAGs by automatic operator fusion
+		DMLConfig dmlconf = ConfigurationManager.getDMLConfig();
+		if( ConfigurationManager.isCodegenEnabled() ){
+			SpoofCompiler.PLAN_CACHE_POLICY = PlanCachePolicy.get(
+				dmlconf.getBooleanValue(DMLConfig.CODEGEN_PLANCACHE),
+				dmlconf.getIntValue(DMLConfig.CODEGEN_LITERALS)==2);
+			SpoofCompiler.setExecTypeSpecificJavaCompiler();
+			if( SpoofCompiler.INTEGRATION==IntegrationType.HOPS )
+				codgenHopsDAG(dmlp);
+		}
 	}
 	
 	public void codgenHopsDAG(DMLProgram dmlp)
@@ -418,6 +445,376 @@ public class DMLTranslator
 		
 	} // end method
 	
+	
+	public Program getRuntimeProgram(DMLProgram prog, DMLConfig config) 
+		throws IOException, LanguageException, DMLRuntimeException, LopsException, HopsException 
+	{	
+		// constructor resets the set of registered functions
+		Program rtprog = new Program();
+		
+		// for all namespaces, translate function statement blocks into function program blocks
+		for (String namespace : prog.getNamespaces().keySet()){
+		
+			for (String fname : prog.getFunctionStatementBlocks(namespace).keySet()){
+				// add program block to program
+				FunctionStatementBlock fsb = prog.getFunctionStatementBlocks(namespace).get(fname);
+				FunctionProgramBlock rtpb = (FunctionProgramBlock)createRuntimeProgramBlock(rtprog, fsb, config);
+				rtprog.addFunctionProgramBlock(namespace, fname, rtpb);
+				rtpb.setRecompileOnce( fsb.isRecompileOnce() );
+			}
+		}
+		
+		// translate all top-level statement blocks to program blocks
+		for (StatementBlock sb : prog.getStatementBlocks() ) {
+		
+			// add program block to program
+			ProgramBlock rtpb = createRuntimeProgramBlock(rtprog, sb, config);
+			rtprog.addProgramBlock(rtpb);
+		}
+		
+		//enhance runtime program by automatic operator fusion
+		if( ConfigurationManager.isCodegenEnabled() 
+			&& SpoofCompiler.INTEGRATION==IntegrationType.RUNTIME ){
+			codgenHopsDAG(rtprog);
+		}
+		
+		return rtprog ;
+	}
+	
+	public ProgramBlock createRuntimeProgramBlock(Program prog, StatementBlock sb, DMLConfig config) 
+		throws IOException, LopsException, DMLRuntimeException 
+	{
+		Dag<Lop> dag = null; 
+		Dag<Lop> pred_dag = null;
+
+		ArrayList<Instruction> instruct;
+		ArrayList<Instruction> pred_instruct = null;
+		
+		ProgramBlock retPB = null;
+		
+		// process While Statement - add runtime program blocks to program
+		if (sb instanceof WhileStatementBlock){
+		
+			// create DAG for loop predicates
+			pred_dag = new Dag<Lop>();
+			((WhileStatementBlock) sb).get_predicateLops().addToDag(pred_dag);
+			
+			// create instructions for loop predicates
+			pred_instruct = new ArrayList<Instruction>();
+			ArrayList<Instruction> pInst = pred_dag.getJobs(null, config);
+			for (Instruction i : pInst ) {
+				pred_instruct.add(i);
+			}
+			
+			// create while program block
+			WhileProgramBlock rtpb = new WhileProgramBlock(prog, pred_instruct);
+			
+			if (rtpb.getPredicateResultVar() == null) {
+				// e.g case : WHILE(continue)
+				if ( ((WhileStatementBlock) sb).get_predicateLops().getExecLocation() == LopProperties.ExecLocation.Data ) {
+					String resultVar = ((WhileStatementBlock) sb).get_predicateLops().getOutputParameters().getLabel();
+					rtpb.setPredicateResultVar( resultVar );
+				}
+				else {
+					LOG.error(sb.printBlockErrorLocation() + "Error in translating the WHILE predicate."); 
+					throw new LopsException(sb.printBlockErrorLocation() + "Error in translating the WHILE predicate."); 
+			
+				}
+			}			
+			//// process the body of the while statement block ////
+			
+			WhileStatementBlock wsb = (WhileStatementBlock)sb;
+			if (wsb.getNumStatements() > 1){
+				LOG.error(wsb.printBlockErrorLocation() + "WhileStatementBlock should only have 1 statement");
+				throw new LopsException(wsb.printBlockErrorLocation() + "WhileStatementBlock should only have 1 statement");
+			}
+			WhileStatement wstmt = (WhileStatement)wsb.getStatement(0);
+			for (StatementBlock sblock : wstmt.getBody()){
+				
+				// process the body
+				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
+				rtpb.addProgramBlock(childBlock);
+			}
+			
+			// check there are actually Lops in to process (loop stmt body will not have any)
+			if (wsb.getLops() != null && !wsb.getLops().isEmpty() ){
+				LOG.error(wsb.printBlockErrorLocation() + "WhileStatementBlock should have no Lops");
+				throw new LopsException(wsb.printBlockErrorLocation() + "WhileStatementBlock should have no Lops");
+			}
+			
+			
+			retPB = rtpb;
+			
+			//post processing for generating missing instructions
+			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
+			
+			// add statement block
+			retPB.setStatementBlock(sb);
+			
+			// add location information
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+		}
+		
+		// process If Statement - add runtime program blocks to program
+		else if (sb instanceof IfStatementBlock){
+		
+			// create DAG for loop predicates
+			pred_dag = new Dag<Lop>();
+			((IfStatementBlock) sb).get_predicateLops().addToDag(pred_dag);
+			
+			// create instructions for loop predicates
+			pred_instruct = new ArrayList<Instruction>();
+			ArrayList<Instruction> pInst = pred_dag.getJobs(null, config);
+			for (Instruction i : pInst ) {
+				pred_instruct.add(i);
+			}
+			
+			// create if program block
+			IfProgramBlock rtpb = new IfProgramBlock(prog, pred_instruct);
+			
+			if (rtpb.getPredicateResultVar() == null ) {
+				// e.g case : If(continue)
+				if ( ((IfStatementBlock) sb).get_predicateLops().getExecLocation() == LopProperties.ExecLocation.Data ) {
+					String resultVar = ((IfStatementBlock) sb).get_predicateLops().getOutputParameters().getLabel();
+					rtpb.setPredicateResultVar( resultVar );
+				}
+				else {
+					LOG.error(sb.printBlockErrorLocation() + "Error in translating the IF predicate."); 
+					throw new LopsException(sb.printBlockErrorLocation() + "Error in translating the IF predicate."); 
+				}
+			}
+			
+			// process the body of the if statement block
+			IfStatementBlock isb = (IfStatementBlock)sb;
+			if (isb.getNumStatements() > 1){
+				LOG.error(isb.printBlockErrorLocation() + "IfStatementBlock should have only 1 statement");
+				throw new LopsException(isb.printBlockErrorLocation() + "IfStatementBlock should have only 1 statement");
+			}
+			IfStatement istmt = (IfStatement)isb.getStatement(0);
+			
+			// process the if body
+			for (StatementBlock sblock : istmt.getIfBody()){
+				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
+				rtpb.addProgramBlockIfBody(childBlock);
+			}
+			
+			// process the else body
+			for (StatementBlock sblock : istmt.getElseBody()){
+				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
+				rtpb.addProgramBlockElseBody(childBlock); 
+			}
+			
+			// check there are actually Lops in to process (loop stmt body will not have any)
+			if (isb.getLops() != null && !isb.getLops().isEmpty() ){
+				LOG.error(isb.printBlockErrorLocation() + "IfStatementBlock should have no Lops");
+				throw new LopsException(isb.printBlockErrorLocation() + "IfStatementBlock should have no Lops");
+			}
+			
+			retPB = rtpb;
+			
+			//post processing for generating missing instructions
+			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
+			
+			// add statement block
+			retPB.setStatementBlock(sb);
+			
+			// add location information
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+		}
+		
+		// process For Statement - add runtime program blocks to program
+		// NOTE: applies to ForStatementBlock and ParForStatementBlock
+		else if (sb instanceof ForStatementBlock) 
+		{ 
+			ForStatementBlock fsb = (ForStatementBlock) sb;
+			
+			// create DAGs for loop predicates 
+			Dag<Lop> fromDag = new Dag<Lop>();
+			Dag<Lop> toDag = new Dag<Lop>();
+			Dag<Lop> incrementDag = new Dag<Lop>();
+			if( fsb.getFromHops()!=null )
+				fsb.getFromLops().addToDag(fromDag);
+			if( fsb.getToHops()!=null )
+				fsb.getToLops().addToDag(toDag);		
+			if( fsb.getIncrementHops()!=null )
+				fsb.getIncrementLops().addToDag(incrementDag);		
+				
+			// create instructions for loop predicates			
+			ArrayList<Instruction> fromInstructions = fromDag.getJobs(null, config);
+			ArrayList<Instruction> toInstructions = toDag.getJobs(null, config);
+			ArrayList<Instruction> incrementInstructions = incrementDag.getJobs(null, config);		
+
+			// create for program block
+			String sbName = null;
+			ForProgramBlock rtpb = null;
+			IterablePredicate iterPred = fsb.getIterPredicate();
+			String [] iterPredData= IterablePredicate.createIterablePredicateVariables(iterPred.getIterVar().getName(),
+					                                                                   fsb.getFromLops(), fsb.getToLops(), fsb.getIncrementLops()); 
+			
+			if( sb instanceof ParForStatementBlock )
+			{
+				sbName = "ParForStatementBlock";
+				rtpb = new ParForProgramBlock(prog, iterPredData,iterPred.getParForParams());
+				ParForProgramBlock pfrtpb = (ParForProgramBlock)rtpb;
+				pfrtpb.setResultVariables( ((ParForStatementBlock)sb).getResultVariables() );
+				pfrtpb.setStatementBlock((ParForStatementBlock)sb); //used for optimization and creating unscoped variables
+			}
+			else //ForStatementBlock
+			{
+				sbName = "ForStatementBlock";
+				rtpb = new ForProgramBlock(prog, iterPredData);
+			}
+			 
+			rtpb.setFromInstructions(      fromInstructions      );
+			rtpb.setToInstructions(        toInstructions        );
+			rtpb.setIncrementInstructions( incrementInstructions );
+			
+			rtpb.setIterablePredicateVars( iterPredData );
+			
+			// process the body of the for statement block
+			if (fsb.getNumStatements() > 1){
+				LOG.error(fsb.printBlockErrorLocation() + " "  + sbName + " should have 1 statement" );
+				throw new LopsException(fsb.printBlockErrorLocation() + " "  + sbName + " should have 1 statement" );
+			}
+			ForStatement fs = (ForStatement)fsb.getStatement(0);
+			for (StatementBlock sblock : fs.getBody()){
+				ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
+				rtpb.addProgramBlock(childBlock); 
+			}
+		
+			// check there are actually Lops in to process (loop stmt body will not have any)
+			if (fsb.getLops() != null && !fsb.getLops().isEmpty()){
+				LOG.error(fsb.printBlockErrorLocation() + sbName + " should have no Lops" );
+				throw new LopsException(fsb.printBlockErrorLocation() + sbName + " should have no Lops" );
+			}
+			
+			retPB = rtpb;
+			
+			//post processing for generating missing instructions
+			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
+			
+			// add statement block
+			retPB.setStatementBlock(sb);
+			
+			// add location information
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+		}
+		
+		// process function statement block - add runtime program blocks to program
+		else if (sb instanceof FunctionStatementBlock){
+			
+			FunctionStatementBlock fsb = (FunctionStatementBlock)sb;
+			if (fsb.getNumStatements() > 1){
+				LOG.error(fsb.printBlockErrorLocation() + "FunctionStatementBlock should only have 1 statement");
+				throw new LopsException(fsb.printBlockErrorLocation() + "FunctionStatementBlock should only have 1 statement");
+			}
+			FunctionStatement fstmt = (FunctionStatement)fsb.getStatement(0);
+			FunctionProgramBlock rtpb = null;
+			
+			if (fstmt instanceof ExternalFunctionStatement) {
+				 // create external function program block
+				
+				String execType = ((ExternalFunctionStatement) fstmt)
+                				    .getOtherParams().get(ExternalFunctionStatement.EXEC_TYPE);
+				boolean isCP = (execType.equals(ExternalFunctionStatement.IN_MEMORY)) ? true : false;
+				
+				String scratchSpaceLoc = null;
+				try {
+					scratchSpaceLoc = config.getTextValue(DMLConfig.SCRATCH_SPACE);
+				} catch (Exception e){
+					LOG.error(fsb.printBlockErrorLocation() + "could not retrieve parameter " + DMLConfig.SCRATCH_SPACE + " from DMLConfig");
+				}				
+				StringBuilder buff = new StringBuilder();
+				buff.append(scratchSpaceLoc);
+				buff.append(Lop.FILE_SEPARATOR);
+				buff.append(Lop.PROCESS_PREFIX);
+				buff.append(DMLScript.getUUID());
+				buff.append(Lop.FILE_SEPARATOR);
+				buff.append(ProgramConverter.CP_ROOT_THREAD_ID);
+				buff.append(Lop.FILE_SEPARATOR);
+				buff.append("PackageSupport");
+				buff.append(Lop.FILE_SEPARATOR);
+				String basedir =  buff.toString();
+				
+				if( isCP )
+				{
+					
+					rtpb = new ExternalFunctionProgramBlockCP(prog, 
+									fstmt.getInputParams(), fstmt.getOutputParams(), 
+									((ExternalFunctionStatement) fstmt).getOtherParams(),
+									basedir );					
+				}
+				else
+				{
+					rtpb = new ExternalFunctionProgramBlock(prog, 
+									fstmt.getInputParams(), fstmt.getOutputParams(), 
+									((ExternalFunctionStatement) fstmt).getOtherParams(),
+									basedir);
+				}
+				
+				if (!fstmt.getBody().isEmpty()){
+					LOG.error(fstmt.printErrorLocation() + "ExternalFunctionStatementBlock should have no statement blocks in body");
+					throw new LopsException(fstmt.printErrorLocation() + "ExternalFunctionStatementBlock should have no statement blocks in body");
+				}
+			}
+			else 
+			{
+				// create function program block
+				rtpb = new FunctionProgramBlock(prog, fstmt.getInputParams(), fstmt.getOutputParams());
+				
+				// process the function statement body
+				for (StatementBlock sblock : fstmt.getBody()){	
+					// process the body
+					ProgramBlock childBlock = createRuntimeProgramBlock(prog, sblock, config);
+					rtpb.addProgramBlock(childBlock);
+				}
+			}
+			
+			// check there are actually Lops in to process (loop stmt body will not have any)
+			if (fsb.getLops() != null && !fsb.getLops().isEmpty()){
+				LOG.error(fsb.printBlockErrorLocation() + "FunctionStatementBlock should have no Lops");
+				throw new LopsException(fsb.printBlockErrorLocation() + "FunctionStatementBlock should have no Lops");
+			}
+			
+			retPB = rtpb;
+			
+			// add location information
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+		}
+		else {
+	
+			// handle general case
+			ProgramBlock rtpb = new ProgramBlock(prog);
+		
+			// DAGs for Lops
+			dag = new Dag<Lop>();
+
+			// check there are actually Lops in to process (loop stmt body will not have any)
+			if (sb.getLops() != null && !sb.getLops().isEmpty()){
+			
+				for (Lop l : sb.getLops()) {
+					l.addToDag(dag);
+				}
+				
+				// Instructions for Lobs DAGs
+				instruct = dag.getJobs(sb, config);
+				rtpb.addInstructions(instruct);
+			}
+			
+			retPB = rtpb;
+			
+			//post processing for generating missing instructions
+			//retPB = verifyAndCorrectProgramBlock(sb.liveIn(), sb.liveOut(), sb._kill, retPB);
+			
+			// add statement block
+			retPB.setStatementBlock(sb);
+			
+			// add location information
+			retPB.setAllPositions(sb.getFilename(), sb.getBeginLine(), sb.getBeginColumn(), sb.getEndLine(), sb.getEndColumn());
+		}
+		
+		return retPB;
+	}
 		
 	public void printLops(DMLProgram dmlp) throws ParseException, LanguageException, HopsException, LopsException {
 		if (LOG.isDebugEnabled()){
@@ -1533,8 +1930,7 @@ public class DMLTranslator
 			throw new ParseException(target.printErrorLocation() + " must define matrix " + target.getName() + " before indexing operations are allowed ");
 		}
 		
-		//TODO Doug, please verify this (we need probably a cleaner way than this postprocessing)
-		if( sourceOp.getDataType() == DataType.MATRIX && source.getOutput().getDataType() == DataType.SCALAR )
+		if( sourceOp.getDataType().isMatrix() && source.getOutput().getDataType().isScalar() )
 			sourceOp.setDataType(DataType.SCALAR);
 		
 		Hop leftIndexOp = new LeftIndexingOp(target.getName(), target.getDataType(), ValueType.DOUBLE, 

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/ProgramRecompiler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/ProgramRecompiler.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/ProgramRecompiler.java
index 2d03af3..189d061 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/ProgramRecompiler.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/ProgramRecompiler.java
@@ -71,8 +71,7 @@ public class ProgramRecompiler
 		
 		//construct runtime program from lops
 		for( StatementBlock sb : sbs ) {
-			DMLProgram prog = sb.getDMLProg();
-			ret.add( prog.createRuntimeProgramBlock(rtprog, sb, config) );
+			ret.add(dmlt.createRuntimeProgramBlock(rtprog, sb, config));
 		}
 		
 		return ret;

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/main/java/org/apache/sysml/utils/Statistics.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/Statistics.java b/src/main/java/org/apache/sysml/utils/Statistics.java
index a72b89e..847682c 100644
--- a/src/main/java/org/apache/sysml/utils/Statistics.java
+++ b/src/main/java/org/apache/sysml/utils/Statistics.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.LongAdder;
 
 import org.apache.sysml.api.DMLScript;
 import org.apache.sysml.conf.ConfigurationManager;
-import org.apache.sysml.conf.DMLConfig;
 import org.apache.sysml.hops.OptimizerUtils;
 import org.apache.sysml.runtime.controlprogram.caching.CacheStatistics;
 import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
@@ -729,7 +728,7 @@ public class Statistics
 				sb.append("Functions recompiled:\t\t" + getFunRecompiles() + ".\n");
 				sb.append("Functions recompile time:\t" + String.format("%.3f", ((double)getFunRecompileTime())/1000000000) + " sec.\n");	
 			}
-			if( ConfigurationManager.getDMLConfig().getBooleanValue(DMLConfig.CODEGEN) ) {
+			if( ConfigurationManager.isCodegenEnabled() ) {
 				sb.append("Codegen compile (DAG, CP, JC):\t" + getCodegenDAGCompile() + "/" + getCodegenCPlanCompile() + "/" + getCodegenClassCompile() + ".\n");
 				sb.append("Codegen compile times (DAG,JC):\t" + String.format("%.3f", (double)getCodegenCompileTime()/1000000000) + "/" + 
 						String.format("%.3f", (double)getCodegenClassCompileTime()/1000000000)  + " sec.\n");

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/test/java/org/apache/sysml/test/integration/functions/codegen/APICodegenTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/codegen/APICodegenTest.java b/src/test/java/org/apache/sysml/test/integration/functions/codegen/APICodegenTest.java
new file mode 100644
index 0000000..7e6ead1
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/codegen/APICodegenTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.codegen;
+
+import static org.apache.sysml.api.mlcontext.ScriptFactory.dml;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.sysml.api.DMLScript;
+import org.apache.sysml.api.jmlc.Connection;
+import org.apache.sysml.api.jmlc.PreparedScript;
+import org.apache.sysml.api.mlcontext.MLContext;
+import org.apache.sysml.api.mlcontext.Script;
+import org.apache.sysml.conf.CompilerConfig.ConfigType;
+import org.apache.sysml.runtime.controlprogram.context.SparkExecutionContext;
+import org.apache.sysml.runtime.matrix.data.MatrixBlock;
+import org.apache.sysml.runtime.util.DataConverter;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.utils.Statistics;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class APICodegenTest extends AutomatedTestBase
+{
+	protected final static String TEST_DIR = "org/apache/sysml/api/mlcontext";
+	protected final static String TEST_NAME = "MLContext";
+
+	private final static int rows = 100;
+	private final static int cols = 1600;
+	private final static double sparsity = 0.7;
+
+	@Override
+	public void setUp() {
+		addTestConfiguration(TEST_DIR, TEST_NAME);
+		getAndLoadTestConfiguration(TEST_NAME);
+	}
+	
+	@Test
+	public void testCodegenMLContext() {
+		runMLContextParforDatasetTest(false);
+	}
+	
+	@Test
+	public void testCodegenJMLCTest() {
+		runMLContextParforDatasetTest(true);
+	}
+
+	private void runMLContextParforDatasetTest(boolean jmlc) 
+	{
+		try {
+			double[][] X = getRandomMatrix(rows, cols, -10, 10, sparsity, 76543); 
+			MatrixBlock mX = DataConverter.convertToMatrixBlock(X); 
+			
+			String s = "X = read(\"/tmp\");"
+				+ "R = colSums(X/rowSums(X));"
+				+ "write(R, \"tmp2\")";
+			
+			//execute scripts
+			if( jmlc ) {
+				DMLScript.STATISTICS = true;
+				Connection conn = new Connection(ConfigType.CODEGEN_ENABLED, 
+					ConfigType.ALLOW_DYN_RECOMPILATION);
+				PreparedScript pscript = conn.prepareScript(
+					s, new String[]{"X"}, new String[]{"R"}, false); 
+				pscript.setMatrix("X", mX, false);
+				pscript.executeScript();
+				conn.close();
+				System.out.println(Statistics.display());
+			}
+			else {
+				SparkConf conf = SparkExecutionContext.createSystemMLSparkConf()
+					.setAppName("MLContextTest").setMaster("local");
+				JavaSparkContext sc = new JavaSparkContext(conf);
+				MLContext ml = new MLContext(sc);
+				ml.setConfigProperty("codegen.enabled", "true");
+				ml.setStatistics(true);
+				Script script = dml(s).in("X", mX).out("R");
+				ml.execute(script);
+				ml.resetConfig();
+				sc.stop();
+				ml.close();
+			}
+			
+			//check for generated operator
+			Assert.assertTrue(heavyHittersContainsSubString("spoofRA"));
+		}
+		catch(Exception ex) {
+			throw new RuntimeException(ex);
+		}
+	}
+
+	@After
+	public void tearDown() {
+		super.tearDown();
+	}
+}

http://git-wip-us.apache.org/repos/asf/systemml/blob/856230c5/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
----------------------------------------------------------------------
diff --git a/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java b/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
index a7d45bf..63be419 100644
--- a/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
+++ b/src/test_suites/java/org/apache/sysml/test/integration/functions/codegen/ZPackageSuite.java
@@ -33,6 +33,7 @@ import org.junit.runners.Suite;
 	AlgorithmMLogreg.class,
 	AlgorithmMSVM.class,
 	AlgorithmPNMF.class,
+	APICodegenTest.class,
 	CellwiseTmplTest.class,
 	CompressedCellwiseTest.class,
 	CompressedMultiAggregateTest.class,