|
| 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.ResolvedExpression; |
| 26 | +import org.apache.flink.table.planner.plan.utils.FlinkRexUtil; |
| 27 | +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; |
| 28 | +import org.apache.flink.table.types.logical.RowType; |
| 29 | + |
| 30 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; |
| 31 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; |
| 32 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; |
| 33 | +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName; |
| 34 | + |
| 35 | +import org.apache.calcite.rex.RexNode; |
| 36 | + |
| 37 | +import java.util.ArrayList; |
| 38 | +import java.util.List; |
| 39 | +import java.util.Objects; |
| 40 | + |
| 41 | +import static org.apache.flink.util.Preconditions.checkNotNull; |
| 42 | + |
| 43 | +/** |
| 44 | + * Serializes metadata filter predicates and replays them during compiled plan restoration. |
| 45 | + * |
| 46 | + * <p>Predicates are stored with a {@code predicateRowType} that already uses metadata key names |
| 47 | + * (not SQL aliases). The alias-to-key translation happens once at optimization time, so no |
| 48 | + * column-to-key mapping needs to be persisted. |
| 49 | + */ |
| 50 | +@JsonIgnoreProperties(ignoreUnknown = true) |
| 51 | +@JsonTypeName("MetadataFilterPushDown") |
| 52 | +public final class MetadataFilterPushDownSpec extends SourceAbilitySpecBase { |
| 53 | + |
| 54 | + public static final String FIELD_NAME_PREDICATES = "predicates"; |
| 55 | + public static final String FIELD_NAME_PREDICATE_ROW_TYPE = "predicateRowType"; |
| 56 | + |
| 57 | + @JsonProperty(FIELD_NAME_PREDICATES) |
| 58 | + private final List<RexNode> predicates; |
| 59 | + |
| 60 | + /** |
| 61 | + * Row type snapshot using metadata key names. Stored because ProjectPushDownSpec may narrow the |
| 62 | + * context's row type during restore. |
| 63 | + */ |
| 64 | + @JsonProperty(FIELD_NAME_PREDICATE_ROW_TYPE) |
| 65 | + private final RowType predicateRowType; |
| 66 | + |
| 67 | + @JsonCreator |
| 68 | + public MetadataFilterPushDownSpec( |
| 69 | + @JsonProperty(FIELD_NAME_PREDICATES) List<RexNode> predicates, |
| 70 | + @JsonProperty(FIELD_NAME_PREDICATE_ROW_TYPE) RowType predicateRowType) { |
| 71 | + this.predicates = new ArrayList<>(checkNotNull(predicates)); |
| 72 | + this.predicateRowType = checkNotNull(predicateRowType); |
| 73 | + } |
| 74 | + |
| 75 | + public List<RexNode> getPredicates() { |
| 76 | + return predicates; |
| 77 | + } |
| 78 | + |
| 79 | + @Override |
| 80 | + public void apply(DynamicTableSource tableSource, SourceAbilityContext context) { |
| 81 | + // Use stored predicateRowType; context's row type may be narrowed by ProjectPushDownSpec. |
| 82 | + MetadataFilterResult result = |
| 83 | + applyMetadataFilters(predicates, predicateRowType, tableSource, context); |
| 84 | + if (result.getAcceptedFilters().size() != predicates.size()) { |
| 85 | + throw new TableException("All metadata predicates should be accepted here."); |
| 86 | + } |
| 87 | + } |
| 88 | + |
| 89 | + /** |
| 90 | + * Converts RexNode predicates to ResolvedExpressions using the given row type and calls |
| 91 | + * applyMetadataFilters on the source. The row type must already use metadata key names. |
| 92 | + */ |
| 93 | + public static MetadataFilterResult applyMetadataFilters( |
| 94 | + List<RexNode> predicates, |
| 95 | + RowType metadataKeyRowType, |
| 96 | + DynamicTableSource tableSource, |
| 97 | + SourceAbilityContext context) { |
| 98 | + if (!(tableSource instanceof SupportsReadingMetadata)) { |
| 99 | + throw new TableException( |
| 100 | + String.format( |
| 101 | + "%s does not support SupportsReadingMetadata.", |
| 102 | + tableSource.getClass().getName())); |
| 103 | + } |
| 104 | + SupportsReadingMetadata readingMetadata = (SupportsReadingMetadata) tableSource; |
| 105 | + if (!readingMetadata.supportsMetadataFilterPushDown()) { |
| 106 | + throw new TableException( |
| 107 | + String.format( |
| 108 | + "%s no longer supports metadata filter push-down.", |
| 109 | + tableSource.getClass().getName())); |
| 110 | + } |
| 111 | + List<ResolvedExpression> resolved = |
| 112 | + FilterPushDownSpec.resolvePredicates( |
| 113 | + predicates, metadataKeyRowType, tableSource, context); |
| 114 | + return readingMetadata.applyMetadataFilters(resolved); |
| 115 | + } |
| 116 | + |
| 117 | + @Override |
| 118 | + public boolean needAdjustFieldReferenceAfterProjection() { |
| 119 | + return true; |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + public String getDigests(SourceAbilityContext context) { |
| 124 | + final List<String> expressionStrs = new ArrayList<>(); |
| 125 | + for (RexNode rexNode : predicates) { |
| 126 | + expressionStrs.add( |
| 127 | + FlinkRexUtil.getExpressionString( |
| 128 | + rexNode, |
| 129 | + JavaScalaConversionUtil.toScala(predicateRowType.getFieldNames()))); |
| 130 | + } |
| 131 | + |
| 132 | + return String.format( |
| 133 | + "metadataFilter=[%s]", |
| 134 | + expressionStrs.stream() |
| 135 | + .reduce((l, r) -> String.format("and(%s, %s)", l, r)) |
| 136 | + .orElse("")); |
| 137 | + } |
| 138 | + |
| 139 | + @Override |
| 140 | + public boolean equals(Object o) { |
| 141 | + if (this == o) { |
| 142 | + return true; |
| 143 | + } |
| 144 | + if (o == null || getClass() != o.getClass()) { |
| 145 | + return false; |
| 146 | + } |
| 147 | + if (!super.equals(o)) { |
| 148 | + return false; |
| 149 | + } |
| 150 | + MetadataFilterPushDownSpec that = (MetadataFilterPushDownSpec) o; |
| 151 | + return Objects.equals(predicates, that.predicates) |
| 152 | + && Objects.equals(predicateRowType, that.predicateRowType); |
| 153 | + } |
| 154 | + |
| 155 | + @Override |
| 156 | + public int hashCode() { |
| 157 | + return Objects.hash(super.hashCode(), predicates, predicateRowType); |
| 158 | + } |
| 159 | +} |
0 commit comments