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/11/12 08:28:22 UTC

[1/2] systemml git commit: [SYSTEMML-2009] Fix codegen thread contention in multi-threaded parfor

Repository: systemml
Updated Branches:
  refs/heads/master f59a2dc22 -> c04929fab


[SYSTEMML-2009] Fix codegen thread contention in multi-threaded parfor

This patch fixes thread contention issues on cost vector id generation
in the codegen optimizer. In scenarios with concurrent dynamic
recompilation (e.g., in local parfor w/ unknowns), and thus concurrent
codegen optimization, threads were partially contented on the global
sequence generator. These cost vector ids are only used for memoization
during plan costing and hence we now simply use a separate id sequence
for each optimizer instance, which avoids any contention altogether. 


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

Branch: refs/heads/master
Commit: 779d4327b76ff37f142ce44d8ee2987d3f221d58
Parents: f59a2dc
Author: Matthias Boehm <mb...@gmail.com>
Authored: Sat Nov 11 22:25:02 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Sat Nov 11 22:25:02 2017 -0800

----------------------------------------------------------------------
 .../codegen/opt/PlanSelectionFuseCostBasedV2.java    | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/779d4327/src/main/java/org/apache/sysml/hops/codegen/opt/PlanSelectionFuseCostBasedV2.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/opt/PlanSelectionFuseCostBasedV2.java b/src/main/java/org/apache/sysml/hops/codegen/opt/PlanSelectionFuseCostBasedV2.java
index 9302573..80013c5 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/opt/PlanSelectionFuseCostBasedV2.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/opt/PlanSelectionFuseCostBasedV2.java
@@ -99,10 +99,13 @@ public class PlanSelectionFuseCostBasedV2 extends PlanSelection
 	//optimizer configuration
 	public static boolean COST_PRUNING = true;
 	public static boolean STRUCTURAL_PRUNING = true;
-	
-	private static final IDSequence COST_ID = new IDSequence();
 	private static final TemplateRow ROW_TPL = new TemplateRow();
 	
+	//cost vector id generator, whose ids are only used for memoization per call to getPlanCost;
+	//hence, we use a sequence generator per optimizer instance to avoid thread contention in 
+	//multi-threaded parfor scenarios with concurrent dynamic recompilation and thus optimization.
+	private final IDSequence COST_ID = new IDSequence();
+	
 	@Override
 	public void selectPlans(CPlanMemoTable memo, ArrayList<Hop> roots) 
 	{
@@ -196,7 +199,7 @@ public class PlanSelectionFuseCostBasedV2 extends PlanSelection
 	 * @param off offset for recursive invocation, indicating the fixed plan part
 	 * @return optimal assignment of materialization points
 	 */
-	private static boolean[] enumPlans(CPlanMemoTable memo, PlanPartition part, StaticCosts costs, 
+	private boolean[] enumPlans(CPlanMemoTable memo, PlanPartition part, StaticCosts costs, 
 		ReachabilityGraph rgraph, InterestingPoint[] matPoints, int off)
 	{
 		//scan linearized search space, w/ skips for branch and bound pruning
@@ -750,7 +753,7 @@ public class PlanSelectionFuseCostBasedV2 extends PlanSelection
 	// Cost model fused operators w/ materialization points
 	//////////
 	
-	private static double getPlanCost(CPlanMemoTable memo, PlanPartition part, 
+	private double getPlanCost(CPlanMemoTable memo, PlanPartition part, 
 			InterestingPoint[] matPoints,boolean[] plan, HashMap<Long, Double> computeCosts,
 			final double costBound)
 	{
@@ -771,7 +774,7 @@ public class PlanSelectionFuseCostBasedV2 extends PlanSelection
 		return costs;
 	}
 	
-	private static double rGetPlanCosts(CPlanMemoTable memo, final Hop current, HashSet<VisitMarkCost> visited,
+	private double rGetPlanCosts(CPlanMemoTable memo, final Hop current, HashSet<VisitMarkCost> visited,
 			PlanPartition part, InterestingPoint[] matPoints, boolean[] plan, HashMap<Long, Double> computeCosts,
 			CostVector costsCurrent, TemplateType currentType, final double costBound)
 	{
@@ -1046,7 +1049,7 @@ public class PlanSelectionFuseCostBasedV2 extends PlanSelection
 			&& HopRewriteUtils.isTransposeOperation(hop.getInput().get(index)); 
 	}
 	
-	private static class CostVector {
+	private class CostVector {
 		public final long ID;
 		public final double outSize; 
 		public double computeCosts = 0;


[2/2] systemml git commit: [MINOR][SYSTEMML-1917] Refactoring hops get/set, incl exception handling

Posted by mb...@apache.org.
[MINOR][SYSTEMML-1917] Refactoring hops get/set, incl exception handling

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

Branch: refs/heads/master
Commit: c04929fab52248d0777111529c779eaea3e0465e
Parents: 779d432
Author: Matthias Boehm <mb...@gmail.com>
Authored: Sat Nov 11 22:48:01 2017 -0800
Committer: Matthias Boehm <mb...@gmail.com>
Committed: Sat Nov 11 23:56:56 2017 -0800

----------------------------------------------------------------------
 .../sysml/hops/codegen/SpoofCompiler.java       |   4 +-
 .../sysml/hops/globalopt/GDFEnumOptimizer.java  |  55 +++----
 .../hops/globalopt/gdfgraph/GraphBuilder.java   |   2 +-
 .../sysml/hops/ipa/FunctionCallGraph.java       |   2 +-
 .../sysml/hops/ipa/IPAPassInlineFunctions.java  |  10 +-
 .../ipa/IPAPassPropagateReplaceLiterals.java    |   2 +-
 .../ipa/IPAPassRemoveConstantBinaryOps.java     |   8 +-
 .../IPAPassRemoveUnnecessaryCheckpoints.java    |  52 ++++---
 .../sysml/hops/ipa/InterProceduralAnalysis.java |   2 +-
 .../apache/sysml/hops/recompile/Recompiler.java |  14 +-
 .../sysml/hops/rewrite/ProgramRewriter.java     |   2 +-
 .../hops/rewrite/RewriteCompressedReblock.java  |  12 +-
 .../rewrite/RewriteForLoopVectorization.java    |  24 +--
 .../RewriteInjectSparkLoopCheckpointing.java    |   2 +-
 .../RewriteMarkLoopVariablesUpdateInPlace.java  |   4 +-
 .../hops/rewrite/RewriteMergeBlockSequence.java |  14 +-
 .../RewriteSplitDagDataDependentOperators.java  |  13 +-
 .../rewrite/RewriteSplitDagUnknownCSVRead.java  |   7 +-
 .../org/apache/sysml/parser/DMLTranslator.java  | 156 +++++--------------
 .../apache/sysml/parser/ForStatementBlock.java  |  11 +-
 .../sysml/parser/FunctionStatementBlock.java    |  13 --
 .../apache/sysml/parser/IfStatementBlock.java   |   9 --
 .../org/apache/sysml/parser/StatementBlock.java |   9 +-
 .../sysml/parser/WhileStatementBlock.java       |  11 --
 .../runtime/controlprogram/ProgramBlock.java    |   2 +-
 .../controlprogram/parfor/ProgramConverter.java |   6 +-
 .../parfor/opt/OptTreeConverter.java            |   2 +-
 .../parfor/opt/OptTreePlanChecker.java          |   2 +-
 .../parfor/opt/OptimizerRuleBased.java          |   2 +-
 .../parfor/opt/ProgramRecompiler.java           |  10 +-
 .../java/org/apache/sysml/utils/Explain.java    |   4 +-
 .../sysml/yarn/ropt/GridEnumerationMemory.java  |   6 +-
 .../sysml/yarn/ropt/ResourceOptimizer.java      |   8 +-
 33 files changed, 176 insertions(+), 304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java b/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
index 51cd0a2..8aa673b 100644
--- a/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
+++ b/src/main/java/org/apache/sysml/hops/codegen/SpoofCompiler.java
@@ -240,7 +240,7 @@ public class SpoofCompiler
 		}
 		else //generic (last-level)
 		{
-			current.set_hops( generateCodeFromHopDAGs(current.get_hops()) );
+			current.setHops( generateCodeFromHopDAGs(current.getHops()) );
 			current.updateRecompilationFlag();
 		}
 	}
@@ -291,7 +291,7 @@ public class SpoofCompiler
 		else //generic (last-level)
 		{
 			StatementBlock sb = current.getStatementBlock();
-			current.setInstructions( generateCodeFromHopDAGsToInst(sb, sb.get_hops()) );
+			current.setInstructions( generateCodeFromHopDAGsToInst(sb, sb.getHops()) );
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/globalopt/GDFEnumOptimizer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/globalopt/GDFEnumOptimizer.java b/src/main/java/org/apache/sysml/hops/globalopt/GDFEnumOptimizer.java
index 73762ce..50284b1 100644
--- a/src/main/java/org/apache/sysml/hops/globalopt/GDFEnumOptimizer.java
+++ b/src/main/java/org/apache/sysml/hops/globalopt/GDFEnumOptimizer.java
@@ -445,41 +445,34 @@ public class GDFEnumOptimizer extends GlobalOptimizer
 			Hop currentHop = p.getNode().getHop();
 			ProgramBlock pb = p.getNode().getProgramBlock();
 			
-			try
-			{
-				//keep the old dag roots
-				ArrayList<Hop> oldRoots = pb.getStatementBlock().get_hops();
-				Hop tmpHop = null;
-				if( !(currentHop instanceof DataOp && ((DataOp)currentHop).isWrite()) ){
-					ArrayList<Hop> newRoots = new ArrayList<>();
-					tmpHop = new DataOp("_tmp", currentHop.getDataType(), currentHop.getValueType(), currentHop, DataOpTypes.TRANSIENTWRITE, "tmp");
-					tmpHop.setVisited(); //ensure recursive visitstatus reset on recompile
-					newRoots.add(tmpHop);
-					pb.getStatementBlock().set_hops(newRoots);
-				}
-				
-				//recompile modified runtime program
-				Recompiler.recompileProgramBlockHierarchy(prog.getProgramBlocks(),
-					new LocalVariableMap(), 0, ResetType.NO_RESET);
-				_compiledPlans++;
-				
-				//cost partial runtime program up to current hop
-				ExecutionContext ec = ExecutionContextFactory.createContext(prog);
-				costs = CostEstimationWrapper.getTimeEstimate(prog, ec);	
-				
-				//restore original hop dag
-				if( tmpHop !=null )
-					HopRewriteUtils.removeChildReference(tmpHop, currentHop);
-				pb.getStatementBlock().set_hops(oldRoots);	
-			}
-			catch(HopsException ex)
-			{
-				throw new DMLRuntimeException(ex);
+			//keep the old dag roots
+			ArrayList<Hop> oldRoots = pb.getStatementBlock().getHops();
+			Hop tmpHop = null;
+			if( !(currentHop instanceof DataOp && ((DataOp)currentHop).isWrite()) ){
+				ArrayList<Hop> newRoots = new ArrayList<>();
+				tmpHop = new DataOp("_tmp", currentHop.getDataType(), currentHop.getValueType(), currentHop, DataOpTypes.TRANSIENTWRITE, "tmp");
+				tmpHop.setVisited(); //ensure recursive visitstatus reset on recompile
+				newRoots.add(tmpHop);
+				pb.getStatementBlock().setHops(newRoots);
 			}
+			
+			//recompile modified runtime program
+			Recompiler.recompileProgramBlockHierarchy(prog.getProgramBlocks(),
+				new LocalVariableMap(), 0, ResetType.NO_RESET);
+			_compiledPlans++;
+			
+			//cost partial runtime program up to current hop
+			ExecutionContext ec = ExecutionContextFactory.createContext(prog);
+			costs = CostEstimationWrapper.getTimeEstimate(prog, ec);
+			
+			//restore original hop dag
+			if( tmpHop !=null )
+				HopRewriteUtils.removeChildReference(tmpHop, currentHop);
+			pb.getStatementBlock().setHops(oldRoots);
 		}
 		
 		//release forced data flow configuration from program
-		rResetRuntimePlanConfig(p, new HashMap<Long,Plan>());		
+		rResetRuntimePlanConfig(p, new HashMap<Long,Plan>());
 		_costedPlans++;
 		
 		return costs;

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/globalopt/gdfgraph/GraphBuilder.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/globalopt/gdfgraph/GraphBuilder.java b/src/main/java/org/apache/sysml/hops/globalopt/gdfgraph/GraphBuilder.java
index e768854..923807e 100644
--- a/src/main/java/org/apache/sysml/hops/globalopt/gdfgraph/GraphBuilder.java
+++ b/src/main/java/org/apache/sysml/hops/globalopt/gdfgraph/GraphBuilder.java
@@ -147,7 +147,7 @@ public class GraphBuilder
 		else //last-level program block
 		{
 			StatementBlock sb = pb.getStatementBlock();
-			ArrayList<Hop> hops = sb.get_hops();
+			ArrayList<Hop> hops = sb.getHops();
 			if( hops != null )
 			{
 				//create new local memo structure for local dag

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/FunctionCallGraph.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/FunctionCallGraph.java b/src/main/java/org/apache/sysml/hops/ipa/FunctionCallGraph.java
index 4735f47..6ea1338 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/FunctionCallGraph.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/FunctionCallGraph.java
@@ -276,7 +276,7 @@ public class FunctionCallGraph
 		} 
 		else {
 			// For generic StatementBlock
-			ArrayList<Hop> hopsDAG = sb.get_hops();
+			ArrayList<Hop> hopsDAG = sb.getHops();
 			if( hopsDAG == null || hopsDAG.isEmpty() ) 
 				return; //nothing to do
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/IPAPassInlineFunctions.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/IPAPassInlineFunctions.java b/src/main/java/org/apache/sysml/hops/ipa/IPAPassInlineFunctions.java
index c7ee3e4..106eb2b 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/IPAPassInlineFunctions.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/IPAPassInlineFunctions.java
@@ -62,16 +62,16 @@ public class IPAPassInlineFunctions extends IPAPass
 			FunctionStatement fstmt = (FunctionStatement)fsb.getStatement(0);
 			if( fstmt.getBody().size() == 1 
 				&& HopRewriteUtils.isLastLevelStatementBlock(fstmt.getBody().get(0)) 
-				&& !containsFunctionOp(fstmt.getBody().get(0).get_hops())
+				&& !containsFunctionOp(fstmt.getBody().get(0).getHops())
 				&& (fgraph.getFunctionCalls(fkey).size() == 1
-					|| countOperators(fstmt.getBody().get(0).get_hops()) 
+					|| countOperators(fstmt.getBody().get(0).getHops()) 
 						<= InterProceduralAnalysis.INLINING_MAX_NUM_OPS) )
 			{
 				if( LOG.isDebugEnabled() )
 					LOG.debug("IPA: Inline function '"+fkey+"'");
 				
 				//replace all relevant function calls 
-				ArrayList<Hop> hops = fstmt.getBody().get(0).get_hops();
+				ArrayList<Hop> hops = fstmt.getBody().get(0).getHops();
 				List<FunctionOp> fcalls = fgraph.getFunctionCalls(fkey);
 				List<StatementBlock> fcallsSB = fgraph.getFunctionCallsSB(fkey);
 				for(int i=0; i<fcalls.size(); i++) {
@@ -101,8 +101,8 @@ public class IPAPassInlineFunctions extends IPAPass
 						if( HopRewriteUtils.isData(out, DataOpTypes.TRANSIENTWRITE) )
 							out.setName(outMap.get(out.getName()));
 					}
-					fcallsSB.get(i).get_hops().remove(op);
-					fcallsSB.get(i).get_hops().addAll(hops2);
+					fcallsSB.get(i).getHops().remove(op);
+					fcallsSB.get(i).getHops().addAll(hops2);
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/IPAPassPropagateReplaceLiterals.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/IPAPassPropagateReplaceLiterals.java b/src/main/java/org/apache/sysml/hops/ipa/IPAPassPropagateReplaceLiterals.java
index 5e2e86c..664ac7b 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/IPAPassPropagateReplaceLiterals.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/IPAPassPropagateReplaceLiterals.java
@@ -116,7 +116,7 @@ public class IPAPassPropagateReplaceLiterals extends IPAPass
 				rReplaceLiterals(current, constants);
 		}
 		else {
-			replaceLiterals(sb.get_hops(), constants);
+			replaceLiterals(sb.getHops(), constants);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveConstantBinaryOps.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveConstantBinaryOps.java b/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveConstantBinaryOps.java
index 6b110c8..d948128 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveConstantBinaryOps.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveConstantBinaryOps.java
@@ -79,7 +79,7 @@ public class IPAPassRemoveConstantBinaryOps extends IPAPass
 			if( !(sb instanceof IfStatementBlock || sb instanceof WhileStatementBlock 
 				  || sb instanceof ForStatementBlock) )
 			{
-				collectMatrixOfOnes(sb.get_hops(), mOnes);
+				collectMatrixOfOnes(sb.getHops(), mOnes);
 			}
 		}
 	}
@@ -128,9 +128,9 @@ public class IPAPassRemoveConstantBinaryOps extends IPAPass
 		}
 		else
 		{
-			if( sb.get_hops() != null ){
-				Hop.resetVisitStatus(sb.get_hops());
-				for( Hop hop : sb.get_hops() )
+			if( sb.getHops() != null ){
+				Hop.resetVisitStatus(sb.getHops());
+				for( Hop hop : sb.getHops() )
 					rRemoveConstantBinaryOp(hop, mOnes);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveUnnecessaryCheckpoints.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveUnnecessaryCheckpoints.java b/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveUnnecessaryCheckpoints.java
index 1c2c6b9..d90b4b6 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveUnnecessaryCheckpoints.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/IPAPassRemoveUnnecessaryCheckpoints.java
@@ -90,10 +90,10 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 					//data operations like nrow(X) or operations removed by rewrites 
 					//double check hops on basic blocks; otherwise worst-case
 					boolean skipRemove = false;
-					if( sb.get_hops() !=null ) {
-						Hop.resetVisitStatus(sb.get_hops());
+					if( sb.getHops() !=null ) {
+						Hop.resetVisitStatus(sb.getHops());
 						skipRemove = true;
-						for( Hop root : sb.get_hops() )
+						for( Hop root : sb.getHops() )
 							skipRemove &= !HopRewriteUtils.rContainsRead(root, cand, false);
 					}					
 					if( !skipRemove )
@@ -114,10 +114,10 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 			else
 			{
 				for( String cand : cands2 )
-					if( sb.variablesUpdated().containsVariable(cand) && sb.get_hops() != null) 
+					if( sb.variablesUpdated().containsVariable(cand) && sb.getHops() != null) 
 					{
-						Hop.resetVisitStatus(sb.get_hops());
-						for( Hop root : sb.get_hops() )
+						Hop.resetVisitStatus(sb.getHops());
+						for( Hop root : sb.getHops() )
 							if( root.getName().equals(cand) &&
 								!HopRewriteUtils.rHasSimpleReadChain(root, cand) ) {
 								chkpointCand.remove(cand);
@@ -126,14 +126,15 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 			}
 		
 			//collect checkpoints and remove unnecessary checkpoints
-			ArrayList<Hop> tmp = collectCheckpoints(sb.get_hops());
-			for( Hop chkpoint : tmp ) {
-				if( chkpointCand.containsKey(chkpoint.getName()) ) {
-					chkpointCand.get(chkpoint.getName()).setRequiresCheckpoint(false);
+			if( HopRewriteUtils.isLastLevelStatementBlock(sb) ) {
+				ArrayList<Hop> tmp = collectCheckpoints(sb.getHops());
+				for( Hop chkpoint : tmp ) {
+					if( chkpointCand.containsKey(chkpoint.getName()) ) {
+						chkpointCand.get(chkpoint.getName()).setRequiresCheckpoint(false);
+					}
+					chkpointCand.put(chkpoint.getName(), chkpoint);
 				}
-				chkpointCand.put(chkpoint.getName(), chkpoint);
 			}
-			
 		}
 	}
 
@@ -159,10 +160,10 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 					//data operations like nrow(X) or operations removed by rewrites 
 					//double check hops on basic blocks; otherwise worst-case
 					boolean skipRemove = false;
-					if( sb.get_hops() !=null ) {
-						Hop.resetVisitStatus(sb.get_hops());
+					if( sb.getHops() !=null ) {
+						Hop.resetVisitStatus(sb.getHops());
 						skipRemove = true;
-						for( Hop root : sb.get_hops() )
+						for( Hop root : sb.getHops() )
 							skipRemove &= !HopRewriteUtils.rContainsRead(root, cand, false);
 					}					
 					if( !skipRemove )
@@ -185,9 +186,9 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 			else
 			{
 				for( String cand : cands2 )
-					if( sb.variablesUpdated().containsVariable(cand) && sb.get_hops() != null) {
-						Hop.resetVisitStatus(sb.get_hops());
-						for( Hop root : sb.get_hops() )
+					if( sb.variablesUpdated().containsVariable(cand) && sb.getHops() != null) {
+						Hop.resetVisitStatus(sb.getHops());
+						for( Hop root : sb.getHops() )
 							if( root.getName().equals(cand) ) {
 								if( HopRewriteUtils.rHasSimpleReadChain(root, cand) ) {
 									chkpointCand.get(cand).setRequiresCheckpoint(false);
@@ -195,15 +196,16 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 									chkpointCand.put(cand, root.getInput().get(0));
 								}
 								else
-									chkpointCand.remove(cand);		
+									chkpointCand.remove(cand);
 							}
 					}	
 			}
 		
 			//collect checkpoints
-			ArrayList<Hop> tmp = collectCheckpoints(sb.get_hops());
-			for( Hop chkpoint : tmp ) {
-				chkpointCand.put(chkpoint.getName(), chkpoint);
+			if( HopRewriteUtils.isLastLevelStatementBlock(sb) ) {
+				ArrayList<Hop> tmp = collectCheckpoints(sb.getHops());
+				for( Hop chkpoint : tmp )
+					chkpointCand.put(chkpoint.getName(), chkpoint);
 			}
 		}
 	}
@@ -218,9 +220,9 @@ public class IPAPassRemoveUnnecessaryCheckpoints extends IPAPass
 			|| sbs.get(0) instanceof ForStatementBlock) ) 
 		{
 			//recursively process all dag roots
-			if( sbs.get(0).get_hops()!=null ) {
-				Hop.resetVisitStatus(sbs.get(0).get_hops());
-				for( Hop root : sbs.get(0).get_hops() )
+			if( sbs.get(0).getHops()!=null ) {
+				Hop.resetVisitStatus(sbs.get(0).getHops());
+				for( Hop root : sbs.get(0).getHops() )
 					rRemoveCheckpointReadWrite(root);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/ipa/InterProceduralAnalysis.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/ipa/InterProceduralAnalysis.java b/src/main/java/org/apache/sysml/hops/ipa/InterProceduralAnalysis.java
index de12bb1..317f601 100644
--- a/src/main/java/org/apache/sysml/hops/ipa/InterProceduralAnalysis.java
+++ b/src/main/java/org/apache/sysml/hops/ipa/InterProceduralAnalysis.java
@@ -353,7 +353,7 @@ public class InterProceduralAnalysis
 			//remove updated constant scalars
 			Recompiler.removeUpdatedScalars(callVars, sb);
 			//old stats in, new stats out if updated
-			ArrayList<Hop> roots = sb.get_hops();
+			ArrayList<Hop> roots = sb.getHops();
 			DMLProgram prog = sb.getDMLProg();
 			//replace scalar reads with literals
 			Hop.resetVisitStatus(roots);

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/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 c6d74dd..360e261 100644
--- a/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
+++ b/src/main/java/org/apache/sysml/hops/recompile/Recompiler.java
@@ -646,8 +646,8 @@ public class Recompiler
 		{
 			//recompile last-level program block instructions
 			StatementBlock sb = pb.getStatementBlock();
-			if( sb!=null && sb.get_hops()!=null ) {
-				pb.setInstructions(recompileHopsDagInstructions(sb, sb.get_hops()));
+			if( sb!=null && sb.getHops()!=null ) {
+				pb.setInstructions(recompileHopsDagInstructions(sb, sb.getHops()));
 			}
 		}
 	}
@@ -883,18 +883,18 @@ public class Recompiler
 				/*&& !Recompiler.containsNonRecompileInstructions(tmp)*/ )
 			{
 				tmp = Recompiler.recompileHopsDag(
-					sb, sb.get_hops(), vars, status, true, false, tid);
+					sb, sb.getHops(), vars, status, true, false, tid);
 				pb.setInstructions( tmp );
 				
 				//propagate stats across hops (should be executed on clone of vars)
-				Recompiler.extractDAGOutputStatistics(sb.get_hops(), vars);
+				Recompiler.extractDAGOutputStatistics(sb.getHops(), vars);
 				
 				//reset recompilation flags (w/ special handling functions)
 				if( ParForProgramBlock.RESET_RECOMPILATION_FLAGs 
-					&& !containsRootFunctionOp(sb.get_hops())  
+					&& !containsRootFunctionOp(sb.getHops())  
 					&& resetRecompile.isReset() ) 
 				{
-					Hop.resetRecompilationFlag(sb.get_hops(), ExecType.CP, resetRecompile);
+					Hop.resetRecompilationFlag(sb.getHops(), ExecType.CP, resetRecompile);
 					sb.updateRecompilationFlag();
 				}
 			}
@@ -1278,7 +1278,7 @@ public class Recompiler
 			//would be invalid with permutation matrix mult across multiple dags)
 			if(	sb != null ) {
 				ArrayList<Instruction> tmp = pb.getInstructions();
-				tmp = Recompiler.recompileHopsDag2Forced(sb, sb.get_hops(), tid, et);
+				tmp = Recompiler.recompileHopsDag2Forced(sb, sb.getHops(), tid, et);
 				pb.setInstructions( tmp );
 			}
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/ProgramRewriter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/ProgramRewriter.java b/src/main/java/org/apache/sysml/hops/rewrite/ProgramRewriter.java
index c56b2ed..d8acdf6 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/ProgramRewriter.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/ProgramRewriter.java
@@ -256,7 +256,7 @@ public class ProgramRewriter
 		}
 		else //generic (last-level)
 		{
-			current.set_hops( rewriteHopDAG(current.get_hops(), state) );
+			current.setHops( rewriteHopDAG(current.getHops(), state) );
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteCompressedReblock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteCompressedReblock.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteCompressedReblock.java
index 7e4567d..fdaad10 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteCompressedReblock.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteCompressedReblock.java
@@ -68,7 +68,7 @@ public class RewriteCompressedReblock extends StatementBlockRewriteRule
 	{
 		//check for inapplicable statement blocks
 		if( !HopRewriteUtils.isLastLevelStatementBlock(sb)
-			|| sb.get_hops() == null )
+			|| sb.getHops() == null )
 			return Arrays.asList(sb);
 		
 		//parse compression config
@@ -78,10 +78,10 @@ public class RewriteCompressedReblock extends StatementBlockRewriteRule
 		
 		//perform compressed reblock rewrite
 		if( compress.isEnabled() ) {
-			Hop.resetVisitStatus(sb.get_hops());
-			for( Hop h : sb.get_hops() ) 
+			Hop.resetVisitStatus(sb.getHops());
+			for( Hop h : sb.getHops() ) 
 				injectCompressionDirective(h, compress, sb.getDMLProg());
-			Hop.resetVisitStatus(sb.get_hops());
+			Hop.resetVisitStatus(sb.getHops());
 		}
 		return Arrays.asList(sb);
 	}
@@ -198,8 +198,8 @@ public class RewriteCompressedReblock extends StatementBlockRewriteRule
 			if( fsb.variablesRead().containsAnyName(status.compMtx) )
 				status.usedInLoop = true;
 		}
-		else if( sb.get_hops() != null ) { //generic (last-level)
-			ArrayList<Hop> roots = sb.get_hops();
+		else if( sb.getHops() != null ) { //generic (last-level)
+			ArrayList<Hop> roots = sb.getHops();
 			Hop.resetVisitStatus(roots);
 			//process entire HOP DAG starting from the roots
 			for( Hop root : roots )

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteForLoopVectorization.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteForLoopVectorization.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteForLoopVectorization.java
index 7d01715..fce1fa1 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteForLoopVectorization.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteForLoopVectorization.java
@@ -119,8 +119,8 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		boolean rightScalar = false;
 		boolean rowIx = false; //row or col
 		
-		if( csb.get_hops()!=null && csb.get_hops().size()==1 ){
-			Hop root = csb.get_hops().get(0);
+		if( csb.getHops()!=null && csb.getHops().size()==1 ){
+			Hop root = csb.getHops().get(0);
 			
 			if( root.getDataType()==DataType.SCALAR && root.getInput().get(0) instanceof BinaryOp ) {
 				BinaryOp bop = (BinaryOp) root.getInput().get(0);
@@ -171,7 +171,7 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		//apply rewrite if possible
 		if( leftScalar || rightScalar ) 
 		{
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			BinaryOp bop = (BinaryOp) root.getInput().get(0);
 			Hop cast = bop.getInput().get( leftScalar?1:0 );
 			Hop ix = cast.getInput().get(0);
@@ -218,9 +218,9 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		//check for applicability
 		boolean apply = false;
 		boolean rowIx = false; //row or col
-		if( csb.get_hops()!=null && csb.get_hops().size()==1 )
+		if( csb.getHops()!=null && csb.getHops().size()==1 )
 		{
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			
 			if( root.getDataType()==DataType.MATRIX && root.getInput().get(0) instanceof LeftIndexingOp )
 			{
@@ -262,7 +262,7 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		//apply rewrite if possible
 		if( apply ) 
 		{
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			LeftIndexingOp lix = (LeftIndexingOp) root.getInput().get(0);
 			BinaryOp bop = (BinaryOp) lix.getInput().get(1);
 			IndexingOp rix0 = (IndexingOp) bop.getInput().get(0);
@@ -302,9 +302,9 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		//check for applicability
 		boolean apply = false;
 		boolean rowIx = false; //row or col
-		if( csb.get_hops()!=null && csb.get_hops().size()==1 )
+		if( csb.getHops()!=null && csb.getHops().size()==1 )
 		{
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			
 			if( root.getDataType()==DataType.MATRIX && root.getInput().get(0) instanceof LeftIndexingOp )
 			{
@@ -326,7 +326,7 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		
 		//apply rewrite if possible
 		if( apply ) {
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			LeftIndexingOp lix = (LeftIndexingOp) root.getInput().get(0);
 			UnaryOp uop = (UnaryOp) lix.getInput().get(1);
 			IndexingOp rix = (IndexingOp) uop.getInput().get(0);
@@ -362,9 +362,9 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		//check for applicability
 		boolean apply = false;
 		boolean rowIx = false; //row or col
-		if( csb.get_hops()!=null && csb.get_hops().size()==1 )
+		if( csb.getHops()!=null && csb.getHops().size()==1 )
 		{
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			
 			if( root.getDataType()==DataType.MATRIX && root.getInput().get(0) instanceof LeftIndexingOp )
 			{
@@ -384,7 +384,7 @@ public class RewriteForLoopVectorization extends StatementBlockRewriteRule
 		
 		//apply rewrite if possible
 		if( apply ) {
-			Hop root = csb.get_hops().get(0);
+			Hop root = csb.getHops().get(0);
 			LeftIndexingOp lix = (LeftIndexingOp) root.getInput().get(0);
 			IndexingOp rix = (IndexingOp) lix.getInput().get(1);
 			int index1 = rowIx ? 2 : 4;

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkLoopCheckpointing.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkLoopCheckpointing.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkLoopCheckpointing.java
index f838952..ddee8ab 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkLoopCheckpointing.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteInjectSparkLoopCheckpointing.java
@@ -107,7 +107,7 @@ public class RewriteInjectSparkLoopCheckpointing extends StatementBlockRewriteRu
 					livein.addVariable(var, read.getVariable(var));
 					liveout.addVariable(var, read.getVariable(var));
 				}
-				sb0.set_hops(hops);
+				sb0.setHops(hops);
 				sb0.setLiveIn(livein);
 				sb0.setLiveOut(liveout);
 				sb0.setSplitDag(true);

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteMarkLoopVariablesUpdateInPlace.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMarkLoopVariablesUpdateInPlace.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMarkLoopVariablesUpdateInPlace.java
index c29820e..c4b3340 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMarkLoopVariablesUpdateInPlace.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMarkLoopVariablesUpdateInPlace.java
@@ -114,8 +114,8 @@ public class RewriteMarkLoopVariablesUpdateInPlace extends StatementBlockRewrite
 					ret &= rIsApplicableForUpdateInPlace(istmt.getElseBody(), varname);	
 			}
 			else {
-				if( sb.get_hops() != null )
-					for( Hop hop : sb.get_hops() ) 
+				if( sb.getHops() != null )
+					for( Hop hop : sb.getHops() ) 
 						ret &= isApplicableForUpdateInPlace(hop, varname);
 			}
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteMergeBlockSequence.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMergeBlockSequence.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMergeBlockSequence.java
index c1a6332..8742ade 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteMergeBlockSequence.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteMergeBlockSequence.java
@@ -72,8 +72,8 @@ public class RewriteMergeBlockSequence extends StatementBlockRewriteRule
 					&& (!hasFunctionOpRoot(sb1) || !hasFunctionIOConflict(sb1,sb2))
 					&& (!hasFunctionOpRoot(sb2) || !hasFunctionIOConflict(sb2,sb1)) )
 				{
-					ArrayList<Hop> sb1Hops = sb1.get_hops();
-					ArrayList<Hop> sb2Hops = sb2.get_hops();
+					ArrayList<Hop> sb1Hops = sb1.getHops();
+					ArrayList<Hop> sb2Hops = sb2.getHops();
 					
 					//determine transient read inputs s2 
 					Hop.resetVisitStatus(sb2Hops);
@@ -162,19 +162,19 @@ public class RewriteMergeBlockSequence extends StatementBlockRewriteRule
 	
 	private static boolean hasFunctionOpRoot(StatementBlock sb) 
 			throws HopsException {
-		if( sb == null || sb.get_hops() == null )
+		if( sb == null || sb.getHops() == null )
 			return false;
 		boolean ret = false;
-		for( Hop root : sb.get_hops() )
+		for( Hop root : sb.getHops() )
 			ret |= (root instanceof FunctionOp);
 		return ret;
 	}
 	
 	private static boolean hasExternalFunctionOpRoot(StatementBlock sb) 
 			throws HopsException {
-		if( sb == null || sb.get_hops() == null )
+		if( sb == null || sb.getHops() == null )
 			return false;
-		for( Hop root : sb.get_hops() )
+		for( Hop root : sb.getHops() )
 			if( root instanceof FunctionOp ) {
 				FunctionStatementBlock fsb = sb.getDMLProg()
 					.getFunctionStatementBlock(((FunctionOp)root).getFunctionKey());
@@ -195,7 +195,7 @@ public class RewriteMergeBlockSequence extends StatementBlockRewriteRule
 		
 		//collect all function output variables of sb1
 		HashSet<String> outSb1 = new HashSet<>();
-		for( Hop root : sb1.get_hops() )
+		for( Hop root : sb1.getHops() )
 			if( root instanceof FunctionOp )
 				outSb1.addAll(Arrays.asList(((FunctionOp)root).getOutputVariableNames()));
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagDataDependentOperators.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagDataDependentOperators.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagDataDependentOperators.java
index 7e7860e..6fbd953 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagDataDependentOperators.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagDataDependentOperators.java
@@ -75,15 +75,16 @@ public class RewriteSplitDagDataDependentOperators extends StatementBlockRewrite
 		throws HopsException 
 	{
 		//DAG splits not required for forced single node
-		if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE )
+		if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE
+			|| !HopRewriteUtils.isLastLevelStatementBlock(sb) )
 			return new ArrayList<>(Arrays.asList(sb));
 		
 		ArrayList<StatementBlock> ret = new ArrayList<>();
 	
 		//collect all unknown csv reads hops
 		ArrayList<Hop> cand = new ArrayList<>();
-		collectDataDependentOperators( sb.get_hops(), cand );
-		Hop.resetVisitStatus(sb.get_hops());
+		collectDataDependentOperators( sb.getHops(), cand );
+		Hop.resetVisitStatus(sb.getHops());
 		
 		//split hop dag on demand
 		if( !cand.isEmpty() )
@@ -141,7 +142,7 @@ public class RewriteSplitDagDataDependentOperators extends StatementBlockRewrite
 								if( parent != twrite )
 									HopRewriteUtils.replaceChildReference(parent, c, tread);
 								else
-									sb.get_hops().remove(parent);
+									sb.getHops().remove(parent);
 							}
 						}
 						
@@ -187,10 +188,10 @@ public class RewriteSplitDagDataDependentOperators extends StatementBlockRewrite
 				}
 				
 				//ensure disjoint operators across DAGs (prevent replicated operations)
-				handleReplicatedOperators( sb1hops, sb.get_hops(), sb1.liveOut(), sb.liveIn() );
+				handleReplicatedOperators( sb1hops, sb.getHops(), sb1.liveOut(), sb.liveIn() );
 				
 				//deep copy new dag (in order to prevent any dangling references)
-				sb1.set_hops(Recompiler.deepCopyHopsDag(sb1hops));
+				sb1.setHops(Recompiler.deepCopyHopsDag(sb1hops));
 				sb1.updateRecompilationFlag();
 				sb1.setSplitDag(true); //avoid later merge by other rewrites
 				

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
index e9a6407..5ec7cb0 100644
--- a/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
+++ b/src/main/java/org/apache/sysml/hops/rewrite/RewriteSplitDagUnknownCSVRead.java
@@ -50,14 +50,15 @@ public class RewriteSplitDagUnknownCSVRead extends StatementBlockRewriteRule
 		throws HopsException 
 	{
 		//DAG splits not required for forced single node
-		if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE )
+		if( DMLScript.rtplatform == RUNTIME_PLATFORM.SINGLE_NODE
+			|| !HopRewriteUtils.isLastLevelStatementBlock(sb) )
 			return Arrays.asList(sb);
 		
 		ArrayList<StatementBlock> ret = new ArrayList<>();
 		
 		//collect all unknown csv reads hops
 		ArrayList<Hop> cand = new ArrayList<>();
-		collectCSVReadHopsUnknownSize( sb.get_hops(), cand );
+		collectCSVReadHopsUnknownSize( sb.getHops(), cand );
 		
 		//split hop dag on demand
 		if( !cand.isEmpty() )
@@ -117,7 +118,7 @@ public class RewriteSplitDagUnknownCSVRead extends StatementBlockRewriteRule
 					}
 				}
 				
-				sb1.set_hops(sb1hops);
+				sb1.setHops(sb1hops);
 				sb1.updateRecompilationFlag();
 				ret.add(sb1); //statement block with csv reblocks
 				ret.add(sb); //statement block with remaining hops

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/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 5a7f60e..71633c4 100644
--- a/src/main/java/org/apache/sysml/parser/DMLTranslator.java
+++ b/src/main/java/org/apache/sysml/parser/DMLTranslator.java
@@ -132,17 +132,9 @@ public class DMLTranslator
 			
 				// add the input variables for the function to input variable list
 				FunctionStatement fstmt = (FunctionStatement)fblock.getStatement(0);
-				if (fblock.getNumStatements() > 1){
-					LOG.error(fstmt.printErrorLocation() + "FunctionStatementBlock can only have 1 FunctionStatement");
-					throw new LanguageException(fstmt.printErrorLocation() + "FunctionStatementBlock can only have 1 FunctionStatement");
-				}
-			
-				for (DataIdentifier currVar : fstmt.getInputParams()) {	
-					
-					if (currVar.getDataType() == DataType.SCALAR){
+				for (DataIdentifier currVar : fstmt.getInputParams()) {
+					if (currVar.getDataType() == DataType.SCALAR)
 						currVar.setDimensions(0, 0);
-					}
-					
 					vs.addVariable(currVar.getName(), currVar);
 				}
 				fblock.validate(dmlp, vs, constVars, false);
@@ -251,7 +243,7 @@ public class DMLTranslator
 	{
 		// Step 1: construct hops for all functions
 		// for each namespace, handle function program blocks
-		for (String namespaceKey : dmlp.getNamespaces().keySet()){		
+		for (String namespaceKey : dmlp.getNamespaces().keySet()){
 			for (String fname: dmlp.getFunctionStatementBlocks(namespaceKey).keySet()) {
 				FunctionStatementBlock current = dmlp.getFunctionStatementBlock(namespaceKey, fname);
 				constructHops(current);
@@ -337,15 +329,10 @@ public class DMLTranslator
 			WhileStatementBlock wsb = (WhileStatementBlock)sb;
 			WhileStatement whileStmt = (WhileStatement)wsb.getStatement(0);
 			ArrayList<StatementBlock> body = whileStmt.getBody();
-				
-			if (sb.get_hops() != null && !sb.get_hops().isEmpty()) {
-				LOG.error(sb.printBlockErrorLocation() + "WhileStatementBlock should not have hops");
-				throw new HopsException(sb.printBlockErrorLocation() + "WhileStatementBlock should not have hops");
-			}
+			
 			// step through stmt blocks in while stmt body
-			for (StatementBlock stmtBlock : body){
+			for (StatementBlock stmtBlock : body)
 				ret |= constructLops(stmtBlock);
-			}
 			
 			// handle while stmt predicate
 			Lop l = wsb.getPredicateHops().constructLops();
@@ -359,11 +346,7 @@ public class DMLTranslator
 			IfStatement ifStmt = (IfStatement)isb.getStatement(0);
 			ArrayList<StatementBlock> ifBody = ifStmt.getIfBody();
 			ArrayList<StatementBlock> elseBody = ifStmt.getElseBody();
-				
-			if (sb.get_hops() != null && !sb.get_hops().isEmpty()){
-				LOG.error(sb.printBlockErrorLocation() + "IfStatementBlock should not have hops");
-				throw new HopsException(sb.printBlockErrorLocation() + "IfStatementBlock should not have hops");
-			}
+			
 			// step through stmt blocks in if stmt ifBody
 			for (StatementBlock stmtBlock : ifBody)
 				ret |= constructLops(stmtBlock);
@@ -383,11 +366,7 @@ public class DMLTranslator
 			ForStatementBlock fsb =  (ForStatementBlock) sb;
 			ForStatement fs = (ForStatement)sb.getStatement(0);
 			ArrayList<StatementBlock> body = fs.getBody();
-						
-			if (sb.get_hops() != null && !sb.get_hops().isEmpty() ) {
-				LOG.error(sb.printBlockErrorLocation() + "ForStatementBlock should not have hops");
-				throw new HopsException(sb.printBlockErrorLocation() + "ForStatementBlock should not have hops");
-			}
+			
 			// step through stmt blocks in FOR stmt body
 			for (StatementBlock stmtBlock : body)
 				ret |= constructLops(stmtBlock);
@@ -411,10 +390,7 @@ public class DMLTranslator
 			FunctionStatementBlock fsb = (FunctionStatementBlock) sb;
 			FunctionStatement functStmt = (FunctionStatement)sb.getStatement(0);
 			ArrayList<StatementBlock> body = functStmt.getBody();
-			if (sb.get_hops() != null && !sb.get_hops().isEmpty()) {
-				LOG.error(sb.printBlockErrorLocation() + "FunctionStatementBlock should not have hops");
-				throw new HopsException(sb.printBlockErrorLocation() + "FunctionStatementBlock should not have hops");
-			}
+			
 			// step through stmt blocks in while stmt body
 			for( StatementBlock stmtBlock : body )
 				ret |= constructLops(stmtBlock);
@@ -424,10 +400,10 @@ public class DMLTranslator
 		
 		// handle default case for regular StatementBlock
 		else {
-			if (sb.get_hops() == null)
-				sb.set_hops(new ArrayList<Hop>());
+			if (sb.getHops() == null)
+				sb.setHops(new ArrayList<Hop>());
 			ArrayList<Lop> lops = new ArrayList<>();
-			for (Hop hop : sb.get_hops())
+			for (Hop hop : sb.getHops())
 				lops.add(hop.constructLops());
 			sb.setLops(lops);
 			ret |= sb.updateRecompilationFlag(); 
@@ -503,10 +479,6 @@ public class DMLTranslator
 			//// 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()){
 				
@@ -515,13 +487,6 @@ public class DMLTranslator
 				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;
 			
 			// add statement block
@@ -550,10 +515,6 @@ public class DMLTranslator
 			
 			// 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
@@ -568,12 +529,6 @@ public class DMLTranslator
 				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
@@ -609,19 +564,16 @@ public class DMLTranslator
 			ArrayList<Instruction> incrementInstructions = incrementDag.getJobs(null, config);
 
 			// create for program block
-			String sbName = null;
 			ForProgramBlock rtpb = null;
 			IterablePredicate iterPred = fsb.getIterPredicate();
 			
 			if( sb instanceof ParForStatementBlock && ConfigurationManager.isParallelParFor() ) {
-				sbName = "ParForStatementBlock";
 				rtpb = new ParForProgramBlock(prog, iterPred.getIterVar().getName(),
 					iterPred.getParForParams(), ((ParForStatementBlock)sb).getResultVariables());
 				ParForProgramBlock pfrtpb = (ParForProgramBlock)rtpb;
 				pfrtpb.setStatementBlock((ParForStatementBlock)sb); //used for optimization and creating unscoped variables
 			}
 			else {//ForStatementBlock
-				sbName = "ForStatementBlock";
 				rtpb = new ForProgramBlock(prog, iterPred.getIterVar().getName());
 			}
 			
@@ -630,22 +582,12 @@ public class DMLTranslator
 			rtpb.setIncrementInstructions(incrementInstructions);
 			
 			// 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;
 			
 			// add statement block
@@ -659,10 +601,6 @@ public class DMLTranslator
 		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;
 			
@@ -670,17 +608,11 @@ public class DMLTranslator
 				 // create external function program block
 				
 				String execType = ((ExternalFunctionStatement) fstmt)
-                				    .getOtherParams().get(ExternalFunctionStatement.EXEC_TYPE);
+						.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(config.getTextValue(DMLConfig.SCRATCH_SPACE));
 				buff.append(Lop.FILE_SEPARATOR);
 				buff.append(Lop.PROCESS_PREFIX);
 				buff.append(DMLScript.getUUID());
@@ -697,7 +629,7 @@ public class DMLTranslator
 					rtpb = new ExternalFunctionProgramBlockCP(prog, 
 									fstmt.getInputParams(), fstmt.getOutputParams(), 
 									((ExternalFunctionStatement) fstmt).getOtherParams(),
-									basedir );					
+									basedir );
 				}
 				else
 				{
@@ -922,12 +854,12 @@ public class DMLTranslator
 	public void refreshMemEstimates(StatementBlock current) throws ParseException, HopsException {
 	
 		MemoTable memo = new MemoTable();
-		ArrayList<Hop> hopsDAG = current.get_hops();
-		if (hopsDAG != null && !hopsDAG.isEmpty()) {
-			Iterator<Hop> iter = hopsDAG.iterator();
-			while (iter.hasNext()) {
-				iter.next().refreshMemEstimates(memo);
-			}
+		
+		if( HopRewriteUtils.isLastLevelStatementBlock(current) ) {
+			ArrayList<Hop> hopsDAG = current.getHops();
+			if (hopsDAG != null && !hopsDAG.isEmpty())
+				for( Hop hop : hopsDAG )
+					hop.refreshMemEstimates(memo);
 		}
 		
 		if (current instanceof FunctionStatementBlock) {
@@ -937,8 +869,7 @@ public class DMLTranslator
 				refreshMemEstimates(sb);
 			}
 		}
-		
-		if (current instanceof WhileStatementBlock) {
+		else if (current instanceof WhileStatementBlock) {
 			// handle predicate
 			WhileStatementBlock wstb = (WhileStatementBlock) current;
 			wstb.getPredicateHops().refreshMemEstimates(new MemoTable());
@@ -951,8 +882,7 @@ public class DMLTranslator
 				refreshMemEstimates(sb);
 			}
 		}
-		
-		if (current instanceof IfStatementBlock) {
+		else if (current instanceof IfStatementBlock) {
 			// handle predicate
 			IfStatementBlock istb = (IfStatementBlock) current;
 			istb.getPredicateHops().refreshMemEstimates(new MemoTable());
@@ -968,8 +898,7 @@ public class DMLTranslator
 				refreshMemEstimates(sb);
 			}
 		}
-		
-		if (current instanceof ForStatementBlock) {
+		else if (current instanceof ForStatementBlock) {
 			// handle predicate
 			ForStatementBlock fsb = (ForStatementBlock) current;
 			if (fsb.getFromHops() != null) 
@@ -1008,51 +937,40 @@ public class DMLTranslator
 			
 	public static void resetHopsDAGVisitStatus(StatementBlock current) throws ParseException, HopsException {
 	
-		ArrayList<Hop> hopsDAG = current.get_hops();
-		if (hopsDAG != null && !hopsDAG.isEmpty() ) {
-			Hop.resetVisitStatus(hopsDAG);
+		if( HopRewriteUtils.isLastLevelStatementBlock(current) ) {
+			ArrayList<Hop> hopsDAG = current.getHops();
+			if (hopsDAG != null && !hopsDAG.isEmpty() ) {
+				Hop.resetVisitStatus(hopsDAG);
+			}
 		}
 		
 		if (current instanceof FunctionStatementBlock) {
-			
 			FunctionStatement fstmt = (FunctionStatement)current.getStatement(0);
 			for (StatementBlock sb : fstmt.getBody()){
 				resetHopsDAGVisitStatus(sb);
 			}
 		}
-		
-		if (current instanceof WhileStatementBlock) {
+		else if (current instanceof WhileStatementBlock) {
 			// handle predicate
 			WhileStatementBlock wstb = (WhileStatementBlock) current;
 			wstb.getPredicateHops().resetVisitStatus();
-		
-			if (wstb.getNumStatements() > 1)
-				LOG.debug("While stmt block has more than 1 stmt");
-			WhileStatement ws = (WhileStatement)wstb.getStatement(0);
 			
-			for (StatementBlock sb : ws.getBody()){
+			WhileStatement ws = (WhileStatement)wstb.getStatement(0);
+			for (StatementBlock sb : ws.getBody())
 				resetHopsDAGVisitStatus(sb);
-			}
 		}
-		
-		if (current instanceof IfStatementBlock) {
+		else if (current instanceof IfStatementBlock) {
 			// handle predicate
 			IfStatementBlock istb = (IfStatementBlock) current;
 			istb.getPredicateHops().resetVisitStatus();
 		
-			if (istb.getNumStatements() > 1)
-				LOG.debug("If statement block has more than 1 stmt");
 			IfStatement is = (IfStatement)istb.getStatement(0);
-			
-			for (StatementBlock sb : is.getIfBody()){
+			for (StatementBlock sb : is.getIfBody())
 				resetHopsDAGVisitStatus(sb);
-			}
-			for (StatementBlock sb : is.getElseBody()){
+			for (StatementBlock sb : is.getElseBody())
 				resetHopsDAGVisitStatus(sb);
-			}
 		}
-		
-		if (current instanceof ForStatementBlock) {
+		else if (current instanceof ForStatementBlock) {
 			// handle predicate
 			ForStatementBlock fsb = (ForStatementBlock) current;
 			if (fsb.getFromHops() != null) 
@@ -1090,7 +1008,7 @@ public class DMLTranslator
 	
 	public void resetLopsDAGVisitStatus(StatementBlock current) throws HopsException {
 		
-		ArrayList<Hop> hopsDAG = current.get_hops();
+		ArrayList<Hop> hopsDAG = current.getHops();
 
 		if (hopsDAG != null && !hopsDAG.isEmpty() ) {
 			Iterator<Hop> iter = hopsDAG.iterator();
@@ -1474,7 +1392,7 @@ public class DMLTranslator
 			
 		}
 		sb.updateLiveVariablesOut(updatedLiveOut);
-		sb.set_hops(output);
+		sb.setHops(output);
 
 	}
 	

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/ForStatementBlock.java b/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
index 686ce7a..ae1aa75 100644
--- a/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/ForStatementBlock.java
@@ -265,16 +265,7 @@ public class ForStatementBlock extends StatementBlock
 		return loReturn;
 	
 	}
-
-	@Override
-	public ArrayList<Hop> get_hops() throws HopsException {
-		if (_hops != null && !_hops.isEmpty()){
-			LOG.error(this.printBlockErrorLocation() + "there should be no HOPs associated with the ForStatementBlock");
-			throw new HopsException(this.printBlockErrorLocation() + "there should be no HOPs associated with the ForStatementBlock");
-		}
-		return _hops;
-	}
-
+	
 	public void setFromHops(Hop hops) { _fromHops = hops; }
 	public void setToHops(Hop hops) { _toHops = hops; }
 	public void setIncrementHops(Hop hops) { _incrementHops = hops; }

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java b/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
index 0f91d00..35ec6d4 100644
--- a/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/FunctionStatementBlock.java
@@ -23,8 +23,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 
-import org.apache.sysml.hops.Hop;
-import org.apache.sysml.hops.HopsException;
 import org.apache.sysml.hops.FunctionOp.FunctionType;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
@@ -267,17 +265,6 @@ public class FunctionStatementBlock extends StatementBlock
 	}
 	
 	@Override
-	public ArrayList<Hop> get_hops() throws HopsException {
-		
-		if (_hops != null && _hops.size() > 0){
-			LOG.error(this.printBlockErrorLocation() + "there should be no HOPs associated with the FunctionStatementBlock");
-			throw new HopsException(this.printBlockErrorLocation() + "there should be no HOPs associated with the FunctionStatementBlock");
-		}
-		
-		return _hops;
-	}
-	
-	@Override
 	public VariableSet analyze(VariableSet loPassed) throws LanguageException{
 		LOG.error(this.printBlockErrorLocation() + "Both liveIn and liveOut variables need to be specified for liveness analysis for FunctionStatementBlock");
 		throw new LanguageException(this.printBlockErrorLocation() + "Both liveIn and liveOut variables need to be specified for liveness analysis for FunctionStatementBlock");	

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/IfStatementBlock.java b/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
index 6803eb2..00295d1 100644
--- a/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/IfStatementBlock.java
@@ -455,15 +455,6 @@ public class IfStatementBlock extends StatementBlock
 		_predicateHops = hops;
 	}
 	
-	@Override
-	public ArrayList<Hop> get_hops() throws HopsException{
-		if (_hops != null && _hops.size() > 0){
-			LOG.error(this.printBlockErrorLocation() + "error there should be no HOPs in IfStatementBlock");
-			throw new HopsException(this.printBlockErrorLocation() + "error there should be no HOPs in IfStatementBlock");
-		}
-		return _hops;
-	}
-	
 	public Hop getPredicateHops(){
 		return _predicateHops;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/parser/StatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/StatementBlock.java b/src/main/java/org/apache/sysml/parser/StatementBlock.java
index c2a3e01..f7f6426 100644
--- a/src/main/java/org/apache/sysml/parser/StatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/StatementBlock.java
@@ -114,8 +114,7 @@ public class StatementBlock extends LiveVariableAnalysis implements ParseInfo
 		return _statements.get(i);
 	}
 
-	public ArrayList<Statement> getStatements()
-	{
+	public ArrayList<Statement> getStatements() {
 		return _statements;
 	}
 
@@ -124,7 +123,7 @@ public class StatementBlock extends LiveVariableAnalysis implements ParseInfo
 		_statements = s;
 	}
 
-	public ArrayList<Hop> get_hops() throws HopsException {
+	public ArrayList<Hop> getHops() {
 		return _hops;
 	}
 
@@ -132,7 +131,7 @@ public class StatementBlock extends LiveVariableAnalysis implements ParseInfo
 		return _lops;
 	}
 
-	public void set_hops(ArrayList<Hop> hops) {
+	public void setHops(ArrayList<Hop> hops) {
 		_hops = hops;
 	}
 
@@ -1047,7 +1046,7 @@ public class StatementBlock extends LiveVariableAnalysis implements ParseInfo
 	public boolean updateRecompilationFlag() throws HopsException {
 		return (_requiresRecompile =
 			ConfigurationManager.isDynamicRecompilation()
-			&& Recompiler.requiresRecompilation(get_hops()));
+			&& Recompiler.requiresRecompilation(getHops()));
 	}
 
 	public boolean requiresRecompilation() {

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java b/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
index 05e2c2c..d34353a 100644
--- a/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
+++ b/src/main/java/org/apache/sysml/parser/WhileStatementBlock.java
@@ -247,17 +247,6 @@ public class WhileStatementBlock extends StatementBlock
 		_predicateHops = hops;
 	}
 	
-	@Override
-	public ArrayList<Hop> get_hops() throws HopsException {
-		
-		if (_hops != null && !_hops.isEmpty()){
-			LOG.error(this._statements.get(0).printErrorLocation() + "there should be no HOPs associated with the WhileStatementBlock");
-			throw new HopsException(this._statements.get(0).printErrorLocation() + "there should be no HOPs associated with the WhileStatementBlock");
-		}
-		
-		return _hops;
-	}
-	
 	public Hop getPredicateHops(){
 		return _predicateHops;
 	}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java b/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
index 34ba2fc..fd6909d 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/ProgramBlock.java
@@ -145,7 +145,7 @@ public class ProgramBlock implements ParseInfo
 				&& _sb.requiresRecompilation() )
 			{
 				tmp = Recompiler.recompileHopsDag(
-					_sb, _sb.get_hops(), ec.getVariables(), null, false, true, _tid);
+					_sb, _sb.getHops(), ec.getVariables(), null, false, true, _tid);
 			}
 			if( DMLScript.STATISTICS ){
 				long t1 = System.nanoTime();

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ProgramConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ProgramConverter.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ProgramConverter.java
index 3e4568b..ea3deb5 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ProgramConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/ProgramConverter.java
@@ -543,7 +543,7 @@ public class ProgramConverter
 		{
 			if( ConfigurationManager.getCompilerConfigFlag(ConfigType.ALLOW_PARALLEL_DYN_RECOMPILATION) 
 				&& sb != null  //forced deep copy for function recompilation
-				&& (Recompiler.requiresRecompilation( sb.get_hops() ) || forceDeepCopy)  )
+				&& (Recompiler.requiresRecompilation( sb.getHops() ) || forceDeepCopy)  )
 			{
 				//create new statement (shallow copy livein/liveout for recompile, line numbers for explain)
 				ret = new StatementBlock();
@@ -555,10 +555,10 @@ public class ProgramConverter
 				ret.setReadVariables( sb.variablesRead() );
 				
 				//deep copy hops dag for concurrent recompile
-				ArrayList<Hop> hops = Recompiler.deepCopyHopsDag( sb.get_hops() );
+				ArrayList<Hop> hops = Recompiler.deepCopyHopsDag( sb.getHops() );
 				if( !plain )
 					Recompiler.updateFunctionNames( hops, pid );
-				ret.set_hops( hops );
+				ret.setHops( hops );
 				ret.updateRecompilationFlag();
 			}
 			else

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreeConverter.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreeConverter.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreeConverter.java
index d5b990b..69ad37a 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreeConverter.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreeConverter.java
@@ -459,7 +459,7 @@ public class OptTreeConverter
 			//process all hops
 			node = new OptNode(NodeType.GENERIC);
 			_hlMap.putProgMapping(sb, pb, node);
-			node.addChilds( createAbstractOptNodes(sb.get_hops(), vars, memo) );
+			node.addChilds( createAbstractOptNodes(sb.getHops(), vars, memo) );
 			node.setExecType(ExecType.CP);
 			node.setLineNumbers(sb.getBeginLine(), sb.getEndLine());
 			

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreePlanChecker.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreePlanChecker.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreePlanChecker.java
index 597b7c0..28368c7 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreePlanChecker.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptTreePlanChecker.java
@@ -117,7 +117,7 @@ public class OptTreePlanChecker
 		}
 		else
 		{
-			checkHopDagCorrectness(prog, dprog, sb.get_hops(), pb.getInstructions(), fnStack);
+			checkHopDagCorrectness(prog, dprog, sb.getHops(), pb.getInstructions(), fnStack);
 			//checkLinksProgramStatementBlock(pb, sb);
 		}
 		

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
index 1294809..eca87da 100644
--- a/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
+++ b/src/main/java/org/apache/sysml/runtime/controlprogram/parfor/opt/OptimizerRuleBased.java
@@ -773,7 +773,7 @@ public class OptimizerRuleBased extends Optimizer
 		
 		//construct new instructions
 		ArrayList<Instruction> newInst = Recompiler.recompileHopsDag(
-			sb, sb.get_hops(), vars, null, false, false, 0);
+			sb, sb.getHops(), vars, null, false, false, 0);
 		pb.setInstructions( newInst );   
 		
 		//reset all rix estimated (modified by recompile)

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/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 12a4ccb..98b104f 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
@@ -164,17 +164,17 @@ public class ProgramRecompiler
 			{
 				//process actual hops
 				boolean ret = false;
-				Hop.resetVisitStatus(sb.get_hops());
+				Hop.resetVisitStatus(sb.getHops());
 				if( force )
 				{
 					//set forced execution type
-					for( Hop h : sb.get_hops() )
+					for( Hop h : sb.getHops() )
 						ret |= rFindAndSetCPIndexingHOP(h, var);
 				}
 				else
 				{
 					//release forced execution type
-					for( Hop h : sb.get_hops() )
+					for( Hop h : sb.getHops() )
 						ret |= rFindAndReleaseIndexingHOP(h, var);
 				}
 				
@@ -183,7 +183,7 @@ public class ProgramRecompiler
 				{
 					//construct new instructions
 					ArrayList<Instruction> newInst = Recompiler.recompileHopsDag(
-						sb, sb.get_hops(), ec.getVariables(), null, true, false, 0);
+						sb, sb.getHops(), ec.getVariables(), null, true, false, 0);
 					pb.setInstructions( newInst ); 
 				}
 			}
@@ -245,7 +245,7 @@ public class ProgramRecompiler
 		}
 		else //last level block
 		{
-			ArrayList<Hop> hops = sb.get_hops();
+			ArrayList<Hop> hops = sb.getHops();
 			if( hops != null ) 
 			{	
 				//replace constant literals

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/utils/Explain.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/utils/Explain.java b/src/main/java/org/apache/sysml/utils/Explain.java
index 6a9d696..b72fe99 100644
--- a/src/main/java/org/apache/sysml/utils/Explain.java
+++ b/src/main/java/org/apache/sysml/utils/Explain.java
@@ -617,7 +617,7 @@ public class Explain
 			if (sb.requiresRecompilation()) {
 				addSubGraphHeader(builder, withSubgraph);
 			}
-			ArrayList<Hop> hopsDAG = sb.get_hops();
+			ArrayList<Hop> hopsDAG = sb.getHops();
 			if (hopsDAG != null && !hopsDAG.isEmpty()) {
 				Hop.resetVisitStatus(hopsDAG);
 				for (Hop hop : hopsDAG)
@@ -710,7 +710,7 @@ public class Explain
 			// For generic StatementBlock
 			builder.append(offset);
 			builder.append("GENERIC (lines "+sb.getBeginLine()+"-"+sb.getEndLine()+") [recompile=" + sb.requiresRecompilation() + "]\n");
-			ArrayList<Hop> hopsDAG = sb.get_hops();
+			ArrayList<Hop> hopsDAG = sb.getHops();
 			if( hopsDAG != null && !hopsDAG.isEmpty() ) {
 				Hop.resetVisitStatus(hopsDAG);
 				for (Hop hop : hopsDAG)

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/yarn/ropt/GridEnumerationMemory.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/yarn/ropt/GridEnumerationMemory.java b/src/main/java/org/apache/sysml/yarn/ropt/GridEnumerationMemory.java
index b3e4ae7..934967e 100644
--- a/src/main/java/org/apache/sysml/yarn/ropt/GridEnumerationMemory.java
+++ b/src/main/java/org/apache/sysml/yarn/ropt/GridEnumerationMemory.java
@@ -130,9 +130,9 @@ public class GridEnumerationMemory extends GridEnumeration
 		else
 		{
 			StatementBlock sb = pb.getStatementBlock();
-			if( sb != null && sb.get_hops() != null ){
-				Hop.resetVisitStatus(sb.get_hops());
-				for( Hop hop : sb.get_hops() )
+			if( sb != null && sb.getHops() != null ){
+				Hop.resetVisitStatus(sb.getHops());
+				for( Hop hop : sb.getHops() )
 					getMemoryEstimates(hop, mem);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/systemml/blob/c04929fa/src/main/java/org/apache/sysml/yarn/ropt/ResourceOptimizer.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/yarn/ropt/ResourceOptimizer.java b/src/main/java/org/apache/sysml/yarn/ropt/ResourceOptimizer.java
index c5cabbe..43030da 100644
--- a/src/main/java/org/apache/sysml/yarn/ropt/ResourceOptimizer.java
+++ b/src/main/java/org/apache/sysml/yarn/ropt/ResourceOptimizer.java
@@ -256,7 +256,7 @@ public class ResourceOptimizer
 		{
 			StatementBlock sb = pb.getStatementBlock();
 			ArrayList<Instruction> inst = Recompiler.recompileHopsDag(
-				sb, sb.get_hops(), new LocalVariableMap(), null, false, false, 0);
+				sb, sb.getHops(), new LocalVariableMap(), null, false, false, 0);
 			pb.setInstructions( inst );
 			B.add(pb);
 			_cntCompilePB ++;
@@ -337,7 +337,7 @@ public class ResourceOptimizer
 		{
 			StatementBlock sb = pb.getStatementBlock();
 			ArrayList<Instruction> inst = Recompiler.recompileHopsDag(
-				sb, sb.get_hops(), new LocalVariableMap(), null, false, false, 0);
+				sb, sb.getHops(), new LocalVariableMap(), null, false, false, 0);
 			inst = annotateMRJobInstructions(inst, cp, mr);
 			pb.setInstructions( inst );
 		}
@@ -388,7 +388,7 @@ public class ResourceOptimizer
 		double val = 0;
 		if( COST_INDIVIDUAL_BLOCKS ) {
 			LocalVariableMap vars = new LocalVariableMap();
-			collectReadVariables(pb.getStatementBlock().get_hops(), vars);
+			collectReadVariables(pb.getStatementBlock().getHops(), vars);
 			ExecutionContext ec = ExecutionContextFactory.createContext(false, null);
 			ec.setVariables(vars);
 			val = CostEstimationWrapper.getTimeEstimate(pb, ec, false);	
@@ -499,7 +499,7 @@ public class ResourceOptimizer
 		else //last-level program blocks
 		{
 			StatementBlock sb = pb.getStatementBlock();
-			return pruneHasOnlyUnknownMR(sb.get_hops());
+			return pruneHasOnlyUnknownMR(sb.getHops());
 		}
 	}