You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org> on 2017/02/28 20:58:48 UTC

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

Taewoo Kim has uploaded a new change for review.

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................

ASTERIXDB-1806: let inject_failure not utilize an index

 - Let Index TransFormation Rule ignore Inject_failure()
   since the function can't utilize the index because of its arguments.

Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
6 files changed, 113 insertions(+), 6 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index ca042f1..e2595c8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -84,6 +84,10 @@
                     BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
                     BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_MOD);
 
+    // Function Identifier sets that cannot utilize any index.
+    private final ImmutableSet<FunctionIdentifier> funcIDSetThatCannotUtilizeIndex =
+            ImmutableSet.of(BuiltinFunctions.OR, BuiltinFunctions.INJECT_FAILURE);
+
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
     protected static void registerAccessMethod(IAccessMethod accessMethod,
@@ -386,9 +390,10 @@
             IVariableTypeEnvironment typeEnvironment) throws AlgebricksException {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+
         // Don't consider optimizing a disjunctive condition with an index (too
-        // complicated for now).
-        if (funcIdent == AlgebricksBuiltinFunctions.OR) {
+        // complicated for now). Also, if a function can't utilize index, then we don't need to check its arguments.
+        if (funcIDSetThatCannotUtilizeIndex.contains(funcIdent)) {
             return false;
         }
         boolean found = analyzeFunctionExpr(funcExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 79ef433..e5050e2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -31,6 +31,8 @@
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -541,7 +543,11 @@
             // If not, we create a new condition based on remaining ones.
             if (!primaryIndexPostProccessingIsNeeded) {
                 List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<>();
-                getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+                try {
+                    getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+                } catch (CompilationException e) {
+                    return null;
+                }
                 // Generate new condition.
                 if (!remainingFuncExprs.isEmpty()) {
                     ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
@@ -630,7 +636,8 @@
     }
 
     private void getNewConditionExprs(Mutable<ILogicalExpression> conditionRef,
-            Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+            Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs)
+            throws CompilationException {
         remainingFuncExprs.clear();
         if (replacedFuncExprs.isEmpty()) {
             return;
@@ -648,7 +655,7 @@
         }
         // The original select cond must be an AND. Check it just to be sure.
         if (funcExpr.getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND) {
-            throw new IllegalStateException();
+            throw new CompilationException(ErrorCode.COMPILATION_NOT_SUPPORTED_FUNC_EXPRESSION, funcExpr.toString());
         }
         // Clean the conjuncts.
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
new file mode 100644
index 0000000..b7ab318
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * 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  : This test case is to verify the fix for issue 1806
+ * https://issues.apache.org/jira/browse/ASTERIXDB-1806
+ * Expected Res : Non-index utilization Plan
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+}
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+SET `import-private-functions` "true";
+
+SELECT  l_returnflag,
+        l_linestatus,
+        sum(l_quantity) AS sum_qty,
+        sum(l_extendedprice) AS sum_base_price,
+        sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+        sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+        avg(l_quantity) AS ave_qty,
+        avg(l_extendedprice) AS ave_price,
+        avg(l_discount) AS ave_disc,
+        count(1) AS count_order
+FROM  LineItem l
+WHERE inject_failure(l.l_shipdate <= '1998-09-02', l.l_orderkey=5988)
+GROUP BY l_returnflag, l_linestatus
+ORDER BY l_returnflag, l_linestatus
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
new file mode 100644
index 0000000..0ef1ee3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$112, $$113]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$80, $$81]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 6dbafd1..c1d1df8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.common.exceptions;
 
-import java.io.File;
 import java.io.InputStream;
 import java.util.Map;
 
@@ -81,6 +80,7 @@
     public static final int COMPILATION_CANT_DROP_ACTIVE_DATASET = 1023;
     public static final int COMPILATION_AQLPLUS_IDENTIFIER_NOT_FOUND = 1024;
     public static final int COMPILATION_AQLPLUS_NO_SUCH_JOIN_TYPE = 1025;
+    public static final int COMPILATION_NOT_SUPPORTED_FUNC_EXPRESSION = 1026;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 14bd0c7..f33bd4f 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -65,6 +65,7 @@
 1023 = Can't drop dataset %1$s since it is connected to active entity: %2$s
 1024 = Identifier %1$s is not found in AQL+ meta-scope
 1025 = There is no such join type in AQL+
+1026 = Not supported Function Expression: %1$s
 
 # Feed Errors
 3001 = Illegal state.

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4476/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4: BAD-1

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/660/ : FAILURE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1536/1/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java:

PS1, Line 83: COMPILATION_NOT_SUPPORTED_FUNC_EXPRESSION
The name is confusing.  The function is supported but not able to leverage index.


https://asterix-gerrit.ics.uci.edu/#/c/1536/1/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS1, Line 67:  
The message is confusing. The function is supported, but cannot take advantage of indexes.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4480/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

Posted by "Taewoo Kim (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/1536

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................

ASTERIXDB-1806: let inject_failure not utilize an index

 - Change the condition check logic of index transformation rule
   so that only arguments of AND function can be checked.
 - Let Index TransFormation Rule ignore Inject_failure()
   since the function can't utilize the index because of its arguments.

Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
6 files changed, 124 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/36/1536/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1536
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1536/1/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java:

PS1, Line 83: COMPILATION_NOT_SUPPORTED_FUNC_EXPRESSION
> The name is confusing.  The function is supported but not able to leverage 
Done


https://asterix-gerrit.ics.uci.edu/#/c/1536/1/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS1, Line 67:  
> The message is confusing. The function is supported, but cannot take advant
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/660/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4479/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/673/ : FAILURE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5: BAD+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/681/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/681/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/673/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1: BAD-1

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/657/ : FAILURE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

Posted by "Taewoo Kim (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/1536

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................

ASTERIXDB-1806: let inject_failure not utilize an index

 - Change the condition check logic of index transformation rule
   so that only arguments of AND function can be checked.
 - Let Index TransFormation Rule ignore Inject_failure()
   since the function can't utilize the index because of its arguments.

Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
6 files changed, 123 insertions(+), 14 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


ASTERIXDB-1806: let inject_failure not utilize an index

 - Change the condition check logic of index transformation rule
   so that only arguments of AND function can be checked.
 - Let Index TransFormation Rule ignore Inject_failure()
   since the function can't utilize the index because of its arguments.

Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1536
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
6 files changed, 124 insertions(+), 14 deletions(-)

Approvals:
  Yingyi Bu: Looks good to me, approved
  Jenkins: Verified; No violations found; Verified

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index ca042f1..d7fb0ac 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -390,19 +390,25 @@
         // complicated for now).
         if (funcIdent == AlgebricksBuiltinFunctions.OR) {
             return false;
-        }
-        boolean found = analyzeFunctionExpr(funcExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
-        for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
-            ILogicalExpression argExpr = arg.getValue();
-            if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                continue;
+        } else if (funcIdent == AlgebricksBuiltinFunctions.AND) {
+            // This is the only case that the optimizer can check the given function's arguments to see
+            // if one of its argument can utilize an index.
+            boolean found = false;
+            for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
+                ILogicalExpression argExpr = arg.getValue();
+                if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                    continue;
+                }
+                AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
+                boolean matchFound =
+                        analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
+                found = found || matchFound;
             }
-            AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
-            boolean matchFound =
-                    analyzeFunctionExpr(argFuncExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
-            found = found || matchFound;
+            return found;
+        } else {
+            // For single function or "NOT" case:
+            return analyzeFunctionExpr(funcExpr, assignsAndUnnests, analyzedAMs, context, typeEnvironment);
         }
-        return found;
     }
 
     /**
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 79ef433..3acf1f5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -31,6 +31,8 @@
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -541,7 +543,11 @@
             // If not, we create a new condition based on remaining ones.
             if (!primaryIndexPostProccessingIsNeeded) {
                 List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<>();
-                getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+                try {
+                    getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+                } catch (CompilationException e) {
+                    return null;
+                }
                 // Generate new condition.
                 if (!remainingFuncExprs.isEmpty()) {
                     ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
@@ -630,7 +636,8 @@
     }
 
     private void getNewConditionExprs(Mutable<ILogicalExpression> conditionRef,
-            Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+            Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs)
+            throws CompilationException {
         remainingFuncExprs.clear();
         if (replacedFuncExprs.isEmpty()) {
             return;
@@ -648,7 +655,8 @@
         }
         // The original select cond must be an AND. Check it just to be sure.
         if (funcExpr.getFunctionIdentifier() != AlgebricksBuiltinFunctions.AND) {
-            throw new IllegalStateException();
+            throw new CompilationException(ErrorCode.COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX,
+                    funcExpr.toString());
         }
         // Clean the conjuncts.
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
new file mode 100644
index 0000000..b7ab318
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
@@ -0,0 +1,69 @@
+/*
+ * 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  : This test case is to verify the fix for issue 1806
+ * https://issues.apache.org/jira/browse/ASTERIXDB-1806
+ * Expected Res : Non-index utilization Plan
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+}
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+SET `import-private-functions` "true";
+
+SELECT  l_returnflag,
+        l_linestatus,
+        sum(l_quantity) AS sum_qty,
+        sum(l_extendedprice) AS sum_base_price,
+        sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+        sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+        avg(l_quantity) AS ave_qty,
+        avg(l_extendedprice) AS ave_price,
+        avg(l_discount) AS ave_disc,
+        count(1) AS count_order
+FROM  LineItem l
+WHERE inject_failure(l.l_shipdate <= '1998-09-02', l.l_orderkey=5988)
+GROUP BY l_returnflag, l_linestatus
+ORDER BY l_returnflag, l_linestatus
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
new file mode 100644
index 0000000..0ef1ee3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$16(ASC), $$17(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$112, $$113]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+              -- SORT_GROUP_BY[$$80, $$81]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 70e0ae9..4898e40 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -84,6 +84,7 @@
     public static final int COMPILATION_CANT_DROP_ACTIVE_DATASET = 1023;
     public static final int COMPILATION_AQLPLUS_IDENTIFIER_NOT_FOUND = 1024;
     public static final int COMPILATION_AQLPLUS_NO_SUCH_JOIN_TYPE = 1025;
+    public static final int COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX = 1026;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 2bdb2a3..aae7050 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -69,6 +69,7 @@
 1023 = Can't drop dataset %1$s since it is connected to active entity: %2$s
 1024 = Identifier %1$s is not found in AQL+ meta-scope
 1025 = There is no such join type in AQL+
+1026 = The given function expression %1$s cannot utilize index
 
 # Feed Errors
 3001 = Illegal state.

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

Posted by "Taewoo Kim (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/1536

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................

ASTERIXDB-1806: let inject_failure not utilize an index

 - Change the condition check logic of index transformation rule
   so that only arguments of AND function can be checked.
 - Let Index TransFormation Rule ignore Inject_failure()
   since the function can't utilize the index because of its arguments.

Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
A asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806.sqlpp
A asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
6 files changed, 123 insertions(+), 14 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4497/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

Added Steven since BAD test fails.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/657/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1806: let inject_failure not utilize an index

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

Change subject: ASTERIXDB-1806: let inject_failure not utilize an index
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4481/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I5ca2da1eb08fbb7c27205bdff9795c0aa816794b
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No