diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
index 27979e72ea02..b0eefb31d478 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
@@ -20,7 +20,6 @@
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.util.ImmutableBitSet;
import org.apache.calcite.util.Pair;
import org.apache.calcite.util.Util;
@@ -58,9 +57,13 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
/** As {@link #rowType}, but not necessarily a struct. */
protected @Nullable RelDataType type;
- /** Ordinals of fields that must be filtered. Initially the empty set, but
- * should typically be re-assigned on validate. */
- protected ImmutableBitSet mustFilterFields = ImmutableBitSet.of();
+ /**
+ * Class that holds information about what fields need to be filtered, what bypass-fields
+ * can defuse the errors if they are filtered on as an alternative, and a set used during
+ * validation internally. Initialized as empty object, but should typically be re-assiged
+ * on validate.
+ */
+ protected FilterRequirement filterRequirement = new FilterRequirement();
protected final @Nullable SqlNode enclosingNode;
@@ -164,9 +167,9 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
return ImmutableList.of();
}
- @Override public ImmutableBitSet getMustFilterFields() {
- return requireNonNull(mustFilterFields,
- "mustFilterFields (maybe validation is not complete?)");
+ @Override public FilterRequirement getFilterRequirement() {
+ return requireNonNull(filterRequirement,
+ "filterRequirement (maybe validation is not complete?)");
}
@Override public SqlMonotonicity getMonotonicity(String columnName) {
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/FilterRequirement.java b/core/src/main/java/org/apache/calcite/sql/validate/FilterRequirement.java
new file mode 100644
index 000000000000..22c1826e1a93
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/FilterRequirement.java
@@ -0,0 +1,118 @@
+/*
+ * 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.calcite.sql.validate;
+
+import org.apache.calcite.util.ImmutableBitSet;
+
+import com.google.common.collect.ImmutableSet;
+
+import java.util.Set;
+
+/**
+ * Class that encapsulates filtering requirements when overloading SemanticTable.
+ *
+ *
A few examples of the behavior:
+ *
+ *
Table t
has a must-filter field f
and bypass-fields b0
+ * and b1
.
+ * SQL: select f from t;
-> Errors because there's no filter on f.
+ * SQL: select * from (select f from t);
-> Errors at the inner query because
+ * there's no filter on f.
+ * SQL: select f from t where f = 1;
-> Valid because there is a filter on f.
+ * SQL: select * from (select f from t) where f = 1;
-> Valid because there is a
+ * filter on f.
+ * SQL: select f from t where b0 = 1;
-> Valid because there is a filter on
+ * bypass-field b0.
+ *
+ *
Some notes on remnantFilterFields.
+ * remnantFilterFields is used to identify whether the query should error
+ * at the top level query. It is populated with the filter-field value when a filter-field is not
+ * selected or filtered on, but a bypass-field for the table is selected.
+ * The remnantFilterFields are no longer accessible by the enclosing query and hence can no
+ * longer be defused by filtering on it; however, it can be defused if the bypass-field is
+ * filtered on, hence we need to keep track of it.
+
+ * Example:
+ * Table t
has a must-filter field f
and bypass-fields b0
+ * and b1
.
+ * SQL: select b0, b1 from t;
+ *
+ *
This results in:
+ * filterFields:[]
+ * bypassFields:[b0, b1]
+ * remnantFilterFields: [f]
+ * -> Errors because it is a top level query and remnantFilterFields is not empty.
+ *
+ *
SQL: select * from (select b0, b1 from t) where b0 = 1;
+ * When unwrapping the inner query we get the same FilterRequirement as the previous example:
+ * filterFields:[]
+ * bypassFields:[b0, b1]
+ * remnantFilterFields: [f]
+ * When unwrapping the top level query, the filter on b0 defuses the remnantFilterField requirement
+ * of [f] because it originated from the same table, resulting in the following:
+ * filterFields:[]
+ * bypassFields:[b0, b1]
+ * remnantFilterFields: []
+ * -> Valid because remnantFilterFields is empty now.
+ */
+final class FilterRequirement {
+
+ /** The ordinals (in the row type) of the "must-filter" fields,
+ * fields that must be filtered in a query.*/
+ private final ImmutableBitSet filterFields;
+ /** The ordinals (in the row type) of the "bypass" fields,
+ * fields that can defuse validation errors on filterFields if filtered on. */
+ private final ImmutableBitSet bypassFields;
+ /** Set of filterField SqlQualifieds that have not been defused
+ * in the current query, but can still be defused by filtering on a bypass field in the
+ * enclosing query.*/
+ private final ImmutableSet remnantFilterFields;
+
+ /**
+ * Creates a FilterRequirement
.
+ *
+ * @param filterFields Ordinals of the "must-filter" fields.
+ * @param bypassFields Ordinals of the "bypass" fields.
+ * @param remnantFilterFields Filter fields that can no longer be filtered on,
+ * but can only be defused if a bypass field is filtered on.
+ */
+ FilterRequirement(ImmutableBitSet filterFields,
+ ImmutableBitSet bypassFields, Set remnantFilterFields) {
+ this.filterFields = ImmutableBitSet.of(filterFields);
+ this.bypassFields = ImmutableBitSet.of(bypassFields);
+ this.remnantFilterFields = ImmutableSet.copyOf(remnantFilterFields);
+ }
+
+ /** Creates an empty FilterRequirement. */
+ FilterRequirement() {
+ this(ImmutableBitSet.of(), ImmutableBitSet.of(), ImmutableSet.of());
+ }
+ /** Returns filterFields. */
+ public ImmutableBitSet getFilterFields() {
+ return filterFields;
+ }
+
+ /** Returns bypassFields. */
+ public ImmutableBitSet getBypassFields() {
+ return bypassFields;
+ }
+
+ /** Returns remnantFilterFields. */
+ public ImmutableSet getRemnantFilterFields() {
+ return remnantFilterFields;
+ }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
index 87d088bc8001..9291752b90a3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
@@ -210,7 +210,7 @@ private SqlValidatorNamespace resolveImpl(SqlIdentifier id) {
}
}
- this.mustFilterFields = resolvedNamespace.getMustFilterFields();
+ filterRequirement = resolvedNamespace.getFilterRequirement();
RelDataType rowType = resolvedNamespace.getRowType();
if (extendList != null) {
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SemanticTable.java b/core/src/main/java/org/apache/calcite/sql/validate/SemanticTable.java
index b115dbb4ccce..6b39af7065c7 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SemanticTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SemanticTable.java
@@ -16,8 +16,12 @@
*/
package org.apache.calcite.sql.validate;
+import com.google.common.collect.ImmutableList;
+
import org.checkerframework.checker.nullness.qual.Nullable;
+import java.util.List;
+
/**
* Extension to {@link SqlValidatorTable} with extra, optional metadata.
*
@@ -44,4 +48,12 @@ public interface SemanticTable {
default boolean mustFilter(int column) {
return getFilter(column) != null;
}
+
+ /**
+ * Returns a list of column ordinals (0-based) of fields that defuse
+ * must-filter columns when filtered on.
+ */
+ default List bypassFieldList() {
+ return ImmutableList.of();
+ }
}
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 706c8e6be0e5..cb08493b2474 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -160,6 +160,7 @@
import java.util.function.Supplier;
import java.util.function.UnaryOperator;
import java.util.stream.Collectors;
+import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static com.google.common.base.Preconditions.checkArgument;
@@ -1212,17 +1213,27 @@ protected void validateNamespace(final SqlValidatorNamespace namespace,
RelDataType type = namespace.getType();
if (node == top) {
- // A top-level namespace must not return any must-filter fields.
+ FilterRequirement filterRequirement = namespace.getFilterRequirement();
+ // Either of the following two conditions result in an invalid query:
+ // 1) A top-level namespace must not return any must-filter fields.
// A non-top-level namespace (e.g. a subquery) may return must-filter
// fields; these are neutralized if the consuming query filters on them.
- final ImmutableBitSet mustFilterFields =
- namespace.getMustFilterFields();
- if (!mustFilterFields.isEmpty()) {
+ // 2) A top-level namespace must not have any remnant-must-filter fields.
+ // Remnant must filter fields are fields that are not selected and cannot
+ // be defused unless a bypass field defuses it.
+ if (!filterRequirement.getFilterFields().isEmpty()
+ || !filterRequirement.getRemnantFilterFields().isEmpty()) {
+ Stream mustFilterStream =
+ StreamSupport.stream(filterRequirement.getFilterFields().spliterator(), false)
+ .map(namespace.getRowType().getFieldNames()::get);
+ Stream remnantStream =
+ filterRequirement.getRemnantFilterFields().stream()
+ .map(q -> q.suffix().get(0));
+
// Set of field names, sorted alphabetically for determinism.
Set fieldNameSet =
- StreamSupport.stream(mustFilterFields.spliterator(), false)
- .map(namespace.getRowType().getFieldNames()::get)
- .collect(Collectors.toCollection(TreeSet::new));
+ Stream.concat(mustFilterStream, remnantStream)
+ .collect(Collectors.toCollection(TreeSet::new));
throw newValidationError(node,
RESOURCE.mustFilterFieldsMissing(fieldNameSet.toString()));
}
@@ -4098,64 +4109,7 @@ protected void validateSelect(
validateSelectList(selectItems, select, targetRowType);
ns.setType(rowType);
- // Deduce which columns must be filtered.
- ns.mustFilterFields = ImmutableBitSet.of();
- if (from != null) {
- final Set qualifieds = new LinkedHashSet<>();
- for (ScopeChild child : fromScope.children) {
- final List fieldNames =
- child.namespace.getRowType().getFieldNames();
- child.namespace.getMustFilterFields()
- .forEachInt(i ->
- qualifieds.add(
- SqlQualified.create(fromScope, 1, child.namespace,
- new SqlIdentifier(
- ImmutableList.of(child.name, fieldNames.get(i)),
- SqlParserPos.ZERO))));
- }
- if (!qualifieds.isEmpty()) {
- if (select.getWhere() != null) {
- forEachQualified(select.getWhere(), getWhereScope(select),
- qualifieds::remove);
- }
- if (select.getHaving() != null) {
- forEachQualified(select.getHaving(), getHavingScope(select),
- qualifieds::remove);
- }
-
- // Each of the must-filter fields identified must be returned as a
- // SELECT item, which is then flagged as must-filter.
- final BitSet mustFilterFields = new BitSet();
- final List expandedSelectItems =
- requireNonNull(fromScope.getExpandedSelectList(),
- "expandedSelectList");
- forEach(expandedSelectItems, (selectItem, i) -> {
- selectItem = stripAs(selectItem);
- if (selectItem instanceof SqlIdentifier) {
- SqlQualified qualified =
- fromScope.fullyQualify((SqlIdentifier) selectItem);
- if (qualifieds.remove(qualified)) {
- // SELECT item #i referenced a must-filter column that was not
- // filtered in the WHERE or HAVING. It becomes a must-filter
- // column for our consumer.
- mustFilterFields.set(i);
- }
- }
- });
-
- // If there are must-filter fields that are not in the SELECT clause,
- // this is an error.
- if (!qualifieds.isEmpty()) {
- throw newValidationError(select,
- RESOURCE.mustFilterFieldsMissing(
- qualifieds.stream()
- .map(q -> q.suffix().get(0))
- .collect(Collectors.toCollection(TreeSet::new))
- .toString()));
- }
- ns.mustFilterFields = ImmutableBitSet.fromBitSet(mustFilterFields);
- }
- }
+ validateMustFilterRequirements(select, ns);
// Validate ORDER BY after we have set ns.rowType because in some
// dialects you can refer to columns of the select list, e.g.
@@ -4172,8 +4126,10 @@ protected void validateSelect(
}
}
- /** For each identifier in an expression, resolves it to a qualified name
- * and calls the provided action. */
+ /**
+ * For each identifier in an expression, resolves it to a qualified name
+ * and calls the provided action.
+ */
private static void forEachQualified(SqlNode node, SqlValidatorScope scope,
Consumer consumer) {
node.accept(new SqlBasicVisitor() {
@@ -4185,6 +4141,49 @@ private static void forEachQualified(SqlNode node, SqlValidatorScope scope,
});
}
+ /**
+ * Removes all entries from qualifieds and remnantMustFilterFields if the supplied SqlNode
+ * is a bypassField.
+ */
+ private static void purgeForBypassFields(SqlNode node, SqlValidatorScope scope,
+ Set qualifieds, Set bypassQualifieds,
+ Set remnantMustFilterFields) {
+ node.accept(new SqlBasicVisitor() {
+ @Override public Void visit(SqlIdentifier id) {
+ final SqlQualified qualified = scope.fullyQualify(id);
+ if (bypassQualifieds.contains(qualified)) {
+ // Clear all the must-filter qualifieds from the same table identifier
+ Collection sameIdentifier = qualifieds.stream()
+ .filter(q -> qualifiedMatchesIdentifier(q, qualified))
+ .collect(Collectors.toList());
+ sameIdentifier.forEach(qualifieds::remove);
+
+ // Clear all the remnant must-filter qualifieds from the same table identifier
+ Collection sameIdentifier_ = remnantMustFilterFields.stream()
+ .filter(q -> qualifiedMatchesIdentifier(q, qualified))
+ .collect(Collectors.toList());
+ sameIdentifier_.forEach(remnantMustFilterFields::remove);
+ }
+ return null;
+ }
+ });
+ }
+
+ private static void toQualifieds(ImmutableBitSet fields, Set qualifiedSet,
+ SelectScope fromScope, ScopeChild child, List fieldNames) {
+ fields.forEachInt(i ->
+ qualifiedSet.add(
+ SqlQualified.create(fromScope, 1, child.namespace,
+ new SqlIdentifier(
+ ImmutableList.of(child.name, fieldNames.get(i)),
+ SqlParserPos.ZERO))));
+
+ }
+
+ private static boolean qualifiedMatchesIdentifier(SqlQualified q1, SqlQualified q2) {
+ return q1.identifier.names.get(0).equals(q2.identifier.names.get(0));
+ }
+
private void checkRollUpInSelectList(SqlSelect select) {
SqlValidatorScope scope = getSelectScope(select);
for (SqlNode item : SqlNonNullableAccessors.getSelectList(select)) {
@@ -4638,6 +4637,90 @@ protected void validateQualifyClause(SqlSelect select) {
}
}
+ protected void validateMustFilterRequirements(SqlSelect select, SelectNamespace ns) {
+ ns.filterRequirement = new FilterRequirement();
+ if (select.getFrom() != null) {
+ final SelectScope fromScope = (SelectScope) getFromScope(select);
+ final BitSet projectedNonFilteredBypassField = new BitSet();
+ final Set qualifieds = new LinkedHashSet<>();
+ final Set bypassQualifieds = new LinkedHashSet<>();
+ final Set remnantQualifieds = new LinkedHashSet<>();
+ for (ScopeChild child : fromScope.children) {
+ final List fieldNames =
+ child.namespace.getRowType().getFieldNames();
+ FilterRequirement mustFilterReq = child.namespace.getFilterRequirement();
+ toQualifieds(mustFilterReq.getFilterFields(), qualifieds, fromScope, child,
+ fieldNames);
+ toQualifieds(mustFilterReq.getBypassFields(), bypassQualifieds, fromScope,
+ child, fieldNames);
+ remnantQualifieds.addAll(mustFilterReq.getRemnantFilterFields());
+ }
+ if (!qualifieds.isEmpty() || !bypassQualifieds.isEmpty()) {
+ if (select.getWhere() != null) {
+ forEachQualified(select.getWhere(), getWhereScope(select),
+ qualifieds::remove);
+ purgeForBypassFields(select.getWhere(), getWhereScope(select),
+ qualifieds, bypassQualifieds, remnantQualifieds);
+ }
+ if (select.getHaving() != null) {
+ forEachQualified(select.getHaving(), getHavingScope(select),
+ qualifieds::remove);
+ purgeForBypassFields(select.getHaving(), getHavingScope(select),
+ qualifieds, bypassQualifieds, remnantQualifieds);
+ }
+
+ // Each of the must-filter fields identified must be returned as a
+ // SELECT item, which is then flagged as must-filter.
+ final BitSet mustFilterFields = new BitSet();
+ // Each of the bypass fields identified must be returned as a
+ // SELECT item, which is then flagged as a bypass field for the consumer.
+ final BitSet mustFilterBypassFields = new BitSet();
+ final List expandedSelectItems =
+ requireNonNull(fromScope.getExpandedSelectList(),
+ "expandedSelectList");
+ forEach(expandedSelectItems, (selectItem, i) -> {
+ selectItem = stripAs(selectItem);
+ if (selectItem instanceof SqlIdentifier) {
+ SqlQualified qualified =
+ fromScope.fullyQualify((SqlIdentifier) selectItem);
+ if (qualifieds.remove(qualified)) {
+ // SELECT item #i referenced a must-filter column that was not
+ // filtered in the WHERE or HAVING. It becomes a must-filter
+ // column for our consumer.
+ mustFilterFields.set(i);
+ }
+ if (bypassQualifieds.remove(qualified)) {
+ // SELECT item #i referenced a bypass column that was not filtered in the WHERE
+ // or HAVING. It becomes a bypass column for our consumer.
+ mustFilterBypassFields.set(i);
+ projectedNonFilteredBypassField.set(0);
+ }
+ }
+ });
+
+ // If there are must-filter fields that are not in the SELECT clause and there were no
+ // bypass-fields on this table, this is an error.
+ if (!qualifieds.isEmpty() && !projectedNonFilteredBypassField.get(0)) {
+ throw newValidationError(select,
+ RESOURCE.mustFilterFieldsMissing(
+ qualifieds.stream()
+ .map(q -> q.suffix().get(0))
+ .collect(Collectors.toCollection(TreeSet::new))
+ .toString()));
+ }
+ // Remaining must-filter fields can be defused by a bypass-field,
+ // so we pass this to the consumer.
+ ImmutableSet remnantMustFilterFields =
+ Stream.of(remnantQualifieds, qualifieds)
+ .flatMap(Set::stream).collect(ImmutableSet.toImmutableSet());
+ ns.filterRequirement =
+ new FilterRequirement(ImmutableBitSet.fromBitSet(mustFilterFields),
+ ImmutableBitSet.fromBitSet(mustFilterBypassFields),
+ remnantMustFilterFields);
+ }
+ }
+ }
+
@Override public void validateWith(SqlWith with, SqlValidatorScope scope) {
final SqlValidatorNamespace namespace = getNamespaceOrThrow(with);
validateNamespace(namespace, unknownType);
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
index 77a8dc070e2c..77679237fd6a 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorNamespace.java
@@ -19,7 +19,6 @@
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.util.ImmutableBitSet;
import org.apache.calcite.util.Pair;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -214,9 +213,8 @@ default boolean fieldExists(String name) {
*/
boolean supportsModality(SqlModality modality);
- /** Returns the ordinals (in the row type) of the "must-filter" fields,
- * fields that that must be filtered in a query. */
- default ImmutableBitSet getMustFilterFields() {
- return ImmutableBitSet.of();
+ /** Returns a FilterRequirement object used during validation. */
+ default FilterRequirement getFilterRequirement() {
+ return new FilterRequirement();
}
}
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
index f6a0069560b7..7cbffb3d1009 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/TableNamespace.java
@@ -30,6 +30,7 @@
import org.apache.calcite.util.Util;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -60,15 +61,18 @@ private TableNamespace(SqlValidatorImpl validator, SqlValidatorTable table,
}
@Override protected RelDataType validateImpl(RelDataType targetRowType) {
- this.mustFilterFields = ImmutableBitSet.of();
table.maybeUnwrap(SemanticTable.class)
- .ifPresent(semanticTable ->
- this.mustFilterFields =
- table.getRowType().getFieldList().stream()
- .map(RelDataTypeField::getIndex)
- .filter(semanticTable::mustFilter)
- .collect(toImmutableBitSet()));
-
+ .ifPresent(semanticTable -> {
+ ImmutableBitSet mustFilterFields = table.getRowType().getFieldList().stream()
+ .map(RelDataTypeField::getIndex)
+ .filter(semanticTable::mustFilter)
+ .collect(toImmutableBitSet());
+ ImmutableBitSet bypassFieldList = ImmutableBitSet.of(semanticTable.bypassFieldList());
+ // We pass in an empty set for remnantMustFilterFields here because it isn't exposed to
+ // SemanticTable and only mustFilterFields and bypassFieldList should be supplied.
+ this.filterRequirement =
+ new FilterRequirement(mustFilterFields, bypassFieldList, ImmutableSet.of());
+ });
if (extendedFields.isEmpty()) {
return table.getRowType();
}
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
index 5c0a85756fab..dc41d92d3c8e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithItemNamespace.java
@@ -41,7 +41,7 @@ class WithItemNamespace extends AbstractNamespace {
final SqlValidatorNamespace childNs =
validator.getNamespaceOrThrow(getQuery());
final RelDataType rowType = childNs.getRowTypeSansSystemColumns();
- mustFilterFields = childNs.getMustFilterFields();
+ filterRequirement = childNs.getFilterRequirement();
SqlNodeList columnList = withItem.columnList;
if (columnList == null) {
return rowType;
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
index bec57b715b1d..286b91d3cce5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithNamespace.java
@@ -64,7 +64,7 @@ public class WithNamespace extends AbstractNamespace {
validator.validateQuery(with.body, scope2, targetRowType);
final RelDataType rowType = validator.getValidatedNodeType(with.body);
validator.setValidatedNodeType(with, rowType);
- mustFilterFields = bodyNamespace.getMustFilterFields();
+ filterRequirement = bodyNamespace.getFilterRequirement();
return rowType;
}
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index c27e0124ab2b..8fc3dec03caa 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -12235,10 +12235,10 @@ private void checkCustomColumnResolving(String table) {
+ " where empno = 1)\n"
+ "where j = 'doctor'")
.ok();
- // Deceitful alias #2. Filter on 'job' is a filter on the underlying
- // 'ename', so the underlying 'job' is missing a filter.
+ // Deceitful alias #2. Filter on 'job' is a filter on the underlying
+ // 'slacker', so the underlying 'job' is missing a filter.
fixture.withSql("^select * from (\n"
- + " select job as j, ename as job\n"
+ + " select job as j, slacker as job\n"
+ " from emp\n"
+ " where empno = 1)\n"
+ "where job = 'doctor'^")
@@ -12386,6 +12386,352 @@ private void checkCustomColumnResolving(String table) {
+ "where empno = 1\n"
+ "and job = 'doctor'")
.ok();
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp where empno = 1)\n"
+ + "SELECT *\n"
+ + "from cte\n"
+ + "where job = 'doctor'")
+ .ok();
+ fixture.withSql("WITH cte AS (\n"
+ + " select empno, job from emp)\n"
+ + "SELECT *\n"
+ + "from cte\n"
+ + "where empno = 1\n"
+ + "and job = 'doctor'")
+ .ok();
+
+ // Filters are missing on EMPNO and JOB, but the error message only
+ // complains about JOB because EMPNO is in the SELECT clause, and could
+ // theoretically be filtered by an enclosing query.
+ fixture.withSql("^select empno\n"
+ + "from emp^")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("^select empno,\n"
+ + " sum(sal) over (order by mgr)\n"
+ + "from emp^")
+ .fails(missingFilters("JOB"));
+ }
+
+ /**
+ * Tests validation of must-filter columns with the inclusion of bypass fields.
+ *
+ * If a table that implements
+ * {@link org.apache.calcite.sql.validate.SemanticTable} tags fields as
+ * 'must-filter', and the SQL query does not contain a WHERE or HAVING clause
+ * on each of the tagged columns, the validator should throw an error.
+ * If any bypass field for a table is in a WHERE/HAVING clause for that SELECT statement,
+ * the must-filter requirements for that table are disabled.
+ */
+ @Test void testMustFilterColumnsWithBypass() {
+ final SqlValidatorFixture fixture = fixture()
+ .withParserConfig(c -> c.withQuoting(Quoting.BACK_TICK))
+ .withOperatorTable(operatorTableFor(SqlLibrary.BIG_QUERY))
+ .withCatalogReader(MustFilterMockCatalogReader::create);
+ // Basic query
+ fixture.withSql("select empno\n"
+ + "from emp\n"
+ + "where job = 'doctor'\n"
+ + "and empno = 1")
+ .ok();
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "where concat(emp.empno, ' ') = 'abc'^")
+ .fails(missingFilters("JOB"));
+
+ // ENAME is a bypass field
+ fixture.withSql("select *\n"
+ + "from emp\n"
+ + "where concat(emp.ename, ' ') = 'abc'^")
+ .ok();
+
+ // SUBQUERIES
+ fixture.withSql("select * from (\n"
+ + " select * from emp where empno = 1)\n"
+ + "where job = 'doctor'")
+ .ok();
+ fixture.withSql("^select * from (\n"
+ + " select ename from emp where empno = 1)^")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("select * from (\n"
+ + " select job, ename from emp where empno = 1)"
+ + "where ename = '1'")
+ .ok();
+ fixture.withSql("select * from (\n"
+ + " select empno, job from emp)\n"
+ + "where job = 'doctor' and empno = 1")
+ .ok();
+
+ // Deceitful alias #1. Filter on 'j' is a filter on the underlying 'job'.
+ fixture.withSql("select * from (\n"
+ + " select job as j, ename as job\n"
+ + " from emp\n"
+ + " where empno = 1)\n"
+ + "where j = 'doctor'")
+ .ok();
+
+ // Deceitful alias #2. Filter on 'job' is a filter on the underlying
+ // 'slacker', so the underlying 'job' is missing a filter.
+ fixture.withSql("^select * from (\n"
+ + " select job as j, slacker as job\n"
+ + " from emp\n"
+ + " where empno = 1)\n"
+ + "where job = 'doctor'^")
+ .fails(missingFilters("J"));
+
+ // Deceitful alias #3. Filter on 'job' is a filter on the underlying
+ // 'ename', which is a bypass field thus no exception.
+ fixture.withSql("select * from (\n"
+ + " select job as j, ename as job\n"
+ + " from emp\n"
+ + " where empno = 1)\n"
+ + "where job = 'doctor'^")
+ .ok();
+ fixture.withSql("select * from (\n"
+ + " select * from emp where job = 'doctor')\n"
+ + "where empno = 1")
+ .ok();
+ fixture.withSql("select * from (\n"
+ + " select empno from emp where job = 'doctor')\n"
+ + "where empno = 1")
+ .ok();
+ fixture.withSql("^select * from (\n"
+ + " select * from emp where empno = 1)^")
+ .fails(missingFilters("JOB"));
+
+ // Query is valid because ENAME is a bypass field
+ fixture.withSql("select * from (\n"
+ + " select * from emp where ename = 1)^")
+ .ok();
+ fixture.withSql("^select * from (select * from `SALES`.`EMP`) as a1^ ")
+ .fails(missingFilters("EMPNO", "JOB"));
+ fixture.withSql("select * from (select * from `SALES`.`EMP`) as a1 where ename = '1'^ ")
+ .ok();
+
+ // JOINs
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "join dept on emp.deptno = dept.deptno^")
+ .fails(missingFilters("EMPNO", "JOB", "NAME"));
+
+ // Query is invalid because ENAME is a bypass field for EMP table, but not the DEPT table.
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "join dept on emp.deptno = dept.deptno where ename = '1'^")
+ .fails(missingFilters("NAME"));
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "join dept on emp.deptno = dept.deptno\n"
+ + "where emp.empno = 1^")
+ .fails(missingFilters("JOB", "NAME"));
+ fixture.withSql("select *\n"
+ + "from emp\n"
+ + "join dept on emp.deptno = dept.deptno\n"
+ + "where emp.empno = 1\n"
+ + "and emp.job = 'doctor'\n"
+ + "and dept.name = 'ACCOUNTING'")
+ .ok();
+ fixture.withSql("select *\n"
+ + "from emp\n"
+ + "join dept on emp.deptno = dept.deptno\n"
+ + "where empno = 1\n"
+ + "and job = 'doctor'\n"
+ + "and dept.name = 'ACCOUNTING'")
+ .ok();
+
+ // Self-join
+ fixture.withSql("^select *\n"
+ + "from `SALES`.emp a1\n"
+ + "join `SALES`.emp a2 on a1.empno = a2.empno^")
+ .fails(missingFilters("EMPNO", "EMPNO0", "JOB", "JOB0"));
+
+ // Query is invalid because filtering on a bypass field in a1 disables must-filter for a1,
+ // but a2 must-filters are still required.
+ fixture.withSql("^select *\n"
+ + "from `SALES`.emp a1\n"
+ + "join `SALES`.emp a2 on a1.empno = a2.empno where a1.ename = '1'^")
+ .fails(missingFilters("EMPNO0", "JOB0"));
+
+ // Query is invalid because here are two JOB columns but only one is filtered.
+ fixture.withSql("^select *\n"
+ + "from emp a1\n"
+ + "join emp a2 on a1.empno = a2.empno\n"
+ + "where a2.empno = 1\n"
+ + "and a1.empno = 1\n"
+ + "and a2.job = 'doctor'^")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("select *\n"
+ + "from emp a1\n"
+ + "join emp a2 on a1.empno = a2.empno\n"
+ + "where a2.empno = 1\n"
+ + "and a1.empno = 1\n"
+ + "and a2.job = 'doctor'^\n"
+ + "and a1.ename = '1'")
+ .ok();
+ fixture.withSql("select *\n"
+ + "from emp a1\n"
+ + "join emp a2 on a1.empno = a2.empno\n"
+ + "where a1.empno = 1\n"
+ + "and a1.job = 'doctor'\n"
+ + "and a2.empno = 2\n"
+ + "and a2.job = 'undertaker'\n")
+ .ok();
+ fixture.withSql("^select *\n"
+ + " from (select * from `SALES`.`EMP`) as a1\n"
+ + "join (select * from `SALES`.`EMP`) as a2\n"
+ + " on a1.`EMPNO` = a2.`EMPNO`^")
+ .fails(missingFilters("EMPNO", "EMPNO0", "JOB", "JOB0"));
+
+ // Query is invalid because filtering on a bypass field in a1 disables must-filter for a1,
+ // but a2 must-filters are still required.
+ fixture.withSql("^select *\n"
+ + " from (select * from `SALES`.`EMP`) as a1\n"
+ + "join (select * from `SALES`.`EMP`) as a2\n"
+ + " on a1.`EMPNO` = a2.`EMPNO`\n"
+ + "where a1.ename = '1'^")
+ .fails(missingFilters("EMPNO0", "JOB0"));
+ fixture.withSql("^select *\n"
+ + " from (select * from `SALES`.`EMP` where `ENAME` = '1') as a1\n"
+ + "join (select * from `SALES`.`EMP`) as a2\n"
+ + " on a1.`EMPNO` = a2.`EMPNO`^")
+ .fails(missingFilters("EMPNO0", "JOB0"));
+
+ // USING
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "join dept using(deptno)\n"
+ + "where emp.empno = 1^")
+ .fails(missingFilters("JOB", "NAME"));
+
+ // Query is invalid because ENAME is bypass field for EMP, but not for DEPT.
+ fixture.withSql("^select *\n"
+ + "from emp\n"
+ + "join dept using(deptno)\n"
+ + "where emp.ename = '1'^")
+ .fails(missingFilters("NAME"));
+ fixture.withSql("select *\n"
+ + "from emp\n"
+ + "join dept using(deptno)\n"
+ + "where emp.empno = 1\n"
+ + "and emp.job = 'doctor'\n"
+ + "and dept.name = 'ACCOUNTING'")
+ .ok();
+
+ // GROUP BY (HAVING)
+ fixture.withSql("select *\n"
+ + "from dept\n"
+ + "group by deptno, name\n"
+ + "having name = 'accounting_dept'")
+ .ok();
+ fixture.withSql("^select *\n"
+ + "from dept\n"
+ + "group by deptno, name^")
+ .fails(missingFilters("NAME"));
+
+ // Query is valid because DEPTNO is bypass field.
+ fixture.withSql("select *\n"
+ + "from dept\n"
+ + "group by deptno, name\n"
+ + "having deptno > '1'")
+ .ok();
+ fixture.withSql("select name\n"
+ + "from dept\n"
+ + "group by name\n"
+ + "having name = 'accounting'")
+ .ok();
+ fixture.withSql("^select name\n"
+ + "from dept\n"
+ + "group by name^ ")
+ .fails(missingFilters("NAME"));
+ fixture.withSql("select sum(sal)\n"
+ + "from emp\n"
+ + "where empno > 10\n"
+ + "and job = 'doctor'\n"
+ + "group by empno\n"
+ + "having sum(sal) > 100")
+ .ok();
+ fixture.withSql("^select sum(sal)\n"
+ + "from emp\n"
+ + "where empno > 10\n"
+ + "group by empno\n"
+ + "having sum(sal) > 100^")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("^select sum(sal)\n"
+ + "from emp\n"
+ + "where empno > 10\n"
+ + "group by empno\n"
+ + "having sum(sal) > 100^")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("select sum(sal), job\n"
+ + "from emp\n"
+ + "where empno > 10\n"
+ + "group by job\n"
+ + "having job = 'undertaker'")
+ .ok();
+ fixture.withSql("select sum(sal), ename\n"
+ + "from emp\n"
+ + "where empno > 10\n"
+ + "group by empno, ename\n"
+ + "having ename = '1'")
+ .ok();
+ fixture.withSql("select sum(sal)\n"
+ + "from emp\n"
+ + "where ename = '1'\n"
+ + "group by empno, ename\n"
+ + "having sum(sal) > 100")
+ .ok();
+
+ // CTE
+ fixture.withSql("^WITH cte AS (\n"
+ + " select * from emp order by empno)^\n"
+ + "SELECT * from cte")
+ .fails(missingFilters("EMPNO", "JOB"));
+
+ // Query is valid because ENAME is a bypass field.
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp where ename = '1' order by empno)^\n"
+ + "SELECT * from cte")
+ .ok();
+
+ // Query is valid because ENAME is a bypass field.
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp order by empno)^\n"
+ + "SELECT * from cte where ename = '1'")
+ .ok();
+ fixture.withSql("^WITH cte AS (\n"
+ + " select * from emp where empno = 1)^\n"
+ + "SELECT * from cte")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("WITH cte AS (\n"
+ + " select *\n"
+ + " from emp\n"
+ + " where empno = 1\n"
+ + " and job = 'doctor')\n"
+ + "SELECT * from cte")
+ .ok();
+ fixture.withSql("^WITH cte AS (\n"
+ + " select * from emp)^\n"
+ + "SELECT *\n"
+ + "from cte\n"
+ + "where empno = 1")
+ .fails(missingFilters("JOB"));
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp where ename = '1')^\n"
+ + "SELECT *\n"
+ + "from cte\n")
+ .ok();
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp)^\n"
+ + "SELECT *\n"
+ + "from cte\n"
+ + "where ename = '1'")
+ .ok();
+ fixture.withSql("WITH cte AS (\n"
+ + " select * from emp)\n"
+ + "SELECT *\n"
+ + "from cte\n"
+ + "where empno = 1\n"
+ + "and job = 'doctor'")
+ .ok();
fixture.withSql("WITH cte AS (\n"
+ " select * from emp where empno = 1)\n"
+ "SELECT *\n"
@@ -12400,9 +12746,9 @@ private void checkCustomColumnResolving(String table) {
+ "and job = 'doctor'")
.ok();
- // Filters are missing on EMPNO and JOB, but the error message only
- // complains about JOB because EMPNO is in the SELECT clause, and could
- // theoretically be filtered by an enclosing query.
+ // Query is invalid because filters are missing on EMPNO and JOB.
+ // The error message only complains about JOB because EMPNO is in the SELECT clause,
+ // and could theoretically be filtered by an enclosing query.
fixture.withSql("^select empno\n"
+ "from emp^")
.fails(missingFilters("JOB"));
diff --git a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
index 01aeb5b8b2c9..5ee0d113e97a 100644
--- a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
+++ b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
@@ -1026,15 +1026,17 @@ public static class MockDynamicTable
public static class MustFilterMockTable
extends MockTable implements SemanticTable {
private final Map fieldFilters;
+ private final List bypassFieldList;
MustFilterMockTable(MockCatalogReader catalogReader, String catalogName,
String schemaName, String name, boolean stream, boolean temporal,
double rowCount, @Nullable ColumnResolver resolver,
InitializerExpressionFactory initializerExpressionFactory,
- Map fieldFilters) {
+ Map fieldFilters, List bypassFieldList) {
super(catalogReader, catalogName, schemaName, name, stream, temporal,
rowCount, resolver, initializerExpressionFactory);
this.fieldFilters = ImmutableMap.copyOf(fieldFilters);
+ this.bypassFieldList = ImmutableList.copyOf(bypassFieldList);
}
/** Creates a MustFilterMockTable. */
@@ -1042,11 +1044,11 @@ public static MustFilterMockTable create(MockCatalogReader catalogReader,
MockSchema schema, String name, boolean stream, double rowCount,
@Nullable ColumnResolver resolver,
InitializerExpressionFactory initializerExpressionFactory,
- boolean temporal, Map fieldFilters) {
+ boolean temporal, Map fieldFilters, List bypassFieldList) {
MustFilterMockTable table =
new MustFilterMockTable(catalogReader, schema.getCatalogName(),
schema.name, name, stream, temporal, rowCount, resolver,
- initializerExpressionFactory, fieldFilters);
+ initializerExpressionFactory, fieldFilters, bypassFieldList);
schema.addTable(name);
return table;
}
@@ -1060,6 +1062,10 @@ public static MustFilterMockTable create(MockCatalogReader catalogReader,
String columnName = columnList.get(column).getKey();
return fieldFilters.containsKey(columnName);
}
+
+ @Override public List bypassFieldList() {
+ return bypassFieldList;
+ }
}
/** Wrapper around a {@link MockTable}, giving it a {@link Table} interface.
diff --git a/testkit/src/main/java/org/apache/calcite/test/catalog/MustFilterMockCatalogReader.java b/testkit/src/main/java/org/apache/calcite/test/catalog/MustFilterMockCatalogReader.java
index 69cc3efaf76f..ef9f71a616ba 100644
--- a/testkit/src/main/java/org/apache/calcite/test/catalog/MustFilterMockCatalogReader.java
+++ b/testkit/src/main/java/org/apache/calcite/test/catalog/MustFilterMockCatalogReader.java
@@ -22,6 +22,7 @@
import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
+import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
/**
@@ -47,10 +48,11 @@ public static SqlValidatorCatalogReader create(RelDataTypeFactory typeFactory,
registerSchema(salesSchema);
// Register "EMP" table. Must-filter fields are "EMPNO", "JOB".
+ // Bypass field of column (1): ENAME.
MustFilterMockTable empTable =
MustFilterMockTable.create(this, salesSchema, "EMP",
false, 14, null, NullInitializerExpressionFactory.INSTANCE,
- false, ImmutableMap.of("EMPNO", "10", "JOB", "JOB_1"));
+ false, ImmutableMap.of("EMPNO", "10", "JOB", "JOB_1"), ImmutableList.of(1));
final RelDataType integerType =
typeFactory.createSqlType(SqlTypeName.INTEGER);
@@ -72,10 +74,11 @@ public static SqlValidatorCatalogReader create(RelDataTypeFactory typeFactory,
registerTable(empTable);
// Register "DEPT" table. "NAME" is a must-filter field.
+ // Bypass field of column (0): DEPTNO.
MustFilterMockTable deptTable =
MustFilterMockTable.create(this, salesSchema, "DEPT",
false, 14, null, NullInitializerExpressionFactory.INSTANCE,
- false, ImmutableMap.of("NAME", "ACCOUNTING_DEPT"));
+ false, ImmutableMap.of("NAME", "ACCOUNTING_DEPT"), ImmutableList.of(0));
deptTable.addColumn("DEPTNO", integerType, true);
deptTable.addColumn("NAME", varcharType);
registerTable(deptTable);