You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/02 04:50:19 UTC

[flink] branch master updated: [FLINK-13427][hive] HiveCatalog's createFunction fails when function name has upper-case characters

This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 36fdbef  [FLINK-13427][hive] HiveCatalog's createFunction fails when function name has upper-case characters
36fdbef is described below

commit 36fdbefc098421781a7cd56ed3440336d29f5811
Author: Rui Li <li...@apache.org>
AuthorDate: Mon Jul 29 13:27:42 2019 +0800

    [FLINK-13427][hive] HiveCatalog's createFunction fails when function name has upper-case characters
    
    This closes #9254
---
 .../flink/table/catalog/hive/HiveCatalog.java      |  4 +-
 .../hive/HiveCatalogGenericMetadataTest.java       |  3 +-
 .../catalog/hive/HiveCatalogHiveMetadataTest.java  |  3 +-
 .../table/catalog/hive/HiveCatalogTestBase.java    | 45 ++++++++++++++++++++++
 4 files changed, 50 insertions(+), 5 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 ba0921c..261180f 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
@@ -82,6 +82,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.ql.io.StorageFormatDescriptor;
 import org.apache.hadoop.hive.ql.io.StorageFormatFactory;
 import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
+import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1070,7 +1071,8 @@ public class HiveCatalog extends AbstractCatalog {
 			function.getClassName();
 
 		return new Function(
-			functionPath.getObjectName(),
+			// due to https://issues.apache.org/jira/browse/HIVE-22053, we have to normalize function name ourselves
+			HiveStringUtils.normalizeIdentifier(functionPath.getObjectName()),
 			functionPath.getDatabaseName(),
 			functionClassName,
 			null,			// Owner name
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogGenericMetadataTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogGenericMetadataTest.java
index 2bd0310..b9a52b6 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogGenericMetadataTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogGenericMetadataTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.catalog.hive;
 
 import org.apache.flink.table.catalog.CatalogPartition;
-import org.apache.flink.table.catalog.CatalogTestBase;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -27,7 +26,7 @@ import org.junit.Test;
 /**
  * Test for HiveCatalog on generic metadata.
  */
-public class HiveCatalogGenericMetadataTest extends CatalogTestBase {
+public class HiveCatalogGenericMetadataTest extends HiveCatalogTestBase {
 
 	@BeforeClass
 	public static void init() {
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 c10d31c..95e09b4 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
@@ -23,7 +23,6 @@ import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.catalog.CatalogPartitionSpec;
 import org.apache.flink.table.catalog.CatalogTable;
 import org.apache.flink.table.catalog.CatalogTableImpl;
-import org.apache.flink.table.catalog.CatalogTestBase;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBinary;
@@ -47,7 +46,7 @@ import static org.junit.Assert.assertFalse;
 /**
  * Test for HiveCatalog on Hive metadata.
  */
-public class HiveCatalogHiveMetadataTest extends CatalogTestBase {
+public class HiveCatalogHiveMetadataTest extends HiveCatalogTestBase {
 
 	@BeforeClass
 	public static void init() {
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java
new file mode 100644
index 0000000..232c250
--- /dev/null
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java
@@ -0,0 +1,45 @@
+/*
+ * 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.catalog.hive;
+
+import org.apache.flink.table.catalog.CatalogTestBase;
+import org.apache.flink.table.catalog.ObjectPath;
+
+import org.junit.Test;
+
+/**
+ * Base class for testing HiveCatalog.
+ */
+public abstract class HiveCatalogTestBase extends CatalogTestBase {
+
+	// ------ functions ------
+
+	@Test
+	public void testCreateFunctionCaseInsensitive() throws Exception {
+		catalog.createDatabase(db1, createDb(), false);
+
+		String functionName = "myUdf";
+		ObjectPath functionPath = new ObjectPath(db1, functionName);
+		catalog.createFunction(functionPath, createFunction(), false);
+		// make sure we can get the function
+		catalog.getFunction(functionPath);
+
+		catalog.dropFunction(functionPath, false);
+	}
+}