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/07/11 07:23:33 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_r917612921


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/StatisticsReportTestBase.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.planner.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+
+/** The base class for statistics report testing. */
+public abstract class StatisticsReportTestBase extends TestLogger {
+
+    protected TableEnvironment tEnv;
+    protected File folder;
+
+    @BeforeEach
+    public void setup(@TempDir File file) throws Exception {
+        folder = file;
+        tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+    }
+
+    @AfterEach
+    public void after() {
+        TestValuesTableFactory.clearAllData();
+    }
+
+    protected void createFileSystemSource(String format) {
+        // now format can be parquet, orc and csv.
+        String[] properties;
+        if (format.equals("orc")) {
+            properties = formatOrcProperties();
+        } else if (format.equals("parquet")) {
+            properties = formatParquetProperties();
+        } else {
+            // default is Csv format
+            properties = formatCsvProperties();
+        }
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE sourceTable (\n"
+                                + "%s"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        String.join(",\n", ddlTypesMapToStringList(ddlTypesMap())),
+                        folder,
+                        String.join(",\n", properties));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatParquetProperties() {
+        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]);
+    }
+
+    private String[] formatOrcProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='orc'");
+        ret.add("'orc.compress'='snappy'");
+        return ret.toArray(new String[0]);
+    }
+
+    private String[] formatCsvProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format' = 'csv'");
+        return ret.toArray(new String[0]);
+    }
+
+    protected Map<String, String> ddlTypesMap() {
+        Map<String, String> ddlTypesMap = new LinkedHashMap<>();
+        ddlTypesMap.put("boolean", "a");
+        ddlTypesMap.put("tinyint", "b");
+        ddlTypesMap.put("smallint", "c");
+        ddlTypesMap.put("int", "d");
+        ddlTypesMap.put("bigint", "e");
+        ddlTypesMap.put("float", "f");
+        ddlTypesMap.put("double", "g");
+        ddlTypesMap.put("string", "h");
+        ddlTypesMap.put("decimal(5,2)", "i");
+        ddlTypesMap.put("decimal(14,2)", "j");
+        ddlTypesMap.put("decimal(38,2)", "k");
+        ddlTypesMap.put("date", "l");
+        ddlTypesMap.put("timestamp(3)", "m");
+        ddlTypesMap.put("timestamp(9)", "n");
+        ddlTypesMap.put("binary(1)", "o");
+        ddlTypesMap.put("varbinary(1)", "p");
+        ddlTypesMap.put("time", "q");
+
+        return ddlTypesMap;

Review Comment:
   add complex types: row, array, map



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/StatisticsReportTestBase.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.planner.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+
+/** The base class for statistics report testing. */
+public abstract class StatisticsReportTestBase extends TestLogger {
+
+    protected TableEnvironment tEnv;
+    protected File folder;
+
+    @BeforeEach
+    public void setup(@TempDir File file) throws Exception {
+        folder = file;
+        tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+    }
+
+    @AfterEach
+    public void after() {
+        TestValuesTableFactory.clearAllData();
+    }
+
+    protected void createFileSystemSource(String format) {
+        // now format can be parquet, orc and csv.
+        String[] properties;
+        if (format.equals("orc")) {
+            properties = formatOrcProperties();
+        } else if (format.equals("parquet")) {
+            properties = formatParquetProperties();
+        } else {
+            // default is Csv format
+            properties = formatCsvProperties();

Review Comment:
   provide a abstract method, let the sub-classes to implement it



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/StatisticsReportTestBase.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.planner.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+
+/** The base class for statistics report testing. */
+public abstract class StatisticsReportTestBase extends TestLogger {
+
+    protected TableEnvironment tEnv;
+    protected File folder;
+
+    @BeforeEach
+    public void setup(@TempDir File file) throws Exception {
+        folder = file;
+        tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+    }
+
+    @AfterEach
+    public void after() {
+        TestValuesTableFactory.clearAllData();
+    }
+
+    protected void createFileSystemSource(String format) {
+        // now format can be parquet, orc and csv.
+        String[] properties;
+        if (format.equals("orc")) {
+            properties = formatOrcProperties();
+        } else if (format.equals("parquet")) {
+            properties = formatParquetProperties();
+        } else {
+            // default is Csv format
+            properties = formatCsvProperties();
+        }
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE sourceTable (\n"
+                                + "%s"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        String.join(",\n", ddlTypesMapToStringList(ddlTypesMap())),
+                        folder,
+                        String.join(",\n", properties));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatParquetProperties() {
+        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]);
+    }
+
+    private String[] formatOrcProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='orc'");
+        ret.add("'orc.compress'='snappy'");
+        return ret.toArray(new String[0]);
+    }
+
+    private String[] formatCsvProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format' = 'csv'");
+        return ret.toArray(new String[0]);
+    }
+
+    protected Map<String, String> ddlTypesMap() {
+        Map<String, String> ddlTypesMap = new LinkedHashMap<>();
+        ddlTypesMap.put("boolean", "a");
+        ddlTypesMap.put("tinyint", "b");
+        ddlTypesMap.put("smallint", "c");
+        ddlTypesMap.put("int", "d");
+        ddlTypesMap.put("bigint", "e");
+        ddlTypesMap.put("float", "f");
+        ddlTypesMap.put("double", "g");
+        ddlTypesMap.put("string", "h");
+        ddlTypesMap.put("decimal(5,2)", "i");
+        ddlTypesMap.put("decimal(14,2)", "j");
+        ddlTypesMap.put("decimal(38,2)", "k");
+        ddlTypesMap.put("date", "l");
+        ddlTypesMap.put("timestamp(3)", "m");
+        ddlTypesMap.put("timestamp(9)", "n");

Review Comment:
   TIMESTAMP_WITHOUT_TIME_ZONE
   TIMESTAMP_WITH_LOCAL_TIME_ZONE



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/StatisticsReportTestBase.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.planner.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+
+/** The base class for statistics report testing. */
+public abstract class StatisticsReportTestBase extends TestLogger {
+
+    protected TableEnvironment tEnv;
+    protected File folder;
+
+    @BeforeEach
+    public void setup(@TempDir File file) throws Exception {
+        folder = file;
+        tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+    }
+
+    @AfterEach
+    public void after() {
+        TestValuesTableFactory.clearAllData();
+    }
+
+    protected void createFileSystemSource(String format) {
+        // now format can be parquet, orc and csv.
+        String[] properties;
+        if (format.equals("orc")) {
+            properties = formatOrcProperties();
+        } else if (format.equals("parquet")) {
+            properties = formatParquetProperties();
+        } else {
+            // default is Csv format
+            properties = formatCsvProperties();
+        }
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE sourceTable (\n"
+                                + "%s"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        String.join(",\n", ddlTypesMapToStringList(ddlTypesMap())),
+                        folder,
+                        String.join(",\n", properties));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatParquetProperties() {
+        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]);
+    }
+
+    private String[] formatOrcProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='orc'");
+        ret.add("'orc.compress'='snappy'");
+        return ret.toArray(new String[0]);
+    }
+
+    private String[] formatCsvProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format' = 'csv'");
+        return ret.toArray(new String[0]);
+    }
+
+    protected Map<String, String> ddlTypesMap() {
+        Map<String, String> ddlTypesMap = new LinkedHashMap<>();
+        ddlTypesMap.put("boolean", "a");

Review Comment:
   the field name should be meaningful, such as: `f_boolean`



##########
flink-formats/flink-parquet/pom.xml:
##########
@@ -160,6 +160,13 @@ under the License.
 
 		<!-- Tests -->
 
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+			<scope>test</scope>

Review Comment:
   add some comments why we need guava



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/StatisticsReportTestBase.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.planner.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.File;
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
+
+/** The base class for statistics report testing. */
+public abstract class StatisticsReportTestBase extends TestLogger {
+
+    protected TableEnvironment tEnv;
+    protected File folder;
+
+    @BeforeEach
+    public void setup(@TempDir File file) throws Exception {
+        folder = file;
+        tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+    }
+
+    @AfterEach
+    public void after() {
+        TestValuesTableFactory.clearAllData();
+    }
+
+    protected void createFileSystemSource(String format) {
+        // now format can be parquet, orc and csv.
+        String[] properties;
+        if (format.equals("orc")) {
+            properties = formatOrcProperties();
+        } else if (format.equals("parquet")) {
+            properties = formatParquetProperties();
+        } else {
+            // default is Csv format
+            properties = formatCsvProperties();
+        }
+
+        String ddl1 =
+                String.format(
+                        "CREATE TABLE sourceTable (\n"
+                                + "%s"
+                                + ") with (\n"
+                                + " 'connector' = 'filesystem',"
+                                + " 'path' = '%s',"
+                                + "%s )",
+                        String.join(",\n", ddlTypesMapToStringList(ddlTypesMap())),
+                        folder,
+                        String.join(",\n", properties));
+        tEnv.executeSql(ddl1);
+    }
+
+    private String[] formatParquetProperties() {
+        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]);
+    }
+
+    private String[] formatOrcProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format'='orc'");
+        ret.add("'orc.compress'='snappy'");
+        return ret.toArray(new String[0]);
+    }
+
+    private String[] formatCsvProperties() {
+        List<String> ret = new ArrayList<>();
+        ret.add("'format' = 'csv'");
+        return ret.toArray(new String[0]);
+    }
+
+    protected Map<String, String> ddlTypesMap() {
+        Map<String, String> ddlTypesMap = new LinkedHashMap<>();
+        ddlTypesMap.put("boolean", "a");
+        ddlTypesMap.put("tinyint", "b");
+        ddlTypesMap.put("smallint", "c");
+        ddlTypesMap.put("int", "d");
+        ddlTypesMap.put("bigint", "e");
+        ddlTypesMap.put("float", "f");
+        ddlTypesMap.put("double", "g");
+        ddlTypesMap.put("string", "h");
+        ddlTypesMap.put("decimal(5,2)", "i");
+        ddlTypesMap.put("decimal(14,2)", "j");
+        ddlTypesMap.put("decimal(38,2)", "k");
+        ddlTypesMap.put("date", "l");
+        ddlTypesMap.put("timestamp(3)", "m");
+        ddlTypesMap.put("timestamp(9)", "n");
+        ddlTypesMap.put("binary(1)", "o");
+        ddlTypesMap.put("varbinary(1)", "p");
+        ddlTypesMap.put("time", "q");
+
+        return ddlTypesMap;
+    }
+
+    protected Map<String, List<Object>> getDataMap() {

Review Comment:
   this method should be `private`



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