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/16 18:28:55 UTC

incubator-systemml git commit: [SYSTEMML-676] Improve PyDML Slicing

Repository: incubator-systemml
Updated Branches:
  refs/heads/master 34845bb68 -> 18bb8c52c


[SYSTEMML-676] Improve PyDML Slicing

Currently, slicing correctly uses inclusive lower and exclusive upper bounds. However, a really useful piece of slicing syntax we are currently missing from PyDML is the ability to have implicit lower or upper bounds, i.e., `X[:3,]` should return all rows up to (but not including) 3. This means that the implicit lower bound is 0. Similarly, `X[2:,]` should return all rows starting at row 2 (0-based), which implies that the upper bound is equal to the number of rows of `X`.

This change updates the grammar to allow these cases, and more generally, `X[:,:]`, with both row and column implict slicing as in NumPy.

Closes #148.


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

Branch: refs/heads/master
Commit: 18bb8c52cc10abd9679f5de3b6107a19ec79dc21
Parents: 34845bb
Author: Mike Dusenberry <mw...@us.ibm.com>
Authored: Mon May 16 11:28:56 2016 -0700
Committer: Mike Dusenberry <mw...@us.ibm.com>
Committed: Mon May 16 11:28:56 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/sysml/parser/pydml/Pydml.g4 |   4 +-
 .../parser/pydml/PydmlSyntacticValidator.java   |  52 ++++++-
 .../indexing/PyDMLImplicitSlicingBounds.java    | 149 +++++++++++++++++++
 .../LeftImplicitColLowerImplicitRowUpper.pydml  |  27 ++++
 ...ImplicitRowColLowerImplicitRowColUpper.pydml |  26 ++++
 .../LeftImplicitRowLowerImplicitColUpper.pydml  |  27 ++++
 .../LeftImplicitRowLowerImplicitRowUpper.pydml  |  26 ++++
 ...tImplicitRowLowerImplicitRowUpperComma.pydml |  26 ++++
 .../RightImplicitColLowerImplicitRowUpper.pydml |  25 ++++
 ...ImplicitRowColLowerImplicitRowColUpper.pydml |  25 ++++
 .../RightImplicitRowLowerImplicitColUpper.pydml |  25 ++++
 .../RightImplicitRowLowerImplicitRowUpper.pydml |  25 ++++
 ...tImplicitRowLowerImplicitRowUpperComma.pydml |  25 ++++
 13 files changed, 459 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/main/java/org/apache/sysml/parser/pydml/Pydml.g4
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/sysml/parser/pydml/Pydml.g4 b/src/main/java/org/apache/sysml/parser/pydml/Pydml.g4
index fda5c06..5fb4951 100644
--- a/src/main/java/org/apache/sysml/parser/pydml/Pydml.g4
+++ b/src/main/java/org/apache/sysml/parser/pydml/Pydml.g4
@@ -220,8 +220,8 @@ dataIdentifier returns [ org.apache.sysml.parser.common.ExpressionInfo dataInfo
        // $dataInfo.expr = new org.apache.sysml.parser.DataIdentifier();
 } :
     // ------------------------------------------
-    // IndexedIdentifier
-    name=ID OPEN_BRACK (rowLower=expression (':' rowUpper=expression)?)? ',' (colLower=expression (':' colUpper=expression)?)? CLOSE_BRACK # IndexedExpression
+    // IndexedIdentifier -- allows implicit lower and upper bounds
+    name=ID OPEN_BRACK ( (rowLower=expression)? (rowImplicitSlice=':' (rowUpper=expression)?)? )? (',' (((colLower=expression)? (colImplicitSlice=':' (colUpper=expression)?)?)?)?)? CLOSE_BRACK # IndexedExpression
     // ------------------------------------------
     | ID                                            # SimpleDataIdentifierExpression
     | COMMANDLINE_NAMED_ID                          # CommandlineParamExpression

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/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 b070314..7f73507 100644
--- a/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
+++ b/src/main/java/org/apache/sysml/parser/pydml/PydmlSyntacticValidator.java
@@ -36,6 +36,7 @@ import org.apache.sysml.conf.CompilerConfig.ConfigType;
 import org.apache.sysml.conf.ConfigurationManager;
 import org.apache.sysml.parser.AssignmentStatement;
 import org.apache.sysml.parser.BinaryExpression;
+import org.apache.sysml.parser.BuiltinFunctionExpression;
 import org.apache.sysml.parser.ConditionalPredicate;
 import org.apache.sysml.parser.DMLProgram;
 import org.apache.sysml.parser.DataIdentifier;
@@ -325,6 +326,8 @@ public class PydmlSyntacticValidator extends CommonSyntacticValidator implements
 		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));
+		boolean isRowSliceImplicit = ctx.rowImplicitSlice != null;
+		boolean isColSliceImplicit = ctx.colImplicitSlice != null;
         ExpressionInfo rowLower = isRowLower ? ctx.rowLower.info : null;
         ExpressionInfo rowUpper = isRowUpper ? ctx.rowUpper.info : null;
         ExpressionInfo colLower = isColLower ? ctx.colLower.info : null;
@@ -350,8 +353,31 @@ public class PydmlSyntacticValidator extends CommonSyntacticValidator implements
 				rowIndices.add(rowUpper.expr);
 			}
 			else if(isRowLower && !isRowUpper) {
-				// only row set
+				// Add given lower bound
 				rowIndices.add(incrementByOne(rowLower.expr, ctx));
+				if(isRowSliceImplicit) {
+					// Add expression for nrow(X) for implicit upper bound
+					Expression.BuiltinFunctionOp bop = Expression.BuiltinFunctionOp.NROW;
+					DataIdentifier x = new DataIdentifier(ctx.name.getText());
+					int line = ctx.start.getLine();
+					int col = ctx.start.getCharPositionInLine();
+					Expression expr = new BuiltinFunctionExpression(bop, new Expression[]{x},
+							currentFile, line, col, line, col);
+					setFileLineColumn(expr, ctx);
+					rowIndices.add(expr);
+				}
+			}
+			else if(!isRowLower && isRowUpper && isRowSliceImplicit) {
+				// Add expression for `1` for implicit lower bound
+				// Note: We go ahead and increment by 1 to convert from 0-based to 1-based indexing
+				int line = ctx.start.getLine();
+				int col = ctx.start.getCharPositionInLine();
+				IntIdentifier one = new IntIdentifier(1, currentFile, line, col, line, col);
+				setFileLineColumn(one, ctx);
+				rowIndices.add(one);
+
+				// Add given upper bound
+				rowIndices.add(rowUpper.expr);
 			}
 			else {
 				notifyErrorListeners("incorrect index expression for row", ctx.start);
@@ -367,7 +393,31 @@ public class PydmlSyntacticValidator extends CommonSyntacticValidator implements
 				colIndices.add(colUpper.expr);
 			}
 			else if(isColLower && !isColUpper) {
+				// Add given lower bound
 				colIndices.add(incrementByOne(colLower.expr, ctx));
+				if(isColSliceImplicit) {
+					// Add expression for ncol(X) for implicit upper bound
+					Expression.BuiltinFunctionOp bop = Expression.BuiltinFunctionOp.NCOL;
+					DataIdentifier x = new DataIdentifier(ctx.name.getText());
+					int line = ctx.start.getLine();
+					int col = ctx.start.getCharPositionInLine();
+					Expression expr = new BuiltinFunctionExpression(bop, new Expression[]{x},
+							currentFile, line, col, line, col);
+					setFileLineColumn(expr, ctx);
+					colIndices.add(expr);
+				}
+			}
+			else if(!isColLower && isColUpper && isColSliceImplicit) {
+				// Add expression for `1` for implicit lower bound
+				// Note: We go ahead and increment by 1 to convert from 0-based to 1-based indexing
+				int line = ctx.start.getLine();
+				int col = ctx.start.getCharPositionInLine();
+				IntIdentifier one = new IntIdentifier(1, currentFile, line, col, line, col);
+				setFileLineColumn(one, ctx);
+				colIndices.add(one);
+
+				// Add given upper bound
+				colIndices.add(colUpper.expr);
 			}
 			else {
 				notifyErrorListeners("incorrect index expression for column", ctx.start);

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/java/org/apache/sysml/test/integration/functions/indexing/PyDMLImplicitSlicingBounds.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/sysml/test/integration/functions/indexing/PyDMLImplicitSlicingBounds.java b/src/test/java/org/apache/sysml/test/integration/functions/indexing/PyDMLImplicitSlicingBounds.java
new file mode 100644
index 0000000..4dd8a35
--- /dev/null
+++ b/src/test/java/org/apache/sysml/test/integration/functions/indexing/PyDMLImplicitSlicingBounds.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysml.test.integration.functions.indexing;
+
+import org.apache.sysml.runtime.matrix.data.MatrixValue.CellIndex;
+import org.apache.sysml.test.integration.AutomatedTestBase;
+import org.apache.sysml.test.integration.TestConfiguration;
+import org.apache.sysml.test.utils.TestUtils;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+/**
+ * Test the PyDML implicit slicing.
+ */
+public class PyDMLImplicitSlicingBounds extends AutomatedTestBase {
+
+    private static final String TEST_NAME1 = "RightImplicitRowLowerImplicitRowUpper";
+    private static final String TEST_NAME2 = "RightImplicitRowLowerImplicitRowUpperComma";
+    private static final String TEST_NAME3 = "RightImplicitRowColLowerImplicitRowColUpper";
+    private static final String TEST_NAME4 = "RightImplicitRowLowerImplicitColUpper";
+    private static final String TEST_NAME5 = "RightImplicitColLowerImplicitRowUpper";
+    private static final String TEST_NAME6 = "LeftImplicitRowLowerImplicitRowUpper";
+    private static final String TEST_NAME7 = "LeftImplicitRowLowerImplicitRowUpperComma";
+    private static final String TEST_NAME8 = "LeftImplicitRowColLowerImplicitRowColUpper";
+    private static final String TEST_NAME9 = "LeftImplicitRowLowerImplicitColUpper";
+    private static final String TEST_NAME10 = "LeftImplicitColLowerImplicitRowUpper";
+    private static final String TEST_DIR = "functions/indexing/";
+    private static final String TEST_CLASS_DIR =
+            TEST_DIR + PyDMLImplicitSlicingBounds.class.getSimpleName() + "/";
+    private static final String INPUT_NAME = "X";
+    private static final String OUTPUT_NAME_IMPLICIT = "X_implicit";
+    private static final String OUTPUT_NAME_EXPLICIT = "X_explicit";
+
+    private static final int rows = 123;
+    private static final int cols = 143;
+    private static final double sparsity = 0.7;
+    private static final double eps = Math.pow(10, -10);
+
+    @Override
+    public void setUp() {
+        TestUtils.clearAssertionInformation();
+        addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1));
+        addTestConfiguration(TEST_NAME2, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME2));
+        addTestConfiguration(TEST_NAME3, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME3));
+        addTestConfiguration(TEST_NAME4, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME4));
+        addTestConfiguration(TEST_NAME5, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME5));
+        addTestConfiguration(TEST_NAME6, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME6));
+        addTestConfiguration(TEST_NAME7, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME7));
+        addTestConfiguration(TEST_NAME8, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME8));
+        addTestConfiguration(TEST_NAME9, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME9));
+        addTestConfiguration(TEST_NAME10, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME10));
+    }
+
+    // Right indexing
+    @Test
+    public void testRightImplicitRowLowerImplicitRowUpper() {
+       testPyDMLImplicitSlicingBounds(TEST_NAME1);
+    }
+
+    @Test
+    public void testRightImplicitRowLowerImplicitRowUpperComma() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME2);
+    }
+
+    @Test
+    public void testRightImplicitRowColLowerImplicitRowColUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME3);
+    }
+
+    @Test
+    public void testRightImplicitRowLowerImplicitColUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME4);
+    }
+
+    @Test
+    public void testRightImplicitColLowerImplicitRowUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME5);
+    }
+
+    // Left indexing
+    @Test
+    public void testLeftImplicitRowLowerImplicitRowUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME6);
+    }
+
+    @Test
+    public void testLeftImplicitRowLowerImplicitRowUpperComma() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME7);
+    }
+
+    @Test
+    public void testLeftImplicitRowColLowerImplicitRowColUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME8);
+    }
+
+    @Test
+    public void testLeftImplicitRowLowerImplicitColUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME9);
+    }
+
+    @Test
+    public void testLeftImplicitColLowerImplicitRowUpper() {
+        testPyDMLImplicitSlicingBounds(TEST_NAME10);
+    }
+    /**
+     * Test the implicit bounds slicing in PyDML.
+     *
+     * @param testName The name of this test case.
+     */
+    private void testPyDMLImplicitSlicingBounds(String testName) {
+        // Create and load test configuration
+        getAndLoadTestConfiguration(testName);
+        String HOME = SCRIPT_DIR + TEST_DIR;
+        fullDMLScriptName = HOME + testName + ".pydml";
+        programArgs = new String[]{"-python", "-args",
+                input(INPUT_NAME), output(OUTPUT_NAME_IMPLICIT), output(OUTPUT_NAME_EXPLICIT)};
+
+        // Generate data
+        double[][] X = getRandomMatrix(rows, cols, -1, 1, sparsity, 7);
+        writeInputMatrixWithMTD(INPUT_NAME, X, true);
+
+        // Run PyDML script
+        runTest(true, false, null, -1);
+
+        // Compare output matrices
+        HashMap<CellIndex, Double> pydmlImplicit = readDMLMatrixFromHDFS(OUTPUT_NAME_IMPLICIT);
+        HashMap<CellIndex, Double> pydmlExplicit = readDMLMatrixFromHDFS(OUTPUT_NAME_EXPLICIT);
+        TestUtils.compareMatrices(pydmlImplicit, pydmlExplicit, eps, "Implicit", "Explicit");
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/LeftImplicitColLowerImplicitRowUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/LeftImplicitColLowerImplicitRowUpper.pydml b/src/test/scripts/functions/indexing/LeftImplicitColLowerImplicitRowUpper.pydml
new file mode 100644
index 0000000..7dfcd0d
--- /dev/null
+++ b/src/test/scripts/functions/indexing/LeftImplicitColLowerImplicitRowUpper.pydml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = full(0, X.shape(0), X.shape(1))
+X_explicit = full(0, X.shape(0), X.shape(1))
+X_implicit[2:,:4] = X[2:X.shape(0),0:4]
+X_explicit[2:X.shape(0),0:4] = X[2:X.shape(0),0:4]
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/LeftImplicitRowColLowerImplicitRowColUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/LeftImplicitRowColLowerImplicitRowColUpper.pydml b/src/test/scripts/functions/indexing/LeftImplicitRowColLowerImplicitRowColUpper.pydml
new file mode 100644
index 0000000..3316249
--- /dev/null
+++ b/src/test/scripts/functions/indexing/LeftImplicitRowColLowerImplicitRowColUpper.pydml
@@ -0,0 +1,26 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = full(0, X.shape(0), X.shape(1))
+X_implicit[:,:] = X
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitColUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitColUpper.pydml b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitColUpper.pydml
new file mode 100644
index 0000000..14e951b
--- /dev/null
+++ b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitColUpper.pydml
@@ -0,0 +1,27 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = full(0, X.shape(0), X.shape(1))
+X_explicit = full(0, X.shape(0), X.shape(1))
+X_implicit[:2,1:] = X[0:2,1:X.shape(1)]
+X_explicit[0:2,1:X.shape(1)] = X[0:2,1:X.shape(1)]
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpper.pydml b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpper.pydml
new file mode 100644
index 0000000..8875394
--- /dev/null
+++ b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpper.pydml
@@ -0,0 +1,26 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = full(0, X.shape(0), X.shape(1))
+X_implicit[:] = X
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpperComma.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpperComma.pydml b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpperComma.pydml
new file mode 100644
index 0000000..0cbbaf1
--- /dev/null
+++ b/src/test/scripts/functions/indexing/LeftImplicitRowLowerImplicitRowUpperComma.pydml
@@ -0,0 +1,26 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = full(0, X.shape(0), X.shape(1))
+X_implicit[:,] = X
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/RightImplicitColLowerImplicitRowUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RightImplicitColLowerImplicitRowUpper.pydml b/src/test/scripts/functions/indexing/RightImplicitColLowerImplicitRowUpper.pydml
new file mode 100644
index 0000000..b9f7b36
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RightImplicitColLowerImplicitRowUpper.pydml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = X[2:,:4]
+X_explicit = X[2:X.shape(0),0:4]
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/RightImplicitRowColLowerImplicitRowColUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RightImplicitRowColLowerImplicitRowColUpper.pydml b/src/test/scripts/functions/indexing/RightImplicitRowColLowerImplicitRowColUpper.pydml
new file mode 100644
index 0000000..7d07a0b
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RightImplicitRowColLowerImplicitRowColUpper.pydml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = X[:,:]
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitColUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitColUpper.pydml b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitColUpper.pydml
new file mode 100644
index 0000000..5171579
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitColUpper.pydml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = X[:2,1:]
+X_explicit = X[0:2,1:X.shape(1)]
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpper.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpper.pydml b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpper.pydml
new file mode 100644
index 0000000..162930e
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpper.pydml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = X[:]
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)

http://git-wip-us.apache.org/repos/asf/incubator-systemml/blob/18bb8c52/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpperComma.pydml
----------------------------------------------------------------------
diff --git a/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpperComma.pydml b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpperComma.pydml
new file mode 100644
index 0000000..9c36920
--- /dev/null
+++ b/src/test/scripts/functions/indexing/RightImplicitRowLowerImplicitRowUpperComma.pydml
@@ -0,0 +1,25 @@
+#-------------------------------------------------------------
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+# 
+#   http://www.apache.org/licenses/LICENSE-2.0
+# 
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+#-------------------------------------------------------------
+X = load($1)
+X_implicit = X[:,]
+X_explicit = X
+save(X_implicit, $2)
+save(X_explicit, $3)