fsk119 commented on code in PR #20007: URL: https://github.com/apache/flink/pull/20007#discussion_r903487186
########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; Review Comment: I think we can introduce an option and let users determine how many lines to sample. ########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; + try { + long totalSize = 0; + int sampledLineCnt = 0; + long sampledTotalSize = 0; Review Comment: totalSize -> totalFileSize sampledLineCnt -> sampledRowCnt sampledTotalSize -> sampledRowSize ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemStatisticsReportTest.java: ########## @@ -0,0 +1,102 @@ +/* + * 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.formats.csv; + +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelVisitor; +import org.apache.calcite.rel.core.TableScan; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for statistics functionality in {@link CsvFormatFactory}. */ +public class CsvFilesystemStatisticsReportTest extends TableTestBase { + private BatchTableTestUtil util; + private TableEnvironment tEnv; + + @Before + public void setup() throws IOException { + util = batchTestUtil(TableConfig.getDefault()); + tEnv = util.getTableEnv(); + String path1 = tempFolder().newFile().getAbsolutePath(); + writeData(new File(path1), Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world")); + + String ddl1 = + String.format( + "CREATE TABLE sourceTable (\n" + + " a bigint,\n" + + " b int,\n" + + " c varchar\n" + + ") with (\n" + + " 'connector' = 'filesystem'," + + " 'format' = 'csv'," + + " 'path' = '%s')", + path1); + tEnv.executeSql(ddl1); + } + + @Test + public void testCsvFileSystemStatisticsReport() { + FlinkStatistic statistic = getStatisticsFromOptimizedPlan("select * from sourceTable"); + assertThat(statistic.getTableStats()).isEqualTo(new TableStats(3)); + } + + private void writeData(File file, List<String> data) throws IOException { + Files.write(file.toPath(), String.join("\n", data).getBytes()); Review Comment: In windows system, it uses "\r\n" as line feed. ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemStatisticsReportTest.java: ########## @@ -0,0 +1,102 @@ +/* + * 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.formats.csv; + +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelVisitor; +import org.apache.calcite.rel.core.TableScan; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for statistics functionality in {@link CsvFormatFactory}. */ +public class CsvFilesystemStatisticsReportTest extends TableTestBase { + private BatchTableTestUtil util; + private TableEnvironment tEnv; + + @Before + public void setup() throws IOException { + util = batchTestUtil(TableConfig.getDefault()); + tEnv = util.getTableEnv(); + String path1 = tempFolder().newFile().getAbsolutePath(); + writeData(new File(path1), Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world")); + + String ddl1 = Review Comment: ddl1 -> ddl ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemStatisticsReportTest.java: ########## @@ -0,0 +1,102 @@ +/* + * 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.formats.csv; + +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelVisitor; +import org.apache.calcite.rel.core.TableScan; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for statistics functionality in {@link CsvFormatFactory}. */ +public class CsvFilesystemStatisticsReportTest extends TableTestBase { + private BatchTableTestUtil util; + private TableEnvironment tEnv; + + @Before + public void setup() throws IOException { + util = batchTestUtil(TableConfig.getDefault()); + tEnv = util.getTableEnv(); + String path1 = tempFolder().newFile().getAbsolutePath(); + writeData(new File(path1), Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world")); + + String ddl1 = + String.format( + "CREATE TABLE sourceTable (\n" + + " a bigint,\n" + + " b int,\n" + + " c varchar\n" + + ") with (\n" + + " 'connector' = 'filesystem'," + + " 'format' = 'csv'," + + " 'path' = '%s')", + path1); + tEnv.executeSql(ddl1); + } + + @Test + public void testCsvFileSystemStatisticsReport() { + FlinkStatistic statistic = getStatisticsFromOptimizedPlan("select * from sourceTable"); + assertThat(statistic.getTableStats()).isEqualTo(new TableStats(3)); + } + + private void writeData(File file, List<String> data) throws IOException { + Files.write(file.toPath(), String.join("\n", data).getBytes()); + } + + private FlinkStatistic getStatisticsFromOptimizedPlan(String sql) { + RelNode relNode = TableTestUtil.toRelNode(tEnv.sqlQuery(sql)); + RelNode optimized = util.getPlanner().optimize(relNode); + FlinkStatisticVisitor visitor = new FlinkStatisticVisitor(); + visitor.go(optimized); + return visitor.result; + } + + private static class FlinkStatisticVisitor extends RelVisitor { + private FlinkStatistic result = null; + + @Override + public void visit(RelNode node, int ordinal, RelNode parent) { + if (node instanceof TableScan) { + Preconditions.checkArgument(result == null); + TableSourceTable table = (TableSourceTable) node.getTable(); + result = table.getStatistic(); + } + super.visit(node, ordinal, parent); + } + } Review Comment: Why not just use the Reporter to get the statistic? ########## flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemStatisticsReportTest.java: ########## @@ -0,0 +1,102 @@ +/* + * 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.formats.csv; + +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.plan.stats.TableStats; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelVisitor; +import org.apache.calcite.rel.core.TableScan; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for statistics functionality in {@link CsvFormatFactory}. */ +public class CsvFilesystemStatisticsReportTest extends TableTestBase { Review Comment: Could you add some bad cases: - The file doesn't exist. - There are some dirty files in the directory. - ... ########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; + try { + long totalSize = 0; + int sampledLineCnt = 0; + long sampledTotalSize = 0; + for (Path file : files) { + FileSystem fs = FileSystem.get(file.toUri()); + FileStatus status = fs.getFileStatus(file); + totalSize += status.getLen(); + + // sample the line size + if (sampledLineCnt < totalSampleLineCnt) { + try (InputStreamReader isr = + new InputStreamReader( + Files.newInputStream(new File(file.toUri()).toPath()))) { + BufferedReader br = new BufferedReader(isr); Review Comment: ``` try(InputStreamReader isr = new InputStreamReader( Files.newInputStream(new File(file.toUri()).toPath())); BufferedReader br = new BufferedReader(isr)) { } ``` ########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; + try { + long totalSize = 0; + int sampledLineCnt = 0; + long sampledTotalSize = 0; + for (Path file : files) { + FileSystem fs = FileSystem.get(file.toUri()); + FileStatus status = fs.getFileStatus(file); + totalSize += status.getLen(); + + // sample the line size + if (sampledLineCnt < totalSampleLineCnt) { + try (InputStreamReader isr = + new InputStreamReader( + Files.newInputStream(new File(file.toUri()).toPath()))) { + BufferedReader br = new BufferedReader(isr); + String line; + while (sampledLineCnt < totalSampleLineCnt + && (line = br.readLine()) != null) { + sampledLineCnt += 1; + sampledTotalSize += line.length(); + } + } + } + } + if (sampledTotalSize == 0) { + return TableStats.UNKNOWN; + } + + int realSampledLineCnt = Math.min(totalSampleLineCnt, sampledLineCnt); + int estimatedRowCount = (int) (totalSize * realSampledLineCnt / sampledTotalSize); Review Comment: It may overflow in the Big Data. I think we should do something to deal with the corner cases. ########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; + try { + long totalSize = 0; + int sampledLineCnt = 0; + long sampledTotalSize = 0; + for (Path file : files) { + FileSystem fs = FileSystem.get(file.toUri()); + FileStatus status = fs.getFileStatus(file); + totalSize += status.getLen(); + + // sample the line size + if (sampledLineCnt < totalSampleLineCnt) { + try (InputStreamReader isr = + new InputStreamReader( + Files.newInputStream(new File(file.toUri()).toPath()))) { + BufferedReader br = new BufferedReader(isr); + String line; + while (sampledLineCnt < totalSampleLineCnt + && (line = br.readLine()) != null) { + sampledLineCnt += 1; + sampledTotalSize += line.length(); Review Comment: Why use line.length() here? I think it should be ``` line.getBytes(StandardCharsets.UTF_8).length ``` ########## flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java: ########## @@ -136,6 +147,45 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder( public ChangelogMode getChangelogMode() { return ChangelogMode.insertOnly(); } + + @Override + public TableStats reportStatistics(List<Path> files, DataType producedDataType) { + final int totalSampleLineCnt = 100; + try { + long totalSize = 0; + int sampledLineCnt = 0; + long sampledTotalSize = 0; + for (Path file : files) { + FileSystem fs = FileSystem.get(file.toUri()); + FileStatus status = fs.getFileStatus(file); + totalSize += status.getLen(); + + // sample the line size + if (sampledLineCnt < totalSampleLineCnt) { + try (InputStreamReader isr = + new InputStreamReader( + Files.newInputStream(new File(file.toUri()).toPath()))) { + BufferedReader br = new BufferedReader(isr); + String line; + while (sampledLineCnt < totalSampleLineCnt + && (line = br.readLine()) != null) { + sampledLineCnt += 1; + sampledTotalSize += line.length(); + } + } + } + } + if (sampledTotalSize == 0) { + return TableStats.UNKNOWN; + } + + int realSampledLineCnt = Math.min(totalSampleLineCnt, sampledLineCnt); + int estimatedRowCount = (int) (totalSize * realSampledLineCnt / sampledTotalSize); + return new TableStats(estimatedRowCount); Review Comment: Why the estimation doesn't give the column stat? -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org