You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by lz...@apache.org on 2020/09/07 06:27:51 UTC
[flink] branch release-1.11 updated: [FLINK-19061][hive]
HiveCatalog fails to get partition column stats if partition value contains
special characters
This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 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 7047eed [FLINK-19061][hive] HiveCatalog fails to get partition column stats if partition value contains special characters
7047eed is described below
commit 7047eed55f66081048dbb72229141329126fdcda
Author: Rui Li <li...@apache.org>
AuthorDate: Mon Sep 7 14:19:52 2020 +0800
[FLINK-19061][hive] HiveCatalog fails to get partition column stats if partition value contains special characters
This closes #13274
---
.../org/apache/flink/table/catalog/hive/HiveCatalog.java | 15 +++++++--------
.../table/catalog/hive/HiveCatalogHiveMetadataTest.java | 5 ++++-
2 files changed, 11 insertions(+), 9 deletions(-)
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java
index 6dac487..46eb2fd 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java
@@ -78,6 +78,7 @@ import org.apache.flink.util.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.TableType;
@@ -1305,7 +1306,7 @@ public class HiveCatalog extends AbstractCatalog {
try {
Partition hivePartition = getHivePartition(tablePath, partitionSpec);
Table hiveTable = getHiveTable(tablePath);
- String partName = getPartitionName(tablePath, partitionSpec, hiveTable);
+ String partName = getEscapedPartitionName(tablePath, partitionSpec, hiveTable);
client.updatePartitionColumnStatistics(HiveStatsUtil.createPartitionColumnStats(
hivePartition, partName, columnStatistics.getColumnStatisticsData(), hiveVersion));
} catch (TableNotExistException | PartitionSpecInvalidException e) {
@@ -1318,14 +1319,12 @@ public class HiveCatalog extends AbstractCatalog {
}
}
- private String getPartitionName(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, Table hiveTable) throws PartitionSpecInvalidException {
+ // make a valid partition name that escapes special characters
+ private String getEscapedPartitionName(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, Table hiveTable) throws PartitionSpecInvalidException {
List<String> partitionCols = getFieldNames(hiveTable.getPartitionKeys());
List<String> partitionVals = getOrderedFullPartitionValues(partitionSpec, partitionCols, tablePath);
- List<String> partKVs = new ArrayList<>();
- for (int i = 0; i < partitionCols.size(); i++) {
- partKVs.add(partitionCols.get(i) + "=" + partitionVals.get(i));
- }
- return String.join("/", partKVs);
+ String defaultPartName = getHiveConf().getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
+ return FileUtils.makePartName(partitionCols, partitionVals, defaultPartName);
}
@Override
@@ -1375,7 +1374,7 @@ public class HiveCatalog extends AbstractCatalog {
try {
Partition partition = getHivePartition(tablePath, partitionSpec);
Table hiveTable = getHiveTable(tablePath);
- String partName = getPartitionName(tablePath, partitionSpec, hiveTable);
+ String partName = getEscapedPartitionName(tablePath, partitionSpec, hiveTable);
List<String> partNames = new ArrayList<>();
partNames.add(partName);
Map<String, List<ColumnStatisticsObj>> partitionColumnStatistics =
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java
index c0144ab..7cf9a52 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java
@@ -156,7 +156,10 @@ public class HiveCatalogHiveMetadataTest extends HiveCatalogMetadataTestBase {
catalog.createDatabase(db1, createDb(), false);
CatalogTable catalogTable = createPartitionedTable();
catalog.createTable(path1, catalogTable, false);
- CatalogPartitionSpec partitionSpec = createPartitionSpec();
+ CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(new HashMap<String, String>() {{
+ put("second", "2010-04-21 09:45:00");
+ put("third", "2000");
+ }});
catalog.createPartition(path1, partitionSpec, createPartition(), true);
Map<String, CatalogColumnStatisticsDataBase> columnStatisticsDataBaseMap = new HashMap<>();
columnStatisticsDataBaseMap.put("first", new CatalogColumnStatisticsDataString(10L, 5.2, 3L, 100L));