You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "JingsongLi (via GitHub)" <gi...@apache.org> on 2023/03/03 07:55:34 UTC

[GitHub] [flink-table-store] JingsongLi commented on a diff in pull request #426: [FLINK-30323] Support table statistics in table store

JingsongLi commented on code in PR #426:
URL: https://github.com/apache/flink-table-store/pull/426#discussion_r1124125253


##########
flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/source/TableStoreSource.java:
##########
@@ -209,4 +233,19 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
         return TableFunctionProvider.of(
                 new FileStoreLookupFunction(table, projection, joinKey, predicate));
     }
+
+    @Override
+    public TableStats reportStatistics() {

Review Comment:
   Can we document this? This may not be a positive path always. Flink SQL provides `table.optimizer.source.report-statistics-enabled` to close this.



##########
flink-table-store-flink/flink-table-store-flink-1.15/src/main/java/org/apache/flink/table/store/connector/source/TableStoreSource.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.TableFunctionProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.store.CoreOptions.ChangelogProducer;
+import org.apache.flink.table.store.CoreOptions.LogChangelogMode;
+import org.apache.flink.table.store.CoreOptions.LogConsistency;
+import org.apache.flink.table.store.annotation.VisibleForTesting;
+import org.apache.flink.table.store.connector.FlinkConnectorOptions;
+import org.apache.flink.table.store.connector.TableStoreDataStreamScanProvider;
+import org.apache.flink.table.store.connector.lookup.FileStoreLookupFunction;
+import org.apache.flink.table.store.file.predicate.Predicate;
+import org.apache.flink.table.store.log.LogSourceProvider;
+import org.apache.flink.table.store.log.LogStoreTableFactory;
+import org.apache.flink.table.store.options.Options;
+import org.apache.flink.table.store.table.AppendOnlyFileStoreTable;
+import org.apache.flink.table.store.table.ChangelogValueCountFileStoreTable;
+import org.apache.flink.table.store.table.ChangelogWithKeyFileStoreTable;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.utils.Projection;
+
+import javax.annotation.Nullable;
+
+import java.util.stream.IntStream;
+
+import static org.apache.flink.table.store.CoreOptions.CHANGELOG_PRODUCER;
+import static org.apache.flink.table.store.CoreOptions.LOG_CHANGELOG_MODE;
+import static org.apache.flink.table.store.CoreOptions.LOG_CONSISTENCY;
+import static org.apache.flink.table.store.CoreOptions.LOG_SCAN_REMOVE_NORMALIZE;
+
+/**
+ * Table source to create {@link StaticFileStoreSource} or {@link ContinuousFileStoreSource} under
+ * batch mode or change-tracking is disabled. For streaming mode with change-tracking enabled and
+ * FULL scan mode, it will create a {@code HybridSource} of {@link StaticFileStoreSource} and kafka
+ * log source created by {@link LogSourceProvider}.
+ */
+public class TableStoreSource extends FlinkTableSource

Review Comment:
   Can we just introduce a `StatisticTableSource` in flink-common? this can avoid copying for large number of codes.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org