You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemml.apache.org by du...@apache.org on 2016/05/10 17:37:51 UTC

incubator-systemml git commit: [SYSTEMML-672] Fix PyDML 1-Based Indexing Bug To Use 0-Based Indexing

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 8c403410c -> 59c1e5540


[SYSTEMML-672] Fix PyDML 1-Based Indexing Bug To Use 0-Based Indexing

Currently, PyDML has a *bug* in which it uses 1-based indexing, as in R & DML, rather than the correct 0-based indexing that Python uses.  This simply fixes this issue in the Parser, and then updates the few PyDML test scripts to use the correct 0-based indexing.  As a result, we now have correct 0-based indexing in PyDML, and slicing correctly has inclusive lower and exclusive upper bounds.

Closes #145.


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

Branch: refs/heads/master
Commit: 59c1e55408188a21cc83aadad1403b31572cb8f7
Parents: 8c40341
Author: Mike Dusenberry <mw...@us.ibm.com>
Authored: Tue May 10 10:37:23 2016 -0700
Committer: Mike Dusenberry <mw...@us.ibm.com>
Committed: Tue May 10 10:37:23 2016 -0700

----------------------------------------------------------------------
 .../parser/common/CommonSyntacticValidator.java |  64 ----------
 .../sysml/parser/dml/DmlSyntacticValidator.java |  68 ++++++++++-
 .../parser/pydml/PydmlSyntacticValidator.java   |  90 ++++++++++++--
 .../apply-transform/apply-transform.pydml       | 120 ++++++++++---------
 .../applications/arima_box-jenkins/arima.pydml  |  86 ++++++-------
 .../applications/cspline/CsplineCG.pydml        |  20 ++--
 .../applications/cspline/CsplineDS.pydml        |  20 ++--
 src/test/scripts/applications/glm/GLM.pydml     |  78 ++++++------
 src/test/scripts/applications/id3/id3.pydml     |  70 +++++------
 src/test/scripts/applications/m-svm/m-svm.pydml |   6 +-
 .../mdabivar/MDABivariateStats.pydml            |  68 +++++------
 .../naive-bayes-parfor/naive-bayes.pydml        |   4 +-
 .../scripts/functions/misc/FunctionsB.pydml     |   4 +-
 .../scripts/functions/misc/FunctionsH.pydml     |   4 +-
 .../scripts/functions/misc/FunctionsK1.pydml    |   4 +-
 15 files changed, 390 insertions(+), 316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
index 6ec0c6d..e75c31d 100644
--- a/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/common/CommonSyntacticValidator.java
@@ -379,70 +379,6 @@ public abstract class CommonSyntacticValidator {
 		}
 	}
 
-	protected void exitIndexedExpressionHelper(ParserRuleContext ctx, String name, ExpressionInfo dataInfo,
-			ExpressionInfo rowLower, ExpressionInfo rowUpper, ExpressionInfo colLower, ExpressionInfo colUpper) {
-		dataInfo.expr = new IndexedIdentifier(name, false, false);
-		setFileLineColumn(dataInfo.expr, ctx);
-		boolean isRowLower = rowLower != null;
-		boolean isRowUpper = rowUpper != null;
-		boolean isColLower = colLower != null;
-		boolean isColUpper = colUpper != null;
-		try {
-			ArrayList< ArrayList<Expression> > exprList = new ArrayList< ArrayList<Expression> >();
-
-			ArrayList<Expression> rowIndices = new ArrayList<Expression>();
-			ArrayList<Expression> colIndices = new ArrayList<Expression>();
-
-
-			if(!isRowLower && !isRowUpper) {
-				// both not set
-				rowIndices.add(null); rowIndices.add(null);
-			}
-			else if(isRowLower && isRowUpper) {
-				// both set
-				rowIndices.add(incrementByOne(rowLower.expr, ctx));
-				rowIndices.add(rowUpper.expr);
-			}
-			else if(isRowLower && !isRowUpper) {
-				// only row set
-				rowIndices.add(incrementByOne(rowLower.expr, ctx));
-			}
-			else {
-				notifyErrorListeners("incorrect index expression for row", ctx.start);
-				return;
-			}
-
-			if(!isColLower && !isColUpper) {
-				// both not set
-				colIndices.add(null); colIndices.add(null);
-			}
-			else if(isColLower && isColUpper) {
-				colIndices.add(incrementByOne(colLower.expr, ctx));
-				colIndices.add(colUpper.expr);
-			}
-			else if(isColLower && !isColUpper) {
-				colIndices.add(incrementByOne(colLower.expr, ctx));
-			}
-			else {
-				notifyErrorListeners("incorrect index expression for column", ctx.start);
-				return;
-			}
-			exprList.add(rowIndices);
-			exprList.add(colIndices);
-			((IndexedIdentifier) dataInfo.expr).setIndices(exprList);
-		}
-		catch(Exception e) {
-			notifyErrorListeners("cannot set the indices", ctx.start);
-			return;
-		}
-	}
-
-	private Expression incrementByOne(Expression expr, ParserRuleContext ctx) {
-		// For maintaining semantic consistency, we have decided to keep 1-based indexing
-		// If in future, PyDML becomes more popular than DML, this can be switched.
-		return expr;
-	}
-
 	protected ConstIdentifier getConstIdFromString(String varValue, Token start) {
 
 		int linePosition = start.getLine();

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
index dd993f6..5f5b2f3 100644
--- a/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/dml/DmlSyntacticValidator.java
@@ -46,6 +46,7 @@ import org.apache.sysml.parser.FunctionCallIdentifier;
 import org.apache.sysml.parser.FunctionStatement;
 import org.apache.sysml.parser.IfStatement;
 import org.apache.sysml.parser.ImportStatement;
+import org.apache.sysml.parser.IndexedIdentifier;
 import org.apache.sysml.parser.IntIdentifier;
 import org.apache.sysml.parser.IterablePredicate;
 import org.apache.sysml.parser.LanguageException;
@@ -265,18 +266,73 @@ public class DmlSyntacticValidator extends CommonSyntacticValidator implements D
 		setFileLineColumn(ctx.dataInfo.expr, ctx);
 	}
 
+	/**
+	 * DML uses 1-based indexing.;
+	 *
+	 * @param ctx the parse tree
+	 */
 	@Override
 	public void exitIndexedExpression(IndexedExpressionContext ctx) {
 		boolean isRowLower = (ctx.rowLower != null && !ctx.rowLower.isEmpty() && (ctx.rowLower.info.expr != null));
 		boolean isRowUpper = (ctx.rowUpper != null && !ctx.rowUpper.isEmpty() && (ctx.rowUpper.info.expr != null));
 		boolean isColLower = (ctx.colLower != null && !ctx.colLower.isEmpty() && (ctx.colLower.info.expr != null));
 		boolean isColUpper = (ctx.colUpper != null && !ctx.colUpper.isEmpty() && (ctx.colUpper.info.expr != null));
-		String name = ctx.name.getText();
-		exitIndexedExpressionHelper(ctx, name, ctx.dataInfo,
-				isRowLower ? ctx.rowLower.info : null,
-				isRowUpper ? ctx.rowUpper.info : null,
-				isColLower ? ctx.colLower.info : null,
-				isColUpper ? ctx.colUpper.info : null);
+		ExpressionInfo rowLower = isRowLower ? ctx.rowLower.info : null;
+		ExpressionInfo rowUpper = isRowUpper ? ctx.rowUpper.info : null;
+		ExpressionInfo colLower = isColLower ? ctx.colLower.info : null;
+		ExpressionInfo colUpper = isColUpper ? ctx.colUpper.info : null;
+
+		ctx.dataInfo.expr = new IndexedIdentifier(ctx.name.getText(), false, false);
+		setFileLineColumn(ctx.dataInfo.expr, ctx);
+
+		try {
+			ArrayList< ArrayList<Expression> > exprList = new ArrayList< ArrayList<Expression> >();
+
+			ArrayList<Expression> rowIndices = new ArrayList<Expression>();
+			ArrayList<Expression> colIndices = new ArrayList<Expression>();
+
+
+			if(!isRowLower && !isRowUpper) {
+				// both not set
+				rowIndices.add(null); rowIndices.add(null);
+			}
+			else if(isRowLower && isRowUpper) {
+				// both set
+				rowIndices.add(rowLower.expr);
+				rowIndices.add(rowUpper.expr);
+			}
+			else if(isRowLower && !isRowUpper) {
+				// only row set
+				rowIndices.add(rowLower.expr);
+			}
+			else {
+				notifyErrorListeners("incorrect index expression for row", ctx.start);
+				return;
+			}
+
+			if(!isColLower && !isColUpper) {
+				// both not set
+				colIndices.add(null); colIndices.add(null);
+			}
+			else if(isColLower && isColUpper) {
+				colIndices.add(colLower.expr);
+				colIndices.add(colUpper.expr);
+			}
+			else if(isColLower && !isColUpper) {
+				colIndices.add(colLower.expr);
+			}
+			else {
+				notifyErrorListeners("incorrect index expression for column", ctx.start);
+				return;
+			}
+			exprList.add(rowIndices);
+			exprList.add(colIndices);
+			((IndexedIdentifier) ctx.dataInfo.expr).setIndices(exprList);
+		}
+		catch(Exception e) {
+			notifyErrorListeners("cannot set the indices", ctx.start);
+			return;
+		}
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java b/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
index f192ddc..aa09e0d 100644
--- a/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
@@ -39,6 +39,7 @@ import org.apache.sysml.parser.BinaryExpression;
 import org.apache.sysml.parser.ConditionalPredicate;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DataIdentifier;
+import org.apache.sysml.parser.DoubleIdentifier;
 import org.apache.sysml.parser.Expression;
 import org.apache.sysml.parser.Expression.DataType;
 import org.apache.sysml.parser.Expression.ValueType;
@@ -49,6 +50,7 @@ import org.apache.sysml.parser.FunctionStatement;
 import org.apache.sysml.parser.IfStatement;
 import org.apache.sysml.parser.ImportStatement;
 import org.apache.sysml.parser.IntIdentifier;
+import org.apache.sysml.parser.IndexedIdentifier;
 import org.apache.sysml.parser.IterablePredicate;
 import org.apache.sysml.parser.LanguageException;
 import org.apache.sysml.parser.ParForStatement;
@@ -311,21 +313,95 @@ public class PydmlSyntacticValidator extends CommonSyntacticValidator implements
 	}
 
 
+	/**
+	 * PyDML uses 0-based indexing, so we increment lower indices by 1
+	 * when translating to DML.
+	 *
+	 * @param ctx the parse tree
+	 */
 	@Override
 	public void exitIndexedExpression(IndexedExpressionContext ctx) {
 		boolean isRowLower = (ctx.rowLower != null && !ctx.rowLower.isEmpty() && (ctx.rowLower.info.expr != null));
 		boolean isRowUpper = (ctx.rowUpper != null && !ctx.rowUpper.isEmpty() && (ctx.rowUpper.info.expr != null));
 		boolean isColLower = (ctx.colLower != null && !ctx.colLower.isEmpty() && (ctx.colLower.info.expr != null));
 		boolean isColUpper = (ctx.colUpper != null && !ctx.colUpper.isEmpty() && (ctx.colUpper.info.expr != null));
-		String name = ctx.name.getText();
-		exitIndexedExpressionHelper(ctx, name, ctx.dataInfo,
-				isRowLower ? ctx.rowLower.info : null,
-				isRowUpper ? ctx.rowUpper.info : null,
-				isColLower ? ctx.colLower.info : null,
-				isColUpper ? ctx.colUpper.info : null);
-	}
+        ExpressionInfo rowLower = isRowLower ? ctx.rowLower.info : null;
+        ExpressionInfo rowUpper = isRowUpper ? ctx.rowUpper.info : null;
+        ExpressionInfo colLower = isColLower ? ctx.colLower.info : null;
+        ExpressionInfo colUpper = isColUpper ? ctx.colUpper.info : null;
+
+		ctx.dataInfo.expr = new IndexedIdentifier(ctx.name.getText(), false, false);
+		setFileLineColumn(ctx.dataInfo.expr, ctx);
+
+		try {
+			ArrayList< ArrayList<Expression> > exprList = new ArrayList< ArrayList<Expression> >();
+
+			ArrayList<Expression> rowIndices = new ArrayList<Expression>();
+			ArrayList<Expression> colIndices = new ArrayList<Expression>();
+
 
+			if(!isRowLower && !isRowUpper) {
+				// both not set
+				rowIndices.add(null); rowIndices.add(null);
+			}
+			else if(isRowLower && isRowUpper) {
+				// both set
+				rowIndices.add(incrementByOne(rowLower.expr, ctx));
+				rowIndices.add(rowUpper.expr);
+			}
+			else if(isRowLower && !isRowUpper) {
+				// only row set
+				rowIndices.add(incrementByOne(rowLower.expr, ctx));
+			}
+			else {
+				notifyErrorListeners("incorrect index expression for row", ctx.start);
+				return;
+			}
+
+			if(!isColLower && !isColUpper) {
+				// both not set
+				colIndices.add(null); colIndices.add(null);
+			}
+			else if(isColLower && isColUpper) {
+				colIndices.add(incrementByOne(colLower.expr, ctx));
+				colIndices.add(colUpper.expr);
+			}
+			else if(isColLower && !isColUpper) {
+				colIndices.add(incrementByOne(colLower.expr, ctx));
+			}
+			else {
+				notifyErrorListeners("incorrect index expression for column", ctx.start);
+				return;
+			}
+			exprList.add(rowIndices);
+			exprList.add(colIndices);
+			((IndexedIdentifier) ctx.dataInfo.expr).setIndices(exprList);
+		}
+		catch(Exception e) {
+			notifyErrorListeners("cannot set the indices", ctx.start);
+			return;
+		}
+	}
 
+	/**
+	 * Increment lower indices by 1 when translating from PyDML
+	 * (0-based indexing) to DML (1-based indexing).
+	 *
+	 * @param expr
+	 * @param ctx
+	 * @return
+	 */
+	private Expression incrementByOne(Expression expr, ParserRuleContext ctx) {
+		// Addition and subtraction operator same as DML
+		Expression.BinaryOp bop = Expression.getBinaryOp("+");
+		Expression retVal = new BinaryExpression(bop);
+		((BinaryExpression)retVal).setLeft(expr);
+		int line = ctx.start.getLine();
+		int col = ctx.start.getCharPositionInLine();
+		((BinaryExpression)retVal).setRight(new DoubleIdentifier(1.0, currentFile, line, col, line, col));
+		setFileLineColumn(retVal, ctx);
+		return retVal;
+	}
 
 	// -----------------------------------------------------------------
 	//          Command line parameters (begin with a '$')

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/apply-transform/apply-transform.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/apply-transform/apply-transform.pydml b/src/test/scripts/applications/apply-transform/apply-transform.pydml
index cc95e85..d528098 100644
--- a/src/test/scripts/applications/apply-transform/apply-transform.pydml
+++ b/src/test/scripts/applications/apply-transform/apply-transform.pydml
@@ -30,89 +30,95 @@ if(cmdLine_missing_value_maps != " "):
     missing_val_maps = read(cmdLine_missing_value_maps)
 
     last_data_col = ncol(original_X)-nrow(missing_val_maps)
-    X = original_X[,1:last_data_col]
+    X = original_X[,0:last_data_col]
 else:
     X = original_X
 
-# col 1: col index of missing indicator col
-#         0 otherwise
-# col 2: global mean if imputation is needed
-# col 3: num_bins if binning is required
-# col 4: bin width if binning is required
-# col 5: min val if binning is required
-# col 6: begin col if dummy coding is required
-# col 7: end col if dummy coding is required
-# col 8: 1 if normalization is required 0 ow
-# col 9: mean for normalization
-# col 10: std for z-scoring for normalization
-#         -1 indicates mean subtraction  
+# col 0: col index of missing indicator col
+#         0 otherwise. NOTE: Assumes 1-based
+#         indexing, so we adjust to 0-based
+#         for PyDML.
+# col 1: global mean if imputation is needed
+# col 2: num_bins if binning is required
+# col 3: bin width if binning is required
+# col 4: min val if binning is required
+# col 5: begin col if dummy coding is required
+#         NOTE: Assumes 1-based indexing, so
+#         we adjust to 0-based for PyDML.
+# col 6: end col if dummy coding is required
+#         NOTE: Assumes 1-based indexing, so
+#         we adjust to 0-based for PyDML.
+# col 7: 1 if normalization is required 0 ow
+# col 8: mean for normalization
+# col 9: std for z-scoring for normalization
+#         -1 indicates mean subtraction
 attrinfo = full(0, rows=ncol(X), cols=10)
 
 if(cmdLine_missing_value_maps != " "):
-    missing_indicator_mat = original_X[,(last_data_col+1):ncol(original_X)]
-    
+    missing_indicator_mat = original_X[,last_data_col:ncol(original_X)]
+
     parfor(i in 1:nrow(missing_val_maps), check=0):
-        attr_index_mv = scalar(missing_val_maps[i,1])
-        attrinfo[attr_index_mv,1] = i
-        attrinfo[attr_index_mv,2] = missing_val_maps[i,2]
-    
+        attr_index_mv = scalar(missing_val_maps[i-1,0])
+        attrinfo[attr_index_mv-1,0] = i
+        attrinfo[attr_index_mv-1,1] = missing_val_maps[i-1,1]
+
 if(cmdLine_bin_defns != " "):
     bin_defns = read(cmdLine_bin_defns)
     parfor(i in 1:nrow(bin_defns), check=0):
-        attr_index_bin = scalar(bin_defns[i,1])
-        attrinfo[attr_index_bin,3] = bin_defns[i,4]
-        attrinfo[attr_index_bin,4] = bin_defns[i,2]
-        attrinfo[attr_index_bin,5] = bin_defns[i,3]
+        attr_index_bin = scalar(bin_defns[i-1,0])
+        attrinfo[attr_index_bin-1,2] = bin_defns[i-1,3]
+        attrinfo[attr_index_bin-1,3] = bin_defns[i-1,1]
+        attrinfo[attr_index_bin-1,4] = bin_defns[i-1,2]
 
 if(cmdLine_dummy_code_maps != " "):
     dummy_code_maps = read(cmdLine_dummy_code_maps)
     parfor(i in 1:nrow(dummy_code_maps), check=0):
-        attr_index_dc = scalar(dummy_code_maps[i,1])
-        attrinfo[attr_index_dc,6] = dummy_code_maps[i,2]
-        attrinfo[attr_index_dc,7] = dummy_code_maps[i,3]
+        attr_index_dc = scalar(dummy_code_maps[i-1,0])
+        attrinfo[attr_index_dc-1,5] = dummy_code_maps[i-1,1]
+        attrinfo[attr_index_dc-1,6] = dummy_code_maps[i-1,2]
 else:
+    attrinfo[,5] = seq(1, ncol(X), 1)
     attrinfo[,6] = seq(1, ncol(X), 1)
-    attrinfo[,7] = seq(1, ncol(X), 1)
 
 if(cmdLine_normalization_maps != " "):
     normalization_map = read(cmdLine_normalization_maps)
     parfor(i in 1:nrow(normalization_map), check=0):
-        attr_index_normalization = scalar(normalization_map[i,1])
-        attrinfo[attr_index_normalization,8] = 1
-        attrinfo[attr_index_normalization,9] = scalar(normalization_map[i,2])
-        attrinfo[attr_index_normalization,10] = scalar(normalization_map[i,3])
+        attr_index_normalization = scalar(normalization_map[i-1,0])
+        attrinfo[attr_index_normalization-1,7] = 1
+        attrinfo[attr_index_normalization-1,8] = scalar(normalization_map[i-1,1])
+        attrinfo[attr_index_normalization-1,9] = scalar(normalization_map[i-1,2])
 
 #write(attrinfo, "binning/attrinfo.mtx", format="csv")
 
-cols_in_transformed_X = scalar(attrinfo[nrow(attrinfo),6])
+cols_in_transformed_X = scalar(attrinfo[nrow(attrinfo)-1,5])
 new_X = full(0, rows=nrow(X), cols=cols_in_transformed_X)
 log = full(0, rows=ncol(X), cols=2)
 parfor(i in 1:ncol(X), check=0):
-    col = X[,i]
-    
-    mv_col_id = scalar(attrinfo[i,1])
-    global_mean = scalar(attrinfo[i,2])
-    num_bins = scalar(attrinfo[i,3])
-    bin_width = scalar(attrinfo[i,4])
-    min_val = scalar(attrinfo[i,5])
-    dummy_coding_beg_col = scalar(attrinfo[i,6])
-    dummy_coding_end_col = scalar(attrinfo[i,7])
-    normalization_needed = scalar(attrinfo[i,8])
-    normalization_mean = scalar(attrinfo[i,9])
-    normalization_std = scalar(attrinfo[i,10])
-    
+    col = X[,i-1]
+
+    mv_col_id = scalar(attrinfo[i-1,0])
+    global_mean = scalar(attrinfo[i-1,1])
+    num_bins = scalar(attrinfo[i-1,2])
+    bin_width = scalar(attrinfo[i-1,3])
+    min_val = scalar(attrinfo[i-1,4])
+    dummy_coding_beg_col = scalar(attrinfo[i-1,5])
+    dummy_coding_end_col = scalar(attrinfo[i-1,6])
+    normalization_needed = scalar(attrinfo[i-1,7])
+    normalization_mean = scalar(attrinfo[i-1,8])
+    normalization_std = scalar(attrinfo[i-1,9])
+
     if(mv_col_id > 0):
         # fill-in with global mean
-        col = col + missing_indicator_mat[,mv_col_id] * global_mean
-    
+        col = col + missing_indicator_mat[,mv_col_id-1] * global_mean
+
     if(num_bins > 0):
         # only for equiwidth bins
-    
+
         # note that max_val entries will get assigned num_bins+1
         col = round((col - min_val)/bin_width - 0.5) + 1
         less_than_lb = ppred(col, 1, "<")
         more_than_ub = ppred(col, num_bins, ">")
-        
+
         col = (1 - less_than_lb - more_than_ub)*col + more_than_ub*num_bins + less_than_lb
 
     if(dummy_coding_beg_col == dummy_coding_end_col):
@@ -121,26 +127,26 @@ parfor(i in 1:ncol(X), check=0):
                 col = col - normalization_mean
             else:
                 col = (col - normalization_mean)/normalization_std
-        
-        new_X[,dummy_coding_beg_col] = col
+
+        new_X[,dummy_coding_beg_col-1] = col
     else:
         min_val = min(col)
         max_val = max(col)
         if(min_val >= 1 & max_val <= dummy_coding_end_col - dummy_coding_beg_col + 1):
             res = table(seq(1, nrow(X), 1), col, nrow(X), (dummy_coding_end_col-dummy_coding_beg_col+1))
-            new_X[,dummy_coding_beg_col:dummy_coding_end_col] = res
+            new_X[,dummy_coding_beg_col-1:dummy_coding_end_col] = res
         else:
-            log[i,1] = 1
+            log[i-1,0] = 1
             if(min_val < 1):
-                log[i,2] = min_val
+                log[i-1,1] = min_val
             else:
-                log[i,2] = max_val
+                log[i-1,1] = max_val
 
 save(new_X, $transformed_X, format="text")
 
 s = "Warning Messages"
 for(i in 1:nrow(log)):
-    if(scalar(log[i,1]) == 1):
-        s = append(s, "Unseen value in column " + i + " (" + scalar(log[i,2]) + ")")
+    if(scalar(log[i-1,0]) == 1):
+        s = append(s, "Unseen value in column " + i + " (" + scalar(log[i-1,1]) + ")")
 
 save(s, $Log)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/arima_box-jenkins/arima.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/arima_box-jenkins/arima.pydml b/src/test/scripts/applications/arima_box-jenkins/arima.pydml
index 1df70d1..d1299c9 100644
--- a/src/test/scripts/applications/arima_box-jenkins/arima.pydml
+++ b/src/test/scripts/applications/arima_box-jenkins/arima.pydml
@@ -36,21 +36,21 @@
 #changing to additive sar since R's arima seems to do that
 
 def arima_css(w:matrix[float], X:matrix[float], pIn: int, P: int, qIn: int, Q:int, s:int, useJacobi: int) -> (obj: float):
-    b = dot(X[,2:ncol(X)], w)
+    b = dot(X[,1:ncol(X)], w)
     
     R = Rand(rows=nrow(X), cols=nrow(X), min=0, max=0)
     for(i7 in 1:qIn):
         ma_ind_ns = P+pIn+i7
         err_ind_ns = i7
         ones_ns = Rand(rows=nrow(R)-err_ind_ns, cols=1, min=1, max=1)
-        d_ns = ones_ns * scalar(w[ma_ind_ns,1])
-        R[1+err_ind_ns:nrow(R),1:ncol(R)-err_ind_ns] = R[1+err_ind_ns:nrow(R),1:ncol(R)-err_ind_ns] + diag(d_ns)
+        d_ns = ones_ns * scalar(w[ma_ind_ns-1,0])
+        R[err_ind_ns:nrow(R),0:ncol(R)-err_ind_ns] = R[err_ind_ns:nrow(R),0:ncol(R)-err_ind_ns] + diag(d_ns)
     for(i8 in 1:Q):
         ma_ind_s = P+pIn+qIn+i8
         err_ind_s = s*i8
         ones_s = Rand(rows=nrow(R)-err_ind_s, cols=1, min=1, max=1)
-        d_s = ones_s * scalar(w[ma_ind_s,1])
-        R[1+err_ind_s:nrow(R),1:ncol(R)-err_ind_s] = R[1+err_ind_s:nrow(R),1:ncol(R)-err_ind_s] + diag(d_s)
+        d_s = ones_s * scalar(w[ma_ind_s-1,0])
+        R[err_ind_s:nrow(R),0:ncol(R)-err_ind_s] = R[err_ind_s:nrow(R),0:ncol(R)-err_ind_s] + diag(d_s)
     
     #checking for strict diagonal dominance
     #required for jacobi's method
@@ -96,7 +96,7 @@ def arima_css(w:matrix[float], X:matrix[float], pIn: int, P: int, qIn: int, Q:in
             beta = norm_r2 / old_norm_r2
             p = -r + beta * p
             iter = iter + 1    
-    errs = X[,1] - y_hat
+    errs = X[,0] - y_hat
     obj = sum(errs*errs)
 # end arima_css function
 
@@ -130,7 +130,7 @@ if(num_rows <= d):
 Y = X
 for(i in 1:d):
     n1 = nrow(Y)+0.0
-    Y = Y[2:n1,] - Y[1:n1-1,]
+    Y = Y[1:n1,] - Y[0:n1-1,]
 
 num_rows = nrow(Y)+0.0
 if(num_rows <= s*D):
@@ -138,7 +138,7 @@ if(num_rows <= s*D):
 
 for(i in 1:D):
     n1 = nrow(Y)+0.0
-    Y = Y[s+1:n1,] - Y[1:n1-s,]
+    Y = Y[s:n1,] - Y[0:n1-s,]
 
 n = nrow(Y)
 
@@ -157,61 +157,61 @@ if(include_mean == 1):
 totcols = 1+p+P+Q+q #target col (X), p-P cols, q-Q cols  
 
 Z = Rand(rows=n, cols=totcols, min=0, max=0)
-Z[,1] = Y #target col
+Z[,0] = Y #target col
 
 parfor(i1 in 1:p, check=0):
-    Z[i1+1:n,1+i1] = Y[1:n-i1,]
+    Z[i1:n,i1] = Y[0:n-i1,]
 
 parfor(i2 in 1:P, check=0):
-    Z[s*i2+1:n,1+p+i2] = Y[1:n-s*i2,]
+    Z[s*i2:n,p+i2] = Y[0:n-s*i2,]
 
 parfor(i5 in 1:q, check=0):
-    Z[i5+1:n,1+P+p+i5] = Y[1:n-i5,]
+    Z[i5:n,P+p+i5] = Y[0:n-i5,]
 
 parfor(i6 in 1:Q, check=0):
-    Z[s*i6+1:n,1+P+p+q+i6] = Y[1:n-s*i6,]
+    Z[s*i6:n,P+p+q+i6] = Y[0:n-s*i6,]
 
 
 one = Rand(rows=1, cols=1, min=1, max=1)
 
 simplex = Rand(rows=totcols-1, cols=totcols, min=0, max=0)
 for(i in 2:ncol(simplex)):
-    simplex[i-1,i] = 0.1
+    simplex[i-2,i-1] = 0.1
 
 num_func_invoc = 0
 
 objvals = Rand(rows=1, cols=ncol(simplex), min=0, max=0)
-parfor(i3 in 1:ncol(simplex)):
-    arima_css_obj_val = arima_css(simplex[,i3], Z, p, P, q, Q, s, useJacobi)
-    objvals[1,i3] = arima_css_obj_val
+parfor(i3 in 1:ncol(simplex), check=0):
+    arima_css_obj_val = arima_css(simplex[,i3-1], Z, p, P, q, Q, s, useJacobi)
+    objvals[0,i3-1] = arima_css_obj_val
 
 num_func_invoc = num_func_invoc + ncol(simplex)
 
-tol = 1.5 * (10**-8) * scalar(objvals[1,1])
+tol = 1.5 * (10**-8) * scalar(objvals[0,0])
 
 continue = 1
 while(continue == 1 & num_func_invoc <= max_func_invoc):
     best_index = 1
     worst_index = 1
     for(i in 2:ncol(objvals)):
-        this = scalar(objvals[1,i])
-        that = scalar(objvals[1,best_index])
+        this = scalar(objvals[0,i-1])
+        that = scalar(objvals[0,best_index-1])
         if(that > this):
             best_index = i
-        that = scalar(objvals[1,worst_index])
+        that = scalar(objvals[0,worst_index-1])
         if(that < this):
             worst_index = i
     
-    best_obj_val = scalar(objvals[1,best_index])
-    worst_obj_val = scalar(objvals[1,worst_index])
+    best_obj_val = scalar(objvals[0,best_index-1])
+    worst_obj_val = scalar(objvals[0,worst_index-1])
     if(worst_obj_val <= best_obj_val + tol):
         continue = 0
     
     print("#Function calls::" + num_func_invoc + " OBJ: " + best_obj_val)
     
-    c = (rowSums(simplex) - simplex[,worst_index])/(nrow(simplex))
+    c = (rowSums(simplex) - simplex[,worst_index-1])/(nrow(simplex))
     
-    x_r = 2*c - simplex[,worst_index]
+    x_r = 2*c - simplex[,worst_index-1]
     obj_x_r = arima_css(x_r, Z, p, P, q, Q, s, useJacobi)
     num_func_invoc = num_func_invoc + 1
     
@@ -221,38 +221,38 @@ while(continue == 1 & num_func_invoc <= max_func_invoc):
         num_func_invoc = num_func_invoc + 1
         
         if(obj_x_r <= obj_x_e):
-            simplex[,worst_index] = x_r
-            objvals[1,worst_index] = obj_x_r
+            simplex[,worst_index-1] = x_r
+            objvals[0,worst_index-1] = obj_x_r
         else:
-            simplex[,worst_index] = x_e
-            objvals[1,worst_index] = obj_x_e
+            simplex[,worst_index-1] = x_e
+            objvals[0,worst_index-1] = obj_x_e
     else:
         if(obj_x_r < worst_obj_val):
-            simplex[,worst_index] = x_r
-            objvals[1,worst_index] = obj_x_r
+            simplex[,worst_index-1] = x_r
+            objvals[0,worst_index-1] = obj_x_r
         
-        x_c_in = (simplex[,worst_index] + c)/2
+        x_c_in = (simplex[,worst_index-1] + c)/2
         obj_x_c_in = arima_css(x_c_in, Z, p, P, q, Q, s, useJacobi)
         num_func_invoc = num_func_invoc + 1
         
-        if(obj_x_c_in < scalar(objvals[1,worst_index])):
-            simplex[,worst_index] = x_c_in
-            objvals[1,worst_index] = obj_x_c_in
+        if(obj_x_c_in < scalar(objvals[0,worst_index-1])):
+            simplex[,worst_index-1] = x_c_in
+            objvals[0,worst_index-1] = obj_x_c_in
         else:
             if(obj_x_r >= worst_obj_val):
-                best_point = simplex[,best_index]
-                parfor(i4 in 1:ncol(simplex)):
+                best_point = simplex[,best_index-1]
+                parfor(i4 in 1:ncol(simplex), check=0):
                     if(i4 != best_index):
-                        simplex[,i4] = (simplex[,i4] + best_point)/2
-                        tmp = arima_css(simplex[,i4], Z, p, P, q, Q, s, useJacobi)
-                        objvals[1,i4] = tmp*one
+                        simplex[,i4-1] = (simplex[,i4-1] + best_point)/2
+                        tmp = arima_css(simplex[,i4-1], Z, p, P, q, Q, s, useJacobi)
+                        objvals[0,i4-1] = tmp*one
                 num_func_invoc = num_func_invoc + ncol(simplex) - 1
 
-best_point = simplex[,best_index]
+best_point = simplex[,best_index-1]
 if(include_mean == 1):
     tmp2 = Rand(rows=totcols, cols=1, min=0, max=0)
-    tmp2[1:nrow(best_point),1] = best_point
-    tmp2[nrow(tmp2),1] = mu
+    tmp2[0:nrow(best_point),0] = best_point
+    tmp2[nrow(tmp2)-1,0] = mu
     best_point = tmp2
 
 save(best_point, $12, format="text")

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/cspline/CsplineCG.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/cspline/CsplineCG.pydml b/src/test/scripts/applications/cspline/CsplineCG.pydml
index 29c55a1..e7ef6bb 100644
--- a/src/test/scripts/applications/cspline/CsplineCG.pydml
+++ b/src/test/scripts/applications/cspline/CsplineCG.pydml
@@ -148,12 +148,12 @@ def interpSpline(x: float, X: matrix[float], Y: matrix[float], K: matrix[float])
     i = nrow(X) - sum(ppred(X, x, ">=")) + 1
     
     #calc the y as per the algo docs
-    t = (x - X[i-1,1]) / ( X[i,1] - X[i-1,1])
+    t = (x - X[i-2,0]) / ( X[i-1,0] - X[i-2,0])
     
-    a =  K[i-1,1]*(X[i,1]-X[i-1,1]) - (Y[i,1]-Y[i-1,1])
-    b = -K[i,1]*(X[i,1]-X[i-1,1]) + (Y[i,1]-Y[i-1,1])
+    a =  K[i-2,0]*(X[i-1,0]-X[i-2,0]) - (Y[i-1,0]-Y[i-2,0])
+    b = -K[i-1,0]*(X[i-1,0]-X[i-2,0]) + (Y[i-1,0]-Y[i-2,0])
     
-    qm = (1-t)*Y[i-1,1] + t*Y[i,1] + t*(1-t)*(a*(1-t)+b*t)
+    qm = (1-t)*Y[i-2,0] + t*Y[i-1,0] + t*(1-t)*(a*(1-t)+b*t)
     
     q = scalar(qm)
 
@@ -223,10 +223,10 @@ def trunc(X: matrix[float], # nxm matrix
     
     if (r != by ):
         if (dir == "up"):
-            Y[1:r-by,] = X[1+by:r,]
+            Y[0:r-by,] = X[by:r,]
         else:
             if (dir == "down"):
-                Y[1:r-by,] = X[1:r-by,]
+                Y[0:r-by,] = X[0:r-by,]
             else:
                 stop("trunc unsupported direction " + dir)
 
@@ -242,16 +242,16 @@ def resize(X: matrix[float], #nxm matrix
     cn = c + cby
     Y = full(0.0, rn, cn)
     if (dir == "tr"): # top right
-        Y[1+rby:rn, 1:c] = X
+        Y[rby:rn, 0:c] = X
     else:
         if (dir == "bl"): # bottom left
-            Y[1:r, 1+cby:cn] = X
+            Y[0:r, cby:cn] = X
         else:
             if (dir == "tl"): # top left
-                Y[1+rby:rn, 1+cby:cn ] = X
+                Y[rby:rn, cby:cn ] = X
             else:
                 if (dir == "br"): # bottom right
-                    Y[1:r, 1:c] = X
+                    Y[0:r, 0:c] = X
                 else:
                     stop("Unknown direction dir => " + dir)
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/cspline/CsplineDS.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/cspline/CsplineDS.pydml b/src/test/scripts/applications/cspline/CsplineDS.pydml
index 40847a7..363df7d 100644
--- a/src/test/scripts/applications/cspline/CsplineDS.pydml
+++ b/src/test/scripts/applications/cspline/CsplineDS.pydml
@@ -127,12 +127,12 @@ def interpSpline(x: float, X: matrix[float], Y: matrix[float], K: matrix[float])
     i = nrow(X) - sum(ppred(X, x, ">=")) + 1
     
     #calc the y as per the algo docs
-    t = (x - X[i-1,1]) / ( X[i,1] - X[i-1,1])
+    t = (x - X[i-2,0]) / ( X[i-1,0] - X[i-2,0])
     
-    a =  K[i-1,1]*(X[i,1]-X[i-1,1]) - (Y[i,1]-Y[i-1,1])
-    b = -K[i,1]*(X[i,1]-X[i-1,1]) + (Y[i,1]-Y[i-1,1])
+    a =  K[i-2,0]*(X[i-1,0]-X[i-2,0]) - (Y[i-1,0]-Y[i-2,0])
+    b = -K[i-1,0]*(X[i-1,0]-X[i-2,0]) + (Y[i-1,0]-Y[i-2,0])
     
-    qm = (1-t)*Y[i-1,1] + t*Y[i,1] + t*(1-t)*(a*(1-t)+b*t)
+    qm = (1-t)*Y[i-2,0] + t*Y[i-1,0] + t*(1-t)*(a*(1-t)+b*t)
     
     q = scalar(qm)
 
@@ -153,10 +153,10 @@ def trunc(X: matrix[float], # nxm matrix
     
     if (r != by ):
         if (dir == "up"):
-            Y[1:r-by,] = X[1+by:r,]
+            Y[0:r-by,] = X[by:r,]
         else:
             if (dir == "down"):
-                Y[1:r-by,] = X[1:r-by,]
+                Y[0:r-by,] = X[0:r-by,]
             else:
                 stop("trunc unsupported direction " + dir)
 
@@ -172,16 +172,16 @@ def resize(X: matrix[float], #nxm matrix
     cn = c + cby
     Y = full(0.0, rn, cn)
     if (dir == "tr"): # top right
-        Y[1+rby:rn, 1:c] = X
+        Y[rby:rn, 0:c] = X
     else:
         if (dir == "bl"): # bottom left
-            Y[1:r, 1+cby:cn] = X
+            Y[0:r, cby:cn] = X
         else:
             if (dir == "tl"): # top left
-                Y[1+rby:rn, 1+cby:cn ] = X
+                Y[rby:rn, cby:cn ] = X
             else:
                 if (dir == "br"): # bottom right
-                    Y[1:r, 1:c] = X
+                    Y[0:r, 0:c] = X
                 else:
                     stop("Unknown direction dir => " + dir)
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/glm/GLM.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/glm/GLM.pydml b/src/test/scripts/applications/glm/GLM.pydml
index e737a13..5e7269d 100644
--- a/src/test/scripts/applications/glm/GLM.pydml
+++ b/src/test/scripts/applications/glm/GLM.pydml
@@ -189,7 +189,7 @@ if (intercept_status == 1 | intercept_status == 2): # add the intercept column
 
 scale_lambda = full (1, rows = num_features, cols = 1)
 if (intercept_status == 1 | intercept_status == 2):
-    scale_lambda [num_features, 1] = 0
+    scale_lambda [num_features-1, 0] = 0
 
 if (intercept_status == 2): # scale-&-shift X columns to mean 0, variance 1
     # Important assumption: X [, num_features] = ones_r
@@ -197,9 +197,9 @@ if (intercept_status == 2): # scale-&-shift X columns to mean 0, variance 1
     var_X_cols = (transpose(colSums (X ** 2)) - num_records * (avg_X_cols ** 2)) / (num_records - 1)
     is_unsafe = ppred (var_X_cols, 0.0, "<=")
     scale_X = 1.0 / sqrt (var_X_cols * (1 - is_unsafe) + is_unsafe)
-    scale_X [num_features, 1] = 1
+    scale_X [num_features-1, 0] = 1
     shift_X = - avg_X_cols * scale_X
-    shift_X [num_features, 1] = 0
+    shift_X [num_features-1, 0] = 0
     rowSums_X_sq = dot((X ** 2), (scale_X ** 2)) + dot(X, (2 * scale_X * shift_X)) + sum (shift_X ** 2)
 else:
     scale_X = full (1, rows = num_features, cols = 1)
@@ -283,7 +283,7 @@ if (is_supported == 1):
         print ("BEGIN IRLS ITERATIONS...")
         
         ssX_newbeta = dot(diag (scale_X), newbeta)
-        ssX_newbeta [num_features, ] = ssX_newbeta [num_features, ] + dot(transpose(shift_X), newbeta)
+        ssX_newbeta [num_features-1, ] = ssX_newbeta [num_features-1, ] + dot(transpose(shift_X), newbeta)
         all_linear_terms = dot(X, ssX_newbeta)
         
         [new_log_l, isNaN_new_log_l] = glm_log_likelihood_part (all_linear_terms, Y, distribution_type, variance_as_power_of_the_mean, link_type, link_as_power_of_the_mean)
@@ -348,7 +348,7 @@ if (is_supported == 1):
                 #     w   = 1.0 / (y_var * link_grad * link_grad)
                 
                 gXY = dot(- transpose(X), g_Y)
-                g = dot(diag (scale_X), gXY) + dot(shift_X, gXY [num_features, ])
+                g = dot(diag (scale_X), gXY) + dot(shift_X, gXY [num_features-1, ])
                 g_norm = sqrt (sum ((g + lambda * beta) ** 2))
                 
                 if (fileLog != " "):
@@ -361,7 +361,7 @@ if (is_supported == 1):
             newbeta = beta + z
             
             ssX_newbeta = dot(diag (scale_X), newbeta)
-            ssX_newbeta [num_features, ] = ssX_newbeta [num_features, ] + dot(transpose(shift_X), newbeta)
+            ssX_newbeta [num_features-1, ] = ssX_newbeta [num_features-1, ] + dot(transpose(shift_X), newbeta)
             all_linear_terms = dot(X, ssX_newbeta)
             
             [new_log_l, isNaN_new_log_l] = glm_log_likelihood_part (all_linear_terms, Y, distribution_type, variance_as_power_of_the_mean, link_type, link_as_power_of_the_mean)
@@ -425,7 +425,7 @@ if (is_supported == 1):
         
         
         ssX_beta = dot(diag (scale_X), beta)
-        ssX_beta [num_features, ] = ssX_beta [num_features, ] + dot(transpose(shift_X), beta)
+        ssX_beta [num_features-1, ] = ssX_beta [num_features-1, ] + dot(transpose(shift_X), beta)
         if (intercept_status == 2):
             beta_out = append (ssX_beta, beta)
         else:
@@ -435,17 +435,17 @@ if (is_supported == 1):
         save (beta_out, fileB, format=fmtB)
         
         if (intercept_status == 1 | intercept_status == 2):
-            intercept_value = scalar (beta_out [num_features, 1])
-            beta_noicept = beta_out [1 : (num_features - 1), 1]
+            intercept_value = scalar (beta_out [num_features-1, 0])
+            beta_noicept = beta_out [0 : (num_features - 1), 0]
         else:
-            beta_noicept = beta_out [1 : num_features, 1]
+            beta_noicept = beta_out [0 : num_features, 0]
         
         min_beta = min (beta_noicept)
         max_beta = max (beta_noicept)
         tmp_i_min_beta = rowIndexMin (transpose(beta_noicept))
-        i_min_beta = scalar (tmp_i_min_beta [1, 1])
+        i_min_beta = scalar (tmp_i_min_beta [0, 0])
         tmp_i_max_beta = rowIndexMax (transpose(beta_noicept))
-        i_max_beta = scalar (tmp_i_max_beta [1, 1])
+        i_max_beta = scalar (tmp_i_max_beta [0, 0])
         
         #####  OVER-DISPERSION PART  #####
         
@@ -641,11 +641,11 @@ def glm_initialize(X: matrix[float], Y: matrix[float], dist_type: int, var_power
     max_iter_CG: int) -> (beta: matrix[float], saturated_log_l: float, isNaN: int):
     saturated_log_l = 0.0
     isNaN = 0
-    y_corr = Y [, 1]
+    y_corr = Y [, 0]
     if (dist_type == 2):
         n_corr = rowSums (Y)
         is_n_zero = ppred (n_corr, 0.0, "==")
-        y_corr = Y [, 1] / (n_corr + is_n_zero) + (0.5 - Y [, 1]) * is_n_zero
+        y_corr = Y [, 0] / (n_corr + is_n_zero) + (0.5 - Y [, 0]) * is_n_zero
     
     linear_terms = y_corr
     if (dist_type == 1 & link_type == 1): # POWER DISTRIBUTION
@@ -761,7 +761,7 @@ def glm_initialize(X: matrix[float], Y: matrix[float], dist_type: int, var_power
     
     if (desired_eta != 0.0):
         if (icept_status == 1 | icept_status == 2):
-            beta [nrow(beta), 1] = desired_eta
+            beta [nrow(beta)-1, 0] = desired_eta
         else:
             # We want: avg (dot(dot(X, ssX_transform), beta)) = desired_eta
             # Note that "ssX_transform" is trivial here, hence ignored
@@ -792,19 +792,19 @@ def glm_dist(linear_terms: matrix[float], Y: matrix[float], dist_type: int, var_
     one_over_sqrt_two_pi = 0.39894228040143267793994605993438
     ones_2 = full (1.0, rows = 1, cols = 2)
     p_one_m_one = ones_2
-    p_one_m_one [1, 2] = -1.0
+    p_one_m_one [0, 1] = -1.0
     m_one_p_one = ones_2
-    m_one_p_one [1, 1] = -1.0
+    m_one_p_one [0, 0] = -1.0
     zero_one = ones_2
-    zero_one [1, 1] = 0.0
+    zero_one [0, 0] = 0.0
     one_zero = ones_2
-    one_zero [1, 2] = 0.0
+    one_zero [0, 1] = 0.0
     flip_pos = full (0, rows = 2, cols = 2)
     flip_neg = flip_pos
-    flip_pos [1, 2] = 1
-    flip_pos [2, 1] = 1
-    flip_neg [1, 2] = -1
-    flip_neg [2, 1] = 1
+    flip_pos [0, 1] = 1
+    flip_pos [1, 0] = 1
+    flip_neg [0, 1] = -1
+    flip_neg [1, 0] = 1
     
     if (dist_type == 1 & link_type == 1): # POWER DISTRIBUTION
         y_mean = zeros_r
@@ -831,7 +831,7 @@ def glm_dist(linear_terms: matrix[float], Y: matrix[float], dist_type: int, var_
         if (link_type == 1): # BINOMIAL.POWER LINKS
             if (link_power == 0.0): # Binomial.log
                 vec1 = 1 / (exp (- linear_terms) - 1)
-                g_Y = Y [, 1] - Y [, 2] * vec1
+                g_Y = Y [, 0] - Y [, 1] * vec1
                 w   = rowSums (Y) * vec1
             else: # Binomial.nonlog
                 vec1 = zeros_r
@@ -846,9 +846,9 @@ def glm_dist(linear_terms: matrix[float], Y: matrix[float], dist_type: int, var_
                 
                 # We want a "zero-protected" version of
                 # vec2 = Y [, 1] / linear_terms
-                is_y_0 = ppred (Y [, 1], 0.0, "==")
-                vec2 = (Y [, 1] + is_y_0) / (linear_terms * (1 - is_y_0) + is_y_0) - is_y_0
-                g_Y =  (vec2 - Y [, 2] * vec1 * linear_terms) / link_power
+                is_y_0 = ppred (Y [, 0], 0.0, "==")
+                vec2 = (Y [, 0] + is_y_0) / (linear_terms * (1 - is_y_0) + is_y_0) - is_y_0
+                g_Y =  (vec2 - Y [, 1] * vec1 * linear_terms) / link_power
                 w   =  rowSums (Y) * vec1 / link_power ** 2
             
         else:
@@ -874,7 +874,7 @@ def glm_dist(linear_terms: matrix[float], Y: matrix[float], dist_type: int, var_
                         + t_gp *   1.061405429))))
                     the_gauss_exp = exp (- (linear_terms ** 2) / 2.0)
                     vec1 = 0.25 * pt_gp * (2 - the_gauss_exp * pt_gp)
-                    vec2 = Y [, 1] - rowSums (Y) * is_lt_pos + the_gauss_exp * pt_gp * rowSums (Y) * (is_lt_pos - 0.5)
+                    vec2 = Y [, 0] - rowSums (Y) * is_lt_pos + the_gauss_exp * pt_gp * rowSums (Y) * (is_lt_pos - 0.5)
                     w   = the_gauss_exp * (one_over_sqrt_two_pi ** 2) * rowSums (Y) / vec1
                     g_Y = one_over_sqrt_two_pi * vec2 / vec1
                 else:
@@ -883,14 +883,14 @@ def glm_dist(linear_terms: matrix[float], Y: matrix[float], dist_type: int, var_
                         the_exp_exp = exp (- the_exp)
                         is_too_small = ppred (10000000 + the_exp, 10000000, "==")
                         the_exp_ratio = (1 - is_too_small) * (1 - the_exp_exp) / (the_exp + is_too_small) + is_too_small * (1 - the_exp / 2)
-                        g_Y =  (rowSums (Y) * the_exp_exp - Y [, 2]) / the_exp_ratio
+                        g_Y =  (rowSums (Y) * the_exp_exp - Y [, 1]) / the_exp_ratio
                         w   =  the_exp_exp * the_exp * rowSums (Y) / the_exp_ratio
                     else:
                         if (link_type == 5): # Binomial.cauchit
                             Y_prob = 0.5 + (dot(atan (finite_linear_terms), p_one_m_one)) / 3.1415926535897932384626433832795
                             Y_prob = Y_prob * (dot((1.0 - rowSums (is_LT_infinite)), ones_2)) + is_LT_infinite
-                            y_residual = Y [, 1] * Y_prob [, 2] - Y [, 2] * Y_prob [, 1]
-                            var_function = rowSums (Y) * Y_prob [, 1] * Y_prob [, 2]
+                            y_residual = Y [, 0] * Y_prob [, 1] - Y [, 1] * Y_prob [, 0]
+                            var_function = rowSums (Y) * Y_prob [, 0] * Y_prob [, 1]
                             link_gradient_normalized = (1 + linear_terms ** 2) * 3.1415926535897932384626433832795
                             g_Y =  rowSums (Y) * y_residual / (var_function * link_gradient_normalized)
                             w   = (rowSums (Y) ** 2) / (var_function * link_gradient_normalized ** 2)
@@ -1074,13 +1074,13 @@ def binomial_probability_two_column(linear_terms: matrix[float], link_type: int,
     
     ones_2 = full (1.0, rows = 1, cols = 2)
     p_one_m_one = ones_2
-    p_one_m_one [1, 2] = -1.0
+    p_one_m_one [0, 1] = -1.0
     m_one_p_one = ones_2
-    m_one_p_one [1, 1] = -1.0
+    m_one_p_one [0, 0] = -1.0
     zero_one = ones_2
-    zero_one [1, 1] = 0.0
+    zero_one [0, 0] = 0.0
     one_zero = ones_2
-    one_zero [1, 2] = 0.0
+    one_zero [0, 1] = 0.0
     
     zeros_r = full (0.0, rows = num_records, cols = 1)
     ones_r = 1.0 + zeros_r
@@ -1124,8 +1124,8 @@ def binomial_probability_two_column(linear_terms: matrix[float], link_type: int,
                     the_exp = exp (finite_linear_terms)
                     the_exp_exp = exp (- the_exp)
                     is_too_small = ppred (10000000 + the_exp, 10000000, "==")
-                    Y_prob [, 1] = (1 - is_too_small) * (1 - the_exp_exp) + is_too_small * the_exp * (1 - the_exp / 2)
-                    Y_prob [, 2] = the_exp_exp
+                    Y_prob [, 0] = (1 - is_too_small) * (1 - the_exp_exp) + is_too_small * the_exp * (1 - the_exp / 2)
+                    Y_prob [, 1] = the_exp_exp
                 else:
                     if (link_type == 5): # Binomial.cauchit
                         Y_prob = 0.5 + (dot(atan (finite_linear_terms), p_one_m_one)) / 3.1415926535897932384626433832795
@@ -1165,9 +1165,9 @@ def get_CG_Steihaug_point(X: matrix[float], scale_X: matrix[float], shift_X: mat
     while (converged_CG == 0):
         i_CG = i_CG + 1
         ssX_p_CG = dot(diag (scale_X), p_CG)
-        ssX_p_CG [size_CG, ] = ssX_p_CG [size_CG, ] + dot(transpose(shift_X), p_CG)
+        ssX_p_CG [size_CG-1, ] = ssX_p_CG [size_CG-1, ] + dot(transpose(shift_X), p_CG)
         temp_CG = dot(transpose(X), (w * (dot(X, ssX_p_CG))))
-        q_CG = (lambda * p_CG) + dot(diag (scale_X), temp_CG) + dot(shift_X, temp_CG [size_CG, ])
+        q_CG = (lambda * p_CG) + dot(diag (scale_X), temp_CG) + dot(shift_X, temp_CG [size_CG-1, ])
         pq_CG = sum (p_CG * q_CG)
         if (pq_CG <= 0):
             pp_CG = sum (p_CG * p_CG)  

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/id3/id3.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/id3/id3.pydml b/src/test/scripts/applications/id3/id3.pydml
index 17a10e4..02b80b5 100644
--- a/src/test/scripts/applications/id3/id3.pydml
+++ b/src/test/scripts/applications/id3/id3.pydml
@@ -118,8 +118,8 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
     #if no attributes remain then return a leaf node with the most popular label    
     if(num_non_zero_labels == 1 | num_remaining_attrs == 0 | num_samples < minsplit):
         nodes = full(0, rows=1, cols=2)
-        nodes[1,1] = -1
-        nodes[1,2] = most_popular_label
+        nodes[0,0] = -1
+        nodes[0,1] = most_popular_label
         edges = full(-1, rows=1, cols=1)
     else:
         #computing gains for all available attributes using parfor
@@ -135,16 +135,16 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
         sz = nrow(attributes)
         gains = full(0, rows=sz, cols=1)
         for(i in 1:nrow(attributes)):
-            if(scalar(attributes[i,1]) == 1):
-                attr_vals = X[,i]
+            if(scalar(attributes[i-1,0]) == 1):
+                attr_vals = X[,i-1]
                 attr_domain = aggregate(target=X_subset, groups=attr_vals, fn="sum")
                 
                 hxt_vector = full(0, rows=nrow(attr_domain), cols=1)
                 
                 for(j in 1:nrow(attr_domain), check=0):
-                    if(scalar(attr_domain[j,1]) != 0):
+                    if(scalar(attr_domain[j-1,0]) != 0):
                         val = j
-                        Tj = X_subset * ppred(X[,i], val, "==")
+                        Tj = X_subset * ppred(X[,i-1], val, "==")
                         
                         #entropy = compute_entropy(Tj, y)
                         hist_labels1 = aggregate(target=Tj, groups=y, fn="sum")
@@ -155,17 +155,17 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
                         entropy_vector = -pi*log(log_term)
                         entropy = sum(entropy_vector)
                         
-                        hxt_vector[j,1] = sum(Tj)/sum(X_subset)*entropy
+                        hxt_vector[j-1,0] = sum(Tj)/sum(X_subset)*entropy
                 hxt = sum(hxt_vector)
-                gains[i,1] = (ht - hxt)
+                gains[i-1,0] = (ht - hxt)
         
         #pick out attr with highest gain
         best_attr = -1
         max_gain = 0
         for(i4 in 1:nrow(gains)):
             #print("best attr " + best_attr + " max gain " + max_gain)
-            if(scalar(attributes[i4,1]) == 1):
-                g = scalar(gains[i4,1])
+            if(scalar(attributes[i4-1,0]) == 1):
+                g = scalar(gains[i4-1,0])
                 if(best_attr == -1 | max_gain <= g):
                     max_gain = g
                     best_attr = i4
@@ -173,13 +173,13 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
         print("best attribute is: " + best_attr)
         print("gain is: " + max_gain)
         
-        attr_vals = X[,best_attr]
+        attr_vals = X[,best_attr-1]
         attr_domain = aggregate(target=X_subset, groups=attr_vals, fn="sum")
         if (1==1):
             print(" ")
         
         new_attributes = attributes
-        new_attributes[best_attr, 1] = 0
+        new_attributes[best_attr-1, 0] = 0
         
         max_sz_subtree = 2*sum(X_subset)
         sz1 = 0
@@ -194,20 +194,20 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
         
         for(i1 in 1:nrow(attr_domain), check=0):
             
-            Ti = X_subset * ppred(X[,best_attr], i1, "==")
+            Ti = X_subset * ppred(X[,best_attr-1], i1, "==")
             num_nodes_Ti = sum(Ti)
             
             if(num_nodes_Ti > 0):
                 [nodesi, edgesi] = id3_learn(X, y, Ti, new_attributes, minsplit)
             
                 start_pt = 1+(i1-1)*max_sz_subtree
-                tempNodeStore[,start_pt:(start_pt+nrow(nodesi)-1)] = t(nodesi)
-                numSubtreeNodes[i1,1] = nrow(nodesi)
-                if(nrow(edgesi)!=1 | ncol(edgesi)!=1 | scalar(edgesi[1,1])!=-1):
-                    tempEdgeStore[,start_pt:(start_pt+nrow(edgesi)-1)] = t(edgesi)
-                    numSubtreeEdges[i1,1] = nrow(edgesi)
+                tempNodeStore[,start_pt-1:(start_pt+nrow(nodesi)-1)] = t(nodesi)
+                numSubtreeNodes[i1-1,0] = nrow(nodesi)
+                if(nrow(edgesi)!=1 | ncol(edgesi)!=1 | scalar(edgesi[0,0])!=-1):
+                    tempEdgeStore[,start_pt-1:(start_pt+nrow(edgesi)-1)] = t(edgesi)
+                    numSubtreeEdges[i1-1,0] = nrow(edgesi)
                 else:
-                    numSubtreeEdges[i1,1] = 0
+                    numSubtreeEdges[i1-1,0] = 0
         
         num_nodes_in_subtrees = sum(numSubtreeNodes)
         num_edges_in_subtrees = sum(numSubtreeEdges)
@@ -217,7 +217,7 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
             sz = 1+num_nodes_in_subtrees
         
         nodes = full(0, rows=sz, cols=2)
-        nodes[1,1] = best_attr
+        nodes[0,0] = best_attr
         numNodes = 1
         
         #edges from root to children
@@ -227,29 +227,29 @@ def id3_learn(X:matrix[float], y:matrix[float], X_subset:matrix[float], attribut
         edges = full(1, rows=sz, cols=3)
         numEdges = 0
         for(i6 in 1:nrow(attr_domain)):
-            num_nodesi = scalar(numSubtreeNodes[i6,1])
+            num_nodesi = scalar(numSubtreeNodes[i6-1,0])
             if(num_nodesi > 0):
-                edges[numEdges+1,2] = i6
+                edges[numEdges,1] = i6
                 numEdges = numEdges + 1
         
         nonEmptyAttri = 0
         for(i7 in 1:nrow(attr_domain)):
-            numNodesInSubtree = scalar(numSubtreeNodes[i7,1])
+            numNodesInSubtree = scalar(numSubtreeNodes[i7-1,0])
             
             if(numNodesInSubtree > 0):
                 start_pt1 = 1 + (i7-1)*max_sz_subtree
-                nodes[numNodes+1:numNodes+numNodesInSubtree,] = transpose(tempNodeStore[,start_pt1:(start_pt1+numNodesInSubtree-1)])
+                nodes[numNodes:numNodes+numNodesInSubtree,] = transpose(tempNodeStore[,start_pt1-1:(start_pt1+numNodesInSubtree-1)])
                 
-                numEdgesInSubtree = scalar(numSubtreeEdges[i7,1])
+                numEdgesInSubtree = scalar(numSubtreeEdges[i7-1,0])
                 
                 if(numEdgesInSubtree!=0):
-                    edgesi1 = transpose(tempEdgeStore[,start_pt1:(start_pt1+numEdgesInSubtree-1)])
-                    edgesi1[,1] = edgesi1[,1] + numNodes
-                    edgesi1[,3] = edgesi1[,3] + numNodes
-                    edges[numEdges+1:numEdges+numEdgesInSubtree,] = edgesi1
+                    edgesi1 = transpose(tempEdgeStore[,start_pt1-1:(start_pt1+numEdgesInSubtree-1)])
+                    edgesi1[,0] = edgesi1[,0] + numNodes
+                    edgesi1[,2] = edgesi1[,2] + numNodes
+                    edges[numEdges:numEdges+numEdgesInSubtree,] = edgesi1
                     numEdges = numEdges + numEdgesInSubtree
                 
-                edges[nonEmptyAttri+1,3] = numNodes + 1
+                edges[nonEmptyAttri,2] = numNodes + 1
                 nonEmptyAttri = nonEmptyAttri + 1
                 
                 numNodes = numNodes + numNodesInSubtree
@@ -276,13 +276,13 @@ y = y + labelCorrection + 0
 [nodes, edges] = id3_learn(X, y, X_subset, attributes, minsplit)
 
 # decoding outputs
-nodes[,2] = nodes[,2] - labelCorrection * ppred(nodes[,1], -1, "==")
+nodes[,1] = nodes[,1] - labelCorrection * ppred(nodes[,0], -1, "==")
 for(i3 in 1:nrow(edges)):
 #parfor(i3 in 1:nrow(edges)):
-    e_parent = scalar(edges[i3,1])
-    parent_feature = scalar(nodes[e_parent,1])
-    correction = scalar(featureCorrections[1,parent_feature])
-    edges[i3,2] = edges[i3,2] - correction
+    e_parent = scalar(edges[i3-1,0])
+    parent_feature = scalar(nodes[e_parent-1,0])
+    correction = scalar(featureCorrections[0,parent_feature-1])
+    edges[i3-1,1] = edges[i3-1,1] - correction
 
 save(nodes, $3, format="text")
 save(edges, $4, format="text")

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/m-svm/m-svm.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/m-svm/m-svm.pydml b/src/test/scripts/applications/m-svm/m-svm.pydml
index fc3669c..5ae350f 100644
--- a/src/test/scripts/applications/m-svm/m-svm.pydml
+++ b/src/test/scripts/applications/m-svm/m-svm.pydml
@@ -108,7 +108,7 @@ else:
             
             train_acc = sum(ppred(Y_local*(dot(X, w_class)), 0, ">="))/num_samples*100
             print("For class " + iter_class + " iteration " + iter + " training accuracy: " + train_acc)
-            debug_mat[iter+1,iter_class] = obj
+            debug_mat[iter,iter_class-1] = obj
             
             if((step_sz*tmp < epsilon*obj) | (iter >= max_iterations-1)):
                 continue = 0
@@ -121,7 +121,7 @@ else:
             iter = iter + 1
         # end while(continue == 1)
         
-        w[,iter_class] = w_class
+        w[,iter_class-1] = w_class
     # end parfor(iter_class in 1:num_classes)
     
     save(w, $model, format=cmdLine_fmt)
@@ -129,7 +129,7 @@ else:
     debug_str = "# Class, Iter, Obj"
     for(iter_class in 1:ncol(debug_mat)):
         for(iter in 1:nrow(debug_mat)):
-            obj = scalar(debug_mat[iter, iter_class])
+            obj = scalar(debug_mat[iter-1, iter_class-1])
             if(obj != -1):
                 debug_str = append(debug_str, iter_class + "," + iter + "," + obj)
     save(debug_str, $Log)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/mdabivar/MDABivariateStats.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/mdabivar/MDABivariateStats.pydml b/src/test/scripts/applications/mdabivar/MDABivariateStats.pydml
index 134eb4b..fd3cda6 100644
--- a/src/test/scripts/applications/mdabivar/MDABivariateStats.pydml
+++ b/src/test/scripts/applications/mdabivar/MDABivariateStats.pydml
@@ -47,14 +47,14 @@ covariances = full(0, rows=sz, cols=1)
 # store for standard deviations used to compute pearson's r
 standard_deviations = full(0, rows=sz, cols=1)
 
-labels = D[,label_index]
+labels = D[,label_index-1]
 
 labelCorrection = 0
 if(label_measurement_level == 1):
     numLabels = nrow(labels)
     cmLabels = moment(labels,2)
     stdLabels = sqrt(cmLabels * (numLabels/(numLabels-1.0)) )
-    standard_deviations[label_index,1] = stdLabels
+    standard_deviations[label_index-1,0] = stdLabels
 else:
     labelCorrection = 1 - min(labels)
     labels = labels + labelCorrection
@@ -64,9 +64,9 @@ mn = colMins(D)
 num_distinct_values = mx-mn+1
 max_num_distinct_values = 0
 for(i1 in 1:nrow(feature_indices)):
-    feature_index1 = scalar(feature_indices[i1,1])
-    num = scalar(num_distinct_values[1,feature_index1])
-    if(scalar(feature_measurement_levels[i1,1]) == 0 & num >= max_num_distinct_values):
+    feature_index1 = scalar(feature_indices[i1-1,0])
+    num = scalar(num_distinct_values[0,feature_index1-1])
+    if(scalar(feature_measurement_levels[i1-1,0]) == 0 & num >= max_num_distinct_values):
         max_num_distinct_values = num
 distinct_label_values = full(0, rows=1, cols=1)
 contingencyTableSz = 1
@@ -94,15 +94,15 @@ featureMeans = full(0, rows=sz, cols=maxNumberOfGroups)
 featureSTDs = full(0, rows=sz, cols=maxNumberOfGroups)
 
 if(label_measurement_level == 0):
-    featureCounts[label_index,1:nrow(distinct_label_values)] = transpose(distinct_label_values)
-    parfor(i2 in 1:nrow(distinct_label_values)):
-        featureValues[label_index,i2] = i2-labelCorrection
+    featureCounts[label_index-1,0:nrow(distinct_label_values)] = transpose(distinct_label_values)
+    parfor(i2 in 1:nrow(distinct_label_values), check=0):
+        featureValues[label_index-1,i2-1] = i2-labelCorrection
 
 parfor(i3 in 1:nrow(feature_indices), check=0):
-    feature_index2 = scalar(feature_indices[i3,1])
-    feature_measurement_level = scalar(feature_measurement_levels[i3,1])
+    feature_index2 = scalar(feature_indices[i3-1,0])
+    feature_measurement_level = scalar(feature_measurement_levels[i3-1,0])
     
-    feature = D[,feature_index2]
+    feature = D[,feature_index2-1]
     
     if(feature_measurement_level == 0):
         featureCorrection = 1 - min(feature)
@@ -110,9 +110,9 @@ parfor(i3 in 1:nrow(feature_indices), check=0):
         
         if(label_measurement_level == feature_measurement_level):
             # categorical-categorical
-            tests[feature_index2,1] = 1
+            tests[feature_index2-1,0] = 1
             [pVal, contingencyTable, rowMarginals, colMarginals] = bivar_cc(labels, feature)
-            stats[feature_index2,1] = pVal
+            stats[feature_index2-1,0] = pVal
             
             sz3=1
             if(1==1):
@@ -122,42 +122,42 @@ parfor(i3 in 1:nrow(feature_indices), check=0):
             
             parfor(i4 in 1:nrow(contingencyTable), check=0):
                 parfor(j in 1:ncol(contingencyTable), check=0):
-                    contingencyTableLabelValues[1, ncol(contingencyTable)*(i4-1)+j] = i4-labelCorrection
-                    contingencyTableFeatureValues[1, ncol(contingencyTable)*(i4-1)+j] = j-featureCorrection 
+                    contingencyTableLabelValues[0, ncol(contingencyTable)*(i4-1)+j-1] = i4-labelCorrection
+                    contingencyTableFeatureValues[0, ncol(contingencyTable)*(i4-1)+j-1] = j-featureCorrection
             contingencyTableCounts = contingencyTable.reshape(rows=1, cols=sz3)
-            contingencyTablesCounts[feature_index2,1:sz3] = contingencyTableCounts
+            contingencyTablesCounts[feature_index2-1,0:sz3] = contingencyTableCounts
             
-            contingencyTablesLabelValues[feature_index2,1:sz3] = contingencyTableLabelValues
-            contingencyTablesFeatureValues[feature_index2,1:sz3] = contingencyTableFeatureValues
+            contingencyTablesLabelValues[feature_index2-1,0:sz3] = contingencyTableLabelValues
+            contingencyTablesFeatureValues[feature_index2-1,0:sz3] = contingencyTableFeatureValues
             
-            featureCounts[feature_index2,1:ncol(colMarginals)] = colMarginals
+            featureCounts[feature_index2-1,0:ncol(colMarginals)] = colMarginals
             parfor(i5 in 1:ncol(colMarginals), check=0):
-                featureValues[feature_index2,i5] = i5-featureCorrection
+                featureValues[feature_index2-1,i5-1] = i5-featureCorrection
         else:
             # label is scale, feature is categorical
-            tests[feature_index2,1] = 2
+            tests[feature_index2-1,0] = 2
             [pVal, frequencies, means, variances] = bivar_sc(labels, feature)
-            stats[feature_index2,1] = pVal
-            featureCounts[feature_index2,1:nrow(frequencies)] = transpose(frequencies)
+            stats[feature_index2-1,0] = pVal
+            featureCounts[feature_index2-1,0:nrow(frequencies)] = transpose(frequencies)
             parfor(i6 in 1:nrow(frequencies), check=0):
-                featureValues[feature_index2,i6] = i6 - featureCorrection
-            featureMeans[feature_index2,1:nrow(means)] = transpose(means)
-            featureSTDs[feature_index2,1:nrow(variances)] = transpose(sqrt(variances))
+                featureValues[feature_index2-1,i6-1] = i6 - featureCorrection
+            featureMeans[feature_index2-1,0:nrow(means)] = transpose(means)
+            featureSTDs[feature_index2-1,0:nrow(variances)] = transpose(sqrt(variances))
     else:
         if(label_measurement_level == feature_measurement_level):
             # scale-scale
-            tests[feature_index2,1] = 3
+            tests[feature_index2-1,0] = 3
             [r, covariance, stdX, stdY] = bivar_ss(labels, feature)
-            stats[feature_index2,1] = r
-            covariances[feature_index2,1] = covariance
-            standard_deviations[feature_index2,1] = stdY
+            stats[feature_index2-1,0] = r
+            covariances[feature_index2-1,0] = covariance
+            standard_deviations[feature_index2-1,0] = stdY
         else:
             # label is categorical, feature is scale
-            tests[feature_index2,1] = 2
+            tests[feature_index2-1,0] = 2
             [pVal, frequencies, means, variances] = bivar_sc(feature, labels)
-            stats[feature_index2,1] = pVal
-            featureMeans[feature_index2,1:nrow(means)] = transpose(means)
-            featureSTDs[feature_index2,1:nrow(variances)] = transpose(sqrt(variances))
+            stats[feature_index2-1,0] = pVal
+            featureMeans[feature_index2-1,0:nrow(means)] = transpose(means)
+            featureSTDs[feature_index2-1,0:nrow(variances)] = transpose(sqrt(variances))
     # end if(feature_measurement_level == 0)
 # end parfor(i3 in 1:nrow(feature_indices), check=0)
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
index 462b330..45e4700 100644
--- a/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
+++ b/src/test/scripts/applications/naive-bayes-parfor/naive-bayes.pydml
@@ -43,8 +43,8 @@ numFeatures = ncol(D)
 # Compute the feature counts for each class
 classFeatureCounts = full(0, rows=numClasses, cols=numFeatures)
 parfor (i in 1:numFeatures):
-    Col = D[,i]
-    classFeatureCounts[,i] = aggregate(target=Col, groups=C, fn="sum", ngroups=numClasses)
+    Col = D[,i-1]
+    classFeatureCounts[,i-1] = aggregate(target=Col, groups=C, fn="sum", ngroups=numClasses)
 
 # Compute the total feature count for each class 
 # and add the number of features to this sum

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/functions/misc/FunctionsB.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/FunctionsB.pydml b/src/test/scripts/functions/misc/FunctionsB.pydml
index ed1d963..91c3d51 100644
--- a/src/test/scripts/functions/misc/FunctionsB.pydml
+++ b/src/test/scripts/functions/misc/FunctionsB.pydml
@@ -19,8 +19,8 @@
 #
 #-------------------------------------------------------------
 def matrixPrint(X: matrix[float]) -> ():
-    for (i in 1:nrow(X)):
-        for (j in 1:ncol(X)) :
+    for (i in 0:nrow(X)-1):
+        for (j in 0:ncol(X)-1) :
             xij = scalar(X[i,j])
             print("[" + i + "," + j + "] " + xij)
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/functions/misc/FunctionsH.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/FunctionsH.pydml b/src/test/scripts/functions/misc/FunctionsH.pydml
index bed66f9..acdbdd8 100644
--- a/src/test/scripts/functions/misc/FunctionsH.pydml
+++ b/src/test/scripts/functions/misc/FunctionsH.pydml
@@ -25,8 +25,8 @@ source("./src/test/scripts/functions/misc/FunctionsG.pydml") as Utils
 source("./src/test/scripts/functions/misc/FunctionsG.pydml") as FunctionsG
 
 def matrixPrint(X: matrix[float]) -> ():
-    for (i in 1:nrow(X)):
-        for (j in 1:ncol(X)):
+    for (i in 0:nrow(X)-1):
+        for (j in 0:ncol(X)-1):
             xij = scalar(X[i,j])
             print("[" + i + "," + j + "] " + xij)
 

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/59c1e554/src/test/scripts/functions/misc/FunctionsK1.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/misc/FunctionsK1.pydml b/src/test/scripts/functions/misc/FunctionsK1.pydml
index 3022839..b6d7d9e 100644
--- a/src/test/scripts/functions/misc/FunctionsK1.pydml
+++ b/src/test/scripts/functions/misc/FunctionsK1.pydml
@@ -21,8 +21,8 @@
 setwd("./src/test/scripts/functions")
 source("misc/FunctionsK2.pydml") as Functions
 def matrixPrint(X: matrix[float]) -> ():
-    for (i in 1:nrow(X)):
-        for (j in 1:ncol(X)) :
+    for (i in 0:nrow(X)-1):
+        for (j in 0:ncol(X)-1) :
             xij = scalar(X[i,j])
             print("[" + i + "," + j + "] " + xij)