kbendick commented on code in PR #4553: URL: https://github.com/apache/iceberg/pull/4553#discussion_r849989015
########## flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java: ########## @@ -0,0 +1,193 @@ +/* + * 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.iceberg.flink; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.FlinkSource; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * Flink Iceberg table source. + */ +public class IcebergTableSource + implements ScanTableSource, SupportsProjectionPushDown, SupportsFilterPushDown, SupportsLimitPushDown { + + private int[] projectedFields; + private long limit; + private List<Expression> filters; + + private final TableLoader loader; + private final TableSchema schema; + private final Map<String, String> properties; + private final boolean isLimitPushDown; + private final ReadableConfig readableConfig; + + private IcebergTableSource(IcebergTableSource toCopy) { + this.loader = toCopy.loader; + this.schema = toCopy.schema; + this.properties = toCopy.properties; + this.projectedFields = toCopy.projectedFields; + this.isLimitPushDown = toCopy.isLimitPushDown; + this.limit = toCopy.limit; + this.filters = toCopy.filters; + this.readableConfig = toCopy.readableConfig; + } + + public IcebergTableSource(TableLoader loader, TableSchema schema, Map<String, String> properties, + ReadableConfig readableConfig) { + this(loader, schema, properties, null, false, -1, ImmutableList.of(), readableConfig); + } + + private IcebergTableSource(TableLoader loader, TableSchema schema, Map<String, String> properties, + int[] projectedFields, boolean isLimitPushDown, + long limit, List<Expression> filters, ReadableConfig readableConfig) { + this.loader = loader; + this.schema = schema; + this.properties = properties; + this.projectedFields = projectedFields; + this.isLimitPushDown = isLimitPushDown; + this.limit = limit; + this.filters = filters; + this.readableConfig = readableConfig; + } + + @Override + public void applyProjection(int[][] projectFields) { + this.projectedFields = new int[projectFields.length]; + for (int i = 0; i < projectFields.length; i++) { + Preconditions.checkArgument(projectFields[i].length == 1, + "Don't support nested projection in iceberg source now."); + this.projectedFields[i] = projectFields[i][0]; + } + } + + private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) { + return FlinkSource.forRowData() + .env(execEnv) + .tableLoader(loader) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConf(readableConfig) + .build(); + } + + private TableSchema getProjectedSchema() { + if (projectedFields == null) { + return schema; + } else { + String[] fullNames = schema.getFieldNames(); + DataType[] fullTypes = schema.getFieldDataTypes(); + return TableSchema.builder().fields( + Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), + Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)).build(); + } + } + + @Override + public void applyLimit(long newLimit) { + this.limit = newLimit; + } + + @Override + public Result applyFilters(List<ResolvedExpression> flinkFilters) { + List<ResolvedExpression> acceptedFilters = Lists.newArrayList(); + List<Expression> expressions = Lists.newArrayList(); + + for (ResolvedExpression resolvedExpression : flinkFilters) { + Optional<Expression> icebergExpression = FlinkFilters.convert(resolvedExpression); + if (icebergExpression.isPresent()) { + expressions.add(icebergExpression.get()); + acceptedFilters.add(resolvedExpression); + } + } + + this.filters = expressions; + return Result.of(acceptedFilters, flinkFilters); + } + + @Override + public boolean supportsNestedProjection() { + // TODO: support nested projection + return false; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + + // UPDATED - Needs to be added as support for other signature is entirely removed. + // This should probably be ported to 1.14 as well to make future changes + // easier to backport. + @Override + public DataStream<RowData> produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + return createDataStream(execEnv); + } Review Comment: Updated function signature to include `ProviderContext`. The old signature, without `ProviderContext` has been deprecated and now can not be used any further. We likely we want to backport this change to 1.14, where the `ProviderContext`-based signature is available, to be able to more easily apply future changes people might make in this file. Associated issue: https://github.com/apache/iceberg/issues/4634 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
