You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/27 13:02:27 UTC

[GitHub] [flink] godfreyhe commented on a diff in pull request #20008: [FLINK-27990][table-planner] Parquet format supports reporting statis…

godfreyhe commented on code in PR #20008:
URL: https://github.com/apache/flink/pull/20008#discussion_r907345600


##########
flink-formats/pom.xml:
##########
@@ -31,6 +31,7 @@ under the License.
 
 	<properties>
 		<flink.format.parquet.version>1.12.2</flink.format.parquet.version>
+		<guava.version>30.0-jre</guava.version>

Review Comment:
   if only parquet requires guava, please move it to flink-parquet module



##########
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemStatisticsReportTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.parquet;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.plan.stats.ColumnStats;
+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.runtime.utils.BatchTableEnvUtil;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+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.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test for statistics functionality in {@link ParquetFileFormatFactory} in the case of file system
+ * source.
+ */
+public class ParquetFileSystemStatisticsReportTest extends TableTestBase {
+    private BatchTableTestUtil util;
+    private TableEnvironment tEnv;
+    private String path1;
+
+    @Before
+    public void setup() throws IOException {
+        util = batchTestUtil(TableConfig.getDefault());
+        tEnv = util.getTableEnv();
+        path1 = tempFolder().newFolder().toURI().getPath();
+
+        BatchTableEnvUtil.registerCollection(
+                tEnv,
+                "originalT",
+                TestData.buildInData(),
+                TestData.buildInType(),
+                "a,b,c,d,e,f,g,h,i,j");
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE parquetFileSystemTable (\n"

Review Comment:
   please defined all types for the test, such as: BINARY, VARBINARY, TIMESTAMP_WITHOUT_TIME_ZONE, decimal(5,2), decimal(38,2), array, etc



##########
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemStatisticsReportTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.parquet;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.plan.stats.ColumnStats;
+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.runtime.utils.BatchTableEnvUtil;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+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.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test for statistics functionality in {@link ParquetFileFormatFactory} in the case of file system
+ * source.
+ */
+public class ParquetFileSystemStatisticsReportTest extends TableTestBase {
+    private BatchTableTestUtil util;
+    private TableEnvironment tEnv;
+    private String path1;
+
+    @Before
+    public void setup() throws IOException {
+        util = batchTestUtil(TableConfig.getDefault());
+        tEnv = util.getTableEnv();
+        path1 = tempFolder().newFolder().toURI().getPath();
+
+        BatchTableEnvUtil.registerCollection(
+                tEnv,
+                "originalT",
+                TestData.buildInData(),
+                TestData.buildInType(),
+                "a,b,c,d,e,f,g,h,i,j");
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE parquetFileSystemTable (\n"
+                                + "  a boolean,\n"
+                                + "  b tinyint,\n"
+                                + "  c int, \n"
+                                + "  d bigint, \n"
+                                + "  e double, \n"
+                                + "  f string, \n"
+                                + "  g decimal(14,2), \n"
+                                + "  h date, \n"
+                                + "  i time, \n"
+                                + "  j timestamp(3) \n"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        path1, String.join(",\n", formatProperties()));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='parquet'");
+        ret.add("'parquet.utc-timezone'='true'");
+        ret.add("'parquet.compression'='gzip'");
+        return ret.toArray(new String[0]);
+    }
+
+    @Test
+    public void testParquetFileSystemStatisticsReportWithSingleFile()
+            throws ExecutionException, InterruptedException {
+        tEnv.executeSql(
+                        "insert into parquetFileSystemTable select a, b, c, d, e, f,"
+                                + " c * 3.14 as g, h, i, j"
+                                + " from originalT")

Review Comment:
   please add some test for filters on DATE, TIME, TIMESTAMP_xxx, DECIMAL, which can check whether the value type is expected



##########
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemStatisticsReportTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.parquet;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.plan.stats.ColumnStats;
+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.runtime.utils.BatchTableEnvUtil;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+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.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test for statistics functionality in {@link ParquetFileFormatFactory} in the case of file system
+ * source.
+ */
+public class ParquetFileSystemStatisticsReportTest extends TableTestBase {
+    private BatchTableTestUtil util;
+    private TableEnvironment tEnv;
+    private String path1;
+
+    @Before
+    public void setup() throws IOException {
+        util = batchTestUtil(TableConfig.getDefault());
+        tEnv = util.getTableEnv();
+        path1 = tempFolder().newFolder().toURI().getPath();
+
+        BatchTableEnvUtil.registerCollection(
+                tEnv,
+                "originalT",
+                TestData.buildInData(),
+                TestData.buildInType(),
+                "a,b,c,d,e,f,g,h,i,j");
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE parquetFileSystemTable (\n"
+                                + "  a boolean,\n"
+                                + "  b tinyint,\n"
+                                + "  c int, \n"
+                                + "  d bigint, \n"
+                                + "  e double, \n"
+                                + "  f string, \n"
+                                + "  g decimal(14,2), \n"
+                                + "  h date, \n"
+                                + "  i time, \n"
+                                + "  j timestamp(3) \n"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        path1, String.join(",\n", formatProperties()));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='parquet'");
+        ret.add("'parquet.utc-timezone'='true'");
+        ret.add("'parquet.compression'='gzip'");
+        return ret.toArray(new String[0]);
+    }
+
+    @Test
+    public void testParquetFileSystemStatisticsReportWithSingleFile()
+            throws ExecutionException, InterruptedException {
+        tEnv.executeSql(
+                        "insert into parquetFileSystemTable select a, b, c, d, e, f,"
+                                + " c * 3.14 as g, h, i, j"
+                                + " from originalT")
+                .await();
+        assertThat(new File(path1).listFiles()).isNotNull().hasSize(1);
+
+        FlinkStatistic statistic =
+                getStatisticsFromOptimizedPlan("select * from parquetFileSystemTable");
+
+        Map<String, ColumnStats> expectedColumnStatsMap = new HashMap<>();
+        expectedColumnStatsMap.put("a", new ColumnStats.Builder().setNullCount(1L).build());
+        expectedColumnStatsMap.put(
+                "b", new ColumnStats.Builder().setMax(3).setMin(1).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "c", new ColumnStats.Builder().setMax(2).setMin(-4).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "d", new ColumnStats.Builder().setMax(3L).setMin(-5L).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "e", new ColumnStats.Builder().setMax(90.08).setMin(-0.8).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "f",
+                new ColumnStats.Builder().setMax("e fg").setMin("abcd").setNullCount(1L).build());
+        expectedColumnStatsMap.put(
+                "g",
+                new ColumnStats.Builder()
+                        .setMax(BigDecimal.valueOf(6.28))
+                        .setMin(BigDecimal.valueOf(-12.56))
+                        .setNullCount(0L)
+                        .build());
+        expectedColumnStatsMap.put(
+                "h",
+                new ColumnStats.Builder()
+                        .setMax(Date.valueOf("2017-12-12"))
+                        .setMin(Date.valueOf("2017-12-12"))
+                        .setNullCount(1L)
+                        .build());
+        expectedColumnStatsMap.put(
+                "i",
+                new ColumnStats.Builder()
+                        .setMax(Time.valueOf("10:08:09"))
+                        .setMin(Time.valueOf("10:08:09"))
+                        .setNullCount(0L)
+                        .build());
+        // Now parquet store timestamp as type int96, and int96 now not support statistics, so
+        // timestamp not support statistics now.
+        expectedColumnStatsMap.put("j", new ColumnStats.Builder().setNullCount(0L).build());
+        int expectedRowCount = 3;
+
+        assertThat(statistic.getTableStats())
+                .isEqualTo(new TableStats(expectedRowCount, expectedColumnStatsMap));
+    }
+
+    @Test
+    public void testParquetFileSystemStatisticsReportWithMultiFile()
+            throws ExecutionException, InterruptedException {
+        // write first parquet file.
+        tEnv.executeSql(
+                        "insert into parquetFileSystemTable select a, b, c, d, e, f,"
+                                + " c * 3.14 as g, h, i, j"
+                                + " from originalT")
+                .await();
+        // write second parquet file.
+        tEnv.executeSql(
+                        "insert into parquetFileSystemTable select a, b, c, d, e, f,"
+                                + " c * 3.14 as g, h, i, j"
+                                + " from originalT")
+                .await();
+        assertThat(new File(path1).listFiles()).isNotNull().hasSize(2);
+
+        FlinkStatistic statistic =
+                getStatisticsFromOptimizedPlan("select * from parquetFileSystemTable");
+
+        Map<String, ColumnStats> expectedColumnStatsMap = new HashMap<>();
+        expectedColumnStatsMap.put("a", new ColumnStats.Builder().setNullCount(2L).build());
+        expectedColumnStatsMap.put(
+                "b", new ColumnStats.Builder().setMax(3).setMin(1).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "c", new ColumnStats.Builder().setMax(2).setMin(-4).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "d", new ColumnStats.Builder().setMax(3L).setMin(-5L).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "e", new ColumnStats.Builder().setMax(90.08).setMin(-0.8).setNullCount(0L).build());
+        expectedColumnStatsMap.put(
+                "f",
+                new ColumnStats.Builder().setMax("e fg").setMin("abcd").setNullCount(2L).build());
+        expectedColumnStatsMap.put(
+                "g",
+                new ColumnStats.Builder()
+                        .setMax(BigDecimal.valueOf(6.28))
+                        .setMin(BigDecimal.valueOf(-12.56))
+                        .setNullCount(0L)
+                        .build());
+        expectedColumnStatsMap.put(
+                "h",
+                new ColumnStats.Builder()
+                        .setMax(Date.valueOf("2017-12-12"))
+                        .setMin(Date.valueOf("2017-12-12"))
+                        .setNullCount(2L)
+                        .build());
+        expectedColumnStatsMap.put(
+                "i",
+                new ColumnStats.Builder()
+                        .setMax(Time.valueOf("10:08:09"))
+                        .setMin(Time.valueOf("10:08:09"))
+                        .setNullCount(0L)
+                        .build());
+        // Now parquet store timestamp as type int96, and int96 now not support statistics, so
+        // timestamp not support statistics now.
+        expectedColumnStatsMap.put("j", new ColumnStats.Builder().setNullCount(0L).build());
+        int expectedRowCount = 6;
+
+        assertThat(statistic.getTableStats())
+                .isEqualTo(new TableStats(expectedRowCount, expectedColumnStatsMap));
+    }
+
+    private FlinkStatistic getStatisticsFromOptimizedPlan(String sql) {

Review Comment:
   extract abstract class for csv, parquet, orc.



##########
flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetFileFormatFactory.java:
##########
@@ -142,5 +180,248 @@ public BulkFormat<RowData, FileSourceSplit> createRuntimeDecoder(
         public ChangelogMode getChangelogMode() {
             return ChangelogMode.insertOnly();
         }
+
+        @Override
+        public TableStats reportStatistics(List<Path> files, DataType producedDataType) {
+            try {
+                Configuration hadoopConfig = getParquetConfiguration(formatOptions);
+                Map<String, Statistics<?>> columnStatisticsMap = new HashMap<>();
+                RowType producedRowType = (RowType) producedDataType.getLogicalType();
+                long rowCount = 0;
+                for (Path file : files) {
+                    rowCount += updateStatistics(hadoopConfig, file, columnStatisticsMap);
+                }
+                Map<String, ColumnStats> columnStatsMap =
+                        convertToColumnStats(columnStatisticsMap, producedRowType);
+                return new TableStats(rowCount, columnStatsMap);
+            } catch (Exception e) {
+                return TableStats.UNKNOWN;
+            }
+        }
+
+        private Map<String, ColumnStats> convertToColumnStats(
+                Map<String, Statistics<?>> columnStatisticsMap, RowType producedRowType) {
+            Map<String, ColumnStats> columnStatMap = new HashMap<>();
+            for (String column : producedRowType.getFieldNames()) {
+                Statistics<?> statistics = columnStatisticsMap.get(column);
+                if (statistics == null) {
+                    continue;
+                }
+                ColumnStats columnStats =
+                        convertToColumnStats(
+                                producedRowType.getTypeAt(producedRowType.getFieldIndex(column)),
+                                statistics);
+                columnStatMap.put(column, columnStats);
+            }
+            return columnStatMap;
+        }
+
+        private ColumnStats convertToColumnStats(
+                LogicalType logicalType, Statistics<?> statistics) {
+            ColumnStats.Builder builder =
+                    new ColumnStats.Builder().setNullCount(statistics.getNumNulls());
+
+            switch (logicalType.getTypeRoot()) {
+                case BOOLEAN:
+                    if (statistics instanceof BooleanStatistics) {
+                        builder.setMin(null).setMax(null);
+                        break;
+                    } else {
+                        return null;
+                    }
+                case TINYINT:
+                case SMALLINT:
+                case INTEGER:
+                case BIGINT:
+                    if (statistics instanceof IntStatistics) {
+                        builder.setMin(((IntStatistics) statistics).getMin())
+                                .setMax(((IntStatistics) statistics).getMax());
+                        break;
+                    } else if (statistics instanceof LongStatistics) {
+                        builder.setMin(((LongStatistics) statistics).getMin())
+                                .setMax(((LongStatistics) statistics).getMax());
+                        break;
+                    } else {
+                        return null;
+                    }
+                case DOUBLE:
+                    if (statistics instanceof DoubleStatistics) {
+                        builder.setMin(((DoubleStatistics) statistics).getMin())
+                                .setMax(((DoubleStatistics) statistics).getMax());
+                        break;
+                    } else {
+                        return null;
+                    }
+                case FLOAT:
+                    if (statistics instanceof FloatStatistics) {
+                        builder.setMin(((FloatStatistics) statistics).getMin())
+                                .setMax(((FloatStatistics) statistics).getMax());
+                        break;
+                    } else {
+                        return null;
+                    }
+                case DATE:
+                    if (statistics instanceof IntStatistics) {
+                        Date min =
+                                Date.valueOf(
+                                        DateTimeUtils.formatDate(
+                                                ((IntStatistics) statistics).getMin()));
+                        Date max =
+                                Date.valueOf(
+                                        DateTimeUtils.formatDate(
+                                                ((IntStatistics) statistics).getMax()));
+                        builder.setMin(min).setMax(max);
+                        break;
+                    } else {
+                        return null;
+                    }
+                case TIME_WITHOUT_TIME_ZONE:
+                    if (statistics instanceof IntStatistics) {
+                        Time min =
+                                Time.valueOf(
+                                        DateTimeUtils.toLocalTime(
+                                                ((IntStatistics) statistics).getMin()));
+                        Time max =
+                                Time.valueOf(
+                                        DateTimeUtils.toLocalTime(
+                                                ((IntStatistics) statistics).getMax()));
+                        builder.setMin(min).setMax(max);
+                        break;
+                    } else {
+                        return null;
+                    }
+                case CHAR:
+                case VARCHAR:
+                case BINARY:
+                case VARBINARY:
+                    if (statistics instanceof BinaryStatistics) {
+                        Binary min = ((BinaryStatistics) statistics).genericGetMin();
+                        Binary max = ((BinaryStatistics) statistics).genericGetMax();
+                        if (min != null) {
+                            builder.setMin(min.toStringUsingUTF8());

Review Comment:
   no max and min for VARBINARY and BINARY



##########
flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFileSystemStatisticsReportTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.parquet;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.plan.stats.ColumnStats;
+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.runtime.utils.BatchTableEnvUtil;
+import org.apache.flink.table.planner.runtime.utils.TestData;
+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.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test for statistics functionality in {@link ParquetFileFormatFactory} in the case of file system
+ * source.
+ */
+public class ParquetFileSystemStatisticsReportTest extends TableTestBase {

Review Comment:
   there are multiple duplicated test with ParquetFormatStatisticsReportTest, please test the special case for Parquet with file system cases



-- 
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