You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by bl...@apache.org on 2019/05/23 17:24:36 UTC

[flink] branch master updated: [FLINK-12236][hive] Support Hive function in HiveCatalog

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

bli 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 f74bbec  [FLINK-12236][hive] Support Hive function in HiveCatalog
f74bbec is described below

commit f74bbec1794a8a836d2ffc8f248e032610f35157
Author: bowen.li <bo...@gmail.com>
AuthorDate: Tue May 21 20:54:59 2019 -0700

    [FLINK-12236][hive] Support Hive function in HiveCatalog
    
    This PR creates HiveCatalogFunction class and adds support for Hive function operations in HiveCatalog.
    
    This closes #8507.
---
 .../flink/table/catalog/hive/HiveCatalog.java      | 27 ++++++++++--
 .../table/catalog/hive/HiveCatalogFunction.java    | 49 +++++++++++++++++++++
 .../catalog/hive/HiveCatalogHiveMetadataTest.java  | 51 +---------------------
 3 files changed, 74 insertions(+), 53 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 22fe08b..1f0bddc 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
@@ -659,6 +659,8 @@ public class HiveCatalog implements Catalog {
 		Function hiveFunction;
 		if (function instanceof GenericCatalogFunction) {
 			hiveFunction = instantiateHiveFunction(functionPath, (GenericCatalogFunction) function);
+		} else if (function instanceof HiveCatalogFunction) {
+			hiveFunction = instantiateHiveFunction(functionPath, (HiveCatalogFunction) function);
 		} else {
 			throw new CatalogException(
 				String.format("Unsupported catalog function type %s", function.getClass().getName()));
@@ -694,8 +696,10 @@ public class HiveCatalog implements Catalog {
 			}
 
 			Function hiveFunction;
-			if (existingFunction instanceof GenericCatalogFunction && newFunction instanceof GenericCatalogFunction) {
-					hiveFunction = instantiateHiveFunction(functionPath, (GenericCatalogFunction) newFunction);
+			if (newFunction instanceof GenericCatalogFunction) {
+				hiveFunction = instantiateHiveFunction(functionPath, (GenericCatalogFunction) newFunction);
+			} else if (newFunction instanceof HiveCatalogFunction) {
+				hiveFunction = instantiateHiveFunction(functionPath, (HiveCatalogFunction) newFunction);
 			} else {
 				throw new CatalogException(
 					String.format("Unsupported catalog function type %s", newFunction.getClass().getName()));
@@ -758,7 +762,7 @@ public class HiveCatalog implements Catalog {
 			Function function = client.getFunction(functionPath.getDatabaseName(), functionPath.getObjectName());
 
 			if (function.getClassName().startsWith(FLINK_FUNCTION_PREFIX)) {
-				// TODO: extract more properties from Hive function and add to CatalogFunction's properties
+				// TODO: extract more properties from Hive function and add to GenericCatalogFunction's properties
 
 				Map<String, String> properties = new HashMap<>();
 				properties.put(GenericInMemoryCatalog.FLINK_IS_GENERIC_KEY, GenericInMemoryCatalog.FLINK_IS_GENERIC_VALUE);
@@ -766,7 +770,9 @@ public class HiveCatalog implements Catalog {
 				return new GenericCatalogFunction(
 					function.getClassName().substring(FLINK_FUNCTION_PREFIX.length()), properties);
 			} else {
-				throw new CatalogException("Hive function is not supported yet");
+				// TODO: extract more properties from Hive function and add to HiveCatalogFunction's properties
+
+				return new HiveCatalogFunction(function.getClassName());
 			}
 		} catch (NoSuchObjectException e) {
 			throw new FunctionNotExistException(catalogName, functionPath, e);
@@ -803,6 +809,19 @@ public class HiveCatalog implements Catalog {
 		);
 	}
 
+	private static Function instantiateHiveFunction(ObjectPath functionPath, HiveCatalogFunction function) {
+		return new Function(
+			functionPath.getObjectName(),
+			functionPath.getDatabaseName(),
+			function.getClassName(),
+			null,			// Owner name
+			PrincipalType.GROUP,	// Temporarily set to GROUP type because it's required by Hive. May change later
+			(int) (System.currentTimeMillis() / 1000),
+			FunctionType.JAVA,		// FunctionType only has JAVA now
+			new ArrayList<>()		// Resource URIs
+		);
+	}
+
 	// ------ stats ------
 
 	@Override
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalogFunction.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalogFunction.java
new file mode 100644
index 0000000..05d562a
--- /dev/null
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalogFunction.java
@@ -0,0 +1,49 @@
+/*
+ * 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.AbstractCatalogFunction;
+
+import java.util.HashMap;
+import java.util.Optional;
+
+/**
+ * A hive catalog function implementation.
+ */
+public class HiveCatalogFunction extends AbstractCatalogFunction {
+
+	public HiveCatalogFunction(String className) {
+		super(className, new HashMap<>());
+	}
+
+	@Override
+	public HiveCatalogFunction copy() {
+		return new HiveCatalogFunction(getClassName());
+	}
+
+	@Override
+	public Optional<String> getDescription() {
+		return Optional.empty();
+	}
+
+	@Override
+	public Optional<String> getDetailedDescription() {
+		return Optional.empty();
+	}
+}
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 6699811..1126564 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
@@ -50,53 +50,6 @@ public class HiveCatalogHiveMetadataTest extends CatalogTestBase {
 	public void testCreateTable_Streaming() throws Exception {
 	}
 
-	// ------ functions ------
-
-	public void testCreateFunction() throws Exception {
-	}
-
-	public void testCreateFunction_DatabaseNotExistException() throws Exception {
-	}
-
-	public void testCreateFunction_FunctionAlreadyExistException() throws Exception {
-	}
-
-	public void testCreateFunction_FunctionAlreadyExist_ignored() throws Exception {
-	}
-
-	public void testAlterFunction() throws Exception {
-	}
-
-	public void testAlterFunction_differentTypedFunction() throws Exception {
-	}
-
-	public void testAlterFunction_FunctionNotExistException() throws Exception {
-	}
-
-	public void testAlterFunction_FunctionNotExist_ignored() throws Exception {
-	}
-
-	public void testListFunctions() throws Exception {
-	}
-
-	public void testListFunctions_DatabaseNotExistException() throws Exception{
-	}
-
-	public void testGetFunction_FunctionNotExistException() throws Exception {
-	}
-
-	public void testGetFunction_FunctionNotExistException_NoDb() throws Exception {
-	}
-
-	public void testDropFunction() throws Exception {
-	}
-
-	public void testDropFunction_FunctionNotExistException() throws Exception {
-	}
-
-	public void testDropFunction_FunctionNotExist_ignored() throws Exception {
-	}
-
 	// ------ utils ------
 
 	@Override
@@ -182,12 +135,12 @@ public class HiveCatalogHiveMetadataTest extends CatalogTestBase {
 
 	@Override
 	protected CatalogFunction createFunction() {
-		throw new UnsupportedOperationException();
+		return new HiveCatalogFunction("test.class.name");
 	}
 
 	@Override
 	protected CatalogFunction createAnotherFunction() {
-		throw new UnsupportedOperationException();
+		return new HiveCatalogFunction("test.another.class.name");
 	}
 
 	@Override