JingsongLi commented on a change in pull request #41: URL: https://github.com/apache/flink-table-store/pull/41#discussion_r825969369
########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreTableSink.java ########## @@ -0,0 +1,109 @@ +/* + * 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.flink.table.store.connector.sink; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.catalog.CatalogLock; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.RequireCatalogLock; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.log.LogSinkProvider; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** Table sink to create {@link StoreSink}. */ +public class StoreTableSink + implements DynamicTableSink, SupportsOverwrite, SupportsPartitioning, RequireCatalogLock { + + private final TableStore tableStore; + @Nullable private final LogSinkProvider logSinkProvider; + + private LinkedHashMap<String, String> staticPartitions = new LinkedHashMap<>(); + private boolean overwrite; + @Nullable private CatalogLock.Factory lockFactory; + + public StoreTableSink(TableStore tableStore, @Nullable LogSinkProvider logSinkProvider) { + this.tableStore = tableStore; + this.logSinkProvider = logSinkProvider; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.all(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + return (DataStreamSinkProvider) + (providerContext, dataStream) -> + tableStore + .sinkBuilder() + .withInput( + new DataStream<>( + dataStream.getExecutionEnvironment(), + dataStream.getTransformation())) + .withLockFactory(lockFactory) + .withLogSinkProvider(logSinkProvider) + .withOverwritePartition(staticPartitions) + .build(); + } + + @Override + public DynamicTableSink copy() { + StoreTableSink copied = new StoreTableSink(tableStore, logSinkProvider); + copied.staticPartitions = new LinkedHashMap<>(staticPartitions); + copied.overwrite = overwrite; + return copied; + } + + @Override + public String asSummaryString() { + return "StoreTableSink"; Review comment: "TableStore" ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreTableSink.java ########## @@ -0,0 +1,109 @@ +/* + * 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.flink.table.store.connector.sink; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.catalog.CatalogLock; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.RequireCatalogLock; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.log.LogSinkProvider; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** Table sink to create {@link StoreSink}. */ +public class StoreTableSink + implements DynamicTableSink, SupportsOverwrite, SupportsPartitioning, RequireCatalogLock { + + private final TableStore tableStore; + @Nullable private final LogSinkProvider logSinkProvider; + + private LinkedHashMap<String, String> staticPartitions = new LinkedHashMap<>(); + private boolean overwrite; + @Nullable private CatalogLock.Factory lockFactory; + + public StoreTableSink(TableStore tableStore, @Nullable LogSinkProvider logSinkProvider) { + this.tableStore = tableStore; + this.logSinkProvider = logSinkProvider; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.all(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + return (DataStreamSinkProvider) + (providerContext, dataStream) -> + tableStore + .sinkBuilder() + .withInput( + new DataStream<>( + dataStream.getExecutionEnvironment(), + dataStream.getTransformation())) + .withLockFactory(lockFactory) + .withLogSinkProvider(logSinkProvider) + .withOverwritePartition(staticPartitions) + .build(); + } + + @Override + public DynamicTableSink copy() { + StoreTableSink copied = new StoreTableSink(tableStore, logSinkProvider); + copied.staticPartitions = new LinkedHashMap<>(staticPartitions); + copied.overwrite = overwrite; + return copied; + } + + @Override + public String asSummaryString() { + return "StoreTableSink"; + } + + @Override + public void applyStaticPartition(Map<String, String> partition) { Review comment: Just `new HashMap<>(partition)` is OK. - File store don't need linked map ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreTableSink.java ########## @@ -0,0 +1,109 @@ +/* + * 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.flink.table.store.connector.sink; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.catalog.CatalogLock; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.RequireCatalogLock; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.log.LogSinkProvider; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** Table sink to create {@link StoreSink}. */ +public class StoreTableSink + implements DynamicTableSink, SupportsOverwrite, SupportsPartitioning, RequireCatalogLock { + + private final TableStore tableStore; + @Nullable private final LogSinkProvider logSinkProvider; + + private LinkedHashMap<String, String> staticPartitions = new LinkedHashMap<>(); + private boolean overwrite; + @Nullable private CatalogLock.Factory lockFactory; + + public StoreTableSink(TableStore tableStore, @Nullable LogSinkProvider logSinkProvider) { + this.tableStore = tableStore; + this.logSinkProvider = logSinkProvider; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.all(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + return (DataStreamSinkProvider) + (providerContext, dataStream) -> + tableStore + .sinkBuilder() + .withInput( + new DataStream<>( + dataStream.getExecutionEnvironment(), + dataStream.getTransformation())) + .withLockFactory(lockFactory) + .withLogSinkProvider(logSinkProvider) + .withOverwritePartition(staticPartitions) + .build(); + } + + @Override + public DynamicTableSink copy() { + StoreTableSink copied = new StoreTableSink(tableStore, logSinkProvider); + copied.staticPartitions = new LinkedHashMap<>(staticPartitions); + copied.overwrite = overwrite; + return copied; Review comment: Lack catalog lock here? ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/StoreTableSink.java ########## @@ -0,0 +1,109 @@ +/* + * 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.flink.table.store.connector.sink; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.catalog.CatalogLock; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.RequireCatalogLock; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.log.LogSinkProvider; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** Table sink to create {@link StoreSink}. */ +public class StoreTableSink + implements DynamicTableSink, SupportsOverwrite, SupportsPartitioning, RequireCatalogLock { + + private final TableStore tableStore; + @Nullable private final LogSinkProvider logSinkProvider; + + private LinkedHashMap<String, String> staticPartitions = new LinkedHashMap<>(); + private boolean overwrite; + @Nullable private CatalogLock.Factory lockFactory; + + public StoreTableSink(TableStore tableStore, @Nullable LogSinkProvider logSinkProvider) { + this.tableStore = tableStore; + this.logSinkProvider = logSinkProvider; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode changelogMode) { + return ChangelogMode.all(); Review comment: We can take a look to `log.changelog-mode` and pk: - has pk: all or upsert - no pk: all Note here: - all is `return requestedMode`. - upsert, you can take a look to `HBaseDynamicTableSink`. ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/source/StoreTableSource.java ########## @@ -0,0 +1,215 @@ +/* + * 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.flink.table.store.connector.source; + +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.file.predicate.And; +import org.apache.flink.table.store.file.predicate.PredicateConverter; +import org.apache.flink.table.store.log.LogSourceProvider; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Table source to create {@link FileStoreSource} under batch mode or change-tracking is disabled. + * For streaming mode with change-tracking enabled and FULL scan mode, it will create a {@link + * org.apache.flink.connector.base.source.hybrid.HybridSource} of {@link FileStoreSource} and kafka + * log source created by {@link LogSourceProvider}. + */ +public class StoreTableSource + implements ScanTableSource, SupportsFilterPushDown, SupportsProjectionPushDown { + + private final TableStore tableStore; + private final boolean streaming; + @Nullable private final LogSourceProvider logSourceProvider; + + @Nullable private List<ResolvedExpression> partitionFilters; + @Nullable private List<ResolvedExpression> fieldFilters; + @Nullable private int[][] projectFields; + + public StoreTableSource( + TableStore tableStore, + boolean streaming, + @Nullable LogSourceProvider logSourceProvider) { + this.tableStore = tableStore; + this.streaming = streaming; + this.logSourceProvider = logSourceProvider; + } + + @Override + public ChangelogMode getChangelogMode() { + return streaming + ? tableStore.valueCountMode() ? ChangelogMode.all() : ChangelogMode.upsert() Review comment: `ChangelogMode.upsert()` can be optimized, we can add a TODO here. ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/source/StoreTableSource.java ########## @@ -0,0 +1,215 @@ +/* + * 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.flink.table.store.connector.source; + +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.file.predicate.And; +import org.apache.flink.table.store.file.predicate.PredicateConverter; +import org.apache.flink.table.store.log.LogSourceProvider; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Table source to create {@link FileStoreSource} under batch mode or change-tracking is disabled. + * For streaming mode with change-tracking enabled and FULL scan mode, it will create a {@link + * org.apache.flink.connector.base.source.hybrid.HybridSource} of {@link FileStoreSource} and kafka + * log source created by {@link LogSourceProvider}. + */ +public class StoreTableSource + implements ScanTableSource, SupportsFilterPushDown, SupportsProjectionPushDown { + + private final TableStore tableStore; + private final boolean streaming; + @Nullable private final LogSourceProvider logSourceProvider; + + @Nullable private List<ResolvedExpression> partitionFilters; + @Nullable private List<ResolvedExpression> fieldFilters; + @Nullable private int[][] projectFields; + + public StoreTableSource( + TableStore tableStore, + boolean streaming, + @Nullable LogSourceProvider logSourceProvider) { + this.tableStore = tableStore; + this.streaming = streaming; + this.logSourceProvider = logSourceProvider; + } + + @Override + public ChangelogMode getChangelogMode() { + return streaming + ? tableStore.valueCountMode() ? ChangelogMode.all() : ChangelogMode.upsert() + : ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + TableStore.SourceBuilder builder = + tableStore + .sourceBuilder() + .withContinuousMode(streaming) + .withHybridMode(streaming && logSourceProvider != null) + .withLogSourceProvider(logSourceProvider) + .withProjection(projectFields) + .withPartitionPredicate( + partitionFilters != null + ? partitionFilters.stream() + .map( + filter -> + filter.accept( + PredicateConverter + .CONVERTER)) + .reduce(And::new) + .orElse(null) + : null) + .withFieldPredicate( + fieldFilters != null + ? fieldFilters.stream() + .map( + filter -> + filter.accept( + PredicateConverter + .CONVERTER)) + .reduce(And::new) + .orElse(null) + : null); + return SourceProvider.of(builder.build()); + } + + @Override + public DynamicTableSource copy() { + StoreTableSource copied = new StoreTableSource(tableStore, streaming, logSourceProvider); + copied.partitionFilters = partitionFilters; + copied.fieldFilters = fieldFilters; + return copied; + } + + @Override + public String asSummaryString() { + return "StoreTableSource"; Review comment: TableStore too ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/source/StoreTableSource.java ########## @@ -0,0 +1,215 @@ +/* + * 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.flink.table.store.connector.source; + +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.file.predicate.And; +import org.apache.flink.table.store.file.predicate.PredicateConverter; +import org.apache.flink.table.store.log.LogSourceProvider; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Table source to create {@link FileStoreSource} under batch mode or change-tracking is disabled. + * For streaming mode with change-tracking enabled and FULL scan mode, it will create a {@link + * org.apache.flink.connector.base.source.hybrid.HybridSource} of {@link FileStoreSource} and kafka + * log source created by {@link LogSourceProvider}. + */ +public class StoreTableSource + implements ScanTableSource, SupportsFilterPushDown, SupportsProjectionPushDown { + + private final TableStore tableStore; + private final boolean streaming; + @Nullable private final LogSourceProvider logSourceProvider; + + @Nullable private List<ResolvedExpression> partitionFilters; + @Nullable private List<ResolvedExpression> fieldFilters; + @Nullable private int[][] projectFields; + + public StoreTableSource( + TableStore tableStore, + boolean streaming, + @Nullable LogSourceProvider logSourceProvider) { + this.tableStore = tableStore; + this.streaming = streaming; + this.logSourceProvider = logSourceProvider; + } + + @Override + public ChangelogMode getChangelogMode() { + return streaming + ? tableStore.valueCountMode() ? ChangelogMode.all() : ChangelogMode.upsert() + : ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + TableStore.SourceBuilder builder = + tableStore + .sourceBuilder() + .withContinuousMode(streaming) + .withHybridMode(streaming && logSourceProvider != null) + .withLogSourceProvider(logSourceProvider) + .withProjection(projectFields) + .withPartitionPredicate( + partitionFilters != null + ? partitionFilters.stream() + .map( + filter -> Review comment: Add a static method `convert(List<Expression>)` to `PredicateConverter ` ########## File path: flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/source/StoreTableSource.java ########## @@ -0,0 +1,215 @@ +/* + * 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.flink.table.store.connector.source; + +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TypeLiteralExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.store.connector.TableStore; +import org.apache.flink.table.store.file.predicate.And; +import org.apache.flink.table.store.file.predicate.PredicateConverter; +import org.apache.flink.table.store.log.LogSourceProvider; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Table source to create {@link FileStoreSource} under batch mode or change-tracking is disabled. + * For streaming mode with change-tracking enabled and FULL scan mode, it will create a {@link + * org.apache.flink.connector.base.source.hybrid.HybridSource} of {@link FileStoreSource} and kafka + * log source created by {@link LogSourceProvider}. + */ +public class StoreTableSource + implements ScanTableSource, SupportsFilterPushDown, SupportsProjectionPushDown { + + private final TableStore tableStore; + private final boolean streaming; + @Nullable private final LogSourceProvider logSourceProvider; + + @Nullable private List<ResolvedExpression> partitionFilters; + @Nullable private List<ResolvedExpression> fieldFilters; + @Nullable private int[][] projectFields; + + public StoreTableSource( + TableStore tableStore, + boolean streaming, + @Nullable LogSourceProvider logSourceProvider) { + this.tableStore = tableStore; + this.streaming = streaming; + this.logSourceProvider = logSourceProvider; + } + + @Override + public ChangelogMode getChangelogMode() { + return streaming + ? tableStore.valueCountMode() ? ChangelogMode.all() : ChangelogMode.upsert() + : ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + TableStore.SourceBuilder builder = + tableStore + .sourceBuilder() + .withContinuousMode(streaming) + .withHybridMode(streaming && logSourceProvider != null) + .withLogSourceProvider(logSourceProvider) + .withProjection(projectFields) + .withPartitionPredicate( + partitionFilters != null + ? partitionFilters.stream() + .map( + filter -> + filter.accept( + PredicateConverter + .CONVERTER)) + .reduce(And::new) + .orElse(null) + : null) + .withFieldPredicate( + fieldFilters != null + ? fieldFilters.stream() + .map( + filter -> + filter.accept( + PredicateConverter + .CONVERTER)) + .reduce(And::new) + .orElse(null) + : null); + return SourceProvider.of(builder.build()); + } + + @Override + public DynamicTableSource copy() { + StoreTableSource copied = new StoreTableSource(tableStore, streaming, logSourceProvider); + copied.partitionFilters = partitionFilters; + copied.fieldFilters = fieldFilters; + return copied; + } + + @Override + public String asSummaryString() { + return "StoreTableSource"; + } + + @Override + public Result applyFilters(List<ResolvedExpression> filters) { + if (tableStore.partitioned()) { + classifyFilters(filters); + } else { + fieldFilters = filters; + } + return Result.of( Review comment: The filter is a best effort, so just return: `return Result.of(new ArrayList<>(filters), new ArrayList<>(filters));` Accept all, and remain all. Planner will do filter again. -- 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]
