|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.table.planner.plan.abilities.source; |
| 20 | + |
| 21 | +import org.apache.flink.table.api.TableException; |
| 22 | +import org.apache.flink.table.connector.source.DynamicTableSource; |
| 23 | +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; |
| 24 | +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata.MetadataFilterResult; |
| 25 | +import org.apache.flink.table.expressions.Expression; |
| 26 | +import org.apache.flink.table.expressions.ResolvedExpression; |
| 27 | +import org.apache.flink.table.expressions.resolver.ExpressionResolver; |
| 28 | +import org.apache.flink.table.planner.plan.utils.FlinkRexUtil; |
| 29 | +import org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter; |
| 30 | +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; |
| 31 | +import org.apache.flink.table.types.logical.RowType; |
| 32 | + |
| 33 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; |
| 34 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; |
| 35 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; |
| 36 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName; |
| 37 | + |
| 38 | +import org.apache.calcite.rex.RexBuilder; |
| 39 | +import org.apache.calcite.rex.RexNode; |
| 40 | + |
| 41 | +import java.util.ArrayList; |
| 42 | +import java.util.List; |
| 43 | +import java.util.Objects; |
| 44 | +import java.util.Optional; |
| 45 | +import java.util.stream.Collectors; |
| 46 | + |
| 47 | +import scala.Option; |
| 48 | + |
| 49 | +import static org.apache.flink.util.Preconditions.checkNotNull; |
| 50 | + |
| 51 | +/** |
| 52 | + * Serializes metadata filter predicates and replays them during compiled plan restoration. |
| 53 | + * |
| 54 | + * <p>Predicates are stored with a {@code predicateRowType} that already uses metadata key names |
| 55 | + * (not SQL aliases). The alias-to-key translation happens once at optimization time, so no |
| 56 | + * column-to-key mapping needs to be persisted. |
| 57 | + */ |
| 58 | +@JsonIgnoreProperties(ignoreUnknown = true) |
| 59 | +@JsonTypeName("MetadataFilterPushDown") |
| 60 | +public final class MetadataFilterPushDownSpec extends SourceAbilitySpecBase { |
| 61 | + |
| 62 | + public static final String FIELD_NAME_PREDICATES = "predicates"; |
| 63 | + public static final String FIELD_NAME_PREDICATE_ROW_TYPE = "predicateRowType"; |
| 64 | + |
| 65 | + @JsonProperty(FIELD_NAME_PREDICATES) |
| 66 | + private final List<RexNode> predicates; |
| 67 | + |
| 68 | + /** |
| 69 | + * Row type snapshot using metadata key names. Stored because ProjectPushDownSpec may narrow the |
| 70 | + * context's row type during restore. |
| 71 | + */ |
| 72 | + @JsonProperty(FIELD_NAME_PREDICATE_ROW_TYPE) |
| 73 | + private final RowType predicateRowType; |
| 74 | + |
| 75 | + @JsonCreator |
| 76 | + public MetadataFilterPushDownSpec( |
| 77 | + @JsonProperty(FIELD_NAME_PREDICATES) List<RexNode> predicates, |
| 78 | + @JsonProperty(FIELD_NAME_PREDICATE_ROW_TYPE) RowType predicateRowType) { |
| 79 | + this.predicates = new ArrayList<>(checkNotNull(predicates)); |
| 80 | + this.predicateRowType = checkNotNull(predicateRowType); |
| 81 | + } |
| 82 | + |
| 83 | + public List<RexNode> getPredicates() { |
| 84 | + return predicates; |
| 85 | + } |
| 86 | + |
| 87 | + @Override |
| 88 | + public void apply(DynamicTableSource tableSource, SourceAbilityContext context) { |
| 89 | + // Use stored predicateRowType; context's row type may be narrowed by ProjectPushDownSpec. |
| 90 | + MetadataFilterResult result = |
| 91 | + applyMetadataFilters(predicates, predicateRowType, tableSource, context); |
| 92 | + if (result.getAcceptedFilters().size() != predicates.size()) { |
| 93 | + throw new TableException("All metadata predicates should be accepted here."); |
| 94 | + } |
| 95 | + } |
| 96 | + |
| 97 | + /** |
| 98 | + * Converts RexNode predicates to ResolvedExpressions using the given row type and calls |
| 99 | + * applyMetadataFilters on the source. The row type must already use metadata key names. |
| 100 | + */ |
| 101 | + public static MetadataFilterResult applyMetadataFilters( |
| 102 | + List<RexNode> predicates, |
| 103 | + RowType metadataKeyRowType, |
| 104 | + DynamicTableSource tableSource, |
| 105 | + SourceAbilityContext context) { |
| 106 | + if (!(tableSource instanceof SupportsReadingMetadata)) { |
| 107 | + throw new TableException( |
| 108 | + String.format( |
| 109 | + "%s does not support SupportsReadingMetadata.", |
| 110 | + tableSource.getClass().getName())); |
| 111 | + } |
| 112 | + |
| 113 | + String[] fieldNames = metadataKeyRowType.getFieldNames().toArray(new String[0]); |
| 114 | + |
| 115 | + RexNodeToExpressionConverter converter = |
| 116 | + new RexNodeToExpressionConverter( |
| 117 | + new RexBuilder(context.getTypeFactory()), |
| 118 | + fieldNames, |
| 119 | + context.getFunctionCatalog(), |
| 120 | + context.getCatalogManager(), |
| 121 | + Option.apply( |
| 122 | + context.getTypeFactory().buildRelNodeRowType(metadataKeyRowType))); |
| 123 | + |
| 124 | + List<Expression> filters = |
| 125 | + predicates.stream() |
| 126 | + .map( |
| 127 | + p -> { |
| 128 | + scala.Option<ResolvedExpression> expr = p.accept(converter); |
| 129 | + if (expr.isDefined()) { |
| 130 | + return expr.get(); |
| 131 | + } else { |
| 132 | + throw new TableException( |
| 133 | + String.format( |
| 134 | + "%s can not be converted to Expression for metadata filter push-down.", |
| 135 | + p.toString())); |
| 136 | + } |
| 137 | + }) |
| 138 | + .collect(Collectors.toList()); |
| 139 | + |
| 140 | + ExpressionResolver resolver = |
| 141 | + ExpressionResolver.resolverFor( |
| 142 | + context.getTableConfig(), |
| 143 | + context.getClassLoader(), |
| 144 | + name -> Optional.empty(), |
| 145 | + context.getFunctionCatalog() |
| 146 | + .asLookup( |
| 147 | + str -> { |
| 148 | + throw new TableException( |
| 149 | + "We should not need to lookup any expressions at this point"); |
| 150 | + }), |
| 151 | + context.getCatalogManager().getDataTypeFactory(), |
| 152 | + (sqlExpression, inputRowType, outputType) -> { |
| 153 | + throw new TableException( |
| 154 | + "SQL expression parsing is not supported at this location."); |
| 155 | + }) |
| 156 | + .build(); |
| 157 | + |
| 158 | + return ((SupportsReadingMetadata) tableSource) |
| 159 | + .applyMetadataFilters(resolver.resolve(filters)); |
| 160 | + } |
| 161 | + |
| 162 | + @Override |
| 163 | + public boolean needAdjustFieldReferenceAfterProjection() { |
| 164 | + return true; |
| 165 | + } |
| 166 | + |
| 167 | + @Override |
| 168 | + public String getDigests(SourceAbilityContext context) { |
| 169 | + final List<String> expressionStrs = new ArrayList<>(); |
| 170 | + for (RexNode rexNode : predicates) { |
| 171 | + expressionStrs.add( |
| 172 | + FlinkRexUtil.getExpressionString( |
| 173 | + rexNode, |
| 174 | + JavaScalaConversionUtil.toScala(predicateRowType.getFieldNames()))); |
| 175 | + } |
| 176 | + |
| 177 | + return String.format( |
| 178 | + "metadataFilter=[%s]", |
| 179 | + expressionStrs.stream() |
| 180 | + .reduce((l, r) -> String.format("and(%s, %s)", l, r)) |
| 181 | + .orElse("")); |
| 182 | + } |
| 183 | + |
| 184 | + @Override |
| 185 | + public boolean equals(Object o) { |
| 186 | + if (this == o) { |
| 187 | + return true; |
| 188 | + } |
| 189 | + if (o == null || getClass() != o.getClass()) { |
| 190 | + return false; |
| 191 | + } |
| 192 | + if (!super.equals(o)) { |
| 193 | + return false; |
| 194 | + } |
| 195 | + MetadataFilterPushDownSpec that = (MetadataFilterPushDownSpec) o; |
| 196 | + return Objects.equals(predicates, that.predicates) |
| 197 | + && Objects.equals(predicateRowType, that.predicateRowType); |
| 198 | + } |
| 199 | + |
| 200 | + @Override |
| 201 | + public int hashCode() { |
| 202 | + return Objects.hash(super.hashCode(), predicates, predicateRowType); |
| 203 | + } |
| 204 | +} |
0 commit comments