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/09/11 18:44:34 UTC

[flink] branch master updated: [FLINK-13931][hive] Support Hive version 2.0.x

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 353de39  [FLINK-13931][hive] Support Hive version 2.0.x
353de39 is described below

commit 353de39cc7630b0843e8b395397d4e359dcb5b81
Author: Xuefu Zhang <xu...@alibaba-inc.com>
AuthorDate: Sat Sep 7 04:39:51 2019 +0800

    [FLINK-13931][hive] Support Hive version 2.0.x
    
    This closes #9644.
---
 flink-connectors/flink-connector-hive/pom.xml      |  6 +++++
 .../hive/client/HiveMetastoreClientWrapper.java    |  4 +++
 .../table/catalog/hive/client/HiveShimLoader.java  |  8 ++++++
 .../{HiveShimV210.java => HiveShimV200.java}       | 30 ++--------------------
 .../table/catalog/hive/client/HiveShimV201.java    | 26 +++++++++++++++++++
 .../table/catalog/hive/client/HiveShimV210.java    | 20 +--------------
 .../connectors/hive/HiveRunnerShimLoader.java      |  2 ++
 .../connectors/hive/TableEnvHiveConnectorTest.java |  4 +++
 8 files changed, 53 insertions(+), 47 deletions(-)

diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml
index deca4bc..857affe 100644
--- a/flink-connectors/flink-connector-hive/pom.xml
+++ b/flink-connectors/flink-connector-hive/pom.xml
@@ -661,6 +661,12 @@ under the License.
 			</properties>
 		</profile>
 		<profile>
+			<id>hive-2.0.0</id>
+			<properties>
+				<hive.version>2.0.0</hive.version>
+			</properties>
+		</profile>
+		<profile>
 			<id>hive-2.1.1</id>
 			<properties>
 				<hive.version>2.1.1</hive.version>
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java
index c7f8895..77d8039 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java
@@ -237,4 +237,8 @@ public class HiveMetastoreClientWrapper implements AutoCloseable {
 		hiveShim.alterPartition(client, databaseName, tableName, partition);
 	}
 
+	public String getHiveVersion() {
+		return hiveVersion;
+	}
+
 }
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 c9a3ca0..b3e3903 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_0_0 = "2.0.0";
+	public static final String HIVE_VERSION_V2_0_1 = "2.0.1";
 	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_2_0 = "2.2.0";
@@ -67,6 +69,12 @@ public class HiveShimLoader {
 			if (v.startsWith(HIVE_VERSION_V1_2_2)) {
 				return new HiveShimV122();
 			}
+			if (v.startsWith(HIVE_VERSION_V2_0_0)) {
+				return new HiveShimV200();
+			}
+			if (v.startsWith(HIVE_VERSION_V2_0_1)) {
+				return new HiveShimV201();
+			}
 			if (v.startsWith(HIVE_VERSION_V2_1_0)) {
 				return new HiveShimV210();
 			}
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/HiveShimV200.java
similarity index 57%
copy from flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java
copy to flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV200.java
index 0fb6e4a..a4e30b5 100644
--- 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/HiveShimV200.java
@@ -24,19 +24,13 @@ 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.
+ * Shim for Hive version 2.0.0.
  */
-public class HiveShimV210 extends HiveShimV122 {
+public class HiveShimV200 extends HiveShimV122 {
 
 	@Override
 	public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) {
@@ -53,24 +47,4 @@ public class HiveShimV210 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);
-		}
-	}
-
 }
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV201.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV201.java
new file mode 100644
index 0000000..9511eb5
--- /dev/null
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV201.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.0.1.
+ */
+public class HiveShimV201 extends HiveShimV200 {
+
+}
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
index 0fb6e4a..20a3429 100644
--- 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
@@ -20,10 +20,7 @@ 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;
@@ -36,22 +33,7 @@ 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);
-		}
-	}
+public class HiveShimV210 extends HiveShimV201 {
 
 	@Override
 	public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition)
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 b5aae24..ba2ff57 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_0_0:
+				case HiveShimLoader.HIVE_VERSION_V2_0_1:
 				case HiveShimLoader.HIVE_VERSION_V2_1_0:
 				case HiveShimLoader.HIVE_VERSION_V2_1_1:
 				case HiveShimLoader.HIVE_VERSION_V2_2_0:
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
index e39999a..784bcc8 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
@@ -114,6 +114,10 @@ public class TableEnvHiveConnectorTest {
 	public void testDifferentFormats() throws Exception {
 		String[] formats = new String[]{"orc", "parquet", "sequencefile", "csv"};
 		for (String format : formats) {
+			if (format.equals("orc") && hmsClient.getHiveVersion().startsWith("2.0")) {
+				// Ignore orc test for Hive version 2.0.x for now due to FLINK-13998
+				continue;
+			}
 			readWriteFormat(format);
 		}
 	}