This is an automated email from the ASF dual-hosted git repository. jark pushed a commit to branch release-1.11 in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.11 by this push: new 33f8524 [FLINK-17996][table-planner-blink] Fix NPE in CatalogTableStatisticsConverter.convertToColumnStats method 33f8524 is described below commit 33f852417e7cdca0211232d60ba340927f3f0d81 Author: SteNicholas <programg...@163.com> AuthorDate: Tue Jun 2 17:57:46 2020 +0800 [FLINK-17996][table-planner-blink] Fix NPE in CatalogTableStatisticsConverter.convertToColumnStats method This closes #12419 --- .../utils/CatalogTableStatisticsConverter.java | 10 +++-- .../utils/CatalogTableStatisticsConverterTest.java | 51 ++++++++++++++++++++++ 2 files changed, 58 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverter.java index 7b396bd..e6a10ea 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverter.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.utils; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; @@ -61,12 +62,15 @@ public class CatalogTableStatisticsConverter { return new TableStats(rowCount, columnStatsMap); } - private static Map<String, ColumnStats> convertToColumnStatsMap( + @VisibleForTesting + public static Map<String, ColumnStats> convertToColumnStatsMap( Map<String, CatalogColumnStatisticsDataBase> columnStatisticsData) { Map<String, ColumnStats> columnStatsMap = new HashMap<>(); for (Map.Entry<String, CatalogColumnStatisticsDataBase> entry : columnStatisticsData.entrySet()) { - ColumnStats columnStats = convertToColumnStats(entry.getValue()); - columnStatsMap.put(entry.getKey(), columnStats); + if (entry.getValue() != null) { + ColumnStats columnStats = convertToColumnStats(entry.getValue()); + columnStatsMap.put(entry.getKey(), columnStats); + } } return columnStatsMap; } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverterTest.java new file mode 100644 index 0000000..c48ce5a --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/CatalogTableStatisticsConverterTest.java @@ -0,0 +1,51 @@ +/* + * 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.catalog.stats.CatalogColumnStatisticsDataBase; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataString; +import org.apache.flink.table.plan.stats.ColumnStats; + +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link CatalogTableStatisticsConverter}. + */ +public class CatalogTableStatisticsConverterTest { + + @Test + public void testConvertToColumnStatsMapWithNullColumnStatisticsData() { + Map<String, CatalogColumnStatisticsDataBase> columnStatisticsDataBaseMap = new HashMap<>(); + columnStatisticsDataBaseMap.put("first", new CatalogColumnStatisticsDataString(10L, 5.2, 3L, 100L)); + columnStatisticsDataBaseMap.put("second", null); + Map<String, ColumnStats> columnStatsMap = CatalogTableStatisticsConverter.convertToColumnStatsMap(columnStatisticsDataBaseMap); + assertNotNull(columnStatsMap); + assertEquals(columnStatisticsDataBaseMap.size() - 1, columnStatsMap.size()); + assertTrue(columnStatsMap.containsKey("first")); + assertFalse(columnStatsMap.containsKey("second")); + } +}