You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2019/05/21 17:33:11 UTC

[asterixdb] branch master updated: [ASTERIXDB-2288][COMP] Fix field access with UNION

This is an automated email from the ASF dual-hosted git repository.

alsuliman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 249c955  [ASTERIXDB-2288][COMP] Fix field access with UNION
249c955 is described below

commit 249c95599c6ca1a61de81ec0a80374eed0cd0784
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Mon May 20 20:33:34 2019 -0700

    [ASTERIXDB-2288][COMP] Fix field access with UNION
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    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 map the
    index to the correct index of the input operator where the assign
    is copied to.
    
    Change-Id: I4d65374aca6959fd5e70279fac8f7f0a6c7c64c5
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3400
    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>
---
 .../asterix/optimizer/base/RuleCollections.java    |  4 +-
 .../rules/AsterixPushAssignBelowUnionAllRule.java  | 97 ++++++++++++++++++++++
 .../field_access_union-ASTERIXDB-2288.1.ddl.sqlpp  | 49 +++++++++++
 ...ield_access_union-ASTERIXDB-2288.2.update.sqlpp | 42 ++++++++++
 ...field_access_union-ASTERIXDB-2288.3.query.sqlpp | 24 ++++++
 ...field_access_union-ASTERIXDB-2288.4.query.sqlpp | 24 ++++++
 ...field_access_union-ASTERIXDB-2288.5.query.sqlpp | 25 ++++++
 ...field_access_union-ASTERIXDB-2288.6.query.sqlpp | 24 ++++++
 .../field_access_union-ASTERIXDB-2288.7.ddl.sqlpp  | 20 +++++
 .../field_access_union-ASTERIXDB-2288.3.adm        | 14 ++++
 .../field_access_union-ASTERIXDB-2288.4.adm        | 14 ++++
 .../field_access_union-ASTERIXDB-2288.5.adm        | 14 ++++
 .../field_access_union-ASTERIXDB-2288.6.adm        | 14 ++++
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  5 ++
 .../rules/PushAssignBelowUnionAllRule.java         | 48 ++++++-----
 15 files changed, 397 insertions(+), 21 deletions(-)

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..f9aecab 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
@@ -30,6 +30,7 @@ import org.apache.asterix.optimizer.rules.AsterixConsolidateWindowOperatorsRule;
 import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
 import org.apache.asterix.optimizer.rules.AsterixIntroduceGroupByCombinerRule;
+import org.apache.asterix.optimizer.rules.AsterixPushAssignBelowUnionAllRule;
 import org.apache.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import org.apache.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import org.apache.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
@@ -117,7 +118,6 @@ import org.apache.hyracks.algebricks.rewriter.rules.IntroduceAggregateCombinerRu
 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;
@@ -330,7 +330,7 @@ public final class RuleCollections {
     public static final List<IAlgebraicRewriteRule> buildPlanCleanupRuleCollection() {
         List<IAlgebraicRewriteRule> planCleanupRules = new LinkedList<>();
         planCleanupRules.add(new SwitchInnerJoinBranchRule());
-        planCleanupRules.add(new PushAssignBelowUnionAllRule());
+        planCleanupRules.add(new AsterixPushAssignBelowUnionAllRule());
         planCleanupRules.add(new ExtractCommonExpressionsRule());
         planCleanupRules.add(new RemoveRedundantVariablesRule());
         planCleanupRules.add(new PushProjectDownRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixPushAssignBelowUnionAllRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixPushAssignBelowUnionAllRule.java
new file mode 100644
index 0000000..23ff7c6
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixPushAssignBelowUnionAllRule.java
@@ -0,0 +1,97 @@
+/*
+ * 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.List;
+
+import org.apache.asterix.om.base.AInt32;
+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.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+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.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
+
+public class AsterixPushAssignBelowUnionAllRule extends PushAssignBelowUnionAllRule {
+
+    // modifies field-access-by-index by adjusting the index to match the one in the branch where assign is moved to
+    @Override
+    protected boolean modifyExpression(ILogicalExpression expression, UnionAllOperator unionOp,
+            IOptimizationContext ctx, int inputIndex) throws AlgebricksException {
+        if (expression.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return true; // expressions other than functions need not be modified
+        }
+        AbstractFunctionCallExpression functionCall = (AbstractFunctionCallExpression) expression;
+        List<Mutable<ILogicalExpression>> arguments = functionCall.getArguments();
+        for (int k = 0, size = arguments.size(); k < size; k++) {
+            if (!modifyExpression(arguments.get(k).getValue(), unionOp, ctx, inputIndex)) {
+                return false;
+            }
+        }
+        // return true if any function other than field-access-by-index. otherwise, try to map the index.
+        return !functionCall.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_INDEX)
+                || mapFieldIndex(functionCall, unionOp, ctx, inputIndex);
+    }
+
+    private static boolean mapFieldIndex(AbstractFunctionCallExpression functionCall, UnionAllOperator unionOp,
+            IOptimizationContext ctx, int inputIndex) throws AlgebricksException {
+        // 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 false;
+        }
+        Integer fieldIndex = ConstantExpressionUtil.getIntArgument(functionCall, 1);
+        if (fieldIndex == null) {
+            return false;
+        }
+        LogicalVariable recordVar = ((VariableReferenceExpression) recordExpr).getVariableReference();
+        for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMap : unionOp.getVariableMappings()) {
+            if (varMap.third.equals(recordVar)) {
+                ARecordType unionType = (ARecordType) ctx.getOutputTypeEnvironment(unionOp).getVarType(varMap.third);
+                ILogicalOperator inputOpToUnion = unionOp.getInputs().get(inputIndex).getValue();
+                ARecordType inputType;
+                if (inputIndex == 0) {
+                    inputType = (ARecordType) ctx.getOutputTypeEnvironment(inputOpToUnion).getVarType(varMap.first);
+                } else {
+                    inputType = (ARecordType) ctx.getOutputTypeEnvironment(inputOpToUnion).getVarType(varMap.second);
+                }
+                String fieldName = unionType.getFieldNames()[fieldIndex];
+                fieldIndex = inputType.getFieldIndex(fieldName);
+                if (fieldIndex < 0) {
+                    return false;
+                }
+                functionCall.getArguments().get(1)
+                        .setValue(new ConstantExpression(new AsterixConstantValue(new AInt32(fieldIndex))));
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.1.ddl.sqlpp
new file mode 100644
index 0000000..a01b14b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+
+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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.2.update.sqlpp
new file mode 100644
index 0000000..efd7565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.2.update.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE 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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.query.sqlpp
new file mode 100644
index 0000000..2e6c3a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.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 related to field access with 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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.4.query.sqlpp
new file mode 100644
index 0000000..c222d57
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.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 related to field access with 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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.query.sqlpp
new file mode 100644
index 0000000..922012b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.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 related to field access with 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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.query.sqlpp
new file mode 100644
index 0000000..8447214
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.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 related to field access with 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/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.7.ddl.sqlpp
new file mode 100644
index 0000000..97845a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.7.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.adm
new file mode 100644
index 0000000..bfbb1bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.3.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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.4.adm
new file mode 100644
index 0000000..8660d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.4.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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.adm
new file mode 100644
index 0000000..4af8057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.5.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_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.adm
new file mode 100644
index 0000000..ea37f8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/field_access_union-ASTERIXDB-2288/field_access_union-ASTERIXDB-2288.6.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/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 9055c85..3ea99ee 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -5640,6 +5640,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="field_access_union-ASTERIXDB-2288">
+        <output-dir compare="Text">field_access_union-ASTERIXDB-2288</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="constant_folding">
         <output-dir compare="Text">constant_folding</output-dir>
       </compilation-unit>
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
index f432c9f..145761c 100644
--- 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
@@ -79,7 +79,7 @@ public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
         }
 
         boolean modified = false;
-        for (int i = 0; i < op.getInputs().size(); i++) {
+        inputs_loop: for (int i = 0; i < op.getInputs().size(); i++) {
             AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
             if (childOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
                 continue;
@@ -87,7 +87,7 @@ public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
             AssignOperator assignOp = (AssignOperator) childOp;
             for (Mutable<ILogicalExpression> expr : assignOp.getExpressions()) {
                 if (!expr.getValue().isFunctional()) {
-                    return false;
+                    continue inputs_loop;
                 }
             }
 
@@ -96,26 +96,23 @@ public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
                 continue;
             }
             UnionAllOperator unionOp = (UnionAllOperator) childOfChildOp;
-
-            Set<LogicalVariable> assignUsedVars = new HashSet<LogicalVariable>();
+            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);
+                if (newAssignOps[j] == null) {
+                    continue inputs_loop;
+                }
             }
             // 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);
+                unionOp.getVariableMappings().add(new Triple<>(first, second, assignVars.get(j)));
             }
             context.computeAndSetTypeEnvironmentForOperator(unionOp);
-
             // Remove original assign operator.
             op.getInputs().set(i, assignOp.getInputs().get(0));
             context.computeAndSetTypeEnvironmentForOperator(op);
@@ -128,9 +125,11 @@ public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
     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()));
+        AssignOperator newAssignOp = cloneAssignOperator(originalAssignOp, context, unionOp, inputIndex);
+        if (newAssignOp == null) {
+            return null;
+        }
+        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++) {
@@ -151,20 +150,31 @@ public class PushAssignBelowUnionAllRule implements IAlgebraicRewriteRule {
 
     /**
      * Clones the given assign operator changing the returned variables to be new ones.
-     * Also, leaves the inputs of the clone clear.
+     * Also, leaves the inputs of the clone clear. It returns null if the assign operator cannot be pushed.
      */
-    private AssignOperator cloneAssignOperator(AssignOperator assignOp, IOptimizationContext context) {
-        List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-        List<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+    private AssignOperator cloneAssignOperator(AssignOperator assignOp, IOptimizationContext context,
+            UnionAllOperator unionOp, int inputIndex) 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());
-            exprs.add(new MutableObject<ILogicalExpression>(
-                    assignOp.getExpressions().get(i).getValue().cloneExpression()));
+            ILogicalExpression clonedExpression = assignOp.getExpressions().get(i).getValue().cloneExpression();
+            if (!modifyExpression(clonedExpression, unionOp, context, inputIndex)) {
+                return null; // bail if the expression couldn't be modified according to the branch it is moved to
+            }
+            exprs.add(new MutableObject<>(clonedExpression));
         }
         AssignOperator assignCloneOp = new AssignOperator(vars, exprs);
         assignCloneOp.setSourceLocation(assignOp.getSourceLocation());
         assignCloneOp.setExecutionMode(assignOp.getExecutionMode());
         return assignCloneOp;
     }
+
+    // modifies the cloned expression according to the branch it'll be moved to. returns true if successful.
+    protected boolean modifyExpression(ILogicalExpression expression, UnionAllOperator unionOp,
+            IOptimizationContext ctx, int inputIndex) throws AlgebricksException {
+        // default implementation does not check specific expressions
+        return true;
+    }
 }