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/07/01 05:03:03 UTC

[flink] branch master updated: [FLINK-13022][table][hive] unify catalog function implementations

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 16cb60f  [FLINK-13022][table][hive] unify catalog function implementations
16cb60f is described below

commit 16cb60ff0df4225e8323bb3153b3ff3ba04781e8
Author: bowen.li <bo...@gmail.com>
AuthorDate: Thu Jun 27 12:01:25 2019 -0700

    [FLINK-13022][table][hive] unify catalog function implementations
    
    This PR unifies catalog function implementations.
    
    This closes #8919.
---
 .../flink/table/catalog/hive/HiveCatalog.java      | 69 +++++++++++-----------
 .../table/catalog/hive/HiveCatalogFunction.java    | 49 ---------------
 .../hive/HiveCatalogGenericMetadataTest.java       | 12 ----
 .../catalog/hive/HiveCatalogHiveMetadataTest.java  | 11 ----
 flink-python/pyflink/table/catalog.py              | 21 +------
 flink-python/pyflink/table/tests/test_catalog.py   |  4 +-
 ...talogFunction.java => CatalogFunctionImpl.java} | 30 +++++++++-
 .../table/catalog/GenericCatalogFunction.java      | 64 --------------------
 .../flink/table/catalog/CatalogTestBase.java       | 16 +++++
 .../table/catalog/GenericInMemoryCatalogTest.java  | 30 ----------
 .../apache/flink/table/catalog/CatalogTest.java    | 64 --------------------
 11 files changed, 80 insertions(+), 290 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 e8c4342..237b105 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
@@ -26,13 +26,13 @@ import org.apache.flink.table.catalog.CatalogBaseTable;
 import org.apache.flink.table.catalog.CatalogDatabase;
 import org.apache.flink.table.catalog.CatalogDatabaseImpl;
 import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogFunctionImpl;
 import org.apache.flink.table.catalog.CatalogPartition;
 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.CatalogView;
 import org.apache.flink.table.catalog.CatalogViewImpl;
-import org.apache.flink.table.catalog.GenericCatalogFunction;
 import org.apache.flink.table.catalog.GenericCatalogPartition;
 import org.apache.flink.table.catalog.ObjectPath;
 import org.apache.flink.table.catalog.config.CatalogConfig;
@@ -882,10 +882,8 @@ public class HiveCatalog extends AbstractCatalog {
 		checkNotNull(function, "function cannot be null");
 
 		Function hiveFunction;
-		if (function instanceof GenericCatalogFunction) {
-			hiveFunction = instantiateHiveFunction(functionPath, (GenericCatalogFunction) function);
-		} else if (function instanceof HiveCatalogFunction) {
-			hiveFunction = instantiateHiveFunction(functionPath, (HiveCatalogFunction) function);
+		if (function instanceof CatalogFunctionImpl) {
+			hiveFunction = instantiateHiveFunction(functionPath, function);
 		} else {
 			throw new CatalogException(
 				String.format("Unsupported catalog function type %s", function.getClass().getName()));
@@ -913,18 +911,18 @@ public class HiveCatalog extends AbstractCatalog {
 
 		try {
 			CatalogFunction existingFunction = getFunction(functionPath);
-
-			if (existingFunction.getClass() != newFunction.getClass()) {
+			boolean existingType = Boolean.valueOf(existingFunction.getProperties().get(CatalogConfig.IS_GENERIC));
+			boolean newType = Boolean.valueOf(newFunction.getProperties().get(CatalogConfig.IS_GENERIC));
+			if (existingType != newType) {
 				throw new CatalogException(
-					String.format("Function types don't match. Existing function is '%s' and new function is '%s'.",
-						existingFunction.getClass().getName(), newFunction.getClass().getName()));
+					String.format("Function types don't match. Existing function %s generic, and new function %s generic.",
+						existingType ? "is" : "isn't",
+						newType ? "is" : "isn't"));
 			}
 
 			Function hiveFunction;
-			if (newFunction instanceof GenericCatalogFunction) {
-				hiveFunction = instantiateHiveFunction(functionPath, (GenericCatalogFunction) newFunction);
-			} else if (newFunction instanceof HiveCatalogFunction) {
-				hiveFunction = instantiateHiveFunction(functionPath, (HiveCatalogFunction) newFunction);
+			if (newFunction instanceof CatalogFunctionImpl) {
+				hiveFunction = instantiateHiveFunction(functionPath, newFunction);
 			} else {
 				throw new CatalogException(
 					String.format("Unsupported catalog function type %s", newFunction.getClass().getName()));
@@ -988,17 +986,20 @@ public class HiveCatalog extends AbstractCatalog {
 			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 GenericCatalogFunction's properties
-
-				Map<String, String> properties = new HashMap<>();
-				properties.put(CatalogConfig.IS_GENERIC, String.valueOf(true));
 
-				return new GenericCatalogFunction(
-					function.getClassName().substring(FLINK_FUNCTION_PREFIX.length()), properties);
+				return new CatalogFunctionImpl(
+					function.getClassName().substring(FLINK_FUNCTION_PREFIX.length()),
+					new HashMap<String, String>() {{
+						put(CatalogConfig.IS_GENERIC, String.valueOf(true));
+					}}
+				);
 			} else {
-				// TODO: extract more properties from Hive function and add to HiveCatalogFunction's properties
-
-				return new HiveCatalogFunction(function.getClassName());
+				return new CatalogFunctionImpl(
+					function.getClassName(),
+					new HashMap<String, String>() {{
+						put(CatalogConfig.IS_GENERIC, String.valueOf(false));
+					}}
+				);
 			}
 		} catch (NoSuchObjectException e) {
 			throw new FunctionNotExistException(getName(), functionPath, e);
@@ -1022,24 +1023,20 @@ public class HiveCatalog extends AbstractCatalog {
 		}
 	}
 
-	private static Function instantiateHiveFunction(ObjectPath functionPath, GenericCatalogFunction function) {
-		return new Function(
-			functionPath.getObjectName(),
-			functionPath.getDatabaseName(),
-			FLINK_FUNCTION_PREFIX + 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
-		);
-	}
+	private static Function instantiateHiveFunction(ObjectPath functionPath, CatalogFunction function) {
+
+		boolean isGeneric = Boolean.valueOf(function.getProperties().get(CatalogConfig.IS_GENERIC));
+
+		// Hive Function does not have properties map
+		// thus, use a prefix in class name to distinguish Flink and Hive functions
+		String functionClassName = isGeneric ?
+			FLINK_FUNCTION_PREFIX + function.getClassName() :
+			function.getClassName();
 
-	private static Function instantiateHiveFunction(ObjectPath functionPath, HiveCatalogFunction function) {
 		return new Function(
 			functionPath.getObjectName(),
 			functionPath.getDatabaseName(),
-			function.getClassName(),
+			functionClassName,
 			null,			// Owner name
 			PrincipalType.GROUP,	// Temporarily set to GROUP type because it's required by Hive. May change later
 			(int) (System.currentTimeMillis() / 1000),
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
deleted file mode 100644
index 05d562a..0000000
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalogFunction.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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/HiveCatalogGenericMetadataTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogGenericMetadataTest.java
index d517a95..5bc6ff6 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
@@ -20,12 +20,10 @@ package org.apache.flink.table.catalog.hive;
 
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableSchema;
-import org.apache.flink.table.catalog.CatalogFunction;
 import org.apache.flink.table.catalog.CatalogPartition;
 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.GenericCatalogFunction;
 import org.apache.flink.table.catalog.exceptions.CatalogException;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.BinaryType;
@@ -276,16 +274,6 @@ public class HiveCatalogGenericMetadataTest extends CatalogTestBase {
 	}
 
 	@Override
-	protected CatalogFunction createFunction() {
-		return new GenericCatalogFunction(MyScalarFunction.class.getName());
-	}
-
-	@Override
-	protected CatalogFunction createAnotherFunction() {
-		return new GenericCatalogFunction(MyOtherScalarFunction.class.getName());
-	}
-
-	@Override
 	public CatalogPartition createPartition() {
 		throw new UnsupportedOperationException();
 	}
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 efe7fec..e641e23 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
@@ -20,7 +20,6 @@ package org.apache.flink.table.catalog.hive;
 
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableSchema;
-import org.apache.flink.table.catalog.CatalogFunction;
 import org.apache.flink.table.catalog.CatalogPartition;
 import org.apache.flink.table.catalog.CatalogPartitionSpec;
 import org.apache.flink.table.catalog.CatalogTable;
@@ -140,16 +139,6 @@ public class HiveCatalogHiveMetadataTest extends CatalogTestBase {
 	}
 
 	@Override
-	protected CatalogFunction createFunction() {
-		return new HiveCatalogFunction("test.class.name");
-	}
-
-	@Override
-	protected CatalogFunction createAnotherFunction() {
-		return new HiveCatalogFunction("test.another.class.name");
-	}
-
-	@Override
 	public CatalogPartition createPartition() {
 		return new HiveCatalogPartition(getBatchTableProperties());
 	}
diff --git a/flink-python/pyflink/table/catalog.py b/flink-python/pyflink/table/catalog.py
index 69252db..d283d16 100644
--- a/flink-python/pyflink/table/catalog.py
+++ b/flink-python/pyflink/table/catalog.py
@@ -23,7 +23,7 @@ from pyflink.table.table_schema import TableSchema
 
 __all__ = ['Catalog', 'CatalogDatabase', 'CatalogBaseTable', 'CatalogPartition', 'CatalogFunction',
            'ObjectPath', 'CatalogPartitionSpec', 'CatalogTableStatistics',
-           'CatalogColumnStatistics', 'HiveCatalog', 'HiveCatalogFunction',
+           'CatalogColumnStatistics', 'HiveCatalog',
            'HiveCatalogPartition']
 
 
@@ -752,11 +752,7 @@ class CatalogFunction(object):
 
     @staticmethod
     def _get(j_catalog_function):
-        if j_catalog_function.getClass().getName() == \
-                "org.apache.flink.table.catalog.hive.HiveCatalogFunction":
-            return HiveCatalogFunction(j_hive_catalog_function=j_catalog_function)
-        else:
-            return CatalogFunction(j_catalog_function)
+        return CatalogFunction(j_catalog_function)
 
     def get_class_name(self):
         """
@@ -979,19 +975,6 @@ class HiveCatalog(Catalog):
         super(HiveCatalog, self).__init__(j_hive_catalog)
 
 
-class HiveCatalogFunction(CatalogFunction):
-    """
-    A catalog function implementation for Hive.
-    """
-
-    def __int__(self, class_name=None, j_hive_catalog_function=None):
-        gateway = get_gateway()
-        if j_hive_catalog_function is None:
-            j_hive_catalog_function = \
-                gateway.jvm.org.apache.flink.table.catalog.hive.HiveCatalogFunction(class_name)
-        super(HiveCatalogFunction, self).__init__(j_hive_catalog_function)
-
-
 class HiveCatalogPartition(CatalogPartition):
     """
     A CatalogPartition implementation that represents a Partition in Hive.
diff --git a/flink-python/pyflink/table/tests/test_catalog.py b/flink-python/pyflink/table/tests/test_catalog.py
index 3f5bf12..5a340f7 100644
--- a/flink-python/pyflink/table/tests/test_catalog.py
+++ b/flink-python/pyflink/table/tests/test_catalog.py
@@ -178,13 +178,13 @@ class CatalogTestBase(PyFlinkTestCase):
     @staticmethod
     def create_function():
         gateway = get_gateway()
-        j_function = gateway.jvm.GenericCatalogFunction("MyFunction", {})
+        j_function = gateway.jvm.CatalogFunctionImpl("MyFunction", {})
         return CatalogFunction(j_function)
 
     @staticmethod
     def create_another_function():
         gateway = get_gateway()
-        j_function = gateway.jvm.GenericCatalogFunction("MyAnotherFunction", {})
+        j_function = gateway.jvm.CatalogFunctionImpl("MyAnotherFunction", {})
         return CatalogFunction(j_function)
 
     @staticmethod
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/AbstractCatalogFunction.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogFunctionImpl.java
similarity index 66%
rename from flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/AbstractCatalogFunction.java
rename to flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogFunctionImpl.java
index f4046b0..b43f4da 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/AbstractCatalogFunction.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogFunctionImpl.java
@@ -20,19 +20,21 @@ package org.apache.flink.table.catalog;
 
 import org.apache.flink.util.StringUtils;
 
+import java.util.HashMap;
 import java.util.Map;
+import java.util.Optional;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * An abstract catalog function implementation.
+ * A catalog function implementation.
  */
-public abstract class AbstractCatalogFunction implements CatalogFunction {
+public class CatalogFunctionImpl implements CatalogFunction {
 	private final String className; // Fully qualified class name of the function
 	private final Map<String, String> properties;
 
-	public AbstractCatalogFunction(String className, Map<String, String> properties) {
+	public CatalogFunctionImpl(String className, Map<String, String> properties) {
 		checkArgument(!StringUtils.isNullOrWhitespaceOnly(className), "className cannot be null or empty");
 
 		this.className = className;
@@ -49,4 +51,26 @@ public abstract class AbstractCatalogFunction implements CatalogFunction {
 		return this.properties;
 	}
 
+	@Override
+	public CatalogFunction copy() {
+		return new CatalogFunctionImpl(getClassName(), new HashMap<>(getProperties()));
+	}
+
+	@Override
+	public Optional<String> getDescription() {
+		return Optional.of("This is a user-defined function");
+	}
+
+	@Override
+	public Optional<String> getDetailedDescription() {
+		return Optional.of("This is a user-defined function");
+	}
+
+	@Override
+	public String toString() {
+		return "CatalogFunctionImpl{" +
+			", className='" + getClassName() + '\'' +
+			", properties=" + getProperties() +
+			'}';
+	}
 }
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericCatalogFunction.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericCatalogFunction.java
deleted file mode 100644
index 24bde8a..0000000
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericCatalogFunction.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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;
-
-import org.apache.flink.table.catalog.config.CatalogConfig;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-/**
- * A generic catalog function implementation.
- */
-public class GenericCatalogFunction extends AbstractCatalogFunction {
-
-	public GenericCatalogFunction(String className) {
-		this(className, new HashMap<>());
-	}
-
-	public GenericCatalogFunction(String className, Map<String, String> properties) {
-		super(className, properties);
-		properties.put(CatalogConfig.IS_GENERIC, String.valueOf(true));
-	}
-
-	@Override
-	public GenericCatalogFunction copy() {
-		return new GenericCatalogFunction(getClassName(), new HashMap<>(getProperties()));
-	}
-
-	@Override
-	public String toString() {
-		return "GenericCatalogFunction{" +
-			", className='" + getClassName() + '\'' +
-			", properties=" + getProperties() +
-			'}';
-	}
-
-	@Override
-	public Optional<String> getDescription() {
-		return Optional.of("This is a user-defined function");
-	}
-
-	@Override
-	public Optional<String> getDetailedDescription() {
-		return Optional.of("This is a user-defined function");
-	}
-
-}
diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogTestBase.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogTestBase.java
index 7779f97..801914f 100644
--- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogTestBase.java
+++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/CatalogTestBase.java
@@ -115,6 +115,22 @@ public abstract class CatalogTestBase extends CatalogTest {
 			"This is another view");
 	}
 
+	@Override
+	protected CatalogFunction createFunction() {
+		return new CatalogFunctionImpl(
+			"test.class.name",
+			getGenericFlag(isGeneric())
+		);
+	}
+
+	@Override
+	protected CatalogFunction createAnotherFunction() {
+		return new CatalogFunctionImpl(
+			"test.another.class.name",
+			getGenericFlag(isGeneric())
+		);
+	}
+
 	protected Map<String, String> getBatchTableProperties() {
 		return new HashMap<String, String>() {{
 			put(IS_STREAMING, "false");
diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogTest.java
index e401223..0014501 100644
--- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogTest.java
+++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogTest.java
@@ -29,7 +29,6 @@ import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataLong;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataString;
 import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
 import org.apache.flink.table.catalog.stats.Date;
-import org.apache.flink.table.functions.ScalarFunction;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -179,33 +178,4 @@ public class GenericInMemoryCatalogTest extends CatalogTestBase {
 		colStatsMap.put("bb6", binaryColStats);
 		return new CatalogColumnStatistics(colStatsMap);
 	}
-
-	@Override
-	protected CatalogFunction createFunction() {
-		return new GenericCatalogFunction(MyScalarFunction.class.getName(), new HashMap<>());
-	}
-
-	@Override
-	protected CatalogFunction createAnotherFunction() {
-		return new GenericCatalogFunction(MyOtherScalarFunction.class.getName(), new HashMap<>());
-	}
-
-	/**
-	 * Test UDF.
-	 */
-	public static class MyScalarFunction extends ScalarFunction {
-		public Integer eval(Integer i) {
-			return i + 1;
-		}
-	}
-
-	/**
-	 * Test UDF.
-	 */
-	public static class MyOtherScalarFunction extends ScalarFunction {
-		public String eval(Integer i) {
-			return String.valueOf(i);
-		}
-	}
-
 }
diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java
index 88c8568..757a4c3 100644
--- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java
+++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/catalog/CatalogTest.java
@@ -33,7 +33,6 @@ import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
 import org.apache.flink.table.catalog.exceptions.TableNotExistException;
 import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
 import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
-import org.apache.flink.table.functions.ScalarFunction;
 
 import org.junit.After;
 import org.junit.AfterClass;
@@ -657,21 +656,6 @@ public abstract class CatalogTest {
 	}
 
 	@Test
-	public void testAlterFunction_differentTypedFunction() throws Exception {
-		catalog.createDatabase(db1, createDb(), false);
-		CatalogFunction function = createFunction();
-		catalog.createFunction(path1, createFunction(), false);
-
-		exception.expect(CatalogException.class);
-		exception.expectMessage(
-			String.format("Function types don't match. " +
-				"Existing function is '%s' and " +
-				"new function is 'org.apache.flink.table.catalog.CatalogTest$TestFunction'.",
-				function.getClass().getName()));
-		catalog.alterFunction(path1, new TestFunction(), false);
-	}
-
-	@Test
 	public void testAlterFunction_FunctionNotExistException() throws Exception {
 		exception.expect(FunctionNotExistException.class);
 		exception.expectMessage("Function db1.nonexist does not exist in Catalog");
@@ -1214,24 +1198,6 @@ public abstract class CatalogTest {
 	}
 
 	/**
-	 * A Flink function for test.
-	 */
-	public static class MyScalarFunction extends ScalarFunction {
-		public Integer eval(Integer i) {
-			return i + 1;
-		}
-	}
-
-	/**
-	 * Another Flink function for test.
-	 */
-	public static class MyOtherScalarFunction extends ScalarFunction {
-		public String eval(Integer i) {
-			return String.valueOf(i);
-		}
-	}
-
-	/**
 	 * Test table used to assert on table of different class.
 	 */
 	public static class TestTable implements CatalogBaseTable {
@@ -1292,36 +1258,6 @@ public abstract class CatalogTest {
 		}
 	}
 
-	/**
-	 * Test function used to assert on function of different class.
-	 */
-	public static class TestFunction implements CatalogFunction {
-		@Override
-		public String getClassName() {
-			return null;
-		}
-
-		@Override
-		public Map<String, String> getProperties() {
-			return null;
-		}
-
-		@Override
-		public CatalogFunction copy() {
-			return null;
-		}
-
-		@Override
-		public Optional<String> getDescription() {
-			return Optional.empty();
-		}
-
-		@Override
-		public Optional<String> getDetailedDescription() {
-			return Optional.empty();
-		}
-	}
-
 	// ------ equality check utils ------
 	// Can be overriden by sub test class