reswqa commented on code in PR #20008: URL: https://github.com/apache/flink/pull/20008#discussion_r901135405
########## flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/ParquetFilesystemStatisticsReportTest.java: ########## @@ -0,0 +1,242 @@ +/* + * 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.Objects; +import java.util.concurrent.ExecutionException; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for statistics functionality in {@link ParquetFileFormatFactory}. */ +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(Objects.requireNonNull(new File(path1).listFiles()).length).isEqualTo(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(Objects.requireNonNull(new File(path1).listFiles()).length).isEqualTo(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) { + 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 { Review Comment: The class `FlinkStatisticVisitor` appears to be repeated in several test classes for support multiple format reporting statistic. Can you avoid repeating yourself by introducing a public test parent class or extracting this class separately? -- 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]
