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/08/29 23:29:28 UTC

[flink] branch master updated: [FLINK-13877][hive] Support Hive version 2.1.0 and 2.1.1

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 a40b31b  [FLINK-13877][hive] Support Hive version 2.1.0 and 2.1.1
a40b31b is described below

commit a40b31bf22efe1bee9d0def1730eaa25ef1c0c52
Author: Xuefu Zhang <xu...@alibaba-inc.com>
AuthorDate: Tue Aug 27 15:29:56 2019 -0700

    [FLINK-13877][hive] Support Hive version 2.1.0 and 2.1.1
    
    Support Hive 2.1.x versions (2.1.0 and 2.1.1).
    
    This closes #9547.
---
 flink-connectors/flink-connector-hive/pom.xml      |  6 ++
 .../table/catalog/hive/client/HiveShimLoader.java  |  8 +++
 .../table/catalog/hive/client/HiveShimV210.java    | 76 ++++++++++++++++++++++
 .../table/catalog/hive/client/HiveShimV211.java    | 26 ++++++++
 .../table/catalog/hive/client/HiveShimV230.java    | 24 +------
 .../connectors/hive/HiveRunnerShimLoader.java      |  2 +
 6 files changed, 119 insertions(+), 23 deletions(-)

diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml
index 0320f01..4f30326 100644
--- a/flink-connectors/flink-connector-hive/pom.xml
+++ b/flink-connectors/flink-connector-hive/pom.xml
@@ -665,6 +665,12 @@ under the License.
 			</properties>
 		</profile>
 		<profile>
+			<id>hive-2.1.1</id>
+			<properties>
+				<hive.version>2.1.1</hive.version>
+			</properties>
+		</profile>
+		<profile>
 			<id>skip-hive-tests</id>
 			<build>
 				<plugins>
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java
index 5681cfb..a45722d 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java
@@ -35,6 +35,8 @@ public class HiveShimLoader {
 	public static final String HIVE_VERSION_V1_2_0 = "1.2.0";
 	public static final String HIVE_VERSION_V1_2_1 = "1.2.1";
 	public static final String HIVE_VERSION_V1_2_2 = "1.2.2";
+	public static final String HIVE_VERSION_V2_1_0 = "2.1.0";
+	public static final String HIVE_VERSION_V2_1_1 = "2.1.1";
 	public static final String HIVE_VERSION_V2_3_0 = "2.3.0";
 	public static final String HIVE_VERSION_V2_3_1 = "2.3.1";
 	public static final String HIVE_VERSION_V2_3_2 = "2.3.2";
@@ -60,6 +62,12 @@ public class HiveShimLoader {
 			if (v.startsWith(HIVE_VERSION_V1_2_2)) {
 				return new HiveShimV122();
 			}
+			if (v.startsWith(HIVE_VERSION_V2_1_0)) {
+				return new HiveShimV210();
+			}
+			if (v.startsWith(HIVE_VERSION_V2_1_1)) {
+				return new HiveShimV211();
+			}
 			if (v.startsWith(HIVE_VERSION_V2_3_0)) {
 				return new HiveShimV230();
 			}
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java
new file mode 100644
index 0000000..0fb6e4a
--- /dev/null
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java
@@ -0,0 +1,76 @@
+/*
+ * 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.client;
+
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.thrift.TException;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+/**
+ * Shim for Hive version 2.1.0.
+ */
+public class HiveShimV210 extends HiveShimV122 {
+
+	@Override
+	public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) {
+		try {
+			Class<?>[] constructorArgTypes = {HiveConf.class};
+			Object[] constructorArgs = {hiveConf};
+			Method method = RetryingMetaStoreClient.class.getMethod("getProxy", HiveConf.class,
+				constructorArgTypes.getClass(), constructorArgs.getClass(), String.class);
+			// getProxy is a static method
+			return (IMetaStoreClient) method.invoke(null, hiveConf, constructorArgTypes, constructorArgs,
+				HiveMetaStoreClient.class.getName());
+		} catch (Exception ex) {
+			throw new CatalogException("Failed to create Hive Metastore client", ex);
+		}
+	}
+
+	@Override
+	public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition)
+			throws InvalidOperationException, MetaException, TException {
+		String errorMsg = "Failed to alter partition for table %s in database %s";
+		try {
+			Method method = client.getClass().getMethod("alter_partition", String.class, String.class,
+				Partition.class, EnvironmentContext.class);
+			method.invoke(client, databaseName, tableName, partition, null);
+		} catch (InvocationTargetException ite) {
+			Throwable targetEx = ite.getTargetException();
+			if (targetEx instanceof TException) {
+				throw (TException) targetEx;
+			} else {
+				throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx);
+			}
+		} catch (NoSuchMethodException | IllegalAccessException e) {
+			throw new CatalogException(String.format(errorMsg, tableName, databaseName), e);
+		}
+	}
+
+}
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV211.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV211.java
new file mode 100644
index 0000000..1783bb1
--- /dev/null
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV211.java
@@ -0,0 +1,26 @@
+/*
+ * 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.client;
+
+/**
+ * Shim for Hive version 2.1.1.
+ */
+public class HiveShimV211 extends HiveShimV210 {
+
+}
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java
index 7965b8f..3e2f3b5 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java
@@ -28,12 +28,10 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
 import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo;
@@ -49,7 +47,7 @@ import java.util.List;
 /**
  * Shim for Hive version 2.3.0.
  */
-public class HiveShimV230 extends HiveShimV122 {
+public class HiveShimV230 extends HiveShimV211 {
 
 	@Override
 	public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) {
@@ -102,26 +100,6 @@ public class HiveShimV230 extends HiveShimV122 {
 	}
 
 	@Override
-	public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition)
-			throws InvalidOperationException, MetaException, TException {
-		String errorMsg = "Failed to alter partition for table %s in database %s";
-		try {
-			Method method = client.getClass().getMethod("alter_partition", String.class, String.class,
-				Partition.class, EnvironmentContext.class);
-			method.invoke(client, databaseName, tableName, partition, null);
-		} catch (InvocationTargetException ite) {
-			Throwable targetEx = ite.getTargetException();
-			if (targetEx instanceof TException) {
-				throw (TException) targetEx;
-			} else {
-				throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx);
-			}
-		} catch (NoSuchMethodException | IllegalAccessException e) {
-			throw new CatalogException(String.format(errorMsg, tableName, databaseName), e);
-		}
-	}
-
-	@Override
 	public SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns) {
 		try {
 			Constructor constructor = SimpleGenericUDAFParameterInfo.class.getConstructor(ObjectInspector[].class,
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java
index 2f9f76f..6903cf6 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java
@@ -41,6 +41,8 @@ public class HiveRunnerShimLoader {
 				case HiveShimLoader.HIVE_VERSION_V1_2_1:
 				case HiveShimLoader.HIVE_VERSION_V1_2_2:
 					return new HiveRunnerShimV3();
+				case HiveShimLoader.HIVE_VERSION_V2_1_0:
+				case HiveShimLoader.HIVE_VERSION_V2_1_1:
 				case HiveShimLoader.HIVE_VERSION_V2_3_0:
 				case HiveShimLoader.HIVE_VERSION_V2_3_1:
 				case HiveShimLoader.HIVE_VERSION_V2_3_2: