You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/08/30 02:32:25 UTC

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Dmitry Lychagin has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/2937

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................

[ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fixed NPE when intersecting two indexes and predicate containing
  between operator:
  a) If index intersection plan is not possible then exit the rule
  b) Rewrite between operator into ge/le pair instead of le/le pair

Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
10 files changed, 234 insertions(+), 15 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/37/2937/1

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index d2b456f..83f1157 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -220,13 +220,17 @@
         List<ILogicalOperator> subRoots = new ArrayList<>();
         for (Pair<IAccessMethod, Index> pair : chosenIndexes) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(pair.first);
-            subRoots.add(pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
-                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx,
-                    AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
-                            subTree.getDataSourceRef().getValue(), afterSelectRefs),
-                    false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
-                            .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                    context, null));
+            boolean retainInput = AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+                    subTree.getDataSourceRef().getValue(), afterSelectRefs);
+            boolean requiresBroadcast = subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
+                    .getExecutionMode() == ExecutionMode.UNPARTITIONED;
+            ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
+                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
+                    requiresBroadcast, context, null);
+            if (subRoot == null) {
+                return false;
+            }
+            subRoots.add(subRoot);
         }
         // Connect each secondary index utilization plan to a common intersect operator.
         ILogicalOperator primaryUnnestOp = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
@@ -406,7 +410,6 @@
                     AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenPrimaryIndex.first);
                     res = chosenPrimaryIndex.first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
                             chosenPrimaryIndex.second, analysisCtx, context);
-                    context.addToDontApplySet(this, selectRef.getValue());
                 } else if (chosenIndexes.size() == 1) {
                     // Index-only plan possible?
                     // Gets the analysis context for the given index.
@@ -418,17 +421,16 @@
                     // Finally, try to apply plan transformation using chosen index.
                     res = chosenIndexes.get(0).first.applySelectPlanTransformation(afterSelectRefs, selectRef, subTree,
                             chosenIndexes.get(0).second, analysisCtx, context);
-                    context.addToDontApplySet(this, selectRef.getValue());
                 } else {
                     // Multiple secondary indexes applicable?
                     res = intersectAllSecondaryIndexes(chosenIndexes, analyzedAMs, context);
-                    context.addToDontApplySet(this, selectRef.getValue());
                 }
 
                 // If the plan transformation is successful, we don't need to traverse
                 // the plan any more, since if there are more SELECT operators, the next
                 // trigger on this plan will find them.
                 if (res) {
+                    context.addToDontApplySet(this, selectRef.getValue());
                     OperatorPropertiesUtil.typeOpRec(opRef, context);
                     return res;
                 }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
new file mode 100644
index 0000000..af04479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
new file mode 100644
index 0000000..80def88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$25(ASC), $$26(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INTERSECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
new file mode 100644
index 0000000..cb51012
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
new file mode 100644
index 0000000..1097325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
new file mode 100644
index 0000000..62f4cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
new file mode 100644
index 0000000..75212af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
@@ -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.
+ */
+
+use tpch;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
new file mode 100644
index 0000000..ad86590
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
@@ -0,0 +1,7 @@
+{ "o_custkey": 40, "o_orderkey": 323, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 3653, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 4934, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 258, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 2596, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 3687, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 5378, "o_orderstatus": "F" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index dc0b84b..904dc61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3375,6 +3375,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="inverted-index-ngram-contains">
         <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 372f0fa..ea86bf5 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -136,13 +136,15 @@
         Expression left = operatorExpr.getExprList().get(1);
         Expression right = operatorExpr.getExprList().get(2);
 
-        // Creates the expression left <= target.
+        // Creates the expression target >= left.
         Expression leftComparison =
-                createLessThanExpression(left, target, operatorExpr.getHints(), operatorExpr.getSourceLocation());
+                createOperatorExpression(OperatorType.GE,
+                        target, left, operatorExpr.getHints(), operatorExpr.getSourceLocation());
         // Creates the expression target <= right.
         Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
         Expression rightComparison =
-                createLessThanExpression(targetCopy, right, operatorExpr.getHints(), operatorExpr.getSourceLocation());
+                createOperatorExpression(OperatorType.LE,
+                        targetCopy, right, operatorExpr.getHints(), operatorExpr.getSourceLocation());
         OperatorExpr andExpr = new OperatorExpr();
         andExpr.addOperand(leftComparison);
         andExpr.addOperand(rightComparison);
@@ -158,12 +160,13 @@
         }
     }
 
-    private Expression createLessThanExpression(Expression lhs, Expression rhs, List<IExpressionAnnotation> hints,
+    private Expression createOperatorExpression(OperatorType opType, Expression lhs, Expression rhs,
+            List<IExpressionAnnotation> hints,
             SourceLocation sourceLoc) {
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(lhs);
         comparison.addOperand(rhs);
-        comparison.addOperator(OperatorType.LE);
+        comparison.addOperator(opType);
         comparison.setSourceLocation(sourceLoc);
         if (hints != null) {
             for (IExpressionAnnotation hint : hints) {

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4821/ (10/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4430/ (4/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4989/ (1/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4397/ (6/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4651/ (3/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4431/ (8/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4433/ (1/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4822/ (8/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4990/ (5/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2442/ (2/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4650/ (3/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4398/ (10/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7257/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2937

to look at the new patch set (#3).

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................

[ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fixed NPE when intersecting two indexes and predicate containing
  between operator:
  a) If index intersection plan is not possible then exit the rule
  b) Rewrite between operator into ge/le pair instead of le/le pair

Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
11 files changed, 234 insertions(+), 16 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/37/2937/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4432/ (8/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7255/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8369/ (9/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4652/ (4/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4820/ (10/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has uploaded a new patch set (#2).

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................

[ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fixed NPE when intersecting two indexes and predicate containing
  between operator:
  a) If index intersection plan is not possible then exit the rule
  b) Rewrite between operator into ge/le pair instead of le/le pair

Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
10 files changed, 233 insertions(+), 15 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/37/2937/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8368/ (6/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4431/ (4/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7256/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/4784/ (11/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9909/ (7/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9907/ (7/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4343/ (9/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4342/ (2/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has submitted this change and it was merged.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


[ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Fixed NPE when intersecting two indexes and predicate containing
  between operator:
  a) If index intersection plan is not possible then exit the rule
  b) Rewrite between operator into ge/le pair instead of le/le pair

Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2937
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
11 files changed, 234 insertions(+), 16 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Till Westmann: Looks good to me, approved
  Jenkins: Verified; No violations found; Verified



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index d2b456f..0ea16ae 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -220,13 +220,17 @@
         List<ILogicalOperator> subRoots = new ArrayList<>();
         for (Pair<IAccessMethod, Index> pair : chosenIndexes) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(pair.first);
-            subRoots.add(pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
-                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx,
-                    AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
-                            subTree.getDataSourceRef().getValue(), afterSelectRefs),
-                    false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
-                            .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                    context, null));
+            boolean retainInput = AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+                    subTree.getDataSourceRef().getValue(), afterSelectRefs);
+            boolean requiresBroadcast = subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
+                    .getExecutionMode() == ExecutionMode.UNPARTITIONED;
+            ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
+                    subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
+                    requiresBroadcast, context, null);
+            if (subRoot == null) {
+                return false;
+            }
+            subRoots.add(subRoot);
         }
         // Connect each secondary index utilization plan to a common intersect operator.
         ILogicalOperator primaryUnnestOp = connectAll2ndarySearchPlanWithIntersect(subRoots, context);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
new file mode 100644
index 0000000..af04479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey;
+
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
new file mode 100644
index 0000000..80def88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$25(ASC), $$26(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- INTERSECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$35(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
new file mode 100644
index 0000000..cb51012
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448)
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
new file mode 100644
index 0000000..1097325
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
new file mode 100644
index 0000000..62f4cc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
new file mode 100644
index 0000000..75212af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/index-selection/intersection-with-between/intersection-with-between.4.query.sqlpp
@@ -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.
+ */
+
+use tpch;
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
new file mode 100644
index 0000000..ad86590
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/index-selection/intersection-with-between/intersection-with-between.4.adm
@@ -0,0 +1,7 @@
+{ "o_custkey": 40, "o_orderkey": 323, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 3653, "o_orderstatus": "F" }
+{ "o_custkey": 40, "o_orderkey": 4934, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 258, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 2596, "o_orderstatus": "O" }
+{ "o_custkey": 43, "o_orderkey": 3687, "o_orderstatus": "F" }
+{ "o_custkey": 43, "o_orderkey": 5378, "o_orderstatus": "F" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
index 7fa0bce..131b860 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.8.adm
@@ -20,7 +20,7 @@
                   -- ASSIGN  |PARTITIONED|
                     exchange
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      data-scan []<-[$$22, $$t] <- test.TweetMessages condition (and(le(0, $$t.getField("user").getField("friends_count")), le($$t.getField("user").getField("friends_count"), 150))) limit 2
+                      data-scan []<-[$$22, $$t] <- test.TweetMessages condition (and(ge($$t.getField("user").getField("friends_count"), 0), le($$t.getField("user").getField("friends_count"), 150))) limit 2
                       -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index dc0b84b..904dc61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3375,6 +3375,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="intersection-with-between">
+        <output-dir compare="Text">intersection-with-between</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="inverted-index-ngram-contains">
         <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 372f0fa..23b4d60 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -136,13 +136,13 @@
         Expression left = operatorExpr.getExprList().get(1);
         Expression right = operatorExpr.getExprList().get(2);
 
-        // Creates the expression left <= target.
-        Expression leftComparison =
-                createLessThanExpression(left, target, operatorExpr.getHints(), operatorExpr.getSourceLocation());
+        // Creates the expression target >= left.
+        Expression leftComparison = createOperatorExpression(OperatorType.GE, target, left, operatorExpr.getHints(),
+                operatorExpr.getSourceLocation());
         // Creates the expression target <= right.
         Expression targetCopy = (Expression) SqlppRewriteUtil.deepCopy(target);
-        Expression rightComparison =
-                createLessThanExpression(targetCopy, right, operatorExpr.getHints(), operatorExpr.getSourceLocation());
+        Expression rightComparison = createOperatorExpression(OperatorType.LE, targetCopy, right,
+                operatorExpr.getHints(), operatorExpr.getSourceLocation());
         OperatorExpr andExpr = new OperatorExpr();
         andExpr.addOperand(leftComparison);
         andExpr.addOperand(rightComparison);
@@ -158,12 +158,12 @@
         }
     }
 
-    private Expression createLessThanExpression(Expression lhs, Expression rhs, List<IExpressionAnnotation> hints,
-            SourceLocation sourceLoc) {
+    private Expression createOperatorExpression(OperatorType opType, Expression lhs, Expression rhs,
+            List<IExpressionAnnotation> hints, SourceLocation sourceLoc) {
         OperatorExpr comparison = new OperatorExpr();
         comparison.addOperand(lhs);
         comparison.addOperand(rhs);
-        comparison.addOperator(OperatorType.LE);
+        comparison.addOperator(opType);
         comparison.setSourceLocation(sourceLoc);
         if (hints != null) {
             for (IExpressionAnnotation hint : hints) {

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4432/ (6/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://goo.gl/QdBXre : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8370/ (3/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/sXGd9s : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4988/ (1/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4396/ (9/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/4782/ (11/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9908/ (5/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2444/ (2/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2443/ (7/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4341/ (5/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7257/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/i4K4sa : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/XX3CBU : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[stabilization-f69489]: [ASTERIXDB-2448][COMP] Fix NullPointerException in Introduce...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2448][COMP] Fix NullPointerException in IntroduceSelectAccessMethodRule
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/4783/ (11/11)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2937
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I34d4cde6adf3785feb6a8280fcdb964a5fb72803
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: stabilization-f69489
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No