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/03/28 21:28:22 UTC

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

Dmitry Lychagin has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................

[ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions

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

Details:
- Constant folding rule should not fail on runtime exceptions
- throw RuntimeDataException instead of java.lang.ArithmeticException
  from numeric operators

Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
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
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
7 files changed, 46 insertions(+), 38 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index fd66821..c3061f8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -97,7 +97,7 @@
     private static final IVariableTypeEnvironment _emptyTypeEnv = new IVariableTypeEnvironment() {
 
         @Override
-        public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) throws AlgebricksException {
+        public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) {
             throw new IllegalStateException();
         }
 
@@ -108,12 +108,12 @@
 
         @Override
         public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
-                List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
+                List<List<LogicalVariable>> correlatedNullableVariableLists) {
             throw new IllegalStateException();
         }
 
         @Override
-        public Object getVarType(LogicalVariable var) throws AlgebricksException {
+        public Object getVarType(LogicalVariable var) {
             throw new IllegalStateException();
         }
 
@@ -170,14 +170,13 @@
         }
 
         @Override
-        public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg)
-                throws AlgebricksException {
+        public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg) {
             return new Pair<>(false, expr);
         }
 
         @Override
         public Pair<Boolean, ILogicalExpression> visitVariableReferenceExpression(VariableReferenceExpression expr,
-                Void arg) throws AlgebricksException {
+                Void arg) {
             return new Pair<>(false, expr);
         }
 
@@ -194,31 +193,32 @@
                 return new Pair<>(false, null);
             }
 
-            //Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors if they are not strongly typed
-            if (expr.getFunctionIdentifier().equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
-                    || expr.getFunctionIdentifier().equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
-                AbstractCollectionType listType = (AbstractCollectionType) TypeCastUtils.getRequiredType(expr);
-                if (listType != null && (listType.getItemType().getTypeTag() == ATypeTag.ANY
-                        || listType.getItemType() instanceof AbstractCollectionType)) {
-                    //case1: listType == null,  could be a nested list inside a list<ANY>
-                    //case2: itemType = ANY
-                    //case3: itemType = a nested list
-                    return new Pair<>(false, null);
-                }
-            }
-            if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
-                ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
-                String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
-                int k = rt.getFieldIndex(str);
-                if (k >= 0) {
-                    // wait for the ByNameToByIndex rule to apply
-                    return new Pair<>(changed, expr);
-                }
-            }
-
-            IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
-                    _emptyTypeEnv, _emptySchemas, jobGenCtx);
             try {
+                //Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors if they are not strongly typed
+                if (expr.getFunctionIdentifier().equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
+                        || expr.getFunctionIdentifier().equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
+                    AbstractCollectionType listType = (AbstractCollectionType) TypeCastUtils.getRequiredType(expr);
+                    if (listType != null && (listType.getItemType().getTypeTag() == ATypeTag.ANY
+                            || listType.getItemType() instanceof AbstractCollectionType)) {
+                        //case1: listType == null,  could be a nested list inside a list<ANY>
+                        //case2: itemType = ANY
+                        //case3: itemType = a nested list
+                        return new Pair<>(false, null);
+                    }
+                }
+                if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+                    ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
+                    String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
+                    int k = rt.getFieldIndex(str);
+                    if (k >= 0) {
+                        // wait for the ByNameToByIndex rule to apply
+                        return new Pair<>(changed, expr);
+                    }
+                }
+
+                IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
+                        _emptyTypeEnv, _emptySchemas, jobGenCtx);
+
                 IScalarEvaluator eval = fact.createScalarEvaluator(null);
                 eval.evaluate(null, p);
                 Object t = _emptyTypeEnv.getType(expr);
@@ -229,8 +229,8 @@
                 bbis.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
                 IAObject o = (IAObject) serde.deserialize(dis);
                 return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
-            } catch (HyracksDataException e) {
-                throw new AlgebricksException(e);
+            } catch (HyracksDataException | AlgebricksException e) {
+                return new Pair<>(false, null);
             }
         }
 
@@ -267,7 +267,7 @@
             return changed;
         }
 
-        private boolean checkArgs(AbstractFunctionCallExpression expr) throws AlgebricksException {
+        private boolean checkArgs(AbstractFunctionCallExpression expr) {
             for (Mutable<ILogicalExpression> r : expr.getArguments()) {
                 if (r.getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
                     return false;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
index 3b12ead..3b395bc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
@@ -39,6 +39,7 @@
     [ 17, ifinf(float("INF"), double("-INF"), 2) ],
     [ 18, isnull(ifinf(double("INF"), double("-INF"), [], 2)) ],
     [ 19, ismissing(if_inf(double("INF"), double("-INF"), missing, 2)) ],
-    [ 20, tostring(ifinf(float("INF"), float("NaN"), 2)) ]
+    [ 20, tostring(ifinf(float("INF"), float("NaN"), 2)) ],
+    [ 21, if_inf(2, 1/0) ]
 ] t
 order by t[0]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
index c45efa6..9f56be4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
@@ -18,4 +18,5 @@
 [ 17, 2 ]
 [ 18, true ]
 [ 19, true ]
-[ 20, "NaN" ]
\ No newline at end of file
+[ 20, "NaN" ]
+[ 21, 2 ]
\ No newline at end of file
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 8146797..b9a5b29 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
@@ -74,6 +74,7 @@
     public static final int INVALID_TYPE_CASTING_MATH_FUNCTION = 31;
     public static final int REJECT_BAD_CLUSTER_STATE = 32;
     public static final int REJECT_NODE_UNREGISTERED = 33;
+    public static final int DIVISION_BY_ZERO = 34;
 
     public static final int INSTANTIATION_ERROR = 100;
 
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 f9188b8..92aac98 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -67,6 +67,7 @@
 31 = Invalid type-casting math function: %1$s for converting %2$s to %3$s
 32 = Cannot execute request, cluster is %1$s
 33 = Node is not registered with the CC
+34 = Division by Zero.
 
 100 = Unable to instantiate class %1$s
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
index 3b72072..0079a8a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -44,7 +45,7 @@
     @Override
     protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
         if (rhs > Integer.MAX_VALUE) {
-            throw new ArithmeticException("Exponent cannot be larger than 2^31-1");
+            throw new OverflowException(getIdentifier());
         }
         return LongMath.checkedPow(lhs, (int) rhs);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 7cda149..77c94bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,10 +47,10 @@
     @Override
     protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
         if (rhs == 0) {
-            throw new ArithmeticException("Division by Zero.");
+            throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
         }
         if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
-            throw new ArithmeticException(("Overflow in integer division"));
+            throw new OverflowException(getIdentifier());
         }
         return lhs / rhs;
     }

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3928/ (12/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3667/ (12/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8673/ (8/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7162/ (8/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3172/ (5/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3213/ (9/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1218/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3742/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1220/ (3/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3923/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7160/ (4/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


[ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions

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

Details:
- Constant folding rule should not fail on runtime exceptions
- throw RuntimeDataException instead of java.lang.ArithmeticException
  from numeric operators

Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2542
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: 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/ConstantFoldingRule.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
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
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
7 files changed, 54 insertions(+), 39 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/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index fd66821..7e9328b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -67,6 +67,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionVisitor;
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
@@ -93,11 +94,13 @@
             BuiltinFunctions.FIELD_ACCESS_BY_INDEX, BuiltinFunctions.CAST_TYPE, BuiltinFunctions.META,
             BuiltinFunctions.META_KEY, BuiltinFunctions.RECORD_CONCAT, BuiltinFunctions.RECORD_CONCAT_STRICT);
 
-    /** Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method. */
+    /**
+     * Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method.
+     */
     private static final IVariableTypeEnvironment _emptyTypeEnv = new IVariableTypeEnvironment() {
 
         @Override
-        public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) throws AlgebricksException {
+        public boolean substituteProducedVariable(LogicalVariable v1, LogicalVariable v2) {
             throw new IllegalStateException();
         }
 
@@ -108,12 +111,12 @@
 
         @Override
         public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
-                List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
+                List<List<LogicalVariable>> correlatedNullableVariableLists) {
             throw new IllegalStateException();
         }
 
         @Override
-        public Object getVarType(LogicalVariable var) throws AlgebricksException {
+        public Object getVarType(LogicalVariable var) {
             throw new IllegalStateException();
         }
 
@@ -170,14 +173,13 @@
         }
 
         @Override
-        public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg)
-                throws AlgebricksException {
+        public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg) {
             return new Pair<>(false, expr);
         }
 
         @Override
         public Pair<Boolean, ILogicalExpression> visitVariableReferenceExpression(VariableReferenceExpression expr,
-                Void arg) throws AlgebricksException {
+                Void arg) {
             return new Pair<>(false, expr);
         }
 
@@ -194,31 +196,33 @@
                 return new Pair<>(false, null);
             }
 
-            //Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors if they are not strongly typed
-            if (expr.getFunctionIdentifier().equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
-                    || expr.getFunctionIdentifier().equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
-                AbstractCollectionType listType = (AbstractCollectionType) TypeCastUtils.getRequiredType(expr);
-                if (listType != null && (listType.getItemType().getTypeTag() == ATypeTag.ANY
-                        || listType.getItemType() instanceof AbstractCollectionType)) {
-                    //case1: listType == null,  could be a nested list inside a list<ANY>
-                    //case2: itemType = ANY
-                    //case3: itemType = a nested list
-                    return new Pair<>(false, null);
-                }
-            }
-            if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
-                ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
-                String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
-                int k = rt.getFieldIndex(str);
-                if (k >= 0) {
-                    // wait for the ByNameToByIndex rule to apply
-                    return new Pair<>(changed, expr);
-                }
-            }
-
-            IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
-                    _emptyTypeEnv, _emptySchemas, jobGenCtx);
             try {
+                // Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors
+                // if they are not strongly typed
+                if (expr.getFunctionIdentifier().equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
+                        || expr.getFunctionIdentifier().equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
+                    AbstractCollectionType listType = (AbstractCollectionType) TypeCastUtils.getRequiredType(expr);
+                    if (listType != null && (listType.getItemType().getTypeTag() == ATypeTag.ANY
+                            || listType.getItemType() instanceof AbstractCollectionType)) {
+                        //case1: listType == null,  could be a nested list inside a list<ANY>
+                        //case2: itemType = ANY
+                        //case3: itemType = a nested list
+                        return new Pair<>(false, null);
+                    }
+                }
+                if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+                    ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
+                    String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
+                    int k = rt.getFieldIndex(str);
+                    if (k >= 0) {
+                        // wait for the ByNameToByIndex rule to apply
+                        return new Pair<>(changed, expr);
+                    }
+                }
+
+                IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
+                        _emptyTypeEnv, _emptySchemas, jobGenCtx);
+
                 IScalarEvaluator eval = fact.createScalarEvaluator(null);
                 eval.evaluate(null, p);
                 Object t = _emptyTypeEnv.getType(expr);
@@ -229,8 +233,11 @@
                 bbis.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
                 IAObject o = (IAObject) serde.deserialize(dis);
                 return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
-            } catch (HyracksDataException e) {
-                throw new AlgebricksException(e);
+            } catch (HyracksDataException | AlgebricksException e) {
+                if (AlgebricksConfig.ALGEBRICKS_LOGGER.isDebugEnabled()) {
+                    AlgebricksConfig.ALGEBRICKS_LOGGER.debug("Exception caught at constant folding: " + e, e);
+                }
+                return new Pair<>(false, null);
             }
         }
 
@@ -267,7 +274,7 @@
             return changed;
         }
 
-        private boolean checkArgs(AbstractFunctionCallExpression expr) throws AlgebricksException {
+        private boolean checkArgs(AbstractFunctionCallExpression expr) {
             for (Mutable<ILogicalExpression> r : expr.getArguments()) {
                 if (r.getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
                     return false;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
index 3b12ead..3b395bc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
@@ -39,6 +39,7 @@
     [ 17, ifinf(float("INF"), double("-INF"), 2) ],
     [ 18, isnull(ifinf(double("INF"), double("-INF"), [], 2)) ],
     [ 19, ismissing(if_inf(double("INF"), double("-INF"), missing, 2)) ],
-    [ 20, tostring(ifinf(float("INF"), float("NaN"), 2)) ]
+    [ 20, tostring(ifinf(float("INF"), float("NaN"), 2)) ],
+    [ 21, if_inf(2, 1/0) ]
 ] t
 order by t[0]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
index c45efa6..9f56be4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
@@ -18,4 +18,5 @@
 [ 17, 2 ]
 [ 18, true ]
 [ 19, true ]
-[ 20, "NaN" ]
\ No newline at end of file
+[ 20, "NaN" ]
+[ 21, 2 ]
\ No newline at end of file
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 8146797..b9a5b29 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
@@ -74,6 +74,7 @@
     public static final int INVALID_TYPE_CASTING_MATH_FUNCTION = 31;
     public static final int REJECT_BAD_CLUSTER_STATE = 32;
     public static final int REJECT_NODE_UNREGISTERED = 33;
+    public static final int DIVISION_BY_ZERO = 34;
 
     public static final int INSTANTIATION_ERROR = 100;
 
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 f9188b8..92aac98 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -67,6 +67,7 @@
 31 = Invalid type-casting math function: %1$s for converting %2$s to %3$s
 32 = Cannot execute request, cluster is %1$s
 33 = Node is not registered with the CC
+34 = Division by Zero.
 
 100 = Unable to instantiate class %1$s
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
index 3b72072..0079a8a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -44,7 +45,7 @@
     @Override
     protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
         if (rhs > Integer.MAX_VALUE) {
-            throw new ArithmeticException("Exponent cannot be larger than 2^31-1");
+            throw new OverflowException(getIdentifier());
         }
         return LongMath.checkedPow(lhs, (int) rhs);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 7cda149..77c94bf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -18,10 +18,13 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.exceptions.OverflowException;
 import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -44,10 +47,10 @@
     @Override
     protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
         if (rhs == 0) {
-            throw new ArithmeticException("Division by Zero.");
+            throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
         }
         if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
-            throw new ArithmeticException(("Overflow in integer division"));
+            throw new OverflowException(getIdentifier());
         }
         return lhs / rhs;
     }

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3197/ (1/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8675/ (2/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3669/ (12/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3921/ (5/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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/2542

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................

[ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions

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

Details:
- Constant folding rule should not fail on runtime exceptions
- throw RuntimeDataException instead of java.lang.ArithmeticException
  from numeric operators

Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/ifinf/ifinf.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/ifinf/ifinf.1.adm
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
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
7 files changed, 54 insertions(+), 39 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3426/ (9/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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

Change subject: [ASTERIXDB-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3124/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I286551a98f57df798ce982228a66d6a1e3fc7304
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
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[release-0.9.4-pre-rc]: [ASTERIXDB-2346][COMP] Constant folding should not fail on r...

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-2346][COMP] Constant folding should not fail on runtime exceptions
......................................................................


Patch Set 2:

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

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

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