You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org> on 2019/05/16 09:01:17 UTC

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/3399


Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................

[ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

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

Details:
This patch fixes field access in the presense of UNION and CASE.
For CASE scenario, push-down-field-access rule throws an exception
if the field access has potentially two sources and it could not
push down the field access to left or right branch. The exception
thinks that the field access has no source when in reality it has
but there were two potential sources because of CASE. Don't throw
an exception if the field access is a valid one and return false
(i.e. field access was not push) instead of throwing an exception.
For UNION, if there is an assign operator above UNION, the optimizer
will move the assign operator under UNION into both branches. If
the assign operator has field access by index, the index is based on
the output record of the union operator which could be different from
the field index in left and right branches. The fix is to convert
the index into a named field, and then move the assign operator.
The optimizer will fire by-name-to-by-index rule after this and
replace the name with the correct index for each record of left and
right branch. by-name-to-by-index rule is already a required rule
to be fired after push-assign-under-union rule.

Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAssignBelowUnionAllRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.7.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.3.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.4.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.6.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.7.adm
D hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
16 files changed, 537 insertions(+), 174 deletions(-)



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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index e4c35e4..79f565f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -66,6 +66,7 @@
 import org.apache.asterix.optimizer.rules.NestGroupByRule;
 import org.apache.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import org.apache.asterix.optimizer.rules.PushAggregateIntoNestedSubplanRule;
+import org.apache.asterix.optimizer.rules.PushAssignBelowUnionAllRule;
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
 import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
 import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
@@ -117,7 +118,6 @@
 import org.apache.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
 import org.apache.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
-import org.apache.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
 import org.apache.hyracks.algebricks.rewriter.rules.PushGroupByIntoSortRule;
 import org.apache.hyracks.algebricks.rewriter.rules.PushMapOperatorDownThroughProductRule;
 import org.apache.hyracks.algebricks.rewriter.rules.PushNestedOrderByUnderPreSortedGroupByRule;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAssignBelowUnionAllRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAssignBelowUnionAllRule.java
new file mode 100644
index 0000000..552b691
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAssignBelowUnionAllRule.java
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * <pre>
+ * Pushes an AssignOperator below a UnionAll operator by creating an new AssignOperator below each of
+ * the UnionAllOperator's branches with appropriate variable replacements.
+ * This rule can help to enable other rules that are difficult to fire across a UnionAllOperator,
+ * for example, eliminating common sub-expressions.
+ * Example:
+ * Before plan:
+ * ...
+ * assign [$$20, $$21] <- [funcA($$3), funcB($$6)]
+ * union ($$1, $$2, $$3) ($$4, $$5, $$6)
+ * union_branch_0
+ * ...
+ * union_branch_1
+ * ...
+ * After plan:
+ * ...
+ * union ($$1, $$2, $$3) ($$4, $$5, $$6) ($$22, $$24, $$20) ($$23, $$25, $$21)
+ * assign [$$22, $$23] <- [funcA($$1), funcB($$4)]
+ * union_branch_0
+ * ...
+ * assign [$$24, $$25] <- [funcA($$2), funcB($$5)]
+ * union_branch_1
+ * ...
+ * </pre>
+ * Post-requirements:
+ * <ol>
+ * <li>{@link ByNameToByIndexFieldAccessRule} should be fired.</li>
+ * </ol>
+ */
+public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (!op.hasInputs()) {
+            return false;
+        }
+
+        boolean modified = false;
+        for (int i = 0; i < op.getInputs().size(); i++) {
+            AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+            if (childOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+                continue;
+            }
+            AssignOperator assignOp = (AssignOperator) childOp;
+            for (Mutable<ILogicalExpression> expr : assignOp.getExpressions()) {
+                if (!expr.getValue().isFunctional()) {
+                    return false;
+                }
+            }
+
+            AbstractLogicalOperator childOfChildOp = (AbstractLogicalOperator) assignOp.getInputs().get(0).getValue();
+            if (childOfChildOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
+                continue;
+            }
+            UnionAllOperator unionOp = (UnionAllOperator) childOfChildOp;
+
+            Set<LogicalVariable> assignUsedVars = new HashSet<>();
+            VariableUtilities.getUsedVariables(assignOp, assignUsedVars);
+
+            List<LogicalVariable> assignVars = assignOp.getVariables();
+
+            AssignOperator[] newAssignOps = new AssignOperator[2];
+            for (int j = 0; j < unionOp.getInputs().size(); j++) {
+                newAssignOps[j] = createAssignBelowUnionAllBranch(unionOp, j, assignOp, assignUsedVars, context);
+            }
+            // Add original assign variables to the union variable mappings.
+            for (int j = 0; j < assignVars.size(); j++) {
+                LogicalVariable first = newAssignOps[0].getVariables().get(j);
+                LogicalVariable second = newAssignOps[1].getVariables().get(j);
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping =
+                        new Triple<>(first, second, assignVars.get(j));
+                unionOp.getVariableMappings().add(varMapping);
+            }
+            context.computeAndSetTypeEnvironmentForOperator(unionOp);
+
+            // Remove original assign operator.
+            op.getInputs().set(i, assignOp.getInputs().get(0));
+            context.computeAndSetTypeEnvironmentForOperator(op);
+            modified = true;
+        }
+
+        return modified;
+    }
+
+    private AssignOperator createAssignBelowUnionAllBranch(UnionAllOperator unionOp, int inputIndex,
+            AssignOperator originalAssignOp, Set<LogicalVariable> assignUsedVars, IOptimizationContext context)
+            throws AlgebricksException {
+        AssignOperator newAssignOp = cloneAssignOperator(originalAssignOp, context, unionOp);
+        newAssignOp.getInputs().add(new MutableObject<>(unionOp.getInputs().get(inputIndex).getValue()));
+        unionOp.getInputs().get(inputIndex).setValue(newAssignOp);
+        int numVarMappings = unionOp.getVariableMappings().size();
+        for (int i = 0; i < numVarMappings; i++) {
+            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping = unionOp.getVariableMappings().get(i);
+            if (assignUsedVars.contains(varMapping.third)) {
+                LogicalVariable replacementVar;
+                if (inputIndex == 0) {
+                    replacementVar = varMapping.first;
+                } else {
+                    replacementVar = varMapping.second;
+                }
+                VariableUtilities.substituteVariables(newAssignOp, varMapping.third, replacementVar, context);
+            }
+        }
+        context.computeAndSetTypeEnvironmentForOperator(newAssignOp);
+        return newAssignOp;
+    }
+
+    /**
+     * Clones the given assign operator changing the returned variables to be new ones.
+     * Also, leaves the inputs of the clone clear.
+     */
+    private AssignOperator cloneAssignOperator(AssignOperator assignOp, IOptimizationContext context,
+            UnionAllOperator unionOp) throws AlgebricksException {
+        List<LogicalVariable> vars = new ArrayList<>();
+        List<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
+        int numVars = assignOp.getVariables().size();
+        for (int i = 0; i < numVars; i++) {
+            vars.add(context.newVar());
+            ILogicalExpression clonedExpression = assignOp.getExpressions().get(i).getValue().cloneExpression();
+            exprs.add(new MutableObject<>(modifyExpression(clonedExpression, unionOp, context)));
+        }
+        AssignOperator assignCloneOp = new AssignOperator(vars, exprs);
+        assignCloneOp.setSourceLocation(assignOp.getSourceLocation());
+        assignCloneOp.setExecutionMode(assignOp.getExecutionMode());
+        return assignCloneOp;
+    }
+
+    private ILogicalExpression modifyExpression(ILogicalExpression expression, UnionAllOperator unionOp,
+            IOptimizationContext context) throws AlgebricksException {
+        switch (expression.getExpressionTag()) {
+            case CONSTANT:
+            case VARIABLE:
+                return expression;
+            case FUNCTION_CALL:
+                AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) expression;
+                List<Mutable<ILogicalExpression>> arguments = functionCall.getArguments();
+                for (int i = 0, size = arguments.size(); i < size; i++) {
+                    ILogicalExpression newArgExpr = modifyExpression(arguments.get(i).getValue(), unionOp, context);
+                    arguments.get(i).setValue(newArgExpr);
+                }
+                return changeFieldIndexToFieldName(functionCall, unionOp, context);
+            default:
+                throw new NotImplementedException();
+        }
+    }
+
+    private static ILogicalExpression changeFieldIndexToFieldName(AbstractFunctionCallExpression functionCall,
+            UnionAllOperator unionOp, IOptimizationContext context) throws AlgebricksException {
+        if (!functionCall.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_INDEX)) {
+            return functionCall;
+        }
+        // the record variable in the field access should match the output variable from union, i.e. $2.getField
+        ILogicalExpression recordExpr = functionCall.getArguments().get(0).getValue();
+        if (recordExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return functionCall;
+        }
+        Integer fieldIndex = ConstantExpressionUtil.getIntArgument(functionCall, 1);
+        if (fieldIndex == null) {
+            return functionCall;
+        }
+        LogicalVariable recordVar = ((VariableReferenceExpression) recordExpr).getVariableReference();
+        AbstractFunctionCallExpression newFunctionCall = functionCall;
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> variableMapping : unionOp
+                .getVariableMappings()) {
+            if (variableMapping.third.equals(recordVar)) {
+                IAType type = (IAType) context.getOutputTypeEnvironment(unionOp).getVarType(variableMapping.third);
+                if (type.getTypeTag() != ATypeTag.OBJECT) {
+                    continue;
+                }
+                ARecordType unionOutputType = (ARecordType) type;
+                String fieldName = unionOutputType.getFieldNames()[fieldIndex];
+                newFunctionCall = new ScalarFunctionCallExpression(
+                        FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
+                        functionCall.getArguments().get(0),
+                        new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AString(fieldName)))));
+                newFunctionCall.setSourceLocation(functionCall.getSourceLocation());
+                return newFunctionCall;
+            }
+        }
+        return newFunctionCall;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index 421b00d..0c3a163 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -69,6 +69,8 @@
 
 public class PushFieldAccessRule implements IAlgebraicRewriteRule {
 
+    private final HashSet<LogicalVariable> allLiveVariables = new HashSet<>();
+
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
@@ -234,8 +236,7 @@
                     LogicalVariable oldVar = assignOp.getVariables().get(0);
                     VariableReferenceExpression v2Ref = new VariableReferenceExpression(v2);
                     v2Ref.setSourceLocation(g.getSourceLocation());
-                    g.getDecorList().add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(oldVar,
-                            new MutableObject<ILogicalExpression>(v2Ref)));
+                    g.getDecorList().add(new Pair<>(oldVar, new MutableObject<>(v2Ref)));
                     changed = true;
                     assignOp.getVariables().set(0, v2);
                     VariableUtilities.substituteVariables(assignOp, m.first, m.second, context);
@@ -255,6 +256,7 @@
                 pushAccessDown(opRef, inputOp, childOfSubplan, context, finalAnnot);
                 return true;
             }
+            allLiveVariables.clear();
             if (inputOp.getInputs().size() == 1 && !inputOp.hasNestedPlans()) {
                 pushAccessDown(opRef, inputOp, inputOp.getInputs().get(0), context, finalAnnot);
                 return true;
@@ -262,6 +264,7 @@
                 for (Mutable<ILogicalOperator> inp : inputOp.getInputs()) {
                     HashSet<LogicalVariable> v2 = new HashSet<>();
                     VariableUtilities.getLiveVariables(inp.getValue(), v2);
+                    allLiveVariables.addAll(v2);
                     if (v2.containsAll(usedInAccess)) {
                         pushAccessDown(opRef, inputOp, inp, context, finalAnnot);
                         return true;
@@ -274,6 +277,7 @@
                     for (Mutable<ILogicalOperator> root : plan.getRoots()) {
                         HashSet<LogicalVariable> v2 = new HashSet<>();
                         VariableUtilities.getLiveVariables(root.getValue(), v2);
+                        allLiveVariables.addAll(v2);
                         if (v2.containsAll(usedInAccess)) {
                             pushAccessDown(opRef, inputOp, root, context, finalAnnot);
                             return true;
@@ -281,8 +285,12 @@
                     }
                 }
             }
+            if (allLiveVariables.containsAll(usedInAccess)) {
+                // all field accesses in the assign operator have corresponding sources, but push down is not possible
+                return false;
+            }
             throw new CompilationException(ErrorCode.COMPILATION_ERROR, assignOp.getSourceLocation(),
-                    "Field access " + assignOp.getExpressions().get(0).getValue() + " doesn't correspond to any input");
+                    "Could not resolve field access " + assignOp.getExpressions().get(0).getValue());
         } else {
             // check if the accessed field is one of the partitioning key fields. If yes, we can equate the 2 variables
             if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.1.ddl.sqlpp
new file mode 100644
index 0000000..692d0fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+USE TinySocial;
+
+CREATE TYPE ChirpUserType AS {
+    screenName: string,
+    lang: string,
+    friendsCount: int,
+    statusesCount: int,
+    name: string,
+    followersCount: int
+};
+
+CREATE TYPE EmploymentType AS {
+    organizationName: string,
+    startDate: date,
+    endDate: date?
+};
+
+CREATE TYPE GleambookUserType AS {
+    id: int,
+    alias: string,
+    name: string,
+    userSince: datetime,
+    friendIds: {{ int }},
+    employment: [EmploymentType]
+};
+
+CREATE DATASET GleambookUsers(GleambookUserType) PRIMARY KEY id;
+CREATE DATASET ChirpUsers(ChirpUserType) PRIMARY KEY screenName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.2.update.sqlpp
new file mode 100644
index 0000000..fe7c165
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.2.update.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+INSERT INTO ChirpUsers
+([
+{"screenName":"NathanGiesen@211","lang":"en","friendsCount":18,"statusesCount":473,"name":"Nathan Giesen","followersCount":49416},
+{"screenName":"ColineGeyer@63","lang":"en","friendsCount":121,"statusesCount":362,"name":"Coline Geyer","followersCount":17159},
+{"screenName":"NilaMilliron_tw","lang":"en","friendsCount":445,"statusesCount":164,"name":"Nila Milliron","followersCount":22649},
+{"screenName":"ChangEwing_573","lang":"en","friendsCount":182,"statusesCount":394,"name":"Chang Ewing","followersCount":32136}
+]);
+
+INSERT INTO GleambookUsers
+([
+{"id":1,"alias":"Margarita","name":"MargaritaStoddard","nickname":"Mags","userSince":datetime("2012-08-20T10:10:00"),"friendIds":{{2,3,6,10}},"employment":[{"organizationName":"Codetechno","startDate":date("2006-08-06")},{"organizationName":"geomedia","startDate":date("2010-06-17"),"endDate":date("2010-01-26")}],"gender":"F"},
+{"id":2,"alias":"Isbel","name":"IsbelDull","nickname":"Izzy","userSince":datetime("2011-01-22T10:10:00"),"friendIds":{{1,4}},"employment":[{"organizationName":"Hexviafind","startDate":date("2010-04-27")}]},
+{"id":3,"alias":"Emory","name":"EmoryUnk","userSince":datetime("2012-07-10T10:10:00"),"friendIds":{{1,5,8,9}},"employment":[{"organizationName":"geomedia","startDate":date("2010-06-17"),"endDate":date("2010-01-26")}]},
+{"id":4,"alias":"Nicholas","name":"NicholasStroh","userSince":datetime("2010-12-27T10:10:00"),"friendIds":{{2}},"employment":[{"organizationName":"Zamcorporation","startDate":date("2010-06-08")}]},
+{"id":5,"alias":"Von","name":"VonKemble","userSince":datetime("2010-01-05T10:10:00"),"friendIds":{{3,6,10}},"employment":[{"organizationName":"Kongreen","startDate":date("2010-11-27")}]},
+{"id":6,"alias":"Willis","name":"WillisWynne","userSince":datetime("2005-01-17T10:10:00"),"friendIds":{{1,3,7}},"employment":[{"organizationName":"jaydax","startDate":date("2009-05-15")}]},
+{"id":7,"alias":"Suzanna","name":"SuzannaTillson","userSince":datetime("2012-08-07T10:10:00"),"friendIds":{{6}},"employment":[{"organizationName":"Labzatron","startDate":date("2011-04-19")}]},
+{"id":8,"alias":"Nila","name":"NilaMilliron","userSince":datetime("2008-01-01T10:10:00"),"friendIds":{{3}},"employment":[{"organizationName":"Plexlane","startDate":date("2010-02-28")}]},
+{"id":9,"alias":"Woodrow","name":"WoodrowNehling","nickname":"Woody","userSince":datetime("2005-09-20T10:10:00"),"friendIds":{{3,10}},"employment":[{"organizationName":"Zuncan","startDate":date("2003-04-22"),"endDate":date("2009-12-13")}]},
+{"id":10,"alias":"Bram","name":"BramHatch","userSince":datetime("2010-10-16T10:10:00"),"friendIds":{{1,5,9}},"employment":[{"organizationName":"physcane","startDate":date("2007-06-05"),"endDate":date("2011-11-05")}]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.3.query.sqlpp
new file mode 100644
index 0000000..727faa2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+SELECT (CASE WHEN g.id = 6 THEN g ELSE u END).name as name
+FROM ChirpUsers u JOIN GleambookUsers g ON u.name < g.name WHERE u.screenName = "ChangEwing_573"
+ORDER BY name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.4.query.sqlpp
new file mode 100644
index 0000000..78a19f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+SELECT VALUE a FROM (SELECT VALUE c FROM ChirpUsers c UNION ALL SELECT VALUE g FROM GleambookUsers g) AS a
+ORDER BY a.name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.5.query.sqlpp
new file mode 100644
index 0000000..7e60d2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.5.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+SELECT name FROM (SELECT VALUE c FROM ChirpUsers c UNION ALL SELECT VALUE g FROM GleambookUsers g) AS a
+ORDER BY name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.6.query.sqlpp
new file mode 100644
index 0000000..b3c44d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+SELECT name, string_length(name) as len
+FROM (SELECT VALUE c FROM ChirpUsers c UNION ALL SELECT VALUE g FROM GleambookUsers g) AS a
+ORDER BY name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.7.query.sqlpp
new file mode 100644
index 0000000..e7729ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access/field_access.7.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2288 and ASTERIXDB-2289 related to field access with CASE and UNION
+USE TinySocial;
+
+SELECT alias FROM (SELECT VALUE c FROM ChirpUsers c UNION ALL SELECT VALUE g FROM GleambookUsers g) AS a
+ORDER BY alias;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.3.adm
new file mode 100644
index 0000000..ec27645
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.3.adm
@@ -0,0 +1,9 @@
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "WillisWynne" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.4.adm
new file mode 100644
index 0000000..bfbb1bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.4.adm
@@ -0,0 +1,14 @@
+{ "name": "BramHatch", "id": 10, "alias": "Bram", "userSince": datetime("2010-10-16T10:10:00.000Z"), "friendIds": {{ 1, 5, 9 }}, "employment": [ { "organizationName": "physcane", "startDate": date("2007-06-05"), "endDate": date("2011-11-05") } ] }
+{ "name": "Chang Ewing", "screenName": "ChangEwing_573", "lang": "en", "friendsCount": 182, "statusesCount": 394, "followersCount": 32136 }
+{ "name": "Coline Geyer", "screenName": "ColineGeyer@63", "lang": "en", "friendsCount": 121, "statusesCount": 362, "followersCount": 17159 }
+{ "name": "EmoryUnk", "id": 3, "alias": "Emory", "userSince": datetime("2012-07-10T10:10:00.000Z"), "friendIds": {{ 1, 5, 8, 9 }}, "employment": [ { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ] }
+{ "name": "IsbelDull", "id": 2, "alias": "Isbel", "userSince": datetime("2011-01-22T10:10:00.000Z"), "friendIds": {{ 1, 4 }}, "employment": [ { "organizationName": "Hexviafind", "startDate": date("2010-04-27") } ], "nickname": "Izzy" }
+{ "name": "MargaritaStoddard", "id": 1, "alias": "Margarita", "userSince": datetime("2012-08-20T10:10:00.000Z"), "friendIds": {{ 2, 3, 6, 10 }}, "employment": [ { "organizationName": "Codetechno", "startDate": date("2006-08-06") }, { "organizationName": "geomedia", "startDate": date("2010-06-17"), "endDate": date("2010-01-26") } ], "nickname": "Mags", "gender": "F" }
+{ "name": "Nathan Giesen", "screenName": "NathanGiesen@211", "lang": "en", "friendsCount": 18, "statusesCount": 473, "followersCount": 49416 }
+{ "name": "NicholasStroh", "id": 4, "alias": "Nicholas", "userSince": datetime("2010-12-27T10:10:00.000Z"), "friendIds": {{ 2 }}, "employment": [ { "organizationName": "Zamcorporation", "startDate": date("2010-06-08") } ] }
+{ "name": "Nila Milliron", "screenName": "NilaMilliron_tw", "lang": "en", "friendsCount": 445, "statusesCount": 164, "followersCount": 22649 }
+{ "name": "NilaMilliron", "id": 8, "alias": "Nila", "userSince": datetime("2008-01-01T10:10:00.000Z"), "friendIds": {{ 3 }}, "employment": [ { "organizationName": "Plexlane", "startDate": date("2010-02-28") } ] }
+{ "name": "SuzannaTillson", "id": 7, "alias": "Suzanna", "userSince": datetime("2012-08-07T10:10:00.000Z"), "friendIds": {{ 6 }}, "employment": [ { "organizationName": "Labzatron", "startDate": date("2011-04-19") } ] }
+{ "name": "VonKemble", "id": 5, "alias": "Von", "userSince": datetime("2010-01-05T10:10:00.000Z"), "friendIds": {{ 3, 6, 10 }}, "employment": [ { "organizationName": "Kongreen", "startDate": date("2010-11-27") } ] }
+{ "name": "WillisWynne", "id": 6, "alias": "Willis", "userSince": datetime("2005-01-17T10:10:00.000Z"), "friendIds": {{ 1, 3, 7 }}, "employment": [ { "organizationName": "jaydax", "startDate": date("2009-05-15") } ] }
+{ "name": "WoodrowNehling", "id": 9, "alias": "Woodrow", "userSince": datetime("2005-09-20T10:10:00.000Z"), "friendIds": {{ 3, 10 }}, "employment": [ { "organizationName": "Zuncan", "startDate": date("2003-04-22"), "endDate": date("2009-12-13") } ], "nickname": "Woody" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.5.adm
new file mode 100644
index 0000000..8660d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.5.adm
@@ -0,0 +1,14 @@
+{ "name": "BramHatch" }
+{ "name": "Chang Ewing" }
+{ "name": "Coline Geyer" }
+{ "name": "EmoryUnk" }
+{ "name": "IsbelDull" }
+{ "name": "MargaritaStoddard" }
+{ "name": "Nathan Giesen" }
+{ "name": "NicholasStroh" }
+{ "name": "Nila Milliron" }
+{ "name": "NilaMilliron" }
+{ "name": "SuzannaTillson" }
+{ "name": "VonKemble" }
+{ "name": "WillisWynne" }
+{ "name": "WoodrowNehling" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.6.adm
new file mode 100644
index 0000000..4af8057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.6.adm
@@ -0,0 +1,14 @@
+{ "name": "BramHatch", "len": 9 }
+{ "name": "Chang Ewing", "len": 11 }
+{ "name": "Coline Geyer", "len": 12 }
+{ "name": "EmoryUnk", "len": 8 }
+{ "name": "IsbelDull", "len": 9 }
+{ "name": "MargaritaStoddard", "len": 17 }
+{ "name": "Nathan Giesen", "len": 13 }
+{ "name": "NicholasStroh", "len": 13 }
+{ "name": "Nila Milliron", "len": 13 }
+{ "name": "NilaMilliron", "len": 12 }
+{ "name": "SuzannaTillson", "len": 14 }
+{ "name": "VonKemble", "len": 9 }
+{ "name": "WillisWynne", "len": 11 }
+{ "name": "WoodrowNehling", "len": 14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.7.adm
new file mode 100644
index 0000000..ea37f8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access/field_access.7.adm
@@ -0,0 +1,14 @@
+{  }
+{  }
+{  }
+{  }
+{ "alias": "Bram" }
+{ "alias": "Emory" }
+{ "alias": "Isbel" }
+{ "alias": "Margarita" }
+{ "alias": "Nicholas" }
+{ "alias": "Nila" }
+{ "alias": "Suzanna" }
+{ "alias": "Von" }
+{ "alias": "Willis" }
+{ "alias": "Woodrow" }
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
deleted file mode 100644
index f432c9f..0000000
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushAssignBelowUnionAllRule.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.algebricks.rewriter.rules;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-/**
- * Pushes an AssignOperator below a UnionAll operator by creating an new AssignOperator below each of
- * the UnionAllOperator's branches with appropriate variable replacements.
- * This rule can help to enable other rules that are difficult to fire across a UnionAllOperator,
- * for example, eliminating common sub-expressions.
- * Example:
- * Before plan:
- * ...
- * assign [$$20, $$21] <- [funcA($$3), funcB($$6)]
- * union ($$1, $$2, $$3) ($$4, $$5, $$6)
- * union_branch_0
- * ...
- * union_branch_1
- * ...
- * After plan:
- * ...
- * union ($$1, $$2, $$3) ($$4, $$5, $$6) ($$22, $$24, $$20) ($$23, $$25, $$21)
- * assign [$$22, $$23] <- [funcA($$1), funcB($$4)]
- * union_branch_0
- * ...
- * assign [$$24, $$25] <- [funcA($$2), funcB($$5)]
- * union_branch_1
- * ...
- */
-public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        return false;
-    }
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (!op.hasInputs()) {
-            return false;
-        }
-
-        boolean modified = false;
-        for (int i = 0; i < op.getInputs().size(); i++) {
-            AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
-            if (childOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-                continue;
-            }
-            AssignOperator assignOp = (AssignOperator) childOp;
-            for (Mutable<ILogicalExpression> expr : assignOp.getExpressions()) {
-                if (!expr.getValue().isFunctional()) {
-                    return false;
-                }
-            }
-
-            AbstractLogicalOperator childOfChildOp = (AbstractLogicalOperator) assignOp.getInputs().get(0).getValue();
-            if (childOfChildOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
-                continue;
-            }
-            UnionAllOperator unionOp = (UnionAllOperator) childOfChildOp;
-
-            Set<LogicalVariable> assignUsedVars = new HashSet<LogicalVariable>();
-            VariableUtilities.getUsedVariables(assignOp, assignUsedVars);
-
-            List<LogicalVariable> assignVars = assignOp.getVariables();
-
-            AssignOperator[] newAssignOps = new AssignOperator[2];
-            for (int j = 0; j < unionOp.getInputs().size(); j++) {
-                newAssignOps[j] = createAssignBelowUnionAllBranch(unionOp, j, assignOp, assignUsedVars, context);
-            }
-            // Add original assign variables to the union variable mappings.
-            for (int j = 0; j < assignVars.size(); j++) {
-                LogicalVariable first = newAssignOps[0].getVariables().get(j);
-                LogicalVariable second = newAssignOps[1].getVariables().get(j);
-                Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping =
-                        new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(first, second, assignVars.get(j));
-                unionOp.getVariableMappings().add(varMapping);
-            }
-            context.computeAndSetTypeEnvironmentForOperator(unionOp);
-
-            // Remove original assign operator.
-            op.getInputs().set(i, assignOp.getInputs().get(0));
-            context.computeAndSetTypeEnvironmentForOperator(op);
-            modified = true;
-        }
-
-        return modified;
-    }
-
-    private AssignOperator createAssignBelowUnionAllBranch(UnionAllOperator unionOp, int inputIndex,
-            AssignOperator originalAssignOp, Set<LogicalVariable> assignUsedVars, IOptimizationContext context)
-            throws AlgebricksException {
-        AssignOperator newAssignOp = cloneAssignOperator(originalAssignOp, context);
-        newAssignOp.getInputs()
-                .add(new MutableObject<ILogicalOperator>(unionOp.getInputs().get(inputIndex).getValue()));
-        unionOp.getInputs().get(inputIndex).setValue(newAssignOp);
-        int numVarMappings = unionOp.getVariableMappings().size();
-        for (int i = 0; i < numVarMappings; i++) {
-            Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping = unionOp.getVariableMappings().get(i);
-            if (assignUsedVars.contains(varMapping.third)) {
-                LogicalVariable replacementVar;
-                if (inputIndex == 0) {
-                    replacementVar = varMapping.first;
-                } else {
-                    replacementVar = varMapping.second;
-                }
-                VariableUtilities.substituteVariables(newAssignOp, varMapping.third, replacementVar, context);
-            }
-        }
-        context.computeAndSetTypeEnvironmentForOperator(newAssignOp);
-        return newAssignOp;
-    }
-
-    /**
-     * Clones the given assign operator changing the returned variables to be new ones.
-     * Also, leaves the inputs of the clone clear.
-     */
-    private AssignOperator cloneAssignOperator(AssignOperator assignOp, IOptimizationContext context) {
-        List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-        List<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
-        int numVars = assignOp.getVariables().size();
-        for (int i = 0; i < numVars; i++) {
-            vars.add(context.newVar());
-            exprs.add(new MutableObject<ILogicalExpression>(
-                    assignOp.getExpressions().get(i).getValue().cloneExpression()));
-        }
-        AssignOperator assignCloneOp = new AssignOperator(vars, exprs);
-        assignCloneOp.setSourceLocation(assignOp.getSourceLocation());
-        assignCloneOp.setExecutionMode(assignOp.getExecutionMode());
-        return assignCloneOp;
-    }
-}

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://cbjenkins.page.link/eXLj4ckr7xideTKy9 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 11:06:30 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/966/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:26 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:44:04 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/968/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Dmitry Lychagin (Code Review)" <de...@asterixdb.apache.org>.
Dmitry Lychagin has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4: Code-Review+2

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3399/4/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java:

https://asterix-gerrit.ics.uci.edu/#/c/3399/4/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java@202
PS4, Line 202:         HashSet<LogicalVariable> usedInAccess = new HashSet<>();
minor. use interface in variable declaration (Set<LogicalVariable> usedInAccess = new HashSet<>()) here and below



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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 23:12:04 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5789/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:19 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:22:50 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:05:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:49:59 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1157/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:55 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1158/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:31:58 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6380/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:56:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:42:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6020/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:57:37 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/nW4JgtzCz2htddfn9 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:23:31 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1156/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:09:11 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:46 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:46 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6010/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:23 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:50:18 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:44:10 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/967/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:51 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3801/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:34 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:51:26 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/570/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:46 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1165/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 19:01:01 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:37 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:29 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4: Contrib+1

BAD Compatibility Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 19:04:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9748/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:26 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/579/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:44:24 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/976/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:44:10 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1064/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:25 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11279/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1065/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:20:02 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1066/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:30 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Jenkins, 

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

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

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

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................

[ASTERIXDB-2289][COMP] Fix field access with CASE

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

Details:
This patch fixes field access in the presense CASE and JOIN.
This is a scenario where push-down-field-access rule throws an exception
if the field access has potentially two sources and it could not
push down the field access to left or right branch. The exception
thinks that the field access has no source when in reality it has
but there were two potential sources because of CASE. Don't throw
an exception if the field access is a valid one and return false
(i.e. field access was not push) instead of throwing an exception.

Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
6 files changed, 143 insertions(+), 3 deletions(-)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/729/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:46 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:02:34 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:04:42 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/569/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 19:14:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/CjENh7ULebmwUb5JA : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:47:01 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/728/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:25 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5704/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:23 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:24 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:53:35 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5807/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 10:39:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:42:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 20:27:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5706/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:12 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Ali Alsuliman has submitted this change and it was merged. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................

[ASTERIXDB-2289][COMP] Fix field access with CASE

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

Details:
This patch fixes field access in the presense CASE and JOIN.
This is a scenario where push-down-field-access rule throws an
exception if the field access has potentially two sources and
it could not push down the field access to left or right branch.
Don't throw an exception and just return false
(i.e. field access was not pushed) instead of throwing an exception.

Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3399
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
8 files changed, 162 insertions(+), 24 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Dmitry Lychagin: Looks good to me, approved

Objections:
  Jenkins: Violations found
  Anon. E. Moose (1000171): Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
index 421b00d..c82aa33 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -84,6 +84,9 @@
         if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
             return false;
         }
+        if (!OperatorPropertiesUtil.isMovable(op)) {
+            return false;
+        }
         AssignOperator access = (AssignOperator) op;
         ILogicalExpression expr = getFirstExpr(access);
         String finalAnnot;
@@ -196,17 +199,17 @@
             pushDownFieldAccessRec(opRef2, context, finalAnnot);
             return true;
         }
-        List<LogicalVariable> usedInAccess = new LinkedList<>();
+        HashSet<LogicalVariable> usedInAccess = new HashSet<>();
         VariableUtilities.getUsedVariables(assignOp, usedInAccess);
 
-        List<LogicalVariable> produced2 = new LinkedList<>();
+        HashSet<LogicalVariable> produced2 = new HashSet<>();
         if (inputOp.getOperatorTag() == LogicalOperatorTag.GROUP) {
             VariableUtilities.getLiveVariables(inputOp, produced2);
         } else {
             VariableUtilities.getProducedVariables(inputOp, produced2);
         }
         boolean pushItDown = false;
-        List<LogicalVariable> inter = new ArrayList<>(usedInAccess);
+        HashSet<LogicalVariable> inter = new HashSet<>(usedInAccess);
         if (inter.isEmpty()) { // ground value
             return false;
         }
@@ -234,8 +237,7 @@
                     LogicalVariable oldVar = assignOp.getVariables().get(0);
                     VariableReferenceExpression v2Ref = new VariableReferenceExpression(v2);
                     v2Ref.setSourceLocation(g.getSourceLocation());
-                    g.getDecorList().add(new Pair<LogicalVariable, Mutable<ILogicalExpression>>(oldVar,
-                            new MutableObject<ILogicalExpression>(v2Ref)));
+                    g.getDecorList().add(new Pair<>(oldVar, new MutableObject<>(v2Ref)));
                     changed = true;
                     assignOp.getVariables().set(0, v2);
                     VariableUtilities.substituteVariables(assignOp, m.first, m.second, context);
@@ -281,8 +283,7 @@
                     }
                 }
             }
-            throw new CompilationException(ErrorCode.COMPILATION_ERROR, assignOp.getSourceLocation(),
-                    "Field access " + assignOp.getExpressions().get(0).getValue() + " doesn't correspond to any input");
+            return false;
         } else {
             // check if the accessed field is one of the partitioning key fields. If yes, we can equate the 2 variables
             if (inputOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
new file mode 100644
index 0000000..46e1f02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+// testing fix for ASTERIXDB-2289 related to field access with CASE
+DROP DATAVERSE TinySocial IF EXISTS;
+CREATE DATAVERSE TinySocial;
+USE TinySocial;
+
+CREATE TYPE ChirpUserType AS {
+    screenName: string,
+    lang: string,
+    friendsCount: int,
+    statusesCount: int,
+    name: string,
+    followersCount: int
+};
+
+CREATE TYPE EmploymentType AS {
+    organizationName: string,
+    startDate: date,
+    endDate: date?
+};
+
+CREATE TYPE GleambookUserType AS {
+    id: int,
+    alias: string,
+    name: string,
+    userSince: datetime,
+    friendIds: {{ int }},
+    employment: [EmploymentType]
+};
+
+CREATE DATASET GleambookUsers(GleambookUserType) PRIMARY KEY id;
+CREATE DATASET ChirpUsers(ChirpUserType) PRIMARY KEY screenName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
new file mode 100644
index 0000000..bcfcd32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2289 related to field access with CASE
+USE TinySocial;
+
+INSERT INTO ChirpUsers
+([
+{"screenName":"NathanGiesen@211","lang":"en","friendsCount":18,"statusesCount":473,"name":"Nathan Giesen","followersCount":49416},
+{"screenName":"ColineGeyer@63","lang":"en","friendsCount":121,"statusesCount":362,"name":"Coline Geyer","followersCount":17159},
+{"screenName":"NilaMilliron_tw","lang":"en","friendsCount":445,"statusesCount":164,"name":"Nila Milliron","followersCount":22649},
+{"screenName":"ChangEwing_573","lang":"en","friendsCount":182,"statusesCount":394,"name":"Chang Ewing","followersCount":32136}
+]);
+
+INSERT INTO GleambookUsers
+([
+{"id":1,"alias":"Margarita","name":"MargaritaStoddard","nickname":"Mags","userSince":datetime("2012-08-20T10:10:00"),"friendIds":{{2,3,6,10}},"employment":[{"organizationName":"Codetechno","startDate":date("2006-08-06")},{"organizationName":"geomedia","startDate":date("2010-06-17"),"endDate":date("2010-01-26")}],"gender":"F"},
+{"id":2,"alias":"Isbel","name":"IsbelDull","nickname":"Izzy","userSince":datetime("2011-01-22T10:10:00"),"friendIds":{{1,4}},"employment":[{"organizationName":"Hexviafind","startDate":date("2010-04-27")}]},
+{"id":3,"alias":"Emory","name":"EmoryUnk","userSince":datetime("2012-07-10T10:10:00"),"friendIds":{{1,5,8,9}},"employment":[{"organizationName":"geomedia","startDate":date("2010-06-17"),"endDate":date("2010-01-26")}]},
+{"id":4,"alias":"Nicholas","name":"NicholasStroh","userSince":datetime("2010-12-27T10:10:00"),"friendIds":{{2}},"employment":[{"organizationName":"Zamcorporation","startDate":date("2010-06-08")}]},
+{"id":5,"alias":"Von","name":"VonKemble","userSince":datetime("2010-01-05T10:10:00"),"friendIds":{{3,6,10}},"employment":[{"organizationName":"Kongreen","startDate":date("2010-11-27")}]},
+{"id":6,"alias":"Willis","name":"WillisWynne","userSince":datetime("2005-01-17T10:10:00"),"friendIds":{{1,3,7}},"employment":[{"organizationName":"jaydax","startDate":date("2009-05-15")}]},
+{"id":7,"alias":"Suzanna","name":"SuzannaTillson","userSince":datetime("2012-08-07T10:10:00"),"friendIds":{{6}},"employment":[{"organizationName":"Labzatron","startDate":date("2011-04-19")}]},
+{"id":8,"alias":"Nila","name":"NilaMilliron","userSince":datetime("2008-01-01T10:10:00"),"friendIds":{{3}},"employment":[{"organizationName":"Plexlane","startDate":date("2010-02-28")}]},
+{"id":9,"alias":"Woodrow","name":"WoodrowNehling","nickname":"Woody","userSince":datetime("2005-09-20T10:10:00"),"friendIds":{{3,10}},"employment":[{"organizationName":"Zuncan","startDate":date("2003-04-22"),"endDate":date("2009-12-13")}]},
+{"id":10,"alias":"Bram","name":"BramHatch","userSince":datetime("2010-10-16T10:10:00"),"friendIds":{{1,5,9}},"employment":[{"organizationName":"physcane","startDate":date("2007-06-05"),"endDate":date("2011-11-05")}]}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
new file mode 100644
index 0000000..d9a5644
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// testing fix for ASTERIXDB-2289 related to field access with CASE
+USE TinySocial;
+
+SELECT (CASE WHEN g.id = 6 THEN g ELSE u END).name as name
+FROM ChirpUsers u JOIN GleambookUsers g ON u.name < g.name WHERE u.screenName = "ChangEwing_573"
+ORDER BY name;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.4.ddl.sqlpp
new file mode 100644
index 0000000..da31d1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.4.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+// testing fix for ASTERIXDB-2289 related to field access with CASE
+DROP DATAVERSE TinySocial IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
new file mode 100644
index 0000000..ec27645
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
@@ -0,0 +1,9 @@
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "Chang Ewing" }
+{ "name": "WillisWynne" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 45f86ec..9055c85 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5625,6 +5625,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="field_access-ASTERIXDB-2289">
+        <output-dir compare="Text">field_access-ASTERIXDB-2289</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="comp-ASTERIXDB-2415">
         <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
       </compilation-unit>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index f43d367..49d4748 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -311,7 +311,7 @@
                 // Can't move nonPures!
                 AssignOperator assign = (AssignOperator) op;
                 for (Mutable<ILogicalExpression> expr : assign.getExpressions()) {
-                    if (containsNonpureCall(expr.getValue())) {
+                    if (!expr.getValue().isFunctional()) {
                         return false;
                     }
                 }
@@ -321,22 +321,6 @@
                 return false;
         }
         return true;
-    }
-
-    private static boolean containsNonpureCall(ILogicalExpression expr) {
-        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression fExpr = (AbstractFunctionCallExpression) expr;
-            if (!fExpr.getFunctionInfo().isFunctional()) {
-                return true;
-            }
-            for (Mutable<ILogicalExpression> subExpr : fExpr.getArguments()) {
-                if (containsNonpureCall(subExpr.getValue())) {
-                    return true;
-                }
-            }
-
-        }
-        return false;
     }
 
     /**

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 5
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:34 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/bKAE6bWRcxbcz62KA : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 22:18:11 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:25:36 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1074/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:53:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6372/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:24 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:32 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5806/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5777/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:42:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Jenkins, 

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

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

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

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................

[ASTERIXDB-2289][COMP] Fix field access with CASE

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

Details:
This patch fixes field access in the presense CASE and JOIN.
This is a scenario where push-down-field-access rule throws an exception
if the field access has potentially two sources and it could not
push down the field access to left or right branch. The exception
thinks that the field access has no source when in reality it has
but there were two potential sources because of CASE. Don't throw
an exception if the field access is a valid one and return false
(i.e. field access was not push) instead of throwing an exception.

Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
6 files changed, 143 insertions(+), 3 deletions(-)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:36 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5714/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:42:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6371/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:20:01 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:27:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/hueHeyQVEeJHnMa96 : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 20:09:58 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3809/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:57:48 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6012/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:22 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/ao4iy6YVMBaznzdk8 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:27:49 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:54 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/e7cHo3CjWDZpaACp6 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:05:30 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/730/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:21:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6370/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:24 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/738/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:42:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Till Westmann, Jenkins, Michael Blow, Dmitry Lychagin, 

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

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

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

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................

[ASTERIXDB-2289][COMP] Fix field access with CASE

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

Details:
This patch fixes field access in the presense CASE and JOIN.
This is a scenario where push-down-field-access rule throws an
exception if the field access has potentially two sources and
it could not push down the field access to left or right branch.
Don't throw an exception and just return false
(i.e. field access was not pushed) instead of throwing an exception.

Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.4.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access-ASTERIXDB-2289/field_access-ASTERIXDB-2289.3.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
8 files changed, 162 insertions(+), 24 deletions(-)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 4:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Thu, 16 May 2019 18:44:25 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with UNION and CASE
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5767/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:01:41 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9747/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:19:59 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2289][COMP] Fix field access with CASE

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3399 )

Change subject: [ASTERIXDB-2289][COMP] Fix field access with CASE
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/571/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I911e4e9018c15e8f226e46fa610d222eb2301fcd
Gerrit-Change-Number: 3399
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 16 May 2019 09:24:12 +0000
Gerrit-HasComments: No