diff --git a/pom.xml b/pom.xml
index bfbed7ee3..2a6993ae9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -29,15 +29,16 @@
waggle-dance-boot
waggle-dance-integration-tests
waggle-dance
- waggle-dance-rpm
+ waggle-dance-glue
+
2.5.3
2.0.4.RELEASE
- 2.7.2
+ 3.1.0
2.2
- 2.3.7
+ 3.0.0
4.13.1
3.5.15
3.1.5
@@ -47,7 +48,8 @@
23.0
4.0
1.2.3
-
+ 1.11.267
+ s
diff --git a/waggle-dance-glue/pom.xml b/waggle-dance-glue/pom.xml
new file mode 100644
index 000000000..d745cca3f
--- /dev/null
+++ b/waggle-dance-glue/pom.xml
@@ -0,0 +1,101 @@
+
+
+ 4.0.0
+
+ com.hotels
+ waggle-dance-parent
+ 3.9.4
+
+
+ waggle-dance-glue
+
+
+
+ com.hotels
+ waggle-dance-core
+ 3.9.4
+
+
+ org.apache.hive
+ hive-metastore
+ ${hive.version}
+ provided
+
+
+ org.apache.hive
+ hive-exec
+ ${hive.version}
+ provided
+
+
+ com.amazonaws
+ aws-java-sdk-glue
+ ${aws.sdk.version}
+ compile
+
+
+ com.amazonaws
+ aws-java-sdk-core
+ ${aws.sdk.version}
+ compile
+
+
+ com.google.guava
+ guava
+ ${guava.version}
+
+
+ junit
+ junit
+ ${junit.version}
+ test
+
+
+ org.hamcrest
+ hamcrest-library
+ ${hamcrest.version}
+ test
+
+
+ org.mockito
+ mockito-core
+ ${mockito.version}
+ test
+
+
+ org.apache.hadoop
+ hadoop-common
+ ${hadoop.version}
+ provided
+
+
+
+ com.google.guava
+ guava
+ ${guava.version}
+
+
+ org.hamcrest
+ hamcrest
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+ 2.2
+
+
+
+ test-jar
+
+
+
+
+
+
+
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java
new file mode 100644
index 000000000..1b9d989aa
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java
@@ -0,0 +1,419 @@
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.ErrorDetail;
+
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.ResourceType;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_DEFERRED_REBUILD;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_TABLE_NAME;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_DB_NAME;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_HANDLER_CLASS;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_ORIGIN_TABLE_NAME;
+
+import static org.apache.commons.lang3.ObjectUtils.firstNonNull;
+
+public class CatalogToHiveConverter {
+
+ private static final Logger logger = Logger.getLogger(CatalogToHiveConverter.class);
+
+ private static final ImmutableMap EXCEPTION_MAP = ImmutableMap.builder()
+ .put("AlreadyExistsException", new HiveException() {
+ public TException get(String msg) {
+ return new AlreadyExistsException(msg);
+ }
+ })
+ .put("InvalidInputException", new HiveException() {
+ public TException get(String msg) {
+ return new InvalidObjectException(msg);
+ }
+ })
+ .put("InternalServiceException", new HiveException() {
+ public TException get(String msg) {
+ return new MetaException(msg);
+ }
+ })
+ .put("ResourceNumberLimitExceededException", new HiveException() {
+ public TException get(String msg) {
+ return new MetaException(msg);
+ }
+ })
+ .put("OperationTimeoutException", new HiveException() {
+ public TException get(String msg) {
+ return new MetaException(msg);
+ }
+ })
+ .put("EntityNotFoundException", new HiveException() {
+ public TException get(String msg) {
+ return new NoSuchObjectException(msg);
+ }
+ })
+ .build();
+
+ interface HiveException {
+ TException get(String msg);
+ }
+
+ public static TException wrapInHiveException(Throwable e) {
+ return getHiveException(e.getClass().getSimpleName(), e.getMessage());
+ }
+
+ public static TException errorDetailToHiveException(ErrorDetail errorDetail) {
+ return getHiveException(errorDetail.getErrorCode(), errorDetail.getErrorMessage());
+ }
+
+ private static TException getHiveException(String errorName, String errorMsg) {
+ if (EXCEPTION_MAP.containsKey(errorName)) {
+ return EXCEPTION_MAP.get(errorName).get(errorMsg);
+ } else {
+ logger.warn("Hive Exception type not found for " + errorName);
+ return new MetaException(errorMsg);
+ }
+ }
+
+ public static Database convertDatabase(com.amazonaws.services.glue.model.Database catalogDatabase) {
+ Database hiveDatabase = new Database();
+ hiveDatabase.setName(catalogDatabase.getName());
+ hiveDatabase.setDescription(catalogDatabase.getDescription());
+ String location = catalogDatabase.getLocationUri();
+ hiveDatabase.setLocationUri(location == null ? "" : location);
+ hiveDatabase.setParameters(firstNonNull(catalogDatabase.getParameters(), Maps.newHashMap()));
+ return hiveDatabase;
+ }
+
+ public static FieldSchema convertFieldSchema(com.amazonaws.services.glue.model.Column catalogFieldSchema) {
+ FieldSchema hiveFieldSchema = new FieldSchema();
+ hiveFieldSchema.setType(catalogFieldSchema.getType());
+ hiveFieldSchema.setName(catalogFieldSchema.getName());
+ hiveFieldSchema.setComment(catalogFieldSchema.getComment());
+
+ return hiveFieldSchema;
+ }
+
+ public static List convertFieldSchemaList(List catalogFieldSchemaList) {
+ List hiveFieldSchemaList = new ArrayList<>();
+ if (catalogFieldSchemaList == null) {
+ return hiveFieldSchemaList;
+ }
+ for (com.amazonaws.services.glue.model.Column catalogFieldSchema : catalogFieldSchemaList){
+ hiveFieldSchemaList.add(convertFieldSchema(catalogFieldSchema));
+ }
+
+ return hiveFieldSchemaList;
+ }
+
+ public static Table convertTable(com.amazonaws.services.glue.model.Table catalogTable, String dbname) {
+ Table hiveTable = new Table();
+ hiveTable.setDbName(dbname);
+ hiveTable.setTableName(catalogTable.getName());
+ Date createTime = catalogTable.getCreateTime();
+ hiveTable.setCreateTime(createTime == null ? 0 : (int) (createTime.getTime() / 1000));
+ hiveTable.setOwner(catalogTable.getOwner());
+ Date lastAccessedTime = catalogTable.getLastAccessTime();
+ hiveTable.setLastAccessTime(lastAccessedTime == null ? 0 : (int) (lastAccessedTime.getTime() / 1000));
+ hiveTable.setRetention(catalogTable.getRetention());
+ hiveTable.setSd(convertStorageDescriptor(catalogTable.getStorageDescriptor()));
+ hiveTable.setPartitionKeys(convertFieldSchemaList(catalogTable.getPartitionKeys()));
+ // Hive may throw a NPE during dropTable if the parameter map is null.
+ Map parameterMap = catalogTable.getParameters();
+ if (parameterMap == null) {
+ parameterMap = Maps.newHashMap();
+ }
+ hiveTable.setParameters(parameterMap);
+ hiveTable.setViewOriginalText(catalogTable.getViewOriginalText());
+ hiveTable.setViewExpandedText(catalogTable.getViewExpandedText());
+ hiveTable.setTableType(catalogTable.getTableType());
+
+ return hiveTable;
+ }
+
+ public static TableMeta convertTableMeta(com.amazonaws.services.glue.model.Table catalogTable, String dbName) {
+ TableMeta tableMeta = new TableMeta();
+ tableMeta.setDbName(dbName);
+ tableMeta.setTableName(catalogTable.getName());
+ tableMeta.setTableType(catalogTable.getTableType());
+ if (catalogTable.getParameters().containsKey("comment")) {
+ tableMeta.setComments(catalogTable.getParameters().get("comment"));
+ }
+ return tableMeta;
+ }
+
+ public static StorageDescriptor convertStorageDescriptor(com.amazonaws.services.glue.model.StorageDescriptor catalogSd) {
+ StorageDescriptor hiveSd = new StorageDescriptor();
+ hiveSd.setCols(convertFieldSchemaList(catalogSd.getColumns()));
+ hiveSd.setLocation(catalogSd.getLocation());
+ hiveSd.setInputFormat(catalogSd.getInputFormat());
+ hiveSd.setOutputFormat(catalogSd.getOutputFormat());
+ hiveSd.setCompressed(catalogSd.getCompressed());
+ hiveSd.setNumBuckets(catalogSd.getNumberOfBuckets());
+ hiveSd.setSerdeInfo(convertSerDeInfo(catalogSd.getSerdeInfo()));
+ hiveSd.setBucketCols(firstNonNull(catalogSd.getBucketColumns(), Lists.newArrayList()));
+ hiveSd.setSortCols(convertOrderList(catalogSd.getSortColumns()));
+ hiveSd.setParameters(firstNonNull(catalogSd.getParameters(), Maps.newHashMap()));
+ hiveSd.setSkewedInfo(convertSkewedInfo(catalogSd.getSkewedInfo()));
+ hiveSd.setStoredAsSubDirectories(catalogSd.getStoredAsSubDirectories());
+
+ return hiveSd;
+ }
+
+ public static Order convertOrder(com.amazonaws.services.glue.model.Order catalogOrder) {
+ Order hiveOrder = new Order();
+ hiveOrder.setCol(catalogOrder.getColumn());
+ hiveOrder.setOrder(catalogOrder.getSortOrder());
+
+ return hiveOrder;
+ }
+
+ public static List convertOrderList(List catalogOrderList) {
+ List hiveOrderList = new ArrayList<>();
+ if (catalogOrderList == null) {
+ return hiveOrderList;
+ }
+ for (com.amazonaws.services.glue.model.Order catalogOrder : catalogOrderList){
+ hiveOrderList.add(convertOrder(catalogOrder));
+ }
+
+ return hiveOrderList;
+ }
+
+ public static SerDeInfo convertSerDeInfo(com.amazonaws.services.glue.model.SerDeInfo catalogSerDeInfo){
+ SerDeInfo hiveSerDeInfo = new SerDeInfo();
+ hiveSerDeInfo.setName(catalogSerDeInfo.getName());
+ hiveSerDeInfo.setParameters(firstNonNull(catalogSerDeInfo.getParameters(), Maps.newHashMap()));
+ hiveSerDeInfo.setSerializationLib(catalogSerDeInfo.getSerializationLibrary());
+
+ return hiveSerDeInfo;
+ }
+
+ public static SkewedInfo convertSkewedInfo(com.amazonaws.services.glue.model.SkewedInfo catalogSkewedInfo) {
+ if (catalogSkewedInfo == null) {
+ return null;
+ }
+
+ SkewedInfo hiveSkewedInfo = new SkewedInfo();
+ hiveSkewedInfo.setSkewedColNames(firstNonNull(catalogSkewedInfo.getSkewedColumnNames(), Lists.newArrayList()));
+ hiveSkewedInfo.setSkewedColValues(convertSkewedValue(catalogSkewedInfo.getSkewedColumnValues()));
+ hiveSkewedInfo.setSkewedColValueLocationMaps(convertSkewedMap(catalogSkewedInfo.getSkewedColumnValueLocationMaps()));
+ return hiveSkewedInfo;
+ }
+
+ public static Index convertTableObjectToIndex(com.amazonaws.services.glue.model.Table catalogTable) {
+ Index hiveIndex = new Index();
+ Map parameters = catalogTable.getParameters();
+ hiveIndex.setIndexName(catalogTable.getName());
+ hiveIndex.setCreateTime((int) (catalogTable.getCreateTime().getTime() / 1000));
+ hiveIndex.setLastAccessTime((int) (catalogTable.getLastAccessTime().getTime() / 1000));
+ hiveIndex.setSd(convertStorageDescriptor(catalogTable.getStorageDescriptor()));
+ hiveIndex.setParameters(catalogTable.getParameters());
+
+ hiveIndex.setDeferredRebuild(parameters.get(INDEX_DEFERRED_REBUILD).equals("TRUE"));
+ hiveIndex.setIndexHandlerClass(parameters.get(INDEX_HANDLER_CLASS));
+ hiveIndex.setDbName(parameters.get(INDEX_DB_NAME));
+ hiveIndex.setOrigTableName(parameters.get(INDEX_ORIGIN_TABLE_NAME));
+ hiveIndex.setIndexTableName(parameters.get(INDEX_TABLE_NAME));
+
+ return hiveIndex;
+ }
+
+ public static Partition convertPartition(com.amazonaws.services.glue.model.Partition src) {
+ Partition tgt = new Partition();
+ Date createTime = src.getCreationTime();
+ if (createTime != null) {
+ tgt.setCreateTime((int) (createTime.getTime() / 1000));
+ tgt.setCreateTimeIsSet(true);
+ } else {
+ tgt.setCreateTimeIsSet(false);
+ }
+ String dbName = src.getDatabaseName();
+ if (dbName != null) {
+ tgt.setDbName(dbName);
+ tgt.setDbNameIsSet(true);
+ } else {
+ tgt.setDbNameIsSet(false);
+ }
+ Date lastAccessTime = src.getLastAccessTime();
+ if (lastAccessTime != null) {
+ tgt.setLastAccessTime((int) (lastAccessTime.getTime() / 1000));
+ tgt.setLastAccessTimeIsSet(true);
+ } else {
+ tgt.setLastAccessTimeIsSet(false);
+ }
+ Map params = src.getParameters();
+
+ // A null parameter map causes Hive to throw a NPE
+ // so ensure we do not return a Partition object with a null parameter map.
+ if (params == null) {
+ params = Maps.newHashMap();
+ }
+
+ tgt.setParameters(params);
+ tgt.setParametersIsSet(true);
+
+ String tableName = src.getTableName();
+ if (tableName != null) {
+ tgt.setTableName(tableName);
+ tgt.setTableNameIsSet(true);
+ } else {
+ tgt.setTableNameIsSet(false);
+ }
+
+ List values = src.getValues();
+ if (values != null) {
+ tgt.setValues(values);
+ tgt.setValuesIsSet(true);
+ } else {
+ tgt.setValuesIsSet(false);
+ }
+
+ com.amazonaws.services.glue.model.StorageDescriptor sd = src.getStorageDescriptor();
+ if (sd != null) {
+ StorageDescriptor hiveSd = convertStorageDescriptor(sd);
+ tgt.setSd(hiveSd);
+ tgt.setSdIsSet(true);
+ } else {
+ tgt.setSdIsSet(false);
+ }
+
+ return tgt;
+ }
+
+ public static List convertPartitions(List src) {
+ if (src == null) {
+ return null;
+ }
+
+ List target = Lists.newArrayList();
+ for (com.amazonaws.services.glue.model.Partition partition : src) {
+ target.add(convertPartition(partition));
+ }
+ return target;
+ }
+
+ public static List convertStringToList(final String s) {
+ if (s == null) {
+ return null;
+ }
+ List listString = new ArrayList<>();
+ for (int i = 0; i < s.length();) {
+ StringBuilder length = new StringBuilder();
+ for (int j = i; j < s.length(); j++){
+ if (s.charAt(j) != '$') {
+ length.append(s.charAt(j));
+ } else {
+ int lengthOfString = Integer.valueOf(length.toString());
+ listString.add(s.substring(j + 1, j + 1 + lengthOfString));
+ i = j + 1 + lengthOfString;
+ break;
+ }
+ }
+ }
+ return listString;
+ }
+
+ @Nonnull
+ public static Map, String> convertSkewedMap(final @Nullable Map catalogSkewedMap) {
+ Map, String> skewedMap = new HashMap<>();
+ if (catalogSkewedMap == null){
+ return skewedMap;
+ }
+
+ for (String coralKey : catalogSkewedMap.keySet()) {
+ skewedMap.put(convertStringToList(coralKey), catalogSkewedMap.get(coralKey));
+ }
+ return skewedMap;
+ }
+
+ @Nonnull
+ public static List> convertSkewedValue(final @Nullable List catalogSkewedValue) {
+ List> skewedValues = new ArrayList<>();
+ if (catalogSkewedValue == null){
+ return skewedValues;
+ }
+
+ for (String skewValue : catalogSkewedValue) {
+ skewedValues.add(convertStringToList(skewValue));
+ }
+ return skewedValues;
+ }
+
+ public static PrincipalType convertPrincipalType(com.amazonaws.services.glue.model.PrincipalType catalogPrincipalType) {
+ if(catalogPrincipalType == null) {
+ return null;
+ }
+
+ if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.GROUP) {
+ return PrincipalType.GROUP;
+ } else if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.USER) {
+ return PrincipalType.USER;
+ } else if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.ROLE) {
+ return PrincipalType.ROLE;
+ }
+ throw new RuntimeException("Unknown principal type:" + catalogPrincipalType.name());
+ }
+
+ public static Function convertFunction(final String dbName,
+ final com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction) {
+ if (catalogFunction == null) {
+ return null;
+ }
+ Function hiveFunction = new Function();
+ hiveFunction.setClassName(catalogFunction.getClassName());
+ hiveFunction.setCreateTime((int)(catalogFunction.getCreateTime().getTime() / 1000));
+ hiveFunction.setDbName(dbName);
+ hiveFunction.setFunctionName(catalogFunction.getFunctionName());
+ hiveFunction.setFunctionType(FunctionType.JAVA);
+ hiveFunction.setOwnerName(catalogFunction.getOwnerName());
+ hiveFunction.setOwnerType(convertPrincipalType(com.amazonaws.services.glue.model.PrincipalType.fromValue(catalogFunction.getOwnerType())));
+ hiveFunction.setResourceUris(convertResourceUriList(catalogFunction.getResourceUris()));
+ return hiveFunction;
+ }
+
+ public static List convertResourceUriList(
+ final List catalogResourceUriList) {
+ if (catalogResourceUriList == null) {
+ return null;
+ }
+ List hiveResourceUriList = new ArrayList<>();
+ for (com.amazonaws.services.glue.model.ResourceUri catalogResourceUri : catalogResourceUriList) {
+ ResourceUri hiveResourceUri = new ResourceUri();
+ hiveResourceUri.setUri(catalogResourceUri.getUri());
+ if (catalogResourceUri.getResourceType() != null) {
+ hiveResourceUri.setResourceType(ResourceType.valueOf(catalogResourceUri.getResourceType()));
+ }
+ hiveResourceUriList.add(hiveResourceUri);
+ }
+
+ return hiveResourceUriList;
+ }
+
+}
\ No newline at end of file
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java
new file mode 100644
index 000000000..7d71af9d2
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java
@@ -0,0 +1,23 @@
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.Table;
+
+import com.google.gson.Gson;
+
+public class ConverterUtils {
+
+ public static final String INDEX_DEFERRED_REBUILD = "DeferredRebuild";
+ public static final String INDEX_TABLE_NAME = "IndexTableName";
+ public static final String INDEX_HANDLER_CLASS = "IndexHandlerClass";
+ public static final String INDEX_DB_NAME = "DbName";
+ public static final String INDEX_ORIGIN_TABLE_NAME = "OriginTableName";
+ private static final Gson gson = new Gson();
+
+ public static String catalogTableToString(final Table table) {
+ return gson.toJson(table);
+ }
+
+ public static Table stringToCatalogTable(final String input) {
+ return gson.fromJson(input, Table.class);
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java
new file mode 100644
index 000000000..703a63eb3
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java
@@ -0,0 +1,96 @@
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class provides methods to convert Hive/Catalog objects to Input objects used
+ * for Glue API parameters
+ */
+public final class GlueInputConverter {
+
+ public static DatabaseInput convertToDatabaseInput(Database hiveDatabase) {
+ return convertToDatabaseInput(HiveToCatalogConverter.convertDatabase(hiveDatabase));
+ }
+
+ public static DatabaseInput convertToDatabaseInput(com.amazonaws.services.glue.model.Database database) {
+ DatabaseInput input = new DatabaseInput();
+
+ input.setName(database.getName());
+ input.setDescription(database.getDescription());
+ input.setLocationUri(database.getLocationUri());
+ input.setParameters(database.getParameters());
+
+ return input;
+ }
+
+ public static TableInput convertToTableInput(Table hiveTable) {
+ return convertToTableInput(HiveToCatalogConverter.convertTable(hiveTable));
+ }
+
+ public static TableInput convertToTableInput(com.amazonaws.services.glue.model.Table table) {
+ TableInput tableInput = new TableInput();
+
+ tableInput.setRetention(table.getRetention());
+ tableInput.setPartitionKeys(table.getPartitionKeys());
+ tableInput.setTableType(table.getTableType());
+ tableInput.setName(table.getName());
+ tableInput.setOwner(table.getOwner());
+ tableInput.setLastAccessTime(table.getLastAccessTime());
+ tableInput.setStorageDescriptor(table.getStorageDescriptor());
+ tableInput.setParameters(table.getParameters());
+ tableInput.setViewExpandedText(table.getViewExpandedText());
+ tableInput.setViewOriginalText(table.getViewOriginalText());
+
+ return tableInput;
+ }
+
+ public static PartitionInput convertToPartitionInput(Partition src) {
+ return convertToPartitionInput(HiveToCatalogConverter.convertPartition(src));
+ }
+
+ public static PartitionInput convertToPartitionInput(com.amazonaws.services.glue.model.Partition src) {
+ PartitionInput partitionInput = new PartitionInput();
+
+ partitionInput.setLastAccessTime(src.getLastAccessTime());
+ partitionInput.setParameters(src.getParameters());
+ partitionInput.setStorageDescriptor(src.getStorageDescriptor());
+ partitionInput.setValues(src.getValues());
+
+ return partitionInput;
+ }
+
+ public static List convertToPartitionInputs(Collection parts) {
+ List inputList = new ArrayList<>();
+
+ for (com.amazonaws.services.glue.model.Partition part : parts) {
+ inputList.add(convertToPartitionInput(part));
+ }
+ return inputList;
+ }
+
+ public static UserDefinedFunctionInput convertToUserDefinedFunctionInput(Function hiveFunction) {
+ UserDefinedFunctionInput functionInput = new UserDefinedFunctionInput();
+
+ functionInput.setClassName(hiveFunction.getClassName());
+ functionInput.setFunctionName(hiveFunction.getFunctionName());
+ functionInput.setOwnerName(hiveFunction.getOwnerName());
+ if(hiveFunction.getOwnerType() != null) {
+ functionInput.setOwnerType(hiveFunction.getOwnerType().name());
+ }
+ functionInput.setResourceUris(HiveToCatalogConverter.covertResourceUriList(hiveFunction.getResourceUris()));
+ return functionInput;
+ }
+
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java
new file mode 100644
index 000000000..3994c0a5c
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java
@@ -0,0 +1,240 @@
+package com.amazonaws.glue.catalog.converters;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_DB_NAME;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_DEFERRED_REBUILD;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_HANDLER_CLASS;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_ORIGIN_TABLE_NAME;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.INDEX_TABLE_NAME;
+
+public class HiveToCatalogConverter {
+
+ public static com.amazonaws.services.glue.model.Database convertDatabase(Database hiveDatabase) {
+ com.amazonaws.services.glue.model.Database catalogDatabase = new com.amazonaws.services.glue.model.Database();
+ catalogDatabase.setName(hiveDatabase.getName());
+ catalogDatabase.setDescription(hiveDatabase.getDescription());
+ catalogDatabase.setLocationUri(hiveDatabase.getLocationUri());
+ catalogDatabase.setParameters(hiveDatabase.getParameters());
+ return catalogDatabase;
+ }
+
+ public static com.amazonaws.services.glue.model.Table convertTable(
+ Table hiveTable) {
+ com.amazonaws.services.glue.model.Table catalogTable = new com.amazonaws.services.glue.model.Table();
+ catalogTable.setRetention(hiveTable.getRetention());
+ catalogTable.setPartitionKeys(convertFieldSchemaList(hiveTable.getPartitionKeys()));
+ catalogTable.setTableType(hiveTable.getTableType());
+ catalogTable.setName(hiveTable.getTableName());
+ catalogTable.setOwner(hiveTable.getOwner());
+ catalogTable.setCreateTime(new Date((long) hiveTable.getCreateTime() * 1000));
+ catalogTable.setLastAccessTime(new Date((long) hiveTable.getLastAccessTime() * 1000));
+ catalogTable.setStorageDescriptor(convertStorageDescriptor(hiveTable.getSd()));
+ catalogTable.setParameters(hiveTable.getParameters());
+ catalogTable.setViewExpandedText(hiveTable.getViewExpandedText());
+ catalogTable.setViewOriginalText(hiveTable.getViewOriginalText());
+
+ return catalogTable;
+ }
+
+ public static com.amazonaws.services.glue.model.StorageDescriptor convertStorageDescriptor(
+ StorageDescriptor hiveSd) {
+ com.amazonaws.services.glue.model.StorageDescriptor catalogSd =
+ new com.amazonaws.services.glue.model.StorageDescriptor();
+ catalogSd.setNumberOfBuckets(hiveSd.getNumBuckets());
+ catalogSd.setCompressed(hiveSd.isCompressed());
+ catalogSd.setParameters(hiveSd.getParameters());
+ catalogSd.setBucketColumns(hiveSd.getBucketCols());
+ catalogSd.setColumns(convertFieldSchemaList(hiveSd.getCols()));
+ catalogSd.setInputFormat(hiveSd.getInputFormat());
+ catalogSd.setLocation(hiveSd.getLocation());
+ catalogSd.setOutputFormat(hiveSd.getOutputFormat());
+ catalogSd.setSerdeInfo(convertSerDeInfo(hiveSd.getSerdeInfo()));
+ catalogSd.setSkewedInfo(convertSkewedInfo(hiveSd.getSkewedInfo()));
+ catalogSd.setSortColumns(convertOrderList(hiveSd.getSortCols()));
+ catalogSd.setStoredAsSubDirectories(hiveSd.isStoredAsSubDirectories());
+
+ return catalogSd;
+ }
+
+ public static com.amazonaws.services.glue.model.Column convertFieldSchema(
+ FieldSchema hiveFieldSchema) {
+ com.amazonaws.services.glue.model.Column catalogFieldSchema =
+ new com.amazonaws.services.glue.model.Column();
+ catalogFieldSchema.setComment(hiveFieldSchema.getComment());
+ catalogFieldSchema.setName(hiveFieldSchema.getName());
+ catalogFieldSchema.setType(hiveFieldSchema.getType());
+
+ return catalogFieldSchema;
+ }
+
+ public static List convertFieldSchemaList(
+ List hiveFieldSchemaList) {
+ List catalogFieldSchemaList =
+ new ArrayList();
+ for (FieldSchema hiveFs : hiveFieldSchemaList){
+ catalogFieldSchemaList.add(convertFieldSchema(hiveFs));
+ }
+
+ return catalogFieldSchemaList;
+ }
+
+ public static com.amazonaws.services.glue.model.SerDeInfo convertSerDeInfo(
+ SerDeInfo hiveSerDeInfo) {
+ com.amazonaws.services.glue.model.SerDeInfo catalogSerDeInfo = new com.amazonaws.services.glue.model.SerDeInfo();
+ catalogSerDeInfo.setName(hiveSerDeInfo.getName());
+ catalogSerDeInfo.setParameters(hiveSerDeInfo.getParameters());
+ catalogSerDeInfo.setSerializationLibrary(hiveSerDeInfo.getSerializationLib());
+
+ return catalogSerDeInfo;
+ }
+
+ public static com.amazonaws.services.glue.model.SkewedInfo convertSkewedInfo(SkewedInfo hiveSkewedInfo) {
+ if (hiveSkewedInfo == null)
+ return null;
+ com.amazonaws.services.glue.model.SkewedInfo catalogSkewedInfo = new com.amazonaws.services.glue.model.SkewedInfo()
+ .withSkewedColumnNames(hiveSkewedInfo.getSkewedColNames())
+ .withSkewedColumnValues(convertSkewedValue(hiveSkewedInfo.getSkewedColValues()))
+ .withSkewedColumnValueLocationMaps(convertSkewedMap(hiveSkewedInfo.getSkewedColValueLocationMaps()));
+ return catalogSkewedInfo;
+ }
+
+ public static com.amazonaws.services.glue.model.Order convertOrder(Order hiveOrder) {
+ com.amazonaws.services.glue.model.Order order = new com.amazonaws.services.glue.model.Order();
+ order.setColumn(hiveOrder.getCol());
+ order.setSortOrder(hiveOrder.getOrder());
+
+ return order;
+ }
+
+ public static List convertOrderList(List hiveOrderList) {
+ if (hiveOrderList == null) {
+ return null;
+ }
+ List catalogOrderList = new ArrayList<>();
+ for (Order hiveOrder : hiveOrderList) {
+ catalogOrderList.add(convertOrder(hiveOrder));
+ }
+
+ return catalogOrderList;
+ }
+
+ public static com.amazonaws.services.glue.model.Table convertIndexToTableObject(Index hiveIndex) {
+ // convert index object to a table object
+ com.amazonaws.services.glue.model.Table catalogIndexTableObject = new com.amazonaws.services.glue.model.Table();
+ catalogIndexTableObject.setName(hiveIndex.getIndexName());
+ catalogIndexTableObject.setCreateTime(new Date((long) (hiveIndex.getCreateTime()) * 1000));
+ catalogIndexTableObject.setLastAccessTime(new Date((long) (hiveIndex.getLastAccessTime()) * 1000));
+ catalogIndexTableObject.setStorageDescriptor(convertStorageDescriptor(hiveIndex.getSd()));
+ catalogIndexTableObject.setParameters(hiveIndex.getParameters());
+
+ // store rest of fields in index to paramter map
+ catalogIndexTableObject.getParameters().put(INDEX_DEFERRED_REBUILD, hiveIndex.isDeferredRebuild() ? "TRUE": "FALSE");
+ catalogIndexTableObject.getParameters().put(INDEX_TABLE_NAME, hiveIndex.getIndexTableName());
+ catalogIndexTableObject.getParameters().put(INDEX_HANDLER_CLASS, hiveIndex.getIndexHandlerClass());
+ catalogIndexTableObject.getParameters().put(INDEX_DB_NAME, hiveIndex.getDbName());
+ catalogIndexTableObject.getParameters().put(INDEX_ORIGIN_TABLE_NAME, hiveIndex.getOrigTableName());
+
+ return catalogIndexTableObject;
+ }
+
+ public static com.amazonaws.services.glue.model.Partition convertPartition(Partition src) {
+ com.amazonaws.services.glue.model.Partition tgt = new com.amazonaws.services.glue.model.Partition();
+
+ tgt.setDatabaseName(src.getDbName());
+ tgt.setTableName(src.getTableName());
+ tgt.setCreationTime(new Date((long) src.getCreateTime() * 1000));
+ tgt.setLastAccessTime(new Date((long) src.getLastAccessTime() * 1000));
+ tgt.setParameters(src.getParameters());
+ tgt.setStorageDescriptor(convertStorageDescriptor(src.getSd()));
+ tgt.setValues(src.getValues());
+
+ return tgt;
+ }
+
+ public static String convertListToString(final List list) {
+ if (list == null) {
+ return null;
+ }
+ StringBuilder sb = new StringBuilder();
+ for (int i = 0; i < list.size(); i++) {
+ String currentString = list.get(i);
+ sb.append(currentString.length() + "$" + currentString);
+ }
+
+ return sb.toString();
+ }
+
+ public static Map convertSkewedMap(final Map, String> coreSkewedMap){
+ if (coreSkewedMap == null){
+ return null;
+ }
+ Map catalogSkewedMap = new HashMap<>();
+ for (List coreKey : coreSkewedMap.keySet()) {
+ catalogSkewedMap.put(convertListToString(coreKey), coreSkewedMap.get(coreKey));
+ }
+ return catalogSkewedMap;
+ }
+
+ public static List convertSkewedValue(final List> coreSkewedValue) {
+ if (coreSkewedValue == null) {
+ return null;
+ }
+ List catalogSkewedValue = new ArrayList<>();
+ for (int i = 0; i < coreSkewedValue.size(); i++) {
+ catalogSkewedValue.add(convertListToString(coreSkewedValue.get(i)));
+ }
+
+ return catalogSkewedValue;
+ }
+
+ public static com.amazonaws.services.glue.model.UserDefinedFunction convertFunction(final Function hiveFunction) {
+ if (hiveFunction == null ){
+ return null;
+ }
+ com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction = new com.amazonaws.services.glue.model.UserDefinedFunction();
+ catalogFunction.setClassName(hiveFunction.getClassName());
+ catalogFunction.setFunctionName(hiveFunction.getFunctionName());
+ catalogFunction.setCreateTime(new Date((long) (hiveFunction.getCreateTime()) * 1000));
+ catalogFunction.setOwnerName(hiveFunction.getOwnerName());
+ if(hiveFunction.getOwnerType() != null) {
+ catalogFunction.setOwnerType(hiveFunction.getOwnerType().name());
+ }
+ catalogFunction.setResourceUris(covertResourceUriList(hiveFunction.getResourceUris()));
+ return catalogFunction;
+ }
+
+ public static List covertResourceUriList(
+ final List hiveResourceUriList) {
+ if (hiveResourceUriList == null) {
+ return null;
+ }
+ List catalogResourceUriList = new ArrayList<>();
+ for (ResourceUri hiveResourceUri : hiveResourceUriList) {
+ com.amazonaws.services.glue.model.ResourceUri catalogResourceUri = new com.amazonaws.services.glue.model.ResourceUri();
+ catalogResourceUri.setUri(hiveResourceUri.getUri());
+ if (hiveResourceUri.getResourceType() != null) {
+ catalogResourceUri.setResourceType(hiveResourceUri.getResourceType().name());
+ }
+ catalogResourceUriList.add(catalogResourceUri);
+ }
+ return catalogResourceUriList;
+ }
+
+}
\ No newline at end of file
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java
new file mode 100644
index 000000000..dc0df8e0e
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java
@@ -0,0 +1,1882 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter;
+import com.amazonaws.glue.catalog.converters.GlueInputConverter;
+import com.amazonaws.glue.catalog.converters.HiveToCatalogConverter;
+import com.amazonaws.glue.catalog.util.BatchDeletePartitionsHelper;
+import com.amazonaws.glue.catalog.util.ExpressionHelper;
+import com.amazonaws.glue.catalog.util.LoggingHelper;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import com.amazonaws.glue.shims.AwsGlueHiveShims;
+import com.amazonaws.glue.shims.ShimsLoader;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.AlreadyExistsException;
+import com.amazonaws.services.glue.model.EntityNotFoundException;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.hotels.bdp.waggledance.client.CloseableThriftHiveMetastoreIface;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.PartitionDropOptions;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest;
+import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
+import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.AddPartitionsResult;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.CheckLockRequest;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionRequest;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
+import org.apache.hadoop.hive.metastore.api.DropPartitionsResult;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PartitionSpec;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.RequestPartsSpec;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionListComposingSpecProxy;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.catalogTableToString;
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.stringToCatalogTable;
+import static com.amazonaws.glue.catalog.metastore.GlueMetastoreClientDelegate.INDEX_PREFIX;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.isExternalTable;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
+public class AWSCatalogMetastoreClient implements CloseableThriftHiveMetastoreIface
+{
+
+ // TODO "hook" into Hive logging (hive or hive.metastore)
+ private static final Logger logger = Logger.getLogger(AWSCatalogMetastoreClient.class);
+
+ private final HiveConf conf;
+ private final AWSGlue glueClient;
+ private final Warehouse wh;
+ private final GlueMetastoreClientDelegate glueMetastoreClientDelegate;
+ private final String catalogId;
+
+ private static final int BATCH_DELETE_PARTITIONS_PAGE_SIZE = 25;
+ private static final int BATCH_DELETE_PARTITIONS_THREADS_COUNT = 5;
+ static final String BATCH_DELETE_PARTITIONS_THREAD_POOL_NAME_FORMAT = "batch-delete-partitions-%d";
+ private static final ExecutorService BATCH_DELETE_PARTITIONS_THREAD_POOL = Executors.newFixedThreadPool(
+ BATCH_DELETE_PARTITIONS_THREADS_COUNT,
+ new ThreadFactoryBuilder()
+ .setNameFormat(BATCH_DELETE_PARTITIONS_THREAD_POOL_NAME_FORMAT)
+ .setDaemon(true).build()
+ );
+
+
+
+ private Map currentMetaVars;
+ private final AwsGlueHiveShims hiveShims = ShimsLoader.getHiveShims();
+
+ public AWSCatalogMetastoreClient(HiveConf conf, HiveMetaHookLoader hook) throws MetaException {
+ this.conf = conf;
+ glueClient = new AWSGlueClientFactory(this.conf).newClient();
+
+ // TODO preserve existing functionality for HiveMetaHook
+ wh = new Warehouse(this.conf);
+
+ AWSGlueMetastore glueMetastore = new AWSGlueMetastoreFactory().newMetastore(conf);
+ glueMetastoreClientDelegate = new GlueMetastoreClientDelegate(this.conf, glueMetastore, wh);
+
+ snapshotActiveConf();
+ catalogId = MetastoreClientUtils.getCatalogId(conf);
+ if (!doesDefaultDBExist()) {
+ createDefaultDatabase();
+ }
+ }
+
+ /**
+ * Currently used for unit tests
+ */
+ public static class Builder {
+
+ private HiveConf conf;
+ private Warehouse wh;
+ private GlueClientFactory clientFactory;
+ private AWSGlueMetastoreFactory metastoreFactory;
+ private boolean createDefaults = true;
+ private String catalogId;
+
+ public Builder withHiveConf(HiveConf conf) {
+ this.conf = conf;
+ return this;
+ }
+
+ public Builder withClientFactory(GlueClientFactory clientFactory) {
+ this.clientFactory = clientFactory;
+ return this;
+ }
+
+ public Builder withMetastoreFactory(AWSGlueMetastoreFactory metastoreFactory) {
+ this.metastoreFactory = metastoreFactory;
+ return this;
+ }
+
+ public Builder withWarehouse(Warehouse wh) {
+ this.wh = wh;
+ return this;
+ }
+
+ public Builder withCatalogId(String catalogId) {
+ this.catalogId = catalogId;
+ return this;
+ }
+
+ public AWSCatalogMetastoreClient build() throws MetaException {
+ return new AWSCatalogMetastoreClient(this);
+ }
+
+ public Builder createDefaults(boolean createDefaultDB) {
+ this.createDefaults = createDefaultDB;
+ return this;
+ }
+ }
+
+ private AWSCatalogMetastoreClient(Builder builder) throws MetaException {
+ conf = Objects.firstNonNull(builder.conf, new HiveConf());
+
+ if (builder.wh != null) {
+ this.wh = builder.wh;
+ } else {
+ this.wh = new Warehouse(conf);
+ }
+
+ if (builder.catalogId != null) {
+ this.catalogId = builder.catalogId;
+ } else {
+ this.catalogId = null;
+ }
+
+ GlueClientFactory clientFactory = Objects.firstNonNull(builder.clientFactory, new AWSGlueClientFactory(conf));
+ AWSGlueMetastoreFactory metastoreFactory = Objects.firstNonNull(builder.metastoreFactory,
+ new AWSGlueMetastoreFactory());
+
+ glueClient = clientFactory.newClient();
+ AWSGlueMetastore glueMetastore = metastoreFactory.newMetastore(conf);
+ glueMetastoreClientDelegate = new GlueMetastoreClientDelegate(this.conf, glueMetastore, wh);
+
+ /**
+ * It seems weird to create databases as part of client construction. This
+ * part should probably be moved to the section in hive code right after the
+ * metastore client is instantiated. For now, simply copying the
+ * functionality in the thrift server
+ */
+ if(builder.createDefaults && !doesDefaultDBExist()) {
+ createDefaultDatabase();
+ }
+ }
+
+ private boolean doesDefaultDBExist() throws MetaException {
+
+ try {
+ GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest().withName(DEFAULT_DATABASE_NAME).withCatalogId(
+ catalogId);
+ glueClient.getDatabase(getDatabaseRequest);
+ } catch (EntityNotFoundException e) {
+ return false;
+ } catch (AmazonServiceException e) {
+ String msg = "Unable to verify existence of default database: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ return true;
+ }
+
+ private void createDefaultDatabase() throws MetaException {
+ Database defaultDB = new Database();
+ defaultDB.setName(DEFAULT_DATABASE_NAME);
+ defaultDB.setDescription(DEFAULT_DATABASE_COMMENT);
+ defaultDB.setLocationUri(wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString());
+
+ org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet principalPrivilegeSet
+ = new org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet();
+ principalPrivilegeSet.setRolePrivileges(Maps.>newHashMap());
+
+ defaultDB.setPrivileges(principalPrivilegeSet);
+
+ /**
+ * TODO: Grant access to role PUBLIC after role support is added
+ */
+ try {
+ create_database(defaultDB);
+ } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+ logger.warn("database - default already exists. Ignoring..");
+ } catch (Exception e) {
+ logger.error("Unable to create default database", e);
+ }
+ }
+
+ @Override
+ public void create_database(Database database) throws InvalidObjectException,
+ org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+ glueMetastoreClientDelegate.createDatabase(database);
+ }
+
+ @Override
+ public Database get_database(String name) throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getDatabase(name);
+ }
+
+ @Override
+ public List get_databases(String pattern) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getDatabases(pattern);
+ }
+
+ @Override
+ public List get_all_databases() throws MetaException, TException {
+ return get_databases(".*");
+ }
+
+
+ @Override
+ public void alter_database(String databaseName, Database database) throws NoSuchObjectException, MetaException,
+ TException {
+ glueMetastoreClientDelegate.alterDatabase(databaseName, database);
+ }
+
+// @Override
+// public void drop_database(String name) throws NoSuchObjectException, InvalidOperationException, MetaException,
+// TException {
+// drop_database(name, true, false, false);
+// }
+
+
+ @Override
+ public void drop_database(String name, boolean deleteData, boolean ignoreUnknownDb) throws NoSuchObjectException,
+ InvalidOperationException, MetaException, TException {
+ drop_database(name, deleteData, ignoreUnknownDb, false);
+ }
+
+// @Override
+ public void drop_database(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
+ throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.dropDatabase(name, deleteData, ignoreUnknownDb, cascade);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition add_partition(org.apache.hadoop.hive.metastore.api.Partition partition)
+ throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException,
+ TException {
+ glueMetastoreClientDelegate.addPartitions(Lists.newArrayList(partition), false, true);
+ return partition;
+ }
+
+ @Override
+ public int add_partitions(List partitions)
+ throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException,
+ TException {
+ return glueMetastoreClientDelegate.addPartitions(partitions, false, true).size();
+ }
+
+
+
+ // @Override
+// public List add_partitions(
+// List partitions,
+// boolean ifNotExists,
+// boolean needResult
+// ) throws TException {
+// return glueMetastoreClientDelegate.addPartitions(partitions, ifNotExists, needResult);
+// }
+//
+
+
+ @Override
+ public AddPartitionsResult add_partitions_req(AddPartitionsRequest addPartitionsRequest)
+ throws TException
+ {
+ List partitions = addPartitionsRequest.getParts();
+ boolean ifNotExists = addPartitionsRequest.isIfNotExists();
+ boolean needResult = addPartitionsRequest.isNeedResult();
+ List addPartitions = glueMetastoreClientDelegate.addPartitions(partitions, ifNotExists, needResult);
+ AddPartitionsResult addPartitionsResult = new AddPartitionsResult();
+ addPartitionsResult.setPartitions(addPartitions);
+ //TODO: check the set of the fields (see getter)
+ addPartitionsResult.setFieldValue(AddPartitionsResult._Fields.PARTITIONS,null);
+ return addPartitionsResult;
+ }
+
+ @Override
+ public int add_partitions_pspec(List list)
+ throws TException
+ {
+ PartitionSpecProxy pSpec = PartitionSpecProxy.Factory.get(list);
+ return glueMetastoreClientDelegate.addPartitionsSpecProxy(pSpec);
+ }
+
+ @Override
+ public void alter_function(String dbName, String functionName, org.apache.hadoop.hive.metastore.api.Function newFunction) throws InvalidObjectException,
+ MetaException, TException {
+ glueMetastoreClientDelegate.alterFunction(dbName, functionName, newFunction);
+ }
+
+ @Override
+ public void alter_index(String dbName, String tblName, String indexName, Index index) throws InvalidOperationException,
+ MetaException, TException {
+ Table catalogIndexTableObject = HiveToCatalogConverter.convertIndexToTableObject(index);
+ org.apache.hadoop.hive.metastore.api.Table originTable = getTable(dbName, tblName);
+ String indexTableObjectName = INDEX_PREFIX + indexName;
+ if (!originTable.getParameters().containsKey(indexTableObjectName)) {
+ throw new NoSuchObjectException("can not find index: " + indexName);
+ }
+
+ originTable.getParameters().put(indexTableObjectName, catalogTableToString(catalogIndexTableObject));
+ alter_table(dbName, tblName, originTable);
+ }
+
+ @Override
+ public void alter_partition(
+ String dbName,
+ String tblName,
+ org.apache.hadoop.hive.metastore.api.Partition partition
+ ) throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterPartitions(dbName, tblName, Lists.newArrayList(partition));
+ }
+
+ @Override
+ public void alter_partition_with_environment_context(
+ String dbName,
+ String tblName,
+ org.apache.hadoop.hive.metastore.api.Partition partition,
+ EnvironmentContext environmentContext
+ ) throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterPartitions(dbName, tblName, Lists.newArrayList(partition));
+ }
+
+
+ @Override
+ public void alter_partitions(
+ String dbName,
+ String tblName,
+ List partitions
+ ) throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterPartitions(dbName, tblName, partitions);
+ }
+
+ @Override
+ public void alter_partitions_with_environment_context(
+ String dbName,
+ String tblName,
+ List partitions,
+ EnvironmentContext environmentContext
+ ) throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterPartitions(dbName, tblName, partitions);
+ }
+
+
+ @Override
+ public void alter_table(String dbName, String tblName, org.apache.hadoop.hive.metastore.api.Table table)
+ throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterTable(dbName, tblName, table, null);
+ }
+
+ @Override
+ public void alter_table_with_cascade(String dbName, String tblName, org.apache.hadoop.hive.metastore.api.Table table, boolean cascade)
+ throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterTable(dbName, tblName, table, null);
+ }
+
+
+ @Override
+ public void alter_table_with_environment_context(
+ String dbName,
+ String tblName,
+ org.apache.hadoop.hive.metastore.api.Table table,
+ EnvironmentContext environmentContext
+ ) throws InvalidOperationException, MetaException, TException {
+ glueMetastoreClientDelegate.alterTable(dbName, tblName, table, environmentContext);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition append_partition(String dbName, String tblName, List values)
+ throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+ return glueMetastoreClientDelegate.appendPartition(dbName, tblName, values);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition append_partition_by_name(String dbName, String tblName, String partitionName) throws InvalidObjectException,
+ org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+ List partVals = partitionNameToVals(partitionName);
+ return glueMetastoreClientDelegate.appendPartition(dbName, tblName, partVals);
+ }
+
+
+ @Override
+ public boolean create_role(org.apache.hadoop.hive.metastore.api.Role role) throws MetaException, TException {
+ return glueMetastoreClientDelegate.createRole(role);
+ }
+
+ @Override
+ public boolean drop_role(String roleName) throws MetaException, TException {
+ return glueMetastoreClientDelegate.dropRole(roleName);
+ }
+
+ @Override
+ public List list_roles(
+ String principalName, org.apache.hadoop.hive.metastore.api.PrincipalType principalType
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.listRoles(principalName, principalType);
+ }
+
+ @Override
+ public List get_role_names() throws MetaException, TException {
+ return glueMetastoreClientDelegate.listRoleNames();
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse get_principals_in_role(
+ org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest request) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getPrincipalsInRole(request);
+ }
+
+ @Override
+ public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(
+ GetRoleGrantsForPrincipalRequest request) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getRoleGrantsForPrincipal(request);
+ }
+
+ @Override
+ public boolean grant_role(
+ String roleName,
+ String userName,
+ org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+ String grantor, org.apache.hadoop.hive.metastore.api.PrincipalType grantorType,
+ boolean grantOption
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.grantRole(roleName, userName, principalType, grantor, grantorType, grantOption);
+ }
+
+// @Override
+// public boolean revoke_role(
+// String roleName,
+// String userName,
+// org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+// boolean grantOption
+// ) throws MetaException, TException {
+// return glueMetastoreClientDelegate.revokeRole(roleName, userName, principalType, grantOption);
+// }
+
+
+ @Override
+ public void cancel_delegation_token(String tokenStrForm) throws MetaException, TException {
+ glueMetastoreClientDelegate.cancelDelegationToken(tokenStrForm);
+ }
+
+// @Override
+// public String getTokenStrForm() throws IOException {
+// return glueMetastoreClientDelegate.getTokenStrForm();
+// }
+
+
+
+ @Override
+ public boolean add_token(String tokenIdentifier, String delegationToken) throws TException {
+ return glueMetastoreClientDelegate.addToken(tokenIdentifier, delegationToken);
+ }
+
+ @Override
+ public boolean remove_token(String tokenIdentifier) throws TException {
+ return glueMetastoreClientDelegate.removeToken(tokenIdentifier);
+ }
+
+ @Override
+ public String get_token(String tokenIdentifier) throws TException {
+ return glueMetastoreClientDelegate.getToken(tokenIdentifier);
+ }
+
+ @Override
+ public List get_all_token_identifiers()
+ throws TException
+ {
+ return glueMetastoreClientDelegate.getAllTokenIdentifiers();
+ }
+
+ @Override
+ public int add_master_key(String key) throws MetaException, TException {
+ return glueMetastoreClientDelegate.addMasterKey(key);
+ }
+
+ @Override
+ public void update_master_key(int seqNo, String key) throws NoSuchObjectException, MetaException, TException {
+ glueMetastoreClientDelegate.updateMasterKey(seqNo, key);
+ }
+
+ @Override
+ public boolean remove_master_key(int keySeq) throws TException {
+ return glueMetastoreClientDelegate.removeMasterKey(keySeq);
+ }
+
+ @Override
+ public List get_master_keys()
+ throws TException
+ {
+ return Arrays.asList(glueMetastoreClientDelegate.getMasterKeys());
+ }
+
+
+ @Override
+ public LockResponse check_lock(CheckLockRequest checkLockRequest)
+ throws TException
+ {
+ return glueMetastoreClientDelegate.checkLock(checkLockRequest.getLockid());
+ }
+
+ @Override
+ public void close() {
+ currentMetaVars = null;
+ }
+
+
+ @Override
+ public void commit_txn(CommitTxnRequest commitTxnRequest)
+ throws TException
+ {
+ glueMetastoreClientDelegate.commitTxn(commitTxnRequest.getTxnid());
+ }
+
+ @Override
+ public void abort_txns(AbortTxnsRequest abortTxnsRequest)
+ throws TException
+ {
+ glueMetastoreClientDelegate.abortTxns(abortTxnsRequest.getTxn_ids());
+ }
+
+ @Deprecated
+ public void compact(
+ String dbName,
+ String tblName,
+ String partitionName,
+ CompactionType compactionType
+ ) throws TException {
+ glueMetastoreClientDelegate.compact(dbName, tblName, partitionName, compactionType);
+ }
+
+ @Deprecated
+ public void compact(
+ String dbName,
+ String tblName,
+ String partitionName,
+ CompactionType compactionType,
+ Map tblProperties
+ ) throws TException {
+ glueMetastoreClientDelegate.compact(dbName, tblName, partitionName, compactionType, tblProperties);
+ }
+
+
+ @Override
+ public CompactionResponse compact2(CompactionRequest compactionRequest)
+ throws TException
+ {
+ String dbName = compactionRequest.getDbname();
+ String tblName = compactionRequest.getTablename();
+ String partitionName = compactionRequest.getPartitionname();
+ CompactionType compactionType = compactionRequest.getType();
+ Map tblProperties = compactionRequest.getProperties();
+ return glueMetastoreClientDelegate.compact2(dbName, tblName, partitionName, compactionType, tblProperties);
+ }
+
+
+ @Override
+ public void create_function(Function function)
+ throws TException
+ {
+ glueMetastoreClientDelegate.createFunction(function);
+ }
+
+ @Override
+ public Index add_index(Index index, org.apache.hadoop.hive.metastore.api.Table indexTable)
+ throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException{
+ boolean dirCreated = glueMetastoreClientDelegate.validateNewTableAndCreateDirectory(indexTable);
+ boolean indexTableCreated = false;
+ String dbName = index.getDbName();
+ String indexTableName = index.getIndexTableName();
+ String originTableName = index.getOrigTableName();
+ Path indexTablePath = new Path(indexTable.getSd().getLocation());
+ Table catalogIndexTableObject = HiveToCatalogConverter.convertIndexToTableObject(index);
+ String indexTableObjectName = INDEX_PREFIX + index.getIndexName();
+
+ try {
+ org.apache.hadoop.hive.metastore.api.Table originTable = getTable(dbName, originTableName);
+ Map parameters = originTable.getParameters();
+ if (parameters.containsKey(indexTableObjectName)){
+ throw new org.apache.hadoop.hive.metastore.api.AlreadyExistsException("Index: " + index.getIndexName() + " already exist");
+ }
+ create_table(indexTable);
+ indexTableCreated = true;
+ originTable.getParameters().put(indexTableObjectName, catalogTableToString(catalogIndexTableObject));
+ alter_table(dbName, originTableName, originTable);
+ //TODO: check the return value
+ return index;
+ } catch (Exception e) {
+ if (dirCreated){
+ wh.deleteDir(indexTablePath, true);
+ }
+ if (indexTableCreated) {
+ dropTable(dbName, indexTableName);
+ }
+ String msg = "Unable to create index: ";
+ logger.error(msg, e);
+ if (e instanceof TException) {
+ throw e;
+ } else {
+ throw new MetaException(msg + e);
+ }
+ }
+
+
+ }
+
+ @Override
+ public void create_table(org.apache.hadoop.hive.metastore.api.Table tbl) throws org.apache.hadoop.hive.metastore.api.AlreadyExistsException, InvalidObjectException, MetaException,
+ NoSuchObjectException, TException {
+ glueMetastoreClientDelegate.createTable(tbl);
+ }
+
+
+ @Override
+ public boolean delete_partition_column_statistics(
+ String dbName, String tableName, String partName, String colName
+ ) throws NoSuchObjectException, MetaException, InvalidObjectException,
+ TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+ return glueMetastoreClientDelegate.deletePartitionColumnStatistics(dbName, tableName, partName, colName);
+ }
+
+
+ @Override
+ public boolean delete_table_column_statistics(
+ String dbName, String tableName, String colName
+ ) throws NoSuchObjectException, MetaException, InvalidObjectException,
+ TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+ return glueMetastoreClientDelegate.deleteTableColumnStatistics(dbName, tableName, colName);
+ }
+
+ @Override
+ public void drop_function(String dbName, String functionName) throws MetaException, NoSuchObjectException,
+ InvalidObjectException, org.apache.hadoop.hive.metastore.api.InvalidInputException, TException {
+ glueMetastoreClientDelegate.dropFunction(dbName, functionName);
+ }
+
+ @Override
+ public boolean drop_index_by_name(String dbName, String tblName, String name, boolean deleteData) throws NoSuchObjectException,
+ MetaException, TException {
+ Index indexToDrop = getIndex(dbName, tblName, name);
+ String indexTableName = indexToDrop.getIndexTableName();
+
+ // Drop the index metadata
+ org.apache.hadoop.hive.metastore.api.Table originTable = getTable(dbName, tblName);
+ Map parameters = originTable.getParameters();
+ String indexTableObjectName = INDEX_PREFIX + name;
+ if (!parameters.containsKey(indexTableObjectName)) {
+ throw new NoSuchObjectException("can not find Index: " + name);
+ }
+ parameters.remove(indexTableObjectName);
+
+ alter_table(dbName, tblName, originTable);
+
+ // Now drop the data associated with the table used to hold the index data
+ if(indexTableName != null && indexTableName.length() > 0) {
+ dropTable(dbName, indexTableName, deleteData, true);
+ }
+
+ return true;
+ }
+
+ private void deleteParentRecursive(Path parent, int depth, boolean mustPurge) throws IOException, MetaException {
+ if (depth > 0 && parent != null && wh.isWritable(parent) && wh.isEmpty(parent)) {
+ wh.deleteDir(parent, true, mustPurge);
+ deleteParentRecursive(parent.getParent(), depth - 1, mustPurge);
+ }
+ }
+
+ // This logic is taken from HiveMetaStore#isMustPurge
+ private boolean isMustPurge(org.apache.hadoop.hive.metastore.api.Table table, boolean ifPurge) {
+ return (ifPurge || "true".equalsIgnoreCase(table.getParameters().get("auto.purge")));
+ }
+
+ @Override
+ public boolean drop_partition(String dbName, String tblName, List values, boolean deleteData)
+ throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+ }
+
+// @Override
+// public boolean dropPartition(String dbName, String tblName, List values, PartitionDropOptions options) throws TException {
+// return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, options.ifExists, options.deleteData, options.purgeData);
+// }
+
+
+
+ @Override
+ public boolean drop_partition_by_name(String dbName, String tblName, String partitionName, boolean deleteData)
+ throws NoSuchObjectException, MetaException, TException {
+ List values = partitionNameToVals(partitionName);
+ return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+ }
+
+
+// @Override
+// public List dropPartitions(
+// String dbName,
+// String tblName,
+// List> partExprs,
+// boolean deleteData,
+// boolean ifExists
+// ) throws NoSuchObjectException, MetaException, TException {
+// //use defaults from PartitionDropOptions for purgeData
+// return dropPartitions_core(dbName, tblName, partExprs, deleteData, false);
+// }
+
+ @Override
+ public DropPartitionsResult drop_partitions_req(DropPartitionsRequest dropPartitionsRequest)
+ throws TException
+ {
+ //TODO:check this method
+ String dbName=dropPartitionsRequest.getDbName();
+ String tblName = dropPartitionsRequest.getTblName();
+ RequestPartsSpec parts = dropPartitionsRequest.getParts();
+ List exprs = parts.getExprs();
+ List> partExprs = exprs.stream().map(
+ dropPartitionsExpr ->
+ new ObjectPair(Integer.valueOf(dropPartitionsExpr.getPartArchiveLevel()), dropPartitionsExpr.getExpr())).collect(Collectors.toList());
+ boolean deleteData = dropPartitionsRequest.isDeleteData();
+
+ List partitions = dropPartitions_core(dbName, tblName, partExprs, deleteData, false);
+ DropPartitionsResult dropPartitionsResult = new DropPartitionsResult();
+ dropPartitionsResult.setPartitions(partitions);
+ dropPartitionsResult.setFieldValue(DropPartitionsResult._Fields.PARTITIONS,null);
+ return dropPartitionsResult;
+ }
+
+// @Override
+// public List dropPartitions(
+// String dbName,
+// String tblName,
+// List> partExprs,
+// boolean deleteData,
+// boolean ifExists,
+// boolean needResults
+// ) throws NoSuchObjectException, MetaException, TException {
+// return dropPartitions_core(dbName, tblName, partExprs, deleteData, false);
+// }
+//
+//
+//
+// @Override
+// public List dropPartitions(
+// String dbName,
+// String tblName,
+// List> partExprs,
+// PartitionDropOptions options
+// ) throws TException {
+// return dropPartitions_core(dbName, tblName, partExprs, options.deleteData, options.purgeData);
+// }
+
+ private List dropPartitions_core(
+ String databaseName,
+ String tableName,
+ List> partExprs,
+ boolean deleteData,
+ boolean purgeData
+ ) throws TException {
+ List deleted = Lists.newArrayList();
+ for (ObjectPair expr : partExprs) {
+ byte[] tmp = expr.getSecond();
+ String exprString = ExpressionHelper.convertHiveExpressionToCatalogExpression(tmp);
+ List catalogPartitionsToDelete = glueMetastoreClientDelegate.getCatalogPartitions(databaseName, tableName, exprString, -1);
+ deleted.addAll(batchDeletePartitions(databaseName, tableName, catalogPartitionsToDelete, deleteData, purgeData));
+ }
+ return deleted;
+ }
+
+ /**
+ * Delete all partitions in the list provided with BatchDeletePartitions request. It doesn't use transaction,
+ * so the call may result in partial failure.
+ * @param dbName
+ * @param tableName
+ * @param partitionsToDelete
+ * @return the partitions successfully deleted
+ * @throws TException
+ */
+ private List batchDeletePartitions(
+ final String dbName, final String tableName, final List partitionsToDelete,
+ final boolean deleteData, final boolean purgeData) throws TException {
+
+ List deleted = Lists.newArrayList();
+ if (partitionsToDelete == null) {
+ return deleted;
+ }
+
+ validateBatchDeletePartitionsArguments(dbName, tableName, partitionsToDelete);
+
+ List> batchDeletePartitionsFutures = Lists.newArrayList();
+
+ int numOfPartitionsToDelete = partitionsToDelete.size();
+ for (int i = 0; i < numOfPartitionsToDelete; i += BATCH_DELETE_PARTITIONS_PAGE_SIZE) {
+ int j = Math.min(i + BATCH_DELETE_PARTITIONS_PAGE_SIZE, numOfPartitionsToDelete);
+ final List partitionsOnePage = partitionsToDelete.subList(i, j);
+
+ batchDeletePartitionsFutures.add(BATCH_DELETE_PARTITIONS_THREAD_POOL.submit(new Callable() {
+ @Override
+ public BatchDeletePartitionsHelper call() throws Exception {
+ return new BatchDeletePartitionsHelper(glueClient, dbName, tableName, catalogId, partitionsOnePage).deletePartitions();
+ }
+ }));
+ }
+
+ TException tException = null;
+ for (Future future : batchDeletePartitionsFutures) {
+ try {
+ BatchDeletePartitionsHelper batchDeletePartitionsHelper = future.get();
+ for (Partition partition : batchDeletePartitionsHelper.getPartitionsDeleted()) {
+ org.apache.hadoop.hive.metastore.api.Partition hivePartition =
+ CatalogToHiveConverter.convertPartition(partition);
+ try {
+ performDropPartitionPostProcessing(dbName, tableName, hivePartition, deleteData, purgeData);
+ } catch (TException e) {
+ logger.error("Drop partition directory failed.", e);
+ tException = tException == null ? e : tException;
+ }
+ deleted.add(hivePartition);
+ }
+ tException = tException == null ? batchDeletePartitionsHelper.getFirstTException() : tException;
+ } catch (Exception e) {
+ logger.error("Exception thrown by BatchDeletePartitions thread pool. ", e);
+ }
+ }
+
+ if (tException != null) {
+ throw tException;
+ }
+ return deleted;
+ }
+
+ private void validateBatchDeletePartitionsArguments(final String dbName, final String tableName,
+ final List partitionsToDelete) {
+
+ Preconditions.checkArgument(dbName != null, "Database name cannot be null");
+ Preconditions.checkArgument(tableName != null, "Table name cannot be null");
+ for (Partition partition : partitionsToDelete) {
+ Preconditions.checkArgument(dbName.equals(partition.getDatabaseName()), "Database name cannot be null");
+ Preconditions.checkArgument(tableName.equals(partition.getTableName()), "Table name cannot be null");
+ Preconditions.checkArgument(partition.getValues() != null, "Partition values cannot be null");
+ }
+ }
+
+ // Preserve the logic from Hive metastore
+ private void performDropPartitionPostProcessing(String dbName, String tblName,
+ org.apache.hadoop.hive.metastore.api.Partition partition, boolean deleteData, boolean ifPurge)
+ throws MetaException, NoSuchObjectException, TException {
+ if (deleteData && partition.getSd() != null && partition.getSd().getLocation() != null) {
+ Path partPath = new Path(partition.getSd().getLocation());
+ org.apache.hadoop.hive.metastore.api.Table table = getTable(dbName, tblName);
+ if (isExternalTable(table)){
+ //Don't delete external table data
+ return;
+ }
+ boolean mustPurge = isMustPurge(table, ifPurge);
+ wh.deleteDir(partPath, true, mustPurge);
+ try {
+ List values = partition.getValues();
+ deleteParentRecursive(partPath.getParent(), values.size() - 1, mustPurge);
+ } catch (IOException e) {
+ throw new MetaException(e.getMessage());
+ }
+ }
+ }
+
+ @Deprecated
+ public void dropTable(String tableName, boolean deleteData) throws MetaException, UnknownTableException, TException,
+ NoSuchObjectException {
+ dropTable(DEFAULT_DATABASE_NAME, tableName, deleteData, false);
+ }
+
+// @Override
+// public void dropTable(String dbname, String tableName) throws MetaException, TException, NoSuchObjectException {
+// dropTable(dbname, tableName, true, true, false);
+// }
+
+
+
+ @Override
+ public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUnknownTab)
+ throws MetaException, TException, NoSuchObjectException {
+ dropTable(dbname, tableName, deleteData, ignoreUnknownTab, false);
+ }
+
+ @Override
+ public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge)
+ throws MetaException, TException, NoSuchObjectException {
+ glueMetastoreClientDelegate.dropTable(dbname, tableName, deleteData, ignoreUnknownTab, ifPurge);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition exchange_partition(
+ Map partitionSpecs,
+ String srcDb,
+ String srcTbl,
+ String dstDb,
+ String dstTbl
+ ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+ return glueMetastoreClientDelegate.exchangePartition(partitionSpecs, srcDb, srcTbl, dstDb, dstTbl);
+ }
+
+ @Override
+ public List exchange_partitions(
+ Map partitionSpecs,
+ String sourceDb,
+ String sourceTbl,
+ String destDb,
+ String destTbl
+ ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+ return glueMetastoreClientDelegate.exchangePartitions(partitionSpecs, sourceDb, sourceTbl, destDb, destTbl);
+ }
+
+ @Override
+ public AggrStats getAggrColStatsFor(String dbName, String tblName, List colNames, List partName)
+ throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getAggrColStatsFor(dbName, tblName, colNames, partName);
+ }
+
+ @Override
+ public List getAllTables(String dbname) throws MetaException, TException, UnknownDBException {
+ return getTables(dbname, ".*");
+ }
+
+ @Override
+ public String getConfigValue(String name, String defaultValue) throws TException, ConfigValSecurityException {
+ if(!Pattern.matches("(hive|hdfs|mapred).*", name)) {
+ throw new ConfigValSecurityException("For security reasons, the config key " + name + " cannot be accessed");
+ }
+
+ return conf.get(name, defaultValue);
+ }
+
+ @Override
+ public String getDelegationToken(
+ String owner, String renewerKerberosPrincipalName
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getDelegationToken(owner, renewerKerberosPrincipalName);
+ }
+
+ @Override
+ public List getFields(String db, String tableName) throws MetaException, TException,
+ UnknownTableException, UnknownDBException {
+ return glueMetastoreClientDelegate.getFields(db, tableName);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Function getFunction(String dbName, String functionName) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getFunction(dbName, functionName);
+ }
+
+ @Override
+ public List getFunctions(String dbName, String pattern) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getFunctions(dbName, pattern);
+ }
+
+ @Override
+ public GetAllFunctionsResponse getAllFunctions() throws MetaException, TException {
+ List databaseNames = getDatabases(".*");
+ List result = new ArrayList<>();
+ try {
+ for (String databaseName : databaseNames) {
+ GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest = new GetUserDefinedFunctionsRequest()
+ .withDatabaseName(databaseName).withPattern(".*").withCatalogId(catalogId);
+
+ List catalogFunctions = glueClient.getUserDefinedFunctions(
+ getUserDefinedFunctionsRequest)
+ .getUserDefinedFunctions();
+ for (UserDefinedFunction catalogFunction : catalogFunctions) {
+ result.add(CatalogToHiveConverter.convertFunction(databaseName, catalogFunction));
+ }
+ }
+
+ GetAllFunctionsResponse response = new GetAllFunctionsResponse();
+ response.setFunctions(result);
+ return response;
+ } catch (AmazonServiceException e) {
+ logger.error(e);
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to get Functions: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ @Override
+ public Index getIndex(String dbName, String tblName, String indexName) throws MetaException, UnknownTableException,
+ NoSuchObjectException, TException {
+ org.apache.hadoop.hive.metastore.api.Table originTable = getTable(dbName, tblName);
+ Map map = originTable.getParameters();
+ String indexTableName = INDEX_PREFIX + indexName;
+ if(!map.containsKey(indexTableName)) {
+ throw new NoSuchObjectException("can not find index: " + indexName);
+ }
+ Table indexTableObject = stringToCatalogTable(map.get(indexTableName));
+ return CatalogToHiveConverter.convertTableObjectToIndex(indexTableObject);
+ }
+
+ @Override
+ public String getMetaConf(String key) throws MetaException, TException {
+ ConfVars metaConfVar = HiveConf.getMetaConf(key);
+ if (metaConfVar == null) {
+ throw new MetaException("Invalid configuration key " + key);
+ }
+ return conf.get(key, metaConfVar.getDefaultValue());
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tblName, List values)
+ throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getPartition(dbName, tblName, values);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tblName, String partitionName)
+ throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+ return glueMetastoreClientDelegate.getPartition(dbName, tblName, partitionName);
+ }
+
+ @Override
+ public Map> getPartitionColumnStatistics(
+ String dbName,
+ String tableName,
+ List partitionNames,
+ List columnNames
+ ) throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getPartitionColumnStatistics(dbName, tableName, partitionNames, columnNames);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Partition getPartitionWithAuthInfo(
+ String databaseName, String tableName, List values,
+ String userName, List groupNames)
+ throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+
+ // TODO move this into the service
+ org.apache.hadoop.hive.metastore.api.Partition partition = getPartition(databaseName, tableName, values);
+ org.apache.hadoop.hive.metastore.api.Table table = getTable(databaseName, tableName);
+ if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
+ String partName = Warehouse.makePartName(table.getPartitionKeys(), values);
+ HiveObjectRef obj = new HiveObjectRef();
+ obj.setObjectType(HiveObjectType.PARTITION);
+ obj.setDbName(databaseName);
+ obj.setObjectName(tableName);
+ obj.setPartValues(values);
+ org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet privilegeSet =
+ this.get_privilege_set(obj, userName, groupNames);
+ partition.setPrivileges(privilegeSet);
+ }
+
+ return partition;
+ }
+
+ @Override
+ public List getPartitionsByNames(
+ String databaseName, String tableName, List partitionNames)
+ throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getPartitionsByNames(databaseName, tableName, partitionNames);
+ }
+
+ @Override
+ public List getSchema(String db, String tableName) throws MetaException, TException, UnknownTableException,
+ UnknownDBException {
+ return glueMetastoreClientDelegate.getSchema(db, tableName);
+ }
+
+ @Deprecated
+ public org.apache.hadoop.hive.metastore.api.Table getTable(String tableName) throws MetaException, TException, NoSuchObjectException {
+ //this has been deprecated
+ return getTable(DEFAULT_DATABASE_NAME, tableName);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.Table getTable(String dbName, String tableName)
+ throws MetaException, TException, NoSuchObjectException {
+ return glueMetastoreClientDelegate.getTable(dbName, tableName);
+ }
+
+ @Override
+ public List getTableColumnStatistics(String dbName, String tableName, List colNames)
+ throws NoSuchObjectException, MetaException, TException {
+ return glueMetastoreClientDelegate.getTableColumnStatistics(dbName, tableName, colNames);
+ }
+
+ @Override
+ public List getTableObjectsByName(String dbName, List tableNames) throws MetaException,
+ InvalidOperationException, UnknownDBException, TException {
+ List hiveTables = Lists.newArrayList();
+ for(String tableName : tableNames) {
+ hiveTables.add(getTable(dbName, tableName));
+ }
+
+ return hiveTables;
+ }
+
+ @Override
+ public List getTables(String dbname, String tablePattern) throws MetaException, TException, UnknownDBException {
+ return glueMetastoreClientDelegate.getTables(dbname, tablePattern);
+ }
+
+ @Override
+ public List getTables(String dbname, String tablePattern, TableType tableType)
+ throws MetaException, TException, UnknownDBException {
+ return glueMetastoreClientDelegate.getTables(dbname, tablePattern, tableType);
+ }
+
+ @Override
+ public List getTableMeta(String dbPatterns, String tablePatterns, List tableTypes)
+ throws MetaException, TException, UnknownDBException {
+ return glueMetastoreClientDelegate.getTableMeta(dbPatterns, tablePatterns, tableTypes);
+ }
+
+ @Override
+ public ValidTxnList getValidTxns() throws TException {
+ return glueMetastoreClientDelegate.getValidTxns();
+ }
+
+ @Override
+ public ValidTxnList getValidTxns(long currentTxn) throws TException {
+ return glueMetastoreClientDelegate.getValidTxns(currentTxn);
+ }
+
+ @Override
+ public org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet get_privilege_set(
+ HiveObjectRef obj,
+ String user, List groups
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.getPrivilegeSet(obj, user, groups);
+ }
+
+ @Override
+ public boolean grant_privileges(org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges)
+ throws MetaException, TException {
+ return glueMetastoreClientDelegate.grantPrivileges(privileges);
+ }
+
+ @Override
+ public boolean revoke_privileges(
+ org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges,
+ boolean grantOption
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.revokePrivileges(privileges, grantOption);
+ }
+
+ @Override
+ public void heartbeat(long txnId, long lockId)
+ throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, TException {
+ glueMetastoreClientDelegate.heartbeat(txnId, lockId);
+ }
+
+ @Override
+ public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) throws TException {
+ return glueMetastoreClientDelegate.heartbeatTxnRange(min, max);
+ }
+
+ @Override
+ public boolean isCompatibleWith(HiveConf conf) {
+ if (currentMetaVars == null) {
+ return false; // recreate
+ }
+ boolean compatible = true;
+ for (ConfVars oneVar : HiveConf.metaVars) {
+ // Since metaVars are all of different types, use string for comparison
+ String oldVar = currentMetaVars.get(oneVar.varname);
+ String newVar = conf.get(oneVar.varname, "");
+ if (oldVar == null ||
+ (oneVar.isCaseSensitive() ? !oldVar.equals(newVar) : !oldVar.equalsIgnoreCase(newVar))) {
+ logger.info("Mestastore configuration " + oneVar.varname +
+ " changed from " + oldVar + " to " + newVar);
+ compatible = false;
+ }
+ }
+ return compatible;
+ }
+
+ @Override
+ public void setHiveAddedJars(String addedJars) {
+ //taken from HiveMetaStoreClient
+ HiveConf.setVar(conf, ConfVars.HIVEADDEDJARS, addedJars);
+ }
+
+ @Override
+ public boolean isLocalMetaStore() {
+ return false;
+ }
+
+ private void snapshotActiveConf() {
+ currentMetaVars = new HashMap(HiveConf.metaVars.length);
+ for (ConfVars oneVar : HiveConf.metaVars) {
+ currentMetaVars.put(oneVar.varname, conf.get(oneVar.varname, ""));
+ }
+ }
+
+ @Override
+ public boolean isPartitionMarkedForEvent(String dbName, String tblName, Map partKVs, PartitionEventType eventType)
+ throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+ UnknownPartitionException, InvalidPartitionException {
+ return glueMetastoreClientDelegate.isPartitionMarkedForEvent(dbName, tblName, partKVs, eventType);
+ }
+
+ @Override
+ public List listIndexNames(String db_name, String tbl_name, short max) throws MetaException, TException {
+ // In current hive implementation, it ignores fields "max"
+ // https://github.com/apache/hive/blob/rel/release-2.3.0/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L3902-L3932
+ List indexes = listIndexes(db_name, tbl_name, max);
+ List indexNames = Lists.newArrayList();
+ for(Index index : indexes) {
+ indexNames.add(index.getIndexName());
+ }
+
+ return indexNames;
+ }
+
+ @Override
+ public List listIndexes(String db_name, String tbl_name, short max) throws NoSuchObjectException, MetaException,
+ TException {
+ // In current hive implementation, it ignores fields "max"
+ // https://github.com/apache/hive/blob/rel/release-2.3.0/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L3867-L3899
+ return glueMetastoreClientDelegate.listIndexes(db_name, tbl_name);
+ }
+
+ @Override
+ public List listPartitionNames(String dbName, String tblName, short max)
+ throws MetaException, TException {
+ try {
+ return listPartitionNames(dbName, tblName, null, max);
+ } catch (NoSuchObjectException e) {
+ // For compatibility with Hive 1.0.0
+ return Collections.emptyList();
+ }
+ }
+
+ @Override
+ public List listPartitionNames(String databaseName, String tableName,
+ List values, short max)
+ throws MetaException, TException, NoSuchObjectException {
+ return glueMetastoreClientDelegate.listPartitionNames(databaseName, tableName, values, max);
+ }
+
+ @Override
+ public int getNumPartitionsByFilter(String dbName, String tableName, String filter)
+ throws MetaException, NoSuchObjectException, TException {
+ return glueMetastoreClientDelegate.getNumPartitionsByFilter(dbName, tableName, filter);
+ }
+
+ @Override
+ public PartitionSpecProxy listPartitionSpecs(String dbName, String tblName, int max) throws TException {
+ return glueMetastoreClientDelegate.listPartitionSpecs(dbName, tblName, max);
+ }
+
+ @Override
+ public PartitionSpecProxy listPartitionSpecsByFilter(String dbName, String tblName, String filter, int max)
+ throws MetaException, NoSuchObjectException, TException {
+ return glueMetastoreClientDelegate.listPartitionSpecsByFilter(dbName, tblName, filter, max);
+ }
+
+ @Override
+ public List listPartitions(String dbName, String tblName, short max)
+ throws NoSuchObjectException, MetaException, TException {
+ return listPartitions(dbName, tblName, null, max);
+ }
+
+ @Override
+ public List listPartitions(
+ String databaseName,
+ String tableName,
+ List values,
+ short max
+ ) throws NoSuchObjectException, MetaException, TException {
+ String expression = null;
+ if (values != null) {
+ org.apache.hadoop.hive.metastore.api.Table table = getTable(databaseName, tableName);
+ expression = ExpressionHelper.buildExpressionFromPartialSpecification(table, values);
+ }
+ return glueMetastoreClientDelegate.getPartitions(databaseName, tableName, expression, (long) max);
+ }
+
+ @Override
+ public boolean listPartitionsByExpr(
+ String databaseName,
+ String tableName,
+ byte[] expr,
+ String defaultPartitionName,
+ short max,
+ List result
+ ) throws TException {
+ checkNotNull(result, "The result argument cannot be null.");
+
+ String catalogExpression = ExpressionHelper.convertHiveExpressionToCatalogExpression(expr);
+ List partitions =
+ glueMetastoreClientDelegate.getPartitions(databaseName, tableName, catalogExpression, (long) max);
+ result.addAll(partitions);
+
+ return false;
+ }
+
+ @Override
+ public List listPartitionsByFilter(
+ String databaseName,
+ String tableName,
+ String filter,
+ short max
+ ) throws MetaException, NoSuchObjectException, TException {
+ // we need to replace double quotes with single quotes in the filter expression
+ // since server side does not accept double quote expressions.
+ if (StringUtils.isNotBlank(filter)) {
+ filter = ExpressionHelper.replaceDoubleQuoteWithSingleQuotes(filter);
+ }
+ return glueMetastoreClientDelegate.getPartitions(databaseName, tableName, filter, (long) max);
+ }
+
+ @Override
+ public List listPartitionsWithAuthInfo(String database, String table, short maxParts,
+ String user, List groups)
+ throws MetaException, TException, NoSuchObjectException {
+ List partitions = listPartitions(database, table, maxParts);
+
+ for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
+ HiveObjectRef obj = new HiveObjectRef();
+ obj.setObjectType(HiveObjectType.PARTITION);
+ obj.setDbName(database);
+ obj.setObjectName(table);
+ obj.setPartValues(p.getValues());
+ org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet set = this.get_privilege_set(obj, user, groups);
+ p.setPrivileges(set);
+ }
+
+ return partitions;
+ }
+
+ @Override
+ public List listPartitionsWithAuthInfo(String database, String table,
+ List partVals, short maxParts,
+ String user, List groups) throws MetaException, TException, NoSuchObjectException {
+ List partitions = listPartitions(database, table, partVals, maxParts);
+
+ for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
+ HiveObjectRef obj = new HiveObjectRef();
+ obj.setObjectType(HiveObjectType.PARTITION);
+ obj.setDbName(database);
+ obj.setObjectName(table);
+ obj.setPartValues(p.getValues());
+ org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet set;
+ try {
+ set = get_privilege_set(obj, user, groups);
+ } catch (MetaException e) {
+ logger.info(String.format("No privileges found for user: %s, "
+ + "groups: [%s]", user, LoggingHelper.concatCollectionToStringForLogging(groups, ",")));
+ set = new org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet();
+ }
+ p.setPrivileges(set);
+ }
+
+ return partitions;
+ }
+
+ @Override
+ public List listTableNamesByFilter(String dbName, String filter, short maxTables) throws MetaException,
+ TException, InvalidOperationException, UnknownDBException {
+ return glueMetastoreClientDelegate.listTableNamesByFilter(dbName, filter, maxTables);
+ }
+
+ @Override
+ public List list_privileges(
+ String principal,
+ org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+ HiveObjectRef objectRef
+ ) throws MetaException, TException {
+ return glueMetastoreClientDelegate.listPrivileges(principal, principalType, objectRef);
+ }
+
+ @Override
+ public LockResponse lock(LockRequest lockRequest) throws NoSuchTxnException, TxnAbortedException, TException {
+ return glueMetastoreClientDelegate.lock(lockRequest);
+ }
+
+ @Override
+ public void markPartitionForEvent(
+ String dbName,
+ String tblName,
+ Map partKVs,
+ PartitionEventType eventType
+ ) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+ UnknownPartitionException, InvalidPartitionException {
+ glueMetastoreClientDelegate.markPartitionForEvent(dbName, tblName, partKVs, eventType);
+ }
+
+ @Override
+ public long openTxn(String user) throws TException {
+ return glueMetastoreClientDelegate.openTxn(user);
+ }
+
+ @Override
+ public OpenTxnsResponse openTxns(String user, int numTxns) throws TException {
+ return glueMetastoreClientDelegate.openTxns(user, numTxns);
+ }
+
+ @Override
+ public Map partitionNameToSpec(String name) throws MetaException, TException {
+ // Lifted from HiveMetaStore
+ if (name.length() == 0) {
+ return new HashMap();
+ }
+ return Warehouse.makeSpecFromName(name);
+ }
+
+ @Override
+ public List partitionNameToVals(String name) throws MetaException, TException {
+ return glueMetastoreClientDelegate.partitionNameToVals(name);
+ }
+
+ @Override
+ public void reconnect() throws MetaException {
+ // TODO reset active Hive confs for metastore glueClient
+ logger.debug("reconnect() was called.");
+ }
+
+ @Override
+ public void renamePartition(String dbName, String tblName, List partitionValues,
+ org.apache.hadoop.hive.metastore.api.Partition newPartition)
+ throws InvalidOperationException, MetaException, TException {
+
+ // Set DDL time to now if not specified
+ setDDLTime(newPartition);
+ org.apache.hadoop.hive.metastore.api.Table tbl;
+ org.apache.hadoop.hive.metastore.api.Partition oldPart;
+
+ try {
+ tbl = getTable(dbName, tblName);
+ oldPart = getPartition(dbName, tblName, partitionValues);
+ } catch(NoSuchObjectException e) {
+ throw new InvalidOperationException(e.getMessage());
+ }
+
+ if(newPartition.getSd() == null || oldPart.getSd() == null ) {
+ throw new InvalidOperationException("Storage descriptor cannot be null");
+ }
+
+ // if an external partition is renamed, the location should not change
+ if (!Strings.isNullOrEmpty(tbl.getTableType()) && tbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+ newPartition.getSd().setLocation(oldPart.getSd().getLocation());
+ renamePartitionInCatalog(dbName, tblName, partitionValues, newPartition);
+ } else {
+
+ Path destPath = getDestinationPathForRename(dbName, tbl, newPartition);
+ Path srcPath = new Path(oldPart.getSd().getLocation());
+ FileSystem srcFs = wh.getFs(srcPath);
+ FileSystem destFs = wh.getFs(destPath);
+
+ verifyDestinationLocation(srcFs, destFs, srcPath, destPath, tbl, newPartition);
+ newPartition.getSd().setLocation(destPath.toString());
+
+ renamePartitionInCatalog(dbName, tblName, partitionValues, newPartition);
+ boolean success = true;
+ try{
+ if (srcFs.exists(srcPath)) {
+ //if destPath's parent path doesn't exist, we should mkdir it
+ Path destParentPath = destPath.getParent();
+ if (!wh.mkdirs(destParentPath, true)) {
+ throw new IOException("Unable to create path " + destParentPath);
+ }
+ wh.renameDir(srcPath, destPath, true);
+ }
+ } catch (IOException e) {
+ success = false;
+ throw new InvalidOperationException("Unable to access old location "
+ + srcPath + " for partition " + tbl.getDbName() + "."
+ + tbl.getTableName() + " " + partitionValues);
+ } finally {
+ if(!success) {
+ // revert metastore operation
+ renamePartitionInCatalog(dbName, tblName, newPartition.getValues(), oldPart);
+ }
+ }
+ }
+ }
+
+ private void verifyDestinationLocation(FileSystem srcFs, FileSystem destFs, Path srcPath, Path destPath, org.apache.hadoop.hive.metastore.api.Table tbl, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+ throws InvalidOperationException {
+ String oldPartLoc = srcPath.toString();
+ String newPartLoc = destPath.toString();
+
+ // check that src and dest are on the same file system
+ if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
+ throw new InvalidOperationException("table new location " + destPath
+ + " is on a different file system than the old location "
+ + srcPath + ". This operation is not supported");
+ }
+ try {
+ srcFs.exists(srcPath); // check that src exists and also checks
+ if (newPartLoc.compareTo(oldPartLoc) != 0 && destFs.exists(destPath)) {
+ throw new InvalidOperationException("New location for this partition "
+ + tbl.getDbName() + "." + tbl.getTableName() + "." + newPartition.getValues()
+ + " already exists : " + destPath);
+ }
+ } catch (IOException e) {
+ throw new InvalidOperationException("Unable to access new location "
+ + destPath + " for partition " + tbl.getDbName() + "."
+ + tbl.getTableName() + " " + newPartition.getValues());
+ }
+ }
+
+ private Path getDestinationPathForRename(String dbName, org.apache.hadoop.hive.metastore.api.Table tbl, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+ throws InvalidOperationException, MetaException, TException {
+ try {
+ Path destPath = new Path(hiveShims.getDefaultTablePath(getDatabase(dbName), tbl.getTableName(), wh),
+ Warehouse.makePartName(tbl.getPartitionKeys(), newPartition.getValues()));
+ return constructRenamedPath(destPath, new Path(newPartition.getSd().getLocation()));
+ } catch (NoSuchObjectException e) {
+ throw new InvalidOperationException(
+ "Unable to change partition or table. Database " + dbName + " does not exist"
+ + " Check metastore logs for detailed stack." + e.getMessage());
+ }
+ }
+
+ private void setDDLTime(org.apache.hadoop.hive.metastore.api.Partition partition) {
+ if (partition.getParameters() == null ||
+ partition.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+ Integer.parseInt(partition.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+ partition.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
+ .currentTimeMillis() / 1000));
+ }
+ }
+
+ private void renamePartitionInCatalog(String databaseName, String tableName,
+ List partitionValues, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+ throws InvalidOperationException, MetaException, TException {
+ try {
+ glueClient.updatePartition(
+ new UpdatePartitionRequest()
+ .withDatabaseName(databaseName)
+ .withTableName(tableName)
+ .withPartitionValueList(partitionValues)
+ .withPartitionInput(GlueInputConverter.convertToPartitionInput(newPartition)));
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ }
+ }
+
+ @Override
+ public long renewDelegationToken(String tokenStrForm) throws MetaException, TException {
+ return glueMetastoreClientDelegate.renewDelegationToken(tokenStrForm);
+ }
+
+ @Override
+ public void rollbackTxn(long txnId) throws NoSuchTxnException, TException {
+ glueMetastoreClientDelegate.rollbackTxn(txnId);
+ }
+
+ @Override
+ public void setMetaConf(String key, String value) throws MetaException, TException {
+ ConfVars confVar = HiveConf.getMetaConf(key);
+ if (confVar == null) {
+ throw new MetaException("Invalid configuration key " + key);
+ }
+ String validate = confVar.validate(value);
+ if (validate != null) {
+ throw new MetaException("Invalid configuration value " + value + " for key " + key + " by " + validate);
+ }
+ conf.set(key, value);
+ }
+
+ @Override
+ public boolean setPartitionColumnStatistics(org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest request)
+ throws NoSuchObjectException, InvalidObjectException,
+ MetaException, TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+ return glueMetastoreClientDelegate.setPartitionColumnStatistics(request);
+ }
+
+ @Override
+ public void flushCache() {
+ //no op
+ }
+
+ @Override
+ public Iterable> getFileMetadata(List fileIds) throws TException {
+ return glueMetastoreClientDelegate.getFileMetadata(fileIds);
+ }
+
+ @Override
+ public Iterable> getFileMetadataBySarg(
+ List fileIds,
+ ByteBuffer sarg,
+ boolean doGetFooters
+ ) throws TException {
+ return glueMetastoreClientDelegate.getFileMetadataBySarg(fileIds, sarg, doGetFooters);
+ }
+
+ @Override
+ public void clearFileMetadata(List fileIds) throws TException {
+ glueMetastoreClientDelegate.clearFileMetadata(fileIds);
+ }
+
+ @Override
+ public void putFileMetadata(List fileIds, List metadata) throws TException {
+ glueMetastoreClientDelegate.putFileMetadata(fileIds, metadata);
+ }
+
+ @Override
+ public boolean isSameConfObj(HiveConf hiveConf) {
+ //taken from HiveMetaStoreClient
+ return this.conf == hiveConf;
+ }
+
+ @Override
+ public boolean cacheFileMetadata(String dbName, String tblName, String partName, boolean allParts) throws TException {
+ return glueMetastoreClientDelegate.cacheFileMetadata(dbName, tblName, partName, allParts);
+ }
+
+ @Override
+ public List getPrimaryKeys(PrimaryKeysRequest primaryKeysRequest) throws MetaException, NoSuchObjectException, TException {
+ // PrimaryKeys are currently unsupported
+ //return null to allow DESCRIBE (FORMATTED | EXTENDED)
+ return null;
+ }
+
+ @Override
+ public List getForeignKeys(ForeignKeysRequest foreignKeysRequest) throws MetaException, NoSuchObjectException, TException {
+ // PrimaryKeys are currently unsupported
+ //return null to allow DESCRIBE (FORMATTED | EXTENDED)
+ return null;
+ }
+
+ @Override
+ public void createTableWithConstraints(
+ org.apache.hadoop.hive.metastore.api.Table table,
+ List primaryKeys,
+ List foreignKeys
+ ) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException {
+ glueMetastoreClientDelegate.createTableWithConstraints(table, primaryKeys, foreignKeys);
+ }
+
+ @Override
+ public void dropConstraint(
+ String dbName,
+ String tblName,
+ String constraintName
+ ) throws MetaException, NoSuchObjectException, TException {
+ glueMetastoreClientDelegate.dropConstraint(dbName, tblName, constraintName);
+ }
+
+ @Override
+ public void addPrimaryKey(List primaryKeyCols)
+ throws MetaException, NoSuchObjectException, TException {
+ glueMetastoreClientDelegate.addPrimaryKey(primaryKeyCols);
+ }
+
+ @Override
+ public void addForeignKey(List foreignKeyCols)
+ throws MetaException, NoSuchObjectException, TException {
+ glueMetastoreClientDelegate.addForeignKey(foreignKeyCols);
+ }
+
+ @Override
+ public ShowCompactResponse showCompactions() throws TException {
+ return glueMetastoreClientDelegate.showCompactions();
+ }
+
+ @Override
+ public void addDynamicPartitions(
+ long txnId,
+ String dbName,
+ String tblName,
+ List partNames
+ ) throws TException {
+ glueMetastoreClientDelegate.addDynamicPartitions(txnId, dbName, tblName, partNames);
+ }
+
+ @Override
+ public void addDynamicPartitions(
+ long txnId,
+ String dbName,
+ String tblName,
+ List partNames,
+ DataOperationType operationType
+ ) throws TException {
+ glueMetastoreClientDelegate.addDynamicPartitions(txnId, dbName, tblName, partNames, operationType);
+ }
+
+ @Override
+ public void insertTable(org.apache.hadoop.hive.metastore.api.Table table, boolean overwrite) throws MetaException {
+ glueMetastoreClientDelegate.insertTable(table, overwrite);
+ }
+
+ @Override
+ public NotificationEventResponse getNextNotification(
+ long lastEventId, int maxEvents, NotificationFilter notificationFilter) throws TException {
+ return glueMetastoreClientDelegate.getNextNotification(lastEventId, maxEvents, notificationFilter);
+ }
+
+ @Override
+ public CurrentNotificationEventId getCurrentNotificationEventId() throws TException {
+ return glueMetastoreClientDelegate.getCurrentNotificationEventId();
+ }
+
+ @Override
+ public FireEventResponse fireListenerEvent(FireEventRequest fireEventRequest) throws TException {
+ return glueMetastoreClientDelegate.fireListenerEvent(fireEventRequest);
+ }
+
+ @Override
+ public ShowLocksResponse showLocks() throws TException {
+ return glueMetastoreClientDelegate.showLocks();
+ }
+
+ @Override
+ public ShowLocksResponse showLocks(ShowLocksRequest showLocksRequest) throws TException {
+ return glueMetastoreClientDelegate.showLocks(showLocksRequest);
+ }
+
+ @Override
+ public GetOpenTxnsInfoResponse showTxns() throws TException {
+ return glueMetastoreClientDelegate.showTxns();
+ }
+
+ @Deprecated
+ public boolean tableExists(String tableName) throws MetaException, TException, UnknownDBException {
+ //this method has been deprecated;
+ return tableExists(DEFAULT_DATABASE_NAME, tableName);
+ }
+
+ @Override
+ public boolean tableExists(String databaseName, String tableName) throws MetaException, TException,
+ UnknownDBException {
+ return glueMetastoreClientDelegate.tableExists(databaseName, tableName);
+ }
+
+ @Override
+ public void unlock(long lockId) throws NoSuchLockException, TxnOpenException, TException {
+ glueMetastoreClientDelegate.unlock(lockId);
+ }
+
+ @Override
+ public boolean updatePartitionColumnStatistics(org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics)
+ throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+ org.apache.hadoop.hive.metastore.api.InvalidInputException {
+ return glueMetastoreClientDelegate.updatePartitionColumnStatistics(columnStatistics);
+ }
+
+ @Override
+ public boolean updateTableColumnStatistics(org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics)
+ throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+ org.apache.hadoop.hive.metastore.api.InvalidInputException {
+ return glueMetastoreClientDelegate.updateTableColumnStatistics(columnStatistics);
+ }
+
+ @Override
+ public void validatePartitionNameCharacters(List part_vals) throws TException, MetaException {
+ try {
+ String partitionValidationRegex = conf.getVar(ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN);
+ Pattern partitionValidationPattern = Strings.isNullOrEmpty(partitionValidationRegex) ? null
+ : Pattern.compile(partitionValidationRegex);
+ MetaStoreUtils.validatePartitionNameCharacters(part_vals, partitionValidationPattern);
+ } catch (Exception e){
+ if (e instanceof MetaException) {
+ throw (MetaException) e;
+ } else {
+ throw new MetaException(e.getMessage());
+ }
+ }
+ }
+
+ private Path constructRenamedPath(Path defaultNewPath, Path currentPath) {
+ URI currentUri = currentPath.toUri();
+
+ return new Path(currentUri.getScheme(), currentUri.getAuthority(),
+ defaultNewPath.toUri().getPath());
+ }
+
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java
new file mode 100644
index 000000000..e135ecfea
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java
@@ -0,0 +1,10 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+
+public interface AWSCredentialsProviderFactory {
+
+ AWSCredentialsProvider buildAWSCredentialsProvider(HiveConf hiveConf);
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java
new file mode 100644
index 000000000..9b4d65c13
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java
@@ -0,0 +1,100 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.AWSGlueClientBuilder;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_CONNECTION_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_ENDPOINT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_MAX_CONNECTIONS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_MAX_RETRY;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_SOCKET_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_REGION;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_CONNECTION_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_MAX_CONNECTIONS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_MAX_RETRY;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_SOCKET_TIMEOUT;
+
+public final class AWSGlueClientFactory implements GlueClientFactory {
+
+ private static final Logger logger = Logger.getLogger(AWSGlueClientFactory.class);
+
+ private final HiveConf conf;
+
+ public AWSGlueClientFactory(HiveConf conf) {
+ Preconditions.checkNotNull(conf, "HiveConf cannot be null");
+ this.conf = conf;
+ }
+
+ @Override
+ public AWSGlue newClient() throws MetaException {
+ try {
+ AWSGlueClientBuilder glueClientBuilder = AWSGlueClientBuilder.standard()
+ .withCredentials(getAWSCredentialsProvider(conf));
+
+ String regionStr = getProperty(AWS_REGION, conf);
+ String glueEndpoint = getProperty(AWS_GLUE_ENDPOINT, conf);
+
+ // ClientBuilder only allows one of EndpointConfiguration or Region to be set
+ if (StringUtils.isNotBlank(glueEndpoint)) {
+ logger.info("Setting glue service endpoint to " + glueEndpoint);
+ glueClientBuilder.setEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(glueEndpoint, null));
+ } else if (StringUtils.isNotBlank(regionStr)) {
+ logger.info("Setting region to : " + regionStr);
+ glueClientBuilder.setRegion(regionStr);
+ } else {
+ Region currentRegion = Regions.getCurrentRegion();
+ if (currentRegion != null) {
+ logger.info("Using region from ec2 metadata : " + currentRegion.getName());
+ glueClientBuilder.setRegion(currentRegion.getName());
+ } else {
+ logger.info("No region info found, using SDK default region: us-east-1");
+ }
+ }
+
+ glueClientBuilder.setClientConfiguration(buildClientConfiguration(conf));
+ return glueClientBuilder.build();
+ } catch (Exception e) {
+ String message = "Unable to build AWSGlueClient: " + e;
+ logger.error(message);
+ throw new MetaException(message);
+ }
+ }
+
+ private AWSCredentialsProvider getAWSCredentialsProvider(HiveConf conf) {
+ Class extends AWSCredentialsProviderFactory> providerFactoryClass = conf
+ .getClass(AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS,
+ DefaultAWSCredentialsProviderFactory.class).asSubclass(
+ AWSCredentialsProviderFactory.class);
+ AWSCredentialsProviderFactory provider = ReflectionUtils.newInstance(
+ providerFactoryClass, conf);
+ return provider.buildAWSCredentialsProvider(conf);
+ }
+
+ private ClientConfiguration buildClientConfiguration(HiveConf hiveConf) {
+ ClientConfiguration clientConfiguration = new ClientConfiguration()
+ .withMaxErrorRetry(hiveConf.getInt(AWS_GLUE_MAX_RETRY, DEFAULT_MAX_RETRY))
+ .withMaxConnections(hiveConf.getInt(AWS_GLUE_MAX_CONNECTIONS, DEFAULT_MAX_CONNECTIONS))
+ .withConnectionTimeout(hiveConf.getInt(AWS_GLUE_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT))
+ .withSocketTimeout(hiveConf.getInt(AWS_GLUE_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT));
+ return clientConfiguration;
+ }
+
+ private static String getProperty(String propertyName, HiveConf conf) {
+ return Strings.isNullOrEmpty(System.getProperty(propertyName)) ?
+ conf.get(propertyName) : System.getProperty(propertyName);
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDataCatalogHiveClientFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDataCatalogHiveClientFactory.java
new file mode 100644
index 000000000..3a455afa9
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDataCatalogHiveClientFactory.java
@@ -0,0 +1,36 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+public class AWSGlueDataCatalogHiveClientFactory implements HiveMetastoreClientFactory
+{
+
+ private HiveConf conf;
+ private HiveMetaHookLoader hookLoader;
+
+ public AWSGlueDataCatalogHiveClientFactory(HiveConf conf, HiveMetaHookLoader hookLoader)
+ {
+ this.conf = conf;
+ this.hookLoader = hookLoader;
+ }
+
+ @Override
+ public IMetaStoreClient getHiveMetastoreClient()
+ throws HiveAuthzPluginException
+ {
+ AWSCatalogMetastoreClient client = null;
+ try {
+ return new AWSCatalogMetastoreClient(conf, hookLoader);
+ }
+ catch (MetaException e) {
+ throw new HiveAuthzPluginException(e);
+ }
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java
new file mode 100644
index 000000000..4a7e59555
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java
@@ -0,0 +1,74 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import org.apache.thrift.TException;
+
+import java.util.List;
+
+/**
+ * This is the accessor interface for using AWS Glue as a metastore.
+ * The generic AWSGlue interface{@link com.amazonaws.services.glue.AWSGlue}
+ * has a number of methods that are irrelevant for clients using Glue only
+ * as a metastore.
+ * Think of this interface as a wrapper over AWSGlue. This additional layer
+ * of abstraction achieves the following -
+ * a) Hides the non-metastore related operations present in AWSGlue
+ * b) Hides away the batching and pagination related limitations of AWSGlue
+ */
+public interface AWSGlueMetastore {
+
+ void createDatabase(DatabaseInput databaseInput);
+
+ Database getDatabase(String dbName);
+
+ List getAllDatabases();
+
+ void updateDatabase(String databaseName, DatabaseInput databaseInput);
+
+ void deleteDatabase(String dbName);
+
+ void createTable(String dbName, TableInput tableInput);
+
+ Table getTable(String dbName, String tableName);
+
+ List getTables(String dbname, String tablePattern);
+
+ void updateTable(String dbName, TableInput tableInput);
+
+ void deleteTable(String dbName, String tableName);
+
+ Partition getPartition(String dbName, String tableName, List partitionValues);
+
+ List getPartitionsByNames(String dbName, String tableName,
+ List partitionsToGet);
+
+ List getPartitions(String dbName, String tableName, String expression,
+ long max) throws TException;
+
+ void updatePartition(String dbName, String tableName, List partitionValues,
+ PartitionInput partitionInput);
+
+ void deletePartition(String dbName, String tableName, List partitionValues);
+
+ List createPartitions(String dbName, String tableName,
+ List partitionInputs);
+
+ void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput);
+
+ UserDefinedFunction getUserDefinedFunction(String dbName, String functionName);
+
+ List getUserDefinedFunctions(String dbName, String pattern);
+
+ void deleteUserDefinedFunction(String dbName, String functionName);
+
+ void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput);
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java
new file mode 100644
index 000000000..623a05e31
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java
@@ -0,0 +1,133 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import org.apache.thrift.TException;
+
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AWSGlueMetastoreBaseDecorator implements AWSGlueMetastore {
+
+ private final AWSGlueMetastore awsGlueMetastore;
+
+ public AWSGlueMetastoreBaseDecorator(AWSGlueMetastore awsGlueMetastore) {
+ checkNotNull(awsGlueMetastore, "awsGlueMetastore can not be null");
+ this.awsGlueMetastore = awsGlueMetastore;
+ }
+
+ @Override
+ public void createDatabase(DatabaseInput databaseInput) {
+ awsGlueMetastore.createDatabase(databaseInput);
+ }
+
+ @Override
+ public Database getDatabase(String dbName) {
+ return awsGlueMetastore.getDatabase(dbName);
+ }
+
+ @Override
+ public List getAllDatabases() {
+ return awsGlueMetastore.getAllDatabases();
+ }
+
+ @Override
+ public void updateDatabase(String databaseName, DatabaseInput databaseInput) {
+ awsGlueMetastore.updateDatabase(databaseName, databaseInput);
+ }
+
+ @Override
+ public void deleteDatabase(String dbName) {
+ awsGlueMetastore.deleteDatabase(dbName);
+ }
+
+ @Override
+ public void createTable(String dbName, TableInput tableInput) {
+ awsGlueMetastore.createTable(dbName, tableInput);
+ }
+
+ @Override
+ public Table getTable(String dbName, String tableName) {
+ return awsGlueMetastore.getTable(dbName, tableName);
+ }
+
+ @Override
+ public List getTables(String dbname, String tablePattern) {
+ return awsGlueMetastore.getTables(dbname, tablePattern);
+ }
+
+ @Override
+ public void updateTable(String dbName, TableInput tableInput) {
+ awsGlueMetastore.updateTable(dbName, tableInput);
+ }
+
+ @Override
+ public void deleteTable(String dbName, String tableName) {
+ awsGlueMetastore.deleteTable(dbName, tableName);
+ }
+
+ @Override
+ public Partition getPartition(String dbName, String tableName, List partitionValues) {
+ return awsGlueMetastore.getPartition(dbName, tableName, partitionValues);
+ }
+
+ @Override
+ public List getPartitionsByNames(String dbName, String tableName, List partitionsToGet) {
+ return awsGlueMetastore.getPartitionsByNames(dbName, tableName, partitionsToGet);
+ }
+
+ @Override
+ public List getPartitions(String dbName, String tableName, String expression, long max) throws TException {
+ return awsGlueMetastore.getPartitions(dbName, tableName, expression, max);
+ }
+
+ @Override
+ public void updatePartition(String dbName, String tableName, List partitionValues, PartitionInput partitionInput) {
+ awsGlueMetastore.updatePartition(dbName, tableName, partitionValues, partitionInput);
+ }
+
+ @Override
+ public void deletePartition(String dbName, String tableName, List partitionValues) {
+ awsGlueMetastore.deletePartition(dbName, tableName, partitionValues);
+ }
+
+ @Override
+ public List createPartitions(String dbName, String tableName, List partitionInputs) {
+ return awsGlueMetastore.createPartitions(dbName, tableName, partitionInputs);
+ }
+
+ @Override
+ public void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput) {
+ awsGlueMetastore.createUserDefinedFunction(dbName, functionInput);
+ }
+
+ @Override
+ public UserDefinedFunction getUserDefinedFunction(String dbName, String functionName) {
+ return awsGlueMetastore.getUserDefinedFunction(dbName, functionName);
+ }
+
+ @Override
+ public List getUserDefinedFunctions(String dbName, String pattern) {
+ return awsGlueMetastore.getUserDefinedFunctions(dbName, pattern);
+ }
+
+ @Override
+ public void deleteUserDefinedFunction(String dbName, String functionName) {
+ awsGlueMetastore.deleteUserDefinedFunction(dbName, functionName);
+ }
+
+ @Override
+ public void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput) {
+ awsGlueMetastore.updateUserDefinedFunction(dbName, functionName, functionInput);
+ }
+
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java
new file mode 100644
index 000000000..02072f962
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java
@@ -0,0 +1,208 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.log4j.Logger;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_SIZE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_TTL_MINS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_SIZE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_TTL_MINS;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AWSGlueMetastoreCacheDecorator extends AWSGlueMetastoreBaseDecorator {
+
+ private static final Logger logger = Logger.getLogger(AWSGlueMetastoreCacheDecorator.class);
+
+ private final HiveConf conf;
+
+ private final boolean databaseCacheEnabled;
+
+ private final boolean tableCacheEnabled;
+
+ @VisibleForTesting
+ protected Cache databaseCache;
+ @VisibleForTesting
+ protected Cache tableCache;
+
+ public AWSGlueMetastoreCacheDecorator(HiveConf conf, AWSGlueMetastore awsGlueMetastore) {
+ super(awsGlueMetastore);
+
+ checkNotNull(conf, "conf can not be null");
+ this.conf = conf;
+
+ databaseCacheEnabled = conf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false);
+ if(databaseCacheEnabled) {
+ int dbCacheSize = conf.getInt(AWS_GLUE_DB_CACHE_SIZE, 0);
+ int dbCacheTtlMins = conf.getInt(AWS_GLUE_DB_CACHE_TTL_MINS, 0);
+
+ //validate config values for size and ttl
+ validateConfigValueIsGreaterThanZero(AWS_GLUE_DB_CACHE_SIZE, dbCacheSize);
+ validateConfigValueIsGreaterThanZero(AWS_GLUE_DB_CACHE_TTL_MINS, dbCacheTtlMins);
+
+ //initialize database cache
+ databaseCache = CacheBuilder.newBuilder().maximumSize(dbCacheSize)
+ .expireAfterWrite(dbCacheTtlMins, TimeUnit.MINUTES).build();
+ } else {
+ databaseCache = null;
+ }
+
+ tableCacheEnabled = conf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false);
+ if(tableCacheEnabled) {
+ int tableCacheSize = conf.getInt(AWS_GLUE_TABLE_CACHE_SIZE, 0);
+ int tableCacheTtlMins = conf.getInt(AWS_GLUE_TABLE_CACHE_TTL_MINS, 0);
+
+ //validate config values for size and ttl
+ validateConfigValueIsGreaterThanZero(AWS_GLUE_TABLE_CACHE_SIZE, tableCacheSize);
+ validateConfigValueIsGreaterThanZero(AWS_GLUE_TABLE_CACHE_TTL_MINS, tableCacheTtlMins);
+
+ //initialize table cache
+ tableCache = CacheBuilder.newBuilder().maximumSize(tableCacheSize)
+ .expireAfterWrite(tableCacheTtlMins, TimeUnit.MINUTES).build();
+ } else {
+ tableCache = null;
+ }
+
+ logger.info("Constructed");
+ }
+
+ private void validateConfigValueIsGreaterThanZero(String configName, int value) {
+ checkArgument(value > 0, String.format("Invalid value for Hive Config %s. " +
+ "Provide a value greater than zero", configName));
+
+ }
+
+ @Override
+ public Database getDatabase(String dbName) {
+ Database result;
+ if(databaseCacheEnabled) {
+ Database valueFromCache = databaseCache.getIfPresent(dbName);
+ if(valueFromCache != null) {
+ logger.info("Cache hit for operation [getDatabase] on key [" + dbName + "]");
+ result = valueFromCache;
+ } else {
+ logger.info("Cache miss for operation [getDatabase] on key [" + dbName + "]");
+ result = super.getDatabase(dbName);
+ databaseCache.put(dbName, result);
+ }
+ } else {
+ result = super.getDatabase(dbName);
+ }
+ return result;
+ }
+
+ @Override
+ public void updateDatabase(String dbName, DatabaseInput databaseInput) {
+ super.updateDatabase(dbName, databaseInput);
+ if(databaseCacheEnabled) {
+ purgeDatabaseFromCache(dbName);
+ }
+ }
+
+ @Override
+ public void deleteDatabase(String dbName) {
+ super.deleteDatabase(dbName);
+ if(databaseCacheEnabled) {
+ purgeDatabaseFromCache(dbName);
+ }
+ }
+
+ private void purgeDatabaseFromCache(String dbName) {
+ databaseCache.invalidate(dbName);
+ }
+
+ @Override
+ public Table getTable(String dbName, String tableName) {
+ Table result;
+ if(tableCacheEnabled) {
+ TableIdentifier key = new TableIdentifier(dbName, tableName);
+ Table valueFromCache = tableCache.getIfPresent(key);
+ if(valueFromCache != null) {
+ logger.info("Cache hit for operation [getTable] on key [" + key + "]");
+ result = valueFromCache;
+ } else {
+ logger.info("Cache miss for operation [getTable] on key [" + key + "]");
+ result = super.getTable(dbName, tableName);
+ tableCache.put(key, result);
+ }
+ } else {
+ result = super.getTable(dbName, tableName);
+ }
+ return result;
+ }
+
+ @Override
+ public void updateTable(String dbName, TableInput tableInput) {
+ super.updateTable(dbName, tableInput);
+ if(tableCacheEnabled) {
+ purgeTableFromCache(dbName, tableInput.getName());
+ }
+ }
+
+ @Override
+ public void deleteTable(String dbName, String tableName) {
+ super.deleteTable(dbName, tableName);
+ if(tableCacheEnabled) {
+ purgeTableFromCache(dbName, tableName);
+ }
+ }
+
+ private void purgeTableFromCache(String dbName, String tableName) {
+ TableIdentifier key = new TableIdentifier(dbName, tableName);
+ tableCache.invalidate(key);
+ }
+
+
+ static class TableIdentifier {
+ private final String dbName;
+ private final String tableName;
+
+ public TableIdentifier(String dbName, String tableName) {
+ this.dbName = dbName;
+ this.tableName = tableName;
+ }
+
+ public String getDbName() {
+ return dbName;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ @Override
+ public String toString() {
+ return "TableIdentifier{" +
+ "dbName='" + dbName + '\'' +
+ ", tableName='" + tableName + '\'' +
+ '}';
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ TableIdentifier that = (TableIdentifier) o;
+ return Objects.equals(dbName, that.dbName) &&
+ Objects.equals(tableName, that.tableName);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(dbName, tableName);
+ }
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java
new file mode 100644
index 000000000..26d030432
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java
@@ -0,0 +1,26 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.AWSGlue;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_ENABLE;
+
+public class AWSGlueMetastoreFactory {
+
+ public AWSGlueMetastore newMetastore(HiveConf conf) throws MetaException {
+ AWSGlue glueClient = new AWSGlueClientFactory(conf).newClient();
+ AWSGlueMetastore defaultMetastore = new DefaultAWSGlueMetastore(conf, glueClient);
+ if(isCacheEnabled(conf)) {
+ return new AWSGlueMetastoreCacheDecorator(conf, defaultMetastore);
+ }
+ return defaultMetastore;
+ }
+
+ private boolean isCacheEnabled(HiveConf conf) {
+ boolean databaseCacheEnabled = conf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false);
+ boolean tableCacheEnabled = conf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false);
+ return (databaseCacheEnabled || tableCacheEnabled);
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java
new file mode 100644
index 000000000..4b81872b9
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java
@@ -0,0 +1,16 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
+
+public class DefaultAWSCredentialsProviderFactory implements
+ AWSCredentialsProviderFactory {
+
+ @Override
+ public AWSCredentialsProvider buildAWSCredentialsProvider(HiveConf hiveConf) {
+ return new DefaultAWSCredentialsProviderChain();
+ }
+
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java
new file mode 100644
index 000000000..9470784b8
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java
@@ -0,0 +1,407 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.BatchCreatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionResult;
+import com.amazonaws.services.glue.model.CreateDatabaseRequest;
+import com.amazonaws.services.glue.model.CreateTableRequest;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.DeleteDatabaseRequest;
+import com.amazonaws.services.glue.model.DeletePartitionRequest;
+import com.amazonaws.services.glue.model.DeleteTableRequest;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetDatabaseResult;
+import com.amazonaws.services.glue.model.GetDatabasesRequest;
+import com.amazonaws.services.glue.model.GetDatabasesResult;
+import com.amazonaws.services.glue.model.GetPartitionRequest;
+import com.amazonaws.services.glue.model.GetPartitionsRequest;
+import com.amazonaws.services.glue.model.GetPartitionsResult;
+import com.amazonaws.services.glue.model.GetTableRequest;
+import com.amazonaws.services.glue.model.GetTableResult;
+import com.amazonaws.services.glue.model.GetTablesRequest;
+import com.amazonaws.services.glue.model.GetTablesResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsResult;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Segment;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UpdateDatabaseRequest;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.amazonaws.services.glue.model.UpdateTableRequest;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.thrift.TException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class DefaultAWSGlueMetastore implements AWSGlueMetastore {
+
+ public static final int BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE = 1000;
+ /**
+ * Based on the maxResults parameter at https://docs.aws.amazon.com/glue/latest/webapi/API_GetPartitions.html
+ */
+ public static final int GET_PARTITIONS_MAX_SIZE = 1000;
+ /**
+ * Maximum number of Glue Segments. A segment defines a non-overlapping region of a table's partitions,
+ * allowing multiple requests to be executed in parallel.
+ */
+ public static final int DEFAULT_NUM_PARTITION_SEGMENTS = 5;
+ /**
+ * Currently the upper limit allowed by Glue is 10.
+ * https://docs.aws.amazon.com/glue/latest/webapi/API_Segment.html
+ */
+ public static final int MAX_NUM_PARTITION_SEGMENTS = 10;
+ public static final String NUM_PARTITION_SEGMENTS_CONF = "aws.glue.partition.num.segments";
+ public static final String CUSTOM_EXECUTOR_FACTORY_CONF = "hive.metastore.executorservice.factory.class";
+
+ private final HiveConf conf;
+ private final AWSGlue glueClient;
+ private final String catalogId;
+ private final ExecutorService executorService;
+ private final int numPartitionSegments;
+
+ protected ExecutorService getExecutorService(HiveConf hiveConf) {
+ Class extends ExecutorServiceFactory> executorFactoryClass = hiveConf
+ .getClass(CUSTOM_EXECUTOR_FACTORY_CONF,
+ DefaultExecutorServiceFactory.class).asSubclass(
+ ExecutorServiceFactory.class);
+ ExecutorServiceFactory factory = ReflectionUtils.newInstance(
+ executorFactoryClass, hiveConf);
+ return factory.getExecutorService(hiveConf);
+ }
+
+ public DefaultAWSGlueMetastore(HiveConf conf, AWSGlue glueClient) {
+ checkNotNull(conf, "Hive Config cannot be null");
+ checkNotNull(glueClient, "glueClient cannot be null");
+ this.numPartitionSegments = conf.getInt(NUM_PARTITION_SEGMENTS_CONF, DEFAULT_NUM_PARTITION_SEGMENTS);
+ checkArgument(numPartitionSegments <= MAX_NUM_PARTITION_SEGMENTS,
+ String.format("Hive Config [%s] can't exceed %d", NUM_PARTITION_SEGMENTS_CONF, MAX_NUM_PARTITION_SEGMENTS));
+ this.conf = conf;
+ this.glueClient = glueClient;
+ this.catalogId = MetastoreClientUtils.getCatalogId(conf);
+ this.executorService = getExecutorService(conf);
+ }
+
+ // ======================= Database =======================
+
+ @Override
+ public void createDatabase(DatabaseInput databaseInput) {
+ CreateDatabaseRequest createDatabaseRequest = new CreateDatabaseRequest().withDatabaseInput(databaseInput)
+ .withCatalogId(catalogId);
+ glueClient.createDatabase(createDatabaseRequest);
+ }
+
+ @Override
+ public Database getDatabase(String dbName) {
+ GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest().withCatalogId(catalogId).withName(dbName);
+ GetDatabaseResult result = glueClient.getDatabase(getDatabaseRequest);
+ return result.getDatabase();
+ }
+
+ @Override
+ public List getAllDatabases() {
+ List ret = Lists.newArrayList();
+ String nextToken = null;
+ do {
+ GetDatabasesRequest getDatabasesRequest = new GetDatabasesRequest().withNextToken(nextToken).withCatalogId(
+ catalogId);
+ GetDatabasesResult result = glueClient.getDatabases(getDatabasesRequest);
+ nextToken = result.getNextToken();
+ ret.addAll(result.getDatabaseList());
+ } while (nextToken != null);
+ return ret;
+ }
+
+ @Override
+ public void updateDatabase(String databaseName, DatabaseInput databaseInput) {
+ UpdateDatabaseRequest updateDatabaseRequest = new UpdateDatabaseRequest().withName(databaseName)
+ .withDatabaseInput(databaseInput).withCatalogId(catalogId);
+ glueClient.updateDatabase(updateDatabaseRequest);
+ }
+
+ @Override
+ public void deleteDatabase(String dbName) {
+ DeleteDatabaseRequest deleteDatabaseRequest = new DeleteDatabaseRequest().withName(dbName).withCatalogId(
+ catalogId);
+ glueClient.deleteDatabase(deleteDatabaseRequest);
+ }
+
+ // ======================== Table ========================
+
+ @Override
+ public void createTable(String dbName, TableInput tableInput) {
+ CreateTableRequest createTableRequest = new CreateTableRequest().withTableInput(tableInput)
+ .withDatabaseName(dbName).withCatalogId(catalogId);
+ glueClient.createTable(createTableRequest);
+ }
+
+ @Override
+ public Table getTable(String dbName, String tableName) {
+ GetTableRequest getTableRequest = new GetTableRequest().withDatabaseName(dbName).withName(tableName)
+ .withCatalogId(catalogId);
+ GetTableResult result = glueClient.getTable(getTableRequest);
+ return result.getTable();
+ }
+
+ @Override
+ public List getTables(String dbname, String tablePattern) {
+ List ret = new ArrayList<>();
+ String nextToken = null;
+ do {
+ GetTablesRequest getTablesRequest = new GetTablesRequest().withDatabaseName(dbname)
+ .withExpression(tablePattern).withNextToken(nextToken).withCatalogId(catalogId);
+ GetTablesResult result = glueClient.getTables(getTablesRequest);
+ ret.addAll(result.getTableList());
+ nextToken = result.getNextToken();
+ } while (nextToken != null);
+ return ret;
+ }
+
+ @Override
+ public void updateTable(String dbName, TableInput tableInput) {
+ UpdateTableRequest updateTableRequest = new UpdateTableRequest().withDatabaseName(dbName)
+ .withTableInput(tableInput).withCatalogId(catalogId);
+ glueClient.updateTable(updateTableRequest);
+ }
+
+ @Override
+ public void deleteTable(String dbName, String tableName) {
+ DeleteTableRequest deleteTableRequest = new DeleteTableRequest().withDatabaseName(dbName).withName(tableName)
+ .withCatalogId(catalogId);
+ glueClient.deleteTable(deleteTableRequest);
+ }
+
+ // =========================== Partition ===========================
+
+ @Override
+ public Partition getPartition(String dbName, String tableName, List partitionValues) {
+ GetPartitionRequest request = new GetPartitionRequest()
+ .withDatabaseName(dbName)
+ .withTableName(tableName)
+ .withPartitionValues(partitionValues)
+ .withCatalogId(catalogId);
+ return glueClient.getPartition(request).getPartition();
+ }
+
+ @Override
+ public List getPartitionsByNames(String dbName, String tableName,
+ List partitionsToGet) {
+
+ List> batchedPartitionsToGet = Lists.partition(partitionsToGet,
+ BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE);
+ List> batchGetPartitionFutures = Lists.newArrayList();
+
+ for (List batch : batchedPartitionsToGet) {
+ final BatchGetPartitionRequest request = new BatchGetPartitionRequest()
+ .withDatabaseName(dbName)
+ .withTableName(tableName)
+ .withPartitionsToGet(batch)
+ .withCatalogId(catalogId);
+ batchGetPartitionFutures.add(this.executorService.submit(new Callable() {
+ @Override
+ public BatchGetPartitionResult call() throws Exception {
+ return glueClient.batchGetPartition(request);
+ }
+ }));
+ }
+
+ List result = Lists.newArrayList();
+ try {
+ for (Future future : batchGetPartitionFutures) {
+ result.addAll(future.get().getPartitions());
+ }
+ } catch (ExecutionException e) {
+ Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+ Throwables.propagate(e.getCause());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ return result;
+ }
+
+ @Override
+ public List getPartitions(String dbName, String tableName, String expression,
+ long max) throws TException {
+ if (max == 0) {
+ return Collections.emptyList();
+ }
+ if (max < 0 || max > GET_PARTITIONS_MAX_SIZE) {
+ return getPartitionsParallel(dbName, tableName, expression, max);
+ } else {
+ // We don't need to get too many partitions, so just do it serially.
+ return getCatalogPartitions(dbName, tableName, expression, max, null);
+ }
+ }
+
+ private List getPartitionsParallel(
+ final String databaseName,
+ final String tableName,
+ final String expression,
+ final long max) throws TException {
+ // Prepare the segments
+ List segments = Lists.newArrayList();
+ for (int i = 0; i < numPartitionSegments; i++) {
+ segments.add(new Segment()
+ .withSegmentNumber(i)
+ .withTotalSegments(numPartitionSegments));
+ }
+ // Submit Glue API calls in parallel using the thread pool.
+ // We could convert this into a parallelStream after upgrading to JDK 8 compiler base.
+ List>> futures = Lists.newArrayList();
+ for (final Segment segment : segments) {
+ futures.add(this.executorService.submit(new Callable>() {
+ @Override
+ public List call() throws Exception {
+ return getCatalogPartitions(databaseName, tableName, expression, max, segment);
+ }
+ }));
+ }
+
+ // Get the results
+ List partitions = Lists.newArrayList();
+ try {
+ for (Future> future : futures) {
+ List segmentPartitions = future.get();
+ if (partitions.size() + segmentPartitions.size() >= max && max > 0) {
+ // Extract the required number of partitions from the segment and we're done.
+ long remaining = max - partitions.size();
+ partitions.addAll(segmentPartitions.subList(0, (int) remaining));
+ break;
+ } else {
+ partitions.addAll(segmentPartitions);
+ }
+ }
+ } catch (ExecutionException e) {
+ Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+ Throwables.propagate(e.getCause());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ return partitions;
+ }
+
+
+ private List getCatalogPartitions(String databaseName, String tableName, String expression,
+ long max, Segment segment) {
+ List partitions = Lists.newArrayList();
+ String nextToken = null;
+ do {
+ GetPartitionsRequest request = new GetPartitionsRequest()
+ .withDatabaseName(databaseName)
+ .withTableName(tableName)
+ .withExpression(expression)
+ .withNextToken(nextToken)
+ .withCatalogId(catalogId)
+ .withSegment(segment);
+ GetPartitionsResult res = glueClient.getPartitions(request);
+ List list = res.getPartitions();
+ if ((partitions.size() + list.size()) >= max && max > 0) {
+ long remaining = max - partitions.size();
+ partitions.addAll(list.subList(0, (int) remaining));
+ break;
+ }
+ partitions.addAll(list);
+ nextToken = res.getNextToken();
+ } while (nextToken != null);
+ return partitions;
+ }
+
+ @Override
+ public void updatePartition(String dbName, String tableName, List partitionValues,
+ PartitionInput partitionInput) {
+ UpdatePartitionRequest updatePartitionRequest = new UpdatePartitionRequest().withDatabaseName(dbName)
+ .withTableName(tableName).withPartitionValueList(partitionValues)
+ .withPartitionInput(partitionInput).withCatalogId(catalogId);
+ glueClient.updatePartition(updatePartitionRequest);
+ }
+
+ @Override
+ public void deletePartition(String dbName, String tableName, List partitionValues) {
+ DeletePartitionRequest request = new DeletePartitionRequest()
+ .withDatabaseName(dbName)
+ .withTableName(tableName)
+ .withPartitionValues(partitionValues)
+ .withCatalogId(catalogId);
+ glueClient.deletePartition(request);
+ }
+
+ @Override
+ public List createPartitions(String dbName, String tableName,
+ List partitionInputs) {
+ BatchCreatePartitionRequest request =
+ new BatchCreatePartitionRequest().withDatabaseName(dbName)
+ .withTableName(tableName).withCatalogId(catalogId)
+ .withPartitionInputList(partitionInputs);
+ return glueClient.batchCreatePartition(request).getErrors();
+ }
+
+ // ====================== User Defined Function ======================
+
+ @Override
+ public void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput) {
+ CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest = new CreateUserDefinedFunctionRequest()
+ .withDatabaseName(dbName).withFunctionInput(functionInput).withCatalogId(catalogId);
+ glueClient.createUserDefinedFunction(createUserDefinedFunctionRequest);
+ }
+
+ @Override
+ public UserDefinedFunction getUserDefinedFunction(String dbName, String functionName) {
+ GetUserDefinedFunctionRequest getUserDefinedFunctionRequest = new GetUserDefinedFunctionRequest()
+ .withDatabaseName(dbName).withFunctionName(functionName).withCatalogId(catalogId);
+ return glueClient.getUserDefinedFunction(getUserDefinedFunctionRequest).getUserDefinedFunction();
+ }
+
+ @Override
+ public List getUserDefinedFunctions(String dbName, String pattern) {
+ List ret = Lists.newArrayList();
+ String nextToken = null;
+ do {
+ GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest = new GetUserDefinedFunctionsRequest()
+ .withDatabaseName(dbName).withPattern(pattern).withNextToken(nextToken).withCatalogId(catalogId);
+ GetUserDefinedFunctionsResult result = glueClient.getUserDefinedFunctions(getUserDefinedFunctionsRequest);
+ nextToken = result.getNextToken();
+ ret.addAll(result.getUserDefinedFunctions());
+ } while (nextToken != null);
+ return ret;
+ }
+
+ @Override
+ public void deleteUserDefinedFunction(String dbName, String functionName) {
+ DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest = new DeleteUserDefinedFunctionRequest()
+ .withDatabaseName(dbName).withFunctionName(functionName).withCatalogId(catalogId);
+ glueClient.deleteUserDefinedFunction(deleteUserDefinedFunctionRequest);
+ }
+
+ @Override
+ public void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput) {
+ UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest = new UpdateUserDefinedFunctionRequest()
+ .withDatabaseName(dbName).withFunctionName(functionName).withFunctionInput(functionInput)
+ .withCatalogId(catalogId);
+ glueClient.updateUserDefinedFunction(updateUserDefinedFunctionRequest);
+ }
+}
\ No newline at end of file
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java
new file mode 100644
index 000000000..c16433236
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java
@@ -0,0 +1,22 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class DefaultExecutorServiceFactory implements ExecutorServiceFactory {
+ private static final int NUM_EXECUTOR_THREADS = 5;
+
+ private static final ExecutorService GLUE_METASTORE_DELEGATE_THREAD_POOL = Executors.newFixedThreadPool(
+ NUM_EXECUTOR_THREADS, new ThreadFactoryBuilder()
+ .setNameFormat(GlueMetastoreClientDelegate.GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT)
+ .setDaemon(true).build()
+ );
+
+ @Override
+ public ExecutorService getExecutorService(HiveConf conf) {
+ return GLUE_METASTORE_DELEGATE_THREAD_POOL;
+ }
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java
new file mode 100644
index 000000000..6168a01f6
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java
@@ -0,0 +1,12 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+import java.util.concurrent.ExecutorService;
+
+/*
+ * Interface for creating an ExecutorService
+ */
+public interface ExecutorServiceFactory {
+ public ExecutorService getExecutorService(HiveConf conf);
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java
new file mode 100644
index 000000000..025812e48
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java
@@ -0,0 +1,13 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.AWSGlue;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+/***
+ * Interface for creating Glue AWS Client
+ */
+public interface GlueClientFactory {
+
+ AWSGlue newClient() throws MetaException;
+
+}
diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java
new file mode 100644
index 000000000..0d3b5440a
--- /dev/null
+++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java
@@ -0,0 +1,1659 @@
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter;
+import com.amazonaws.glue.catalog.converters.GlueInputConverter;
+import com.amazonaws.glue.catalog.converters.HiveToCatalogConverter;
+import com.amazonaws.glue.catalog.util.BatchCreatePartitionsHelper;
+import com.amazonaws.glue.catalog.util.ExpressionHelper;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import com.amazonaws.glue.catalog.util.PartitionKey;
+import com.amazonaws.glue.shims.AwsGlueHiveShims;
+import com.amazonaws.glue.shims.ShimsLoader;
+import com.amazonaws.services.glue.model.Column;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.EntityNotFoundException;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.Index;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+
+import static com.amazonaws.glue.catalog.converters.ConverterUtils.stringToCatalogTable;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.deepCopyMap;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.isExternalTable;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.makeDirs;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.validateGlueTable;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.validateTableObject;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.hadoop.hive.metastore.HiveMetaStore.PUBLIC;
+import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE;
+import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE;
+
+/***
+ * Delegate Class to provide all common functionality
+ * between Spark-hive version, Hive and Presto clients
+ *
+ */
+public class GlueMetastoreClientDelegate {
+
+ private static final Logger logger = Logger.getLogger(GlueMetastoreClientDelegate.class);
+
+ private static final List implicitRoles = Lists.newArrayList(new Role(PUBLIC, 0, PUBLIC));
+ public static final int MILLISECOND_TO_SECOND_FACTOR = 1000;
+ public static final Long NO_MAX = -1L;
+ public static final String MATCH_ALL = ".*";
+
+ public static final String INDEX_PREFIX = "index_prefix";
+
+ private static final int BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE = 100;
+
+ public static final String CUSTOM_EXECUTOR_FACTORY_CONF = "hive.metastore.executorservice.factory.class";
+
+ static final String GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT = "glue-metastore-delegate-%d";
+
+ private final ExecutorService executorService;
+ private final AWSGlueMetastore glueMetastore;
+ private final HiveConf conf;
+ private final Warehouse wh;
+ private final AwsGlueHiveShims hiveShims = ShimsLoader.getHiveShims();
+ private final String catalogId;
+
+ public static final String CATALOG_ID_CONF = "hive.metastore.glue.catalogid";
+ public static final String NUM_PARTITION_SEGMENTS_CONF = "aws.glue.partition.num.segments";
+
+ protected ExecutorService getExecutorService() {
+ Class extends ExecutorServiceFactory> executorFactoryClass = this.conf
+ .getClass(CUSTOM_EXECUTOR_FACTORY_CONF,
+ DefaultExecutorServiceFactory.class).asSubclass(
+ ExecutorServiceFactory.class);
+ ExecutorServiceFactory factory = ReflectionUtils.newInstance(
+ executorFactoryClass, conf);
+ return factory.getExecutorService(conf);
+ }
+
+ public GlueMetastoreClientDelegate(HiveConf conf, AWSGlueMetastore glueMetastore,
+ Warehouse wh) throws MetaException {
+ checkNotNull(conf, "Hive Config cannot be null");
+ checkNotNull(glueMetastore, "glueMetastore cannot be null");
+ checkNotNull(wh, "Warehouse cannot be null");
+
+ this.conf = conf;
+ this.glueMetastore = glueMetastore;
+ this.wh = wh;
+ this.executorService = getExecutorService();
+
+ // TODO - May be validate catalogId confirms to AWS AccountId too.
+ catalogId = MetastoreClientUtils.getCatalogId(conf);
+ }
+
+ // ======================= Database =======================
+
+ public void createDatabase(org.apache.hadoop.hive.metastore.api.Database database) throws TException {
+ checkNotNull(database, "database cannot be null");
+
+ if (StringUtils.isEmpty(database.getLocationUri())) {
+ database.setLocationUri(wh.getDefaultDatabasePath(database.getName()).toString());
+ } else {
+ database.setLocationUri(wh.getDnsPath(new Path(database.getLocationUri())).toString());
+ }
+ Path dbPath = new Path(database.getLocationUri());
+ boolean madeDir = makeDirs(wh, dbPath);
+
+ try {
+ DatabaseInput catalogDatabase = GlueInputConverter.convertToDatabaseInput(database);
+ glueMetastore.createDatabase(catalogDatabase);
+ } catch (AmazonServiceException e) {
+ if (madeDir) {
+ wh.deleteDir(dbPath, true);
+ }
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to create database: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public org.apache.hadoop.hive.metastore.api.Database getDatabase(String name) throws TException {
+ checkArgument(StringUtils.isNotEmpty(name), "name cannot be null or empty");
+
+ try {
+ Database catalogDatabase = glueMetastore.getDatabase(name);
+ return CatalogToHiveConverter.convertDatabase(catalogDatabase);
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to get database object: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public List getDatabases(String pattern) throws TException {
+ // Special handling for compatibility with Hue that passes "*" instead of ".*"
+ if (pattern == null || pattern.equals("*")) {
+ pattern = MATCH_ALL;
+ }
+
+ try {
+ List ret = new ArrayList<>();
+
+ List allDatabases = glueMetastore.getAllDatabases();
+
+ //filter by pattern
+ for (Database db : allDatabases) {
+ String name = db.getName();
+ if (Pattern.matches(pattern, name)) {
+ ret.add(name);
+ }
+ }
+ return ret;
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to get databases: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public void alterDatabase(String databaseName, org.apache.hadoop.hive.metastore.api.Database database) throws TException {
+ checkArgument(StringUtils.isNotEmpty(databaseName), "databaseName cannot be null or empty");
+ checkNotNull(database, "database cannot be null");
+
+ try {
+ DatabaseInput catalogDatabase = GlueInputConverter.convertToDatabaseInput(database);
+ glueMetastore.updateDatabase(databaseName, catalogDatabase);
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to alter database: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade) throws TException {
+ checkArgument(StringUtils.isNotEmpty(name), "name cannot be null or empty");
+
+ String dbLocation;
+ try {
+ List tables = getTables(name, MATCH_ALL);
+ boolean isEmptyDatabase = tables.isEmpty();
+
+ org.apache.hadoop.hive.metastore.api.Database db = getDatabase(name);
+ dbLocation = db.getLocationUri();
+
+ // TODO: handle cascade
+ if (isEmptyDatabase || cascade) {
+ glueMetastore.deleteDatabase(name);
+ } else {
+ throw new InvalidOperationException("Database " + name + " is not empty.");
+ }
+ } catch (NoSuchObjectException e) {
+ if (ignoreUnknownDb) {
+ return;
+ } else {
+ throw e;
+ }
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to drop database: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+
+ if (deleteData) {
+ try {
+ wh.deleteDir(new Path(dbLocation), true);
+ } catch (Exception e) {
+ logger.error("Unable to remove database directory " + dbLocation, e);
+ }
+ }
+ }
+
+ public boolean databaseExists(String dbName) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty");
+
+ try {
+ getDatabase(dbName);
+ } catch (NoSuchObjectException e) {
+ return false;
+ } catch (AmazonServiceException e) {
+ throw new TException(e);
+ } catch (Exception e) {
+ throw new MetaException(e.getMessage());
+ }
+ return true;
+ }
+
+ // ======================== Table ========================
+
+ public void createTable(org.apache.hadoop.hive.metastore.api.Table tbl) throws TException {
+ checkNotNull(tbl, "tbl cannot be null");
+ boolean dirCreated = validateNewTableAndCreateDirectory(tbl);
+ try {
+ // Glue Server side does not set DDL_TIME. Set it here for the time being.
+ // TODO: Set DDL_TIME parameter in Glue service
+ tbl.setParameters(deepCopyMap(tbl.getParameters()));
+ tbl.getParameters().put(hive_metastoreConstants.DDL_TIME,
+ Long.toString(System.currentTimeMillis() / MILLISECOND_TO_SECOND_FACTOR));
+
+ TableInput tableInput = GlueInputConverter.convertToTableInput(tbl);
+ glueMetastore.createTable(tbl.getDbName(), tableInput);
+ } catch (AmazonServiceException e) {
+ if (dirCreated) {
+ Path tblPath = new Path(tbl.getSd().getLocation());
+ wh.deleteDir(tblPath, true);
+ }
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to create table: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public boolean tableExists(String databaseName, String tableName) throws TException {
+ checkArgument(StringUtils.isNotEmpty(databaseName), "databaseName cannot be null or empty");
+ checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty");
+
+ if (!databaseExists(databaseName)) {
+ throw new UnknownDBException("Database: " + databaseName + " does not exist.");
+ }
+ try {
+ glueMetastore.getTable(databaseName, tableName);
+ return true;
+ } catch (EntityNotFoundException e) {
+ return false;
+ } catch (AmazonServiceException e){
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to check table exist: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public org.apache.hadoop.hive.metastore.api.Table getTable(String dbName, String tableName) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty");
+ checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty");
+
+ try {
+ Table table = glueMetastore.getTable(dbName, tableName);
+ validateGlueTable(table);
+ return CatalogToHiveConverter.convertTable(table, dbName);
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to get table: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public List getTables(String dbname, String tablePattern) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbname), "dbName cannot be null or empty");
+
+ List names = Lists.newArrayList();
+ try {
+ List tables = glueMetastore.getTables(dbname, tablePattern);
+ for (Table catalogTable : tables) {
+ names.add(catalogTable.getName());
+ }
+ return names;
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to get tables: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ public List getTableMeta(
+ String dbPatterns,
+ String tablePatterns,
+ List tableTypes
+ ) throws TException {
+ List tables = new ArrayList<>();
+ List databases = getDatabases(dbPatterns);
+ for (String dbName : databases) {
+ String nextToken = null;
+ List dbTables = glueMetastore.getTables(dbName, tablePatterns);
+ for (Table catalogTable : dbTables) {
+ if (tableTypes == null ||
+ tableTypes.isEmpty() ||
+ tableTypes.contains(catalogTable.getTableType())) {
+ tables.add(CatalogToHiveConverter.convertTableMeta(catalogTable, dbName));
+ }
+ }
+ }
+ return tables;
+ }
+
+ /*
+ * Hive reference: https://github.com/apache/hive/blob/rel/release-2.3.0/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java#L88
+ */
+ public void alterTable(
+ String dbName,
+ String oldTableName,
+ org.apache.hadoop.hive.metastore.api.Table newTable,
+ EnvironmentContext environmentContext
+ ) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty");
+ checkArgument(StringUtils.isNotEmpty(oldTableName), "oldTableName cannot be null or empty");
+ checkNotNull(newTable, "newTable cannot be null");
+
+ if (isCascade(environmentContext)) {
+ throw new UnsupportedOperationException("Cascade for alter_table is not supported");
+ }
+
+ if (!oldTableName.equals(newTable.getTableName())) {
+ throw new UnsupportedOperationException("Table rename is not supported");
+ }
+
+ validateTableObject(newTable, conf);
+ if (!tableExists(dbName, oldTableName)) {
+ throw new UnknownTableException("Table: " + oldTableName + " does not exists");
+ }
+
+ // If table properties has EXTERNAL set, update table type accordinly
+ // mimics Hive's ObjectStore#convertToMTable, added in HIVE-1329
+ boolean isExternal = Boolean.parseBoolean(newTable.getParameters().get("EXTERNAL"));
+ if (MANAGED_TABLE.toString().equals(newTable.getTableType()) && isExternal) {
+ newTable.setTableType(EXTERNAL_TABLE.toString());
+ } else if (EXTERNAL_TABLE.toString().equals(newTable.getTableType()) && !isExternal) {
+ newTable.setTableType(MANAGED_TABLE.toString());
+ }
+
+ if (hiveShims.requireCalStats(conf, null, null, newTable, environmentContext) && newTable.getPartitionKeys().isEmpty()) {
+ //update table stats for non-partition Table
+ org.apache.hadoop.hive.metastore.api.Database db = getDatabase(newTable.getDbName());
+ hiveShims.updateTableStatsFast(db, newTable, wh, false, true, environmentContext);
+ }
+
+ try {
+ TableInput newTableInput = GlueInputConverter.convertToTableInput(newTable);
+ glueMetastore.updateTable(dbName, newTableInput);
+ } catch (AmazonServiceException e) {
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e) {
+ String msg = "Unable to alter table: " + oldTableName;
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+ }
+
+ private boolean isCascade(EnvironmentContext environmentContext) {
+ return environmentContext != null &&
+ environmentContext.isSetProperties() &&
+ StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(StatsSetupConst.CASCADE));
+ }
+
+ public void dropTable(
+ String dbName,
+ String tableName,
+ boolean deleteData,
+ boolean ignoreUnknownTbl,
+ boolean ifPurge
+ ) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty");
+ checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty");
+
+ if (!tableExists(dbName, tableName)) {
+ if (!ignoreUnknownTbl) {
+ throw new UnknownTableException("Cannot find table: " + dbName + "." + tableName);
+ } else {
+ return;
+ }
+ }
+
+ org.apache.hadoop.hive.metastore.api.Table tbl = getTable(dbName, tableName);
+ String tblLocation = tbl.getSd().getLocation();
+ boolean isExternal = isExternalTable(tbl);
+ dropPartitionsForTable(dbName, tableName, deleteData && !isExternal);
+ dropIndexesForTable(dbName, tableName, deleteData && !isExternal);
+
+ try {
+ glueMetastore.deleteTable(dbName, tableName);
+ } catch (AmazonServiceException e){
+ throw CatalogToHiveConverter.wrapInHiveException(e);
+ } catch (Exception e){
+ String msg = "Unable to drop table: ";
+ logger.error(msg, e);
+ throw new MetaException(msg + e);
+ }
+
+ if (StringUtils.isNotEmpty(tblLocation) && deleteData && !isExternal) {
+ Path tblPath = new Path(tblLocation);
+ try {
+ wh.deleteDir(tblPath, true, ifPurge);
+ } catch (Exception e){
+ logger.error("Unable to remove table directory " + tblPath, e);
+ }
+ }
+ }
+
+ private void dropPartitionsForTable(String dbName, String tableName, boolean deleteData) throws TException {
+ List partitionsToDelete = getPartitions(dbName, tableName, null, NO_MAX);
+ for (org.apache.hadoop.hive.metastore.api.Partition part : partitionsToDelete) {
+ dropPartition(dbName, tableName, part.getValues(), true, deleteData, false);
+ }
+ }
+
+ private void dropIndexesForTable(String dbName, String tableName, boolean deleteData) throws TException {
+ List indexesToDelete = listIndexes(dbName, tableName);
+ for (Index index : indexesToDelete) {
+ dropTable(dbName, index.getIndexTableName(), deleteData, true, false);
+ }
+ }
+
+ public List getTables(String dbname, String tablePattern, TableType tableType) throws TException {
+ throw new UnsupportedOperationException("getTables with TableType is not supported");
+ }
+
+ public List listTableNamesByFilter(String dbName, String filter, short maxTables) throws TException {
+ throw new UnsupportedOperationException("listTableNamesByFilter is not supported");
+ }
+
+ /**
+ * @return boolean
+ * true -> directory created
+ * false -> directory not created
+ */
+ public boolean validateNewTableAndCreateDirectory(org.apache.hadoop.hive.metastore.api.Table tbl) throws TException {
+ checkNotNull(tbl, "tbl cannot be null");
+ if (tableExists(tbl.getDbName(), tbl.getTableName())) {
+ throw new AlreadyExistsException("Table " + tbl.getTableName() + " already exists.");
+ }
+ validateTableObject(tbl, conf);
+
+ if (TableType.VIRTUAL_VIEW.toString().equals(tbl.getTableType())) {
+ // we don't need to create directory for virtual views
+ return false;
+ }
+
+ if (StringUtils.isEmpty(tbl.getSd().getLocation())) {
+ org.apache.hadoop.hive.metastore.api.Database db = getDatabase(tbl.getDbName());
+ tbl.getSd().setLocation(hiveShims.getDefaultTablePath(db, tbl.getTableName(), wh).toString());
+ } else {
+ tbl.getSd().setLocation(wh.getDnsPath(new Path(tbl.getSd().getLocation())).toString());
+ }
+
+ Path tblPath = new Path(tbl.getSd().getLocation());
+ return makeDirs(wh, tblPath);
+ }
+
+ // =========================== Partition ===========================
+
+ public org.apache.hadoop.hive.metastore.api.Partition appendPartition(
+ String dbName,
+ String tblName,
+ List values
+ ) throws TException {
+ checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty");
+ checkArgument(StringUtils.isNotEmpty(tblName), "tblName cannot be null or empty");
+ checkNotNull(values, "partition values cannot be null");
+ org.apache.hadoop.hive.metastore.api.Table table = getTable(dbName, tblName);
+ checkNotNull(table.getSd(), "StorageDescriptor cannot be null for Table " + tblName);
+ org.apache.hadoop.hive.metastore.api.Partition partition = buildPartitionFromValues(table, values);
+ addPartitions(Lists.newArrayList(partition), false, true);
+ return partition;
+ }
+
+ /**
+ * Taken from HiveMetaStore#append_partition_common
+ */
+ private org.apache.hadoop.hive.metastore.api.Partition buildPartitionFromValues(
+ org.apache.hadoop.hive.metastore.api.Table table, List values) throws MetaException {
+ org.apache.hadoop.hive.metastore.api.Partition partition = new org.apache.hadoop.hive.metastore.api.Partition();
+ partition.setDbName(table.getDbName());
+ partition.setTableName(table.getTableName());
+ partition.setValues(values);
+ partition.setSd(table.getSd().deepCopy());
+
+ Path partLocation = new Path(table.getSd().getLocation(), Warehouse.makePartName(table.getPartitionKeys(), values));
+ partition.getSd().setLocation(partLocation.toString());
+
+ long timeInSecond = System.currentTimeMillis() / MILLISECOND_TO_SECOND_FACTOR;
+ partition.setCreateTime((int) timeInSecond);
+ partition.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(timeInSecond));
+ return partition;
+ }
+
+ public List addPartitions(
+ List