You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by gg...@apache.org on 2021/07/28 17:25:44 UTC

[asterixdb] branch master updated: [ASTERIXDB-2927][SQL] SOME AND EVERY quantified expressions

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

ggalvizo 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 21e5282  [ASTERIXDB-2927][SQL] SOME AND EVERY quantified expressions
21e5282 is described below

commit 21e52822c08f231278110879009d36b6a3492e43
Author: ggalvizo <gg...@uci.edu>
AuthorDate: Tue Jul 27 10:26:31 2021 -1000

    [ASTERIXDB-2927][SQL] SOME AND EVERY quantified expressions
    
    - user mode changes: yes
    - storage format changes: no
    - interface changes: no
    
    User now has the option to use "ANY/SOME AND EVERY" in addition to ANY/SOME
    and EVERY for quantified expressions. "ANY/SOME AND EVERY" is similar to
    "EVERY", except we require that at least one item exists in the
    quantified array / multiset (i.e. no empty sets in the result).
    
    Change-Id: I66315f69eb1579a4c450cc3d0e97b39a03fe5194
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12323
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../translator/LangExpressionToPlanTranslator.java | 181 +++++++++++++++------
 .../optimizerts/queries/any-and-every-1.sqlpp      |  34 ++++
 .../optimizerts/queries/any-and-every-2.sqlpp      |  37 +++++
 .../optimizerts/queries/any-and-every-3.sqlpp      |  34 ++++
 .../optimizerts/results/any-and-every-1.plan       |  19 +++
 .../optimizerts/results/any-and-every-2.plan       |  26 +++
 .../optimizerts/results/any-and-every-3.plan       |  20 +++
 .../any_and_every_01/any_and_every_01.1.ddl.sqlpp  |  29 ++++
 .../any_and_every_01.2.update.sqlpp                |  29 ++++
 .../any_and_every_01.3.query.sqlpp                 |  25 +++
 .../any_and_every_01.4.query.sqlpp                 |  25 +++
 .../any_and_every_01/any_and_every_01.1.adm        |   2 +
 .../any_and_every_01/any_and_every_01.2.adm        |   2 +
 .../test/resources/runtimets/testsuite_sqlpp.xml   |   5 +
 asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf  |   2 +-
 .../asterix-doc/src/main/markdown/sqlpp/2_expr.md  |   2 +-
 .../common/expression/QuantifiedExpression.java    |   1 +
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj    |   6 +-
 .../runtime/formats/NonTaggedDataFormat.java       |   5 +-
 .../algebra/expressions/ConstantExpression.java    |  27 +++
 20 files changed, 456 insertions(+), 55 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index b76fc3e..d664da1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -1326,7 +1326,20 @@ abstract class LangExpressionToPlanTranslator
     public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         SourceLocation sourceLoc = qe.getSourceLocation();
-        Mutable<ILogicalOperator> topOp = tupSource;
+
+        Mutable<ILogicalOperator> topOp;
+        SubplanOperator subplanOp;
+        if (qe.getQuantifier() == Quantifier.SOME_AND_EVERY) {
+            subplanOp = new SubplanOperator();
+            subplanOp.getInputs().add(tupSource);
+            subplanOp.setSourceLocation(sourceLoc);
+            NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
+            ntsOp.setSourceLocation(sourceLoc);
+            topOp = new MutableObject<>(ntsOp);
+        } else {
+            subplanOp = null; // not used
+            topOp = tupSource;
+        }
 
         ILogicalOperator firstOp = null;
         Mutable<ILogicalOperator> lastOp = null;
@@ -1357,47 +1370,104 @@ abstract class LangExpressionToPlanTranslator
 
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = langExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
 
-        AggregateFunctionCallExpression fAgg;
-        SelectOperator s;
-        if (qe.getQuantifier() == Quantifier.SOME) {
-            s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
-            s.getInputs().add(eo2.second);
-            s.setSourceLocation(sourceLoc);
-            fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM,
-                    new ArrayList<>());
-            fAgg.setSourceLocation(sourceLoc);
-        } else { // EVERY
-            // look for input items that do not satisfy the condition, if none found then return true
-            // when inverting the condition account for NULL/MISSING by replacing them with FALSE
-            // condition() -> not(if-missing-or-null(condition(), false))
-
-            List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
-            ConstantExpression eFalse = new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
-            eFalse.setSourceLocation(sourceLoc);
-            ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
-            ifMissingOrNullArgs.add(new MutableObject<>(eFalse));
-
-            List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
-            ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL), ifMissingOrNullArgs);
-            ifMissinOrNullExpr.setSourceLocation(sourceLoc);
-            notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
-
-            ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
-            notExpr.setSourceLocation(sourceLoc);
-            s = new SelectOperator(new MutableObject<>(notExpr), false, null);
-            s.getInputs().add(eo2.second);
-            s.setSourceLocation(sourceLoc);
-            fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM, new ArrayList<>());
-            fAgg.setSourceLocation(sourceLoc);
-        }
-        LogicalVariable qeVar = context.newVar();
-        AggregateOperator a =
-                new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
-        a.getInputs().add(new MutableObject<>(s));
-        a.setSourceLocation(sourceLoc);
-        return new Pair<>(a, qeVar);
+        switch (qe.getQuantifier()) {
+            case SOME:
+                SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
+                s.getInputs().add(eo2.second);
+                s.setSourceLocation(sourceLoc);
+                AggregateFunctionCallExpression fAgg = BuiltinFunctions
+                        .makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM, new ArrayList<>(0));
+                fAgg.setSourceLocation(sourceLoc);
+                LogicalVariable qeVar = context.newVar();
+                AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+                        mkSingletonArrayList(new MutableObject<>(fAgg)));
+                a.getInputs().add(new MutableObject<>(s));
+                a.setSourceLocation(sourceLoc);
+                return new Pair<>(a, qeVar);
+            case EVERY:
+                // look for input items that do not satisfy the condition, if none found then return true
+                // when inverting the condition account for NULL/MISSING by replacing them with FALSE:
+                // condition() -> not(if-missing-or-null(condition(), false))
+                List<Mutable<ILogicalExpression>> ifMissingOrNullArgs = new ArrayList<>(2);
+                ifMissingOrNullArgs.add(new MutableObject<>(eo2.first));
+                ifMissingOrNullArgs.add(new MutableObject<>(ConstantExpression.FALSE));
+                List<Mutable<ILogicalExpression>> notArgs = new ArrayList<>(1);
+                ScalarFunctionCallExpression ifMissinOrNullExpr = new ScalarFunctionCallExpression(
+                        BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.IF_MISSING_OR_NULL),
+                        ifMissingOrNullArgs);
+                ifMissinOrNullExpr.setSourceLocation(sourceLoc);
+                notArgs.add(new MutableObject<>(ifMissinOrNullExpr));
+                ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
+                        BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
+                notExpr.setSourceLocation(sourceLoc);
+                s = new SelectOperator(new MutableObject<>(notExpr), false, null);
+                s.getInputs().add(eo2.second);
+                s.setSourceLocation(sourceLoc);
+                fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM,
+                        new ArrayList<>());
+                fAgg.setSourceLocation(sourceLoc);
+                qeVar = context.newVar();
+                a = new AggregateOperator(mkSingletonArrayList(qeVar), mkSingletonArrayList(new MutableObject<>(fAgg)));
+                a.getInputs().add(new MutableObject<>(s));
+                a.setSourceLocation(sourceLoc);
+                return new Pair<>(a, qeVar);
+            case SOME_AND_EVERY:
+                // return true if the stream was non-empty but there were no items that satisfied the condition
+                AbstractFunctionCallExpression fAgg1 = BuiltinFunctions
+                        .makeAggregateFunctionExpression(BuiltinFunctions.NON_EMPTY_STREAM, new ArrayList<>(0));
+                fAgg1.setSourceLocation(sourceLoc);
+
+                List<Mutable<ILogicalExpression>> switchCaseArgs = new ArrayList<>(4);
+                switchCaseArgs.add(new MutableObject<>(eo2.first));
+                switchCaseArgs.add(new MutableObject<>(ConstantExpression.TRUE));
+                switchCaseArgs.add(new MutableObject<>(ConstantExpression.NULL));
+                switchCaseArgs.add(new MutableObject<>(ConstantExpression.TRUE));
+                ScalarFunctionCallExpression switchCaseExpr = new ScalarFunctionCallExpression(
+                        BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SWITCH_CASE), switchCaseArgs);
+                switchCaseExpr.setSourceLocation(sourceLoc);
+
+                AbstractFunctionCallExpression fAgg2 = BuiltinFunctions.makeAggregateFunctionExpression(
+                        BuiltinFunctions.SQL_COUNT, mkSingletonArrayList(new MutableObject<>(switchCaseExpr)));
+                fAgg2.setSourceLocation(sourceLoc);
+
+                LogicalVariable qeVar1 = context.newVar();
+                LogicalVariable qeVar2 = context.newVar();
+                List<LogicalVariable> qeVarList = new ArrayList<>(2);
+                List<Mutable<ILogicalExpression>> fAggList = new ArrayList<>(2);
+                qeVarList.add(qeVar1);
+                qeVarList.add(qeVar2);
+                fAggList.add(new MutableObject<>(fAgg1));
+                fAggList.add(new MutableObject<>(fAgg2));
+
+                a = new AggregateOperator(qeVarList, fAggList);
+                a.getInputs().add(eo2.second);
+                a.setSourceLocation(sourceLoc);
+
+                subplanOp.setRootOp(new MutableObject<>(a));
+
+                VariableReferenceExpression qeVar1Ref = new VariableReferenceExpression(qeVar1);
+                qeVar1Ref.setSourceLocation(sourceLoc);
+
+                VariableReferenceExpression qeVar2Ref = new VariableReferenceExpression(qeVar2);
+                qeVar2Ref.setSourceLocation(sourceLoc);
+                ScalarFunctionCallExpression qeVar2EqZero = new ScalarFunctionCallExpression(
+                        BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.EQ), new MutableObject<>(qeVar2Ref),
+                        new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(0)))));
+
+                ScalarFunctionCallExpression andExpr =
+                        new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+                                new MutableObject<>(qeVar1Ref), new MutableObject<>(qeVar2EqZero));
+
+                qeVar = context.newVar();
+                AssignOperator assignOp2 = new AssignOperator(qeVar, new MutableObject<>(andExpr));
+                assignOp2.setSourceLocation(sourceLoc);
+                assignOp2.getInputs().add(new MutableObject<>(subplanOp));
+
+                return new Pair<>(assignOp2, qeVar);
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                        qe.getQuantifier().toString());
+        }
     }
 
     @Override
@@ -1738,14 +1808,25 @@ abstract class LangExpressionToPlanTranslator
     }
 
     protected boolean expressionNeedsNoNesting(Expression expr) throws CompilationException {
-        Kind k = expr.getKind();
-        boolean noNesting = k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION;
-        noNesting = noNesting || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION
-                || k == Kind.FIELD_ACCESSOR_EXPRESSION;
-        noNesting = noNesting || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION
-                || k == Kind.IF_EXPRESSION;
-        return noNesting || k == Kind.CASE_EXPRESSION || k == Kind.WINDOW_EXPRESSION;
+        switch (expr.getKind()) {
+            case LITERAL_EXPRESSION:
+            case LIST_CONSTRUCTOR_EXPRESSION:
+            case RECORD_CONSTRUCTOR_EXPRESSION:
+            case VARIABLE_EXPRESSION:
+            case CALL_EXPRESSION:
+            case OP_EXPRESSION:
+            case FIELD_ACCESSOR_EXPRESSION:
+            case INDEX_ACCESSOR_EXPRESSION:
+            case UNARY_EXPRESSION:
+            case IF_EXPRESSION:
+            case CASE_EXPRESSION:
+            case WINDOW_EXPRESSION:
+                return true;
+            case QUANTIFIED_EXPRESSION:
+                return ((QuantifiedExpression) expr).getQuantifier() == Quantifier.SOME_AND_EVERY;
+            default:
+                return false;
+        }
     }
 
     protected <T> List<T> mkSingletonArrayList(T item) {
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp
new file mode 100644
index 0000000..808226a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+  _id: uuid
+};
+CREATE DATASET   Dataset1 (TestType)
+PRIMARY KEY      _id AUTOGENERATED;
+
+-- Quantifying on items in an array.
+FROM    Dataset1 D
+WHERE   ANY AND EVERY I IN D.items
+        SATISFIES I = 1
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp
new file mode 100644
index 0000000..4e2690e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-2.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * 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    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+  _id: uuid
+};
+CREATE DATASET   Dataset1 (TestType)
+PRIMARY KEY      _id AUTOGENERATED;
+
+-- Quantifying on items in an array, that are within an array.
+FROM    Dataset1 D
+WHERE   ANY AND EVERY I IN D.items
+        SATISFIES (
+            SOME AND EVERY J IN I.items
+            SATISFIES J = 1
+        )
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp
new file mode 100644
index 0000000..e0a9673
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/any-and-every-3.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+  _id: uuid
+};
+CREATE DATASET   Dataset1 (TestType)
+PRIMARY KEY      _id AUTOGENERATED;
+
+-- Quantifying on items in two arrays.
+FROM    Dataset1 D
+WHERE   SOME AND EVERY I IN D.items, J IN D.other_items
+        SATISFIES I = 1 AND J = 2
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
new file mode 100644
index 0000000..faac4b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-1.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
new file mode 100644
index 0000000..b482afb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- SUBPLAN  |LOCAL|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
new file mode 100644
index 0000000..d865914
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/any-and-every-3.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- UNNEST  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN (TestDataverse.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp
new file mode 100644
index 0000000..fb19ba8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+  _id: uuid
+};
+
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp
new file mode 100644
index 0000000..97cda55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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               TestDataverse;
+
+INSERT INTO       Dataset1 [
+    { "items": [], "other_items": [1, 2, 3] },
+    { "items": null, "other_items": [1, 2, 3] },
+    { "other_items": [1, 2, 3] },
+    { "items": [1, 2, 3], "other_items": [1, 2, 3 ] },
+    { "items": [1, 2], "other_items": [1, 2, 3] },
+    { "items": [1, 2, 3, 4], "other_items": [1, 2, 3] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.3.query.sqlpp
new file mode 100644
index 0000000..1c87dbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.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.
+ */
+
+USE       TestDataverse;
+FROM      Dataset1 D
+WHERE     ANY AND EVERY I IN D.items
+          SATISFIES I BETWEEN 1 AND 3
+SELECT    D.items
+ORDER BY  D.items;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.query.sqlpp
new file mode 100644
index 0000000..85bf8c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/quantifiers/any_and_every_01/any_and_every_01.4.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.
+ */
+
+USE       TestDataverse;
+FROM      Dataset1 D
+WHERE     ANY AND EVERY I IN D.items, J IN D.other_items
+          SATISFIES I BETWEEN 1 AND 3 AND J BETWEEN 1 AND 3
+SELECT    D.items, D.other_items
+ORDER BY  D.items, D.other_items;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm
new file mode 100644
index 0000000..4746c44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.1.adm
@@ -0,0 +1,2 @@
+{ "items": [ 1, 2 ] }
+{ "items": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm
new file mode 100644
index 0000000..01c0fff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/quantifiers/any_and_every_01/any_and_every_01.2.adm
@@ -0,0 +1,2 @@
+{ "items": [ 1, 2 ], "other_items": [ 1, 2, 3 ] }
+{ "items": [ 1, 2, 3 ], "other_items": [ 1, 2, 3 ] }
\ 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 b1549f6..a663fd3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9386,6 +9386,11 @@
   </test-group>
   <test-group name="quantifiers">
     <test-case FilePath="quantifiers">
+      <compilation-unit name="any_and_every_01">
+        <output-dir compare="Text">any_and_every_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="quantifiers">
       <compilation-unit name="anysat_01">
         <output-dir compare="Text">somesat_01</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index 425a13e..c9cabe9 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -5,7 +5,7 @@ OperatorExpr ::= PathExpr
                  | OperatorExpr Operator OperatorExpr?
                  | OperatorExpr "BETWEEN" OperatorExpr "AND" OperatorExpr
 
-QuantifiedExpr::= ( "SOME" | "EVERY" ) Variable "IN" Expr ( "," Variable "IN" Expr )* "SATISFIES" Expr ("END")?
+QuantifiedExpr::= ( "SOME" | "EVERY" | ( "SOME" "AND" "EVERY" ) ) Variable "IN" Expr ( "," Variable "IN" Expr )* "SATISFIES" Expr ("END")?
 
 PathExpr ::= PrimaryExpr ("." Identifier | "[" Expr (":" (Expr)? )? "]")*
 
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index de7f442..9ecc29b 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -191,7 +191,7 @@ Quantified expressions are used for expressing existential or universal predicat
 The following pair of examples illustrate the use of a quantified expression to test that every (or some) element in the
 set [1, 2, 3] of integers is less than three. The first example yields `FALSE` and second example yields `TRUE`.
 
-It is useful to note that if the set were instead the empty set, the first expression would yield `TRUE` ("every" value in an empty set satisfies the condition) while the second expression would yield `FALSE` (since there isn't "some" value, as there are no values in the set, that satisfies the condition).
+It is useful to note that if the set were instead the empty set, the first expression would yield `TRUE` ("every" value in an empty set satisfies the condition) while the second expression would yield `FALSE` (since there isn't "some" value, as there are no values in the set, that satisfies the condition). To express a universal predicate that yields `FALSE` with the empty set, we would use the quantifier `SOME AND EVERY` in lieu of `EVERY`.
 
 A quantified expression will return a `NULL` (or `MISSING`) if the first expression in it evaluates to `NULL` (or `MISSING`).
 Otherwise, a type error will be raised if the first expression in a quantified expression does not return a collection.
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
index a35d975..8715774 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/QuantifiedExpression.java
@@ -78,6 +78,7 @@ public class QuantifiedExpression extends AbstractExpression {
     }
 
     public enum Quantifier {
+        SOME_AND_EVERY,
         EVERY,
         SOME
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 38529a5..de9ea7a 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -4941,8 +4941,10 @@ QuantifiedExpression QuantifiedExpression()throws ParseException:
     createNewScope();
   }
 
-   ( ((<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); })
-     | (<EVERY> {  startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); }))
+     ( LOOKAHEAD(2)
+       (<ANY>|<SOME>)<AND><EVERY> { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME_AND_EVERY); }
+       | (<ANY>|<SOME>) { startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.SOME); }
+       | <EVERY> {  startToken = token; qc.setQuantifier(QuantifiedExpression.Quantifier.EVERY); } )
     var = Variable() <IN> inExpr = Expression()
     {
       pair = new QuantifiedPair(var, inExpr);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index 31a93dc..05d14b8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -41,6 +41,7 @@ import org.apache.asterix.formats.nontagged.TypeTraitProvider;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
 import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.base.IAObject;
@@ -311,9 +312,11 @@ public class NonTaggedDataFormat implements IDataFormat {
     @SuppressWarnings("unchecked")
     @Override
     public IScalarEvaluatorFactory getConstantEvalFactory(IAlgebricksConstantValue value) throws AlgebricksException {
-        IAObject obj = null;
+        IAObject obj;
         if (value.isMissing()) {
             obj = AMissing.MISSING;
+        } else if (value.isNull()) {
+            obj = ANull.NULL;
         } else if (value.isTrue()) {
             obj = ABoolean.TRUE;
         } else if (value.isFalse()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index 40e729d..c49f62f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -85,6 +85,33 @@ public final class ConstantExpression extends AbstractLogicalExpression {
             return "FALSE";
         }
     });
+    public static final ConstantExpression NULL = new ConstantExpression(new IAlgebricksConstantValue() {
+
+        @Override
+        public boolean isTrue() {
+            return false;
+        }
+
+        @Override
+        public boolean isMissing() {
+            return false;
+        }
+
+        @Override
+        public boolean isNull() {
+            return true;
+        }
+
+        @Override
+        public boolean isFalse() {
+            return false;
+        }
+
+        @Override
+        public String toString() {
+            return "NULL";
+        }
+    });
     public static final ConstantExpression MISSING = new ConstantExpression(new IAlgebricksConstantValue() {
 
         @Override