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 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 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 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 partitions, + boolean ifNotExists, + boolean needResult + ) throws TException { + checkNotNull(partitions, "partitions cannot be null"); + List partitionsCreated = + batchCreatePartitions(partitions, ifNotExists); + if (!needResult) { + return null; + } + return CatalogToHiveConverter.convertPartitions(partitionsCreated); + } + + private List batchCreatePartitions( + final List hivePartitions, + final boolean ifNotExists + ) throws TException { + if (hivePartitions.isEmpty()) { + return Lists.newArrayList(); + } + + final String dbName = hivePartitions.get(0).getDbName(); + final String tableName = hivePartitions.get(0).getTableName(); + org.apache.hadoop.hive.metastore.api.Table tbl = getTable(dbName, tableName); + validateInputForBatchCreatePartitions(tbl, hivePartitions); + + List catalogPartitions = Lists.newArrayList(); + Map addedPath = Maps.newHashMap(); + try { + for (org.apache.hadoop.hive.metastore.api.Partition partition : hivePartitions) { + Path location = getPartitionLocation(tbl, partition); + boolean partDirCreated = false; + if (location != null) { + partition.getSd().setLocation(location.toString()); + partDirCreated = makeDirs(wh, location); + } + Partition catalogPartition = HiveToCatalogConverter.convertPartition(partition); + catalogPartitions.add(catalogPartition); + if (partDirCreated) { + addedPath.put(new PartitionKey(catalogPartition), new Path(partition.getSd().getLocation())); + } + } + } catch (MetaException e) { + for (Path path : addedPath.values()) { + deletePath(path); + } + throw e; + } + + List> batchCreatePartitionsFutures = Lists.newArrayList(); + for (int i = 0; i < catalogPartitions.size(); i += BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE) { + int j = Math.min(i + BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE, catalogPartitions.size()); + final List partitionsOnePage = catalogPartitions.subList(i, j); + + batchCreatePartitionsFutures.add(this.executorService.submit(new Callable() { + @Override + public BatchCreatePartitionsHelper call() throws Exception { + return new BatchCreatePartitionsHelper(glueMetastore, dbName, tableName, catalogId, partitionsOnePage, ifNotExists) + .createPartitions(); + } + })); + } + + TException tException = null; + List partitionsCreated = Lists.newArrayList(); + for (Future future : batchCreatePartitionsFutures) { + try { + BatchCreatePartitionsHelper batchCreatePartitionsHelper = future.get(); + partitionsCreated.addAll(batchCreatePartitionsHelper.getPartitionsCreated()); + tException = tException == null ? batchCreatePartitionsHelper.getFirstTException() : tException; + deletePathForPartitions(batchCreatePartitionsHelper.getPartitionsFailed(), addedPath); + } catch (Exception e) { + logger.error("Exception thrown by BatchCreatePartitions thread pool. ", e); + } + } + + if (tException != null) { + throw tException; + } + return partitionsCreated; + } + + private void validateInputForBatchCreatePartitions( + org.apache.hadoop.hive.metastore.api.Table tbl, + List hivePartitions) { + checkNotNull(tbl.getPartitionKeys(), "Partition keys cannot be null"); + for (org.apache.hadoop.hive.metastore.api.Partition partition : hivePartitions) { + checkArgument(tbl.getDbName().equals(partition.getDbName()), "Partitions must be in the same DB"); + checkArgument(tbl.getTableName().equals(partition.getTableName()), "Partitions must be in the same table"); + checkNotNull(partition.getValues(), "Partition values cannot be null"); + checkArgument(tbl.getPartitionKeys().size() == partition.getValues().size(), "Number of table partition keys must match number of partition values"); + } + } + + private void deletePathForPartitions(List partitions, Map addedPath) { + for (Partition partition : partitions) { + Path path = addedPath.get(new PartitionKey(partition)); + if (path != null) { + deletePath(path); + } + } + } + + private void deletePath(Path path) { + try { + wh.deleteDir(path, true); + } catch (MetaException e) { + logger.error("Warehouse delete directory failed. ", e); + } + } + + /** + * Taken from HiveMetastore#createLocationForAddedPartition + */ + private Path getPartitionLocation( + org.apache.hadoop.hive.metastore.api.Table tbl, + org.apache.hadoop.hive.metastore.api.Partition part) throws MetaException { + Path partLocation = null; + String partLocationStr = null; + if (part.getSd() != null) { + partLocationStr = part.getSd().getLocation(); + } + + if (StringUtils.isEmpty(partLocationStr)) { + // set default location if not specified and this is + // a physical table partition (not a view) + if (tbl.getSd().getLocation() != null) { + partLocation = new Path(tbl.getSd().getLocation(), + Warehouse.makePartName(tbl.getPartitionKeys(), part.getValues())); + } + } else { + if (tbl.getSd().getLocation() == null) { + throw new MetaException("Cannot specify location for a view partition"); + } + partLocation = wh.getDnsPath(new Path(partLocationStr)); + } + return partLocation; + } + + public List listPartitionNames( + String databaseName, + String tableName, + List values, + short max + ) throws TException { + String expression = null; + org.apache.hadoop.hive.metastore.api.Table table = getTable(databaseName, tableName); + if (values != null) { + expression = ExpressionHelper.buildExpressionFromPartialSpecification(table, values); + } + + List names = Lists.newArrayList(); + List partitions = getPartitions(databaseName, tableName, expression, max); + for(org.apache.hadoop.hive.metastore.api.Partition p : partitions) { + names.add(Warehouse.makePartName(table.getPartitionKeys(), p.getValues())); + } + return names; + } + + public List getPartitionsByNames( + String databaseName, + String tableName, + List partitionNames + ) throws TException { + checkArgument(StringUtils.isNotEmpty(databaseName), "databaseName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty"); + checkNotNull(partitionNames, "partitionNames cannot be null"); + + List partitionsToGet = Lists.newArrayList(); + for (String partitionName : partitionNames) { + partitionsToGet.add(new PartitionValueList().withValues(partitionNameToVals(partitionName))); + } + try { + List partitions = + glueMetastore.getPartitionsByNames(databaseName, tableName, partitionsToGet); + + return CatalogToHiveConverter.convertPartitions(partitions); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get partition by names: " + StringUtils.join(partitionNames, "/"); + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tblName, String partitionName) + throws TException { + checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tblName), "tblName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(partitionName), "partitionName cannot be null or empty"); + List values = partitionNameToVals(partitionName); + return getPartition(dbName, tblName, values); + } + + public org.apache.hadoop.hive.metastore.api.Partition getPartition(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, "values cannot be null"); + + + Partition partition; + try { + partition = glueMetastore.getPartition(dbName, tblName, values); + if (partition == null) { + logger.debug("No partitions were return for dbName = " + dbName + ", tblName = " + tblName + ", values = " + values); + return null; + } + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get partition with values: " + StringUtils.join(values, "/"); + logger.error(msg, e); + throw new MetaException(msg + e); + } + return CatalogToHiveConverter.convertPartition(partition); + } + + public List getPartitions( + String databaseName, + String tableName, + String filter, + long max + ) throws TException { + checkArgument(StringUtils.isNotEmpty(databaseName), "databaseName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty"); + List partitions = getCatalogPartitions(databaseName, tableName, filter, max); + return CatalogToHiveConverter.convertPartitions(partitions); + } + + public List getCatalogPartitions( + final String databaseName, + final String tableName, + final String expression, + final long max + ) throws TException { + checkArgument(StringUtils.isNotEmpty(databaseName), "databaseName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tableName), "tableName cannot be null or empty"); + try{ + return glueMetastore.getPartitions(databaseName, tableName, expression, max); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get partitions with expression: " + expression; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + public boolean dropPartition( + String dbName, + String tblName, + Listvalues, + boolean ifExist, + boolean deleteData, + boolean purgeData + ) throws TException { + checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tblName), "tblName cannot be null or empty"); + checkNotNull(values, "values cannot be null"); + + org.apache.hadoop.hive.metastore.api.Partition partition = null; + try { + partition = getPartition(dbName, tblName, values); + } catch (NoSuchObjectException e) { + if (ifExist) { + return true; + } + } + + try { + glueMetastore.deletePartition(dbName, tblName, partition.getValues()); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to drop partition with values: " + StringUtils.join(values, "/"); + logger.error(msg, e); + throw new MetaException(msg + e); + } + + performDropPartitionPostProcessing(dbName, tblName, partition, deleteData, purgeData); + return true; + } + + private void performDropPartitionPostProcessing( + String dbName, + String tblName, + org.apache.hadoop.hive.metastore.api.Partition partition, + boolean deleteData, + boolean ifPurge + ) throws 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()); + } + } + } + + /** + * 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"))); + } + + /** + * Taken from HiveMetaStore#deleteParentRecursive + */ + 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); + } + } + + public void alterPartitions( + String dbName, + String tblName, + List partitions + ) throws TException { + checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tblName), "tblName cannot be null or empty"); + checkNotNull(partitions, "partitions cannot be null"); + + for (org.apache.hadoop.hive.metastore.api.Partition part : partitions) { + part.setParameters(deepCopyMap(part.getParameters())); + if (part.getParameters().get(hive_metastoreConstants.DDL_TIME) == null || + Integer.parseInt(part.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) { + part.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System.currentTimeMillis() / MILLISECOND_TO_SECOND_FACTOR)); + } + + PartitionInput partitionInput = GlueInputConverter.convertToPartitionInput(part); + + try { + glueMetastore.updatePartition(dbName, tblName, part.getValues(), partitionInput); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to alter partition: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + } + + /** + * Taken from HiveMetaStore#partition_name_to_vals + */ + public List partitionNameToVals(String name) throws TException { + checkNotNull(name, "name cannot be null"); + if (name.isEmpty()) { + return Lists.newArrayList(); + } + LinkedHashMap map = Warehouse.makeSpecFromName(name); + List vals = Lists.newArrayList(); + vals.addAll(map.values()); + return vals; + } + + // ============================ Index ============================== + + public List listIndexes(String dbName, String tblName) throws TException { + checkArgument(StringUtils.isNotEmpty(dbName), "dbName cannot be null or empty"); + checkArgument(StringUtils.isNotEmpty(tblName), "tblName cannot be null or empty"); + + org.apache.hadoop.hive.metastore.api.Table originTable = getTable(dbName, tblName); + Map parameters = originTable.getParameters(); + List
indexTableObjects = Lists.newArrayList(); + for(String key : parameters.keySet()) { + if(key.startsWith(INDEX_PREFIX)) { + String serialisedString = parameters.get(key); + indexTableObjects.add(stringToCatalogTable(serialisedString)); + } + } + + List hiveIndexList = Lists.newArrayList(); + for (Table catalogIndexTableObject : indexTableObjects) { + hiveIndexList.add(CatalogToHiveConverter.convertTableObjectToIndex(catalogIndexTableObject)); + } + return hiveIndexList; + } + + // ======================= Roles & Privilege ======================= + + public boolean createRole(org.apache.hadoop.hive.metastore.api.Role role) throws TException { + throw new UnsupportedOperationException("createRole is not supported"); + } + + public boolean dropRole(String roleName) throws TException { + throw new UnsupportedOperationException("dropRole is not supported"); + } + + public List listRoles( + String principalName, + org.apache.hadoop.hive.metastore.api.PrincipalType principalType + ) throws TException { + // All users belong to public role implicitly, add that role + // Bring logic from Hive's ObjectStore + // https://code.amazon.com/packages/Aws157Hive/blobs/48f6e30080df475ffe54c39f70dd134268e30358/ + // --/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L4208 + if (principalType == PrincipalType.USER) { + return implicitRoles; + } else { + throw new UnsupportedOperationException( + "listRoles is only supported for " + PrincipalType.USER + " Principal type"); + } + } + + public List listRoleNames() throws TException { + // return PUBLIC role as implicit role to prevent unnecessary failure, + // even though Glue doesn't support Role API yet + // https://code.amazon.com/packages/Aws157Hive/blobs/48f6e30080df475ffe54c39f70dd134268e30358/ + // --/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L4325 + return Lists.newArrayList(PUBLIC); + } + + public org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse getPrincipalsInRole( + org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest request + ) throws TException { + throw new UnsupportedOperationException("getPrincipalsInRole is not supported"); + } + + public GetRoleGrantsForPrincipalResponse getRoleGrantsForPrincipal( + GetRoleGrantsForPrincipalRequest request + ) throws TException { + throw new UnsupportedOperationException("getRoleGrantsForPrincipal is not supported"); + } + + public boolean grantRole( + 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 TException { + throw new UnsupportedOperationException("grantRole is not supported"); + } + + public boolean revokeRole( + String roleName, + String userName, + org.apache.hadoop.hive.metastore.api.PrincipalType principalType, + boolean grantOption + ) throws TException { + throw new UnsupportedOperationException("revokeRole is not supported"); + } + + public boolean revokePrivileges( + org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges, + boolean grantOption + ) throws TException { + throw new UnsupportedOperationException("revokePrivileges is not supported"); + } + + public boolean grantPrivileges(org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges) + throws TException { + throw new UnsupportedOperationException("grantPrivileges is not supported"); + } + + public org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet getPrivilegeSet( + HiveObjectRef objectRef, + String user, List groups + ) throws TException { + // getPrivilegeSet is NOT yet supported. + // return null not to break due to optional info + // Hive return null when every condition fail + // https://code.amazon.com/packages/Aws157Hive/blobs/c1ced60e67765d27086b3621255cd843947c151e/ + // --/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java#L5237 + return null; + } + + public List listPrivileges( + String principal, + org.apache.hadoop.hive.metastore.api.PrincipalType principalType, + HiveObjectRef objectRef + ) throws TException { + throw new UnsupportedOperationException("listPrivileges is not supported"); + } + + // ========================== Statistics ========================== + + public boolean deletePartitionColumnStatistics( + String dbName, + String tableName, + String partName, + String colName + ) throws TException { + throw new UnsupportedOperationException("deletePartitionColumnStatistics is not supported"); + } + + public boolean deleteTableColumnStatistics( + String dbName, + String tableName, + String colName + ) throws TException { + throw new UnsupportedOperationException("deleteTableColumnStatistics is not supported"); + } + + public Map> getPartitionColumnStatistics( + String dbName, + String tableName, + List partitionNames, List columnNames + ) throws TException { + throw new UnsupportedOperationException("getPartitionColumnStatistics is not supported"); + } + + public List getTableColumnStatistics( + String dbName, + String tableName, + List colNames + ) throws TException { + throw new UnsupportedOperationException("getTableColumnStatistics is not supported"); + } + + public boolean updatePartitionColumnStatistics( + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics + ) throws TException { + throw new UnsupportedOperationException("updatePartitionColumnStatistics is not supported"); + } + + public boolean updateTableColumnStatistics( + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics + ) throws TException { + throw new UnsupportedOperationException("updateTableColumnStatistics is not supported"); + } + + public AggrStats getAggrColStatsFor( + String dbName, + String tblName, + List colNames, + List partName + ) throws TException { + throw new UnsupportedOperationException("getAggrColStatsFor is not supported"); + } + + public void cancelDelegationToken(String tokenStrForm) throws TException { + throw new UnsupportedOperationException("cancelDelegationToken is not supported"); + } + + public String getTokenStrForm() throws IOException { + throw new UnsupportedOperationException("getTokenStrForm is not supported"); + } + + public boolean addToken(String tokenIdentifier, String delegationToken) throws TException { + throw new UnsupportedOperationException("addToken is not supported"); + } + + public boolean removeToken(String tokenIdentifier) throws TException { + throw new UnsupportedOperationException("removeToken is not supported"); + } + + public String getToken(String tokenIdentifier) throws TException { + throw new UnsupportedOperationException("getToken is not supported"); + } + + public List getAllTokenIdentifiers() throws TException { + throw new UnsupportedOperationException("getAllTokenIdentifiers is not supported"); + } + + public int addMasterKey(String key) throws TException { + throw new UnsupportedOperationException("addMasterKey is not supported"); + } + + public void updateMasterKey(Integer seqNo, String key) throws TException { + throw new UnsupportedOperationException("updateMasterKey is not supported"); + } + + public boolean removeMasterKey(Integer keySeq) throws TException { + throw new UnsupportedOperationException("removeMasterKey is not supported"); + } + + public String[] getMasterKeys() throws TException { + throw new UnsupportedOperationException("getMasterKeys is not supported"); + } + + public LockResponse checkLock(long lockId) throws TException { + throw new UnsupportedOperationException("checkLock is not supported"); + } + + public void commitTxn(long txnId) throws TException { + throw new UnsupportedOperationException("commitTxn is not supported"); + } + + public void abortTxns(List txnIds) throws TException { + throw new UnsupportedOperationException("abortTxns is not supported"); + } + + public void compact( + String dbName, + String tblName, + String partitionName, + CompactionType compactionType + ) throws TException { + throw new UnsupportedOperationException("compact is not supported"); + } + + public void compact( + String dbName, + String tblName, + String partitionName, + CompactionType compactionType, + Map tblProperties + ) throws TException { + throw new UnsupportedOperationException("compact is not supported"); + } + + public CompactionResponse compact2( + String dbName, + String tblName, + String partitionName, + CompactionType compactionType, + Map tblProperties + ) throws TException { + throw new UnsupportedOperationException("compact2 is not supported"); + } + + public ValidTxnList getValidTxns() throws TException { + throw new UnsupportedOperationException("getValidTxns is not supported"); + } + + public ValidTxnList getValidTxns(long currentTxn) throws TException { + throw new UnsupportedOperationException("getValidTxns is not supported"); + } + + public org.apache.hadoop.hive.metastore.api.Partition exchangePartition( + Map partitionSpecs, + String srcDb, + String srcTbl, + String dstDb, + String dstTbl + ) throws TException { + throw new UnsupportedOperationException("exchangePartition not yet supported."); + } + + public List exchangePartitions( + Map partitionSpecs, + String sourceDb, + String sourceTbl, + String destDb, + String destTbl + ) throws TException { + throw new UnsupportedOperationException("exchangePartitions is not yet supported"); + } + + public String getDelegationToken( + String owner, + String renewerKerberosPrincipalName + ) throws TException { + throw new UnsupportedOperationException("getDelegationToken is not supported"); + } + + public void heartbeat(long txnId, long lockId) throws TException { + throw new UnsupportedOperationException("heartbeat is not supported"); + } + + public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) throws TException { + throw new UnsupportedOperationException("heartbeatTxnRange is not supported"); + } + + public boolean isPartitionMarkedForEvent( + String dbName, + String tblName, + Map partKVs, + PartitionEventType eventType + ) throws TException { + throw new UnsupportedOperationException("isPartitionMarkedForEvent is not supported"); + } + + public int getNumPartitionsByFilter( + String dbName, + String tableName, + String filter + ) throws TException { + throw new UnsupportedOperationException("getNumPartitionsByFilter is not supported."); + } + + public PartitionSpecProxy listPartitionSpecs( + String dbName, + String tblName, + int max + ) throws TException { + throw new UnsupportedOperationException("listPartitionSpecs is not supported."); + } + + public PartitionSpecProxy listPartitionSpecsByFilter( + String dbName, + String tblName, + String filter, + int max + ) throws TException { + throw new UnsupportedOperationException("listPartitionSpecsByFilter is not supported"); + } + + public LockResponse lock(LockRequest lockRequest) throws TException { + throw new UnsupportedOperationException("lock is not supported"); + } + + public void markPartitionForEvent( + String dbName, + String tblName, + Map partKeyValues, + PartitionEventType eventType + ) throws TException { + throw new UnsupportedOperationException("markPartitionForEvent is not supported"); + } + + public long openTxn(String user) throws TException { + throw new UnsupportedOperationException("openTxn is not supported"); + } + + public OpenTxnsResponse openTxns(String user, int numTxns) throws TException { + throw new UnsupportedOperationException("openTxns is not supported"); + } + + public long renewDelegationToken(String tokenStrForm) throws TException { + throw new UnsupportedOperationException("renewDelegationToken is not supported"); + } + + public void rollbackTxn(long txnId) throws TException { + throw new UnsupportedOperationException("rollbackTxn is not supported"); + } + + public void createTableWithConstraints( + org.apache.hadoop.hive.metastore.api.Table table, + List primaryKeys, + List foreignKeys + ) throws AlreadyExistsException, TException { + throw new UnsupportedOperationException("createTableWithConstraints is not supported"); + } + + public void dropConstraint( + String dbName, + String tblName, + String constraintName + ) throws TException { + throw new UnsupportedOperationException("dropConstraint is not supported"); + } + + public void addPrimaryKey(List primaryKeyCols) throws TException { + throw new UnsupportedOperationException("addPrimaryKey is not supported"); + } + + public void addForeignKey(List foreignKeyCols) throws TException { + throw new UnsupportedOperationException("addForeignKey is not supported"); + } + + public ShowCompactResponse showCompactions() throws TException { + throw new UnsupportedOperationException("showCompactions is not supported"); + } + + public void addDynamicPartitions( + long txnId, + String dbName, + String tblName, + List partNames + ) throws TException { + throw new UnsupportedOperationException("addDynamicPartitions is not supported"); + } + + public void addDynamicPartitions( + long txnId, + String dbName, + String tblName, + List partNames, + DataOperationType operationType + ) throws TException { + throw new UnsupportedOperationException("addDynamicPartitions is not supported"); + } + + public void insertTable(org.apache.hadoop.hive.metastore.api.Table table, boolean overwrite) throws MetaException { + throw new UnsupportedOperationException("insertTable is not supported"); + } + + public NotificationEventResponse getNextNotification( + long lastEventId, + int maxEvents, + IMetaStoreClient.NotificationFilter notificationFilter + ) throws TException { + throw new UnsupportedOperationException("getNextNotification is not supported"); + } + + public CurrentNotificationEventId getCurrentNotificationEventId() throws TException { + throw new UnsupportedOperationException("getCurrentNotificationEventId is not supported"); + } + + public FireEventResponse fireListenerEvent(FireEventRequest fireEventRequest) throws TException { + throw new UnsupportedOperationException("fireListenerEvent is not supported"); + } + + public ShowLocksResponse showLocks() throws TException { + throw new UnsupportedOperationException("showLocks is not supported"); + } + + public ShowLocksResponse showLocks(ShowLocksRequest showLocksRequest) throws TException { + throw new UnsupportedOperationException("showLocks is not supported"); + } + + public GetOpenTxnsInfoResponse showTxns() throws TException { + throw new UnsupportedOperationException("showTxns is not supported"); + } + + public void unlock(long lockId) throws TException { + throw new UnsupportedOperationException("unlock is not supported"); + } + + public Iterable> getFileMetadata(List fileIds) throws TException { + throw new UnsupportedOperationException("getFileMetadata is not supported"); + } + + public Iterable> getFileMetadataBySarg( + List fileIds, + ByteBuffer sarg, + boolean doGetFooters + ) throws TException { + throw new UnsupportedOperationException("getFileMetadataBySarg is not supported"); + } + + public void clearFileMetadata(List fileIds) throws TException { + throw new UnsupportedOperationException("clearFileMetadata is not supported"); + } + + public void putFileMetadata(List fileIds, List metadata) throws TException { + throw new UnsupportedOperationException("putFileMetadata is not supported"); + } + + public boolean setPartitionColumnStatistics( + org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest request + ) throws TException { + throw new UnsupportedOperationException("setPartitionColumnStatistics is not supported"); + } + + public boolean cacheFileMetadata( + String dbName, + String tblName, + String partName, + boolean allParts + ) throws TException { + throw new UnsupportedOperationException("cacheFileMetadata is not supported"); + } + + public int addPartitionsSpecProxy(PartitionSpecProxy pSpec) throws TException { + throw new UnsupportedOperationException("addPartitionsSpecProxy is unsupported"); + } + + public void setUGI(String username) throws TException { + throw new UnsupportedOperationException("setUGI is unsupported"); + } + + /** + * Gets the user defined function in a database stored in metastore and + * converts back to Hive function. + * + * @param dbName + * @param functionName + * @return + * @throws MetaException + * @throws TException + */ + public org.apache.hadoop.hive.metastore.api.Function getFunction(String dbName, String functionName) + throws TException { + try { + UserDefinedFunction userDefinedFunction = glueMetastore.getUserDefinedFunction(dbName, functionName); + return CatalogToHiveConverter.convertFunction(dbName, userDefinedFunction); + } catch (AmazonServiceException e) { + logger.error(e); + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get Function: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Gets user defined functions that match a pattern in database stored in + * metastore and converts back to Hive function. + * + * @param dbName + * @param pattern + * @return + * @throws MetaException + * @throws TException + */ + public List getFunctions(String dbName, String pattern) throws TException { + try { + List functionNames = Lists.newArrayList(); + List functions = + glueMetastore.getUserDefinedFunctions(dbName, pattern); + for (UserDefinedFunction catalogFunction : functions) { + functionNames.add(catalogFunction.getFunctionName()); + } + return functionNames; + } 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); + } + } + + /** + * Creates a new user defined function in the metastore. + * + * @param function + * @throws InvalidObjectException + * @throws MetaException + * @throws TException + */ + public void createFunction(org.apache.hadoop.hive.metastore.api.Function function) throws InvalidObjectException, + TException { + try { + UserDefinedFunctionInput functionInput = GlueInputConverter.convertToUserDefinedFunctionInput(function); + glueMetastore.createUserDefinedFunction(function.getDbName(), functionInput); + } catch (AmazonServiceException e) { + logger.error(e); + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to create Function: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Drops a user defined function in the database stored in metastore. + * + * @param dbName + * @param functionName + * @throws MetaException + * @throws NoSuchObjectException + * @throws InvalidObjectException + * @throws org.apache.hadoop.hive.metastore.api.InvalidInputException + * @throws TException + */ + public void dropFunction(String dbName, String functionName) throws NoSuchObjectException, + InvalidObjectException, org.apache.hadoop.hive.metastore.api.InvalidInputException, TException { + try { + glueMetastore.deleteUserDefinedFunction(dbName, functionName); + } catch (AmazonServiceException e) { + logger.error(e); + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to drop Function: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Updates a user defined function in a database stored in the metastore. + * + * @param dbName + * @param functionName + * @param newFunction + * @throws InvalidObjectException + * @throws MetaException + * @throws TException + */ + public void alterFunction(String dbName, String functionName, + org.apache.hadoop.hive.metastore.api.Function newFunction) throws InvalidObjectException, MetaException, + TException { + try { + UserDefinedFunctionInput functionInput = GlueInputConverter.convertToUserDefinedFunctionInput(newFunction); + glueMetastore.updateUserDefinedFunction(dbName, functionName, functionInput); + } catch (AmazonServiceException e) { + logger.error(e); + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to alter Function: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Fetches the fields for a table in a database. + * + * @param db + * @param tableName + * @return + * @throws MetaException + * @throws TException + * @throws UnknownTableException + * @throws UnknownDBException + */ + public List getFields(String db, String tableName) throws MetaException, TException, + UnknownTableException, UnknownDBException { + try { + Table table = glueMetastore.getTable(db, tableName); + return CatalogToHiveConverter.convertFieldSchemaList(table.getStorageDescriptor().getColumns()); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get field from table: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Fetches the schema for a table in a database. + * + * @param db + * @param tableName + * @return + * @throws MetaException + * @throws TException + * @throws UnknownTableException + * @throws UnknownDBException + */ + public List getSchema(String db, String tableName) throws TException, + UnknownTableException, UnknownDBException { + try { + Table table = glueMetastore.getTable(db, tableName); + List schemas = table.getStorageDescriptor().getColumns(); + if (table.getPartitionKeys() != null && !table.getPartitionKeys().isEmpty()) { + schemas.addAll(table.getPartitionKeys()); + } + return CatalogToHiveConverter.convertFieldSchemaList(schemas); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } catch (Exception e) { + String msg = "Unable to get field from table: "; + logger.error(msg, e); + throw new MetaException(msg + e); + } + } + + /** + * Updates the partition values for a table in database stored in metastore. + * + * @param databaseName + * @param tableName + * @param partitionValues + * @param newPartition + * @throws InvalidOperationException + * @throws MetaException + * @throws TException + */ + public void renamePartitionInCatalog(String databaseName, String tableName, List partitionValues, + org.apache.hadoop.hive.metastore.api.Partition newPartition) throws InvalidOperationException, + TException { + try { + PartitionInput partitionInput = GlueInputConverter.convertToPartitionInput(newPartition); + glueMetastore.updatePartition(databaseName, tableName, partitionValues, partitionInput); + } catch (AmazonServiceException e) { + throw CatalogToHiveConverter.wrapInHiveException(e); + } + } +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/SessionCredentialsProviderFactory.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/SessionCredentialsProviderFactory.java new file mode 100644 index 000000000..bfffc58c0 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/metastore/SessionCredentialsProviderFactory.java @@ -0,0 +1,35 @@ +package com.amazonaws.glue.catalog.metastore; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSSessionCredentials; +import com.amazonaws.auth.BasicSessionCredentials; +import com.amazonaws.internal.StaticCredentialsProvider; + +import org.apache.hadoop.hive.conf.HiveConf; + +import static com.google.common.base.Preconditions.checkArgument; + +public class SessionCredentialsProviderFactory implements AWSCredentialsProviderFactory { + + public final static String AWS_ACCESS_KEY_CONF_VAR = "hive.aws_session_access_id"; + public final static String AWS_SECRET_KEY_CONF_VAR = "hive.aws_session_secret_key"; + public final static String AWS_SESSION_TOKEN_CONF_VAR = "hive.aws_session_token"; + + @Override + public AWSCredentialsProvider buildAWSCredentialsProvider(HiveConf hiveConf) { + + checkArgument(hiveConf != null, "hiveConf cannot be null."); + + String accessKey = hiveConf.get(AWS_ACCESS_KEY_CONF_VAR); + String secretKey = hiveConf.get(AWS_SECRET_KEY_CONF_VAR); + String sessionToken = hiveConf.get(AWS_SESSION_TOKEN_CONF_VAR); + + checkArgument(accessKey != null, AWS_ACCESS_KEY_CONF_VAR + " must be set."); + checkArgument(secretKey != null, AWS_SECRET_KEY_CONF_VAR + " must be set."); + checkArgument(sessionToken != null, AWS_SESSION_TOKEN_CONF_VAR + " must be set."); + + AWSSessionCredentials credentials = new BasicSessionCredentials(accessKey, secretKey, sessionToken); + + return new StaticCredentialsProvider(credentials); + } +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/AWSGlueConfig.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/AWSGlueConfig.java new file mode 100644 index 000000000..b7142d64d --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/AWSGlueConfig.java @@ -0,0 +1,33 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.ClientConfiguration; + +public final class AWSGlueConfig { + + private AWSGlueConfig() {} + + public static final String AWS_GLUE_ENDPOINT = "aws.glue.endpoint"; + public static final String AWS_REGION = "aws.region"; + public static final String AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS + = "aws.catalog.credentials.provider.factory.class"; + + public static final String AWS_GLUE_MAX_RETRY = "aws.glue.max-error-retries"; + public static final int DEFAULT_MAX_RETRY = 5; + + public static final String AWS_GLUE_MAX_CONNECTIONS = "aws.glue.max-connections"; + public static final int DEFAULT_MAX_CONNECTIONS = ClientConfiguration.DEFAULT_MAX_CONNECTIONS; + + public static final String AWS_GLUE_CONNECTION_TIMEOUT = "aws.glue.connection-timeout"; + public static final int DEFAULT_CONNECTION_TIMEOUT = ClientConfiguration.DEFAULT_CONNECTION_TIMEOUT; + + public static final String AWS_GLUE_SOCKET_TIMEOUT = "aws.glue.socket-timeout"; + public static final int DEFAULT_SOCKET_TIMEOUT = ClientConfiguration.DEFAULT_SOCKET_TIMEOUT; + + public static final String AWS_GLUE_DB_CACHE_ENABLE = "aws.glue.cache.db.enable"; + public static final String AWS_GLUE_DB_CACHE_SIZE = "aws.glue.cache.db.size"; + public static final String AWS_GLUE_DB_CACHE_TTL_MINS = "aws.glue.cache.db.ttl-mins"; + + public static final String AWS_GLUE_TABLE_CACHE_ENABLE = "aws.glue.cache.table.enable"; + public static final String AWS_GLUE_TABLE_CACHE_SIZE = "aws.glue.cache.table.size"; + public static final String AWS_GLUE_TABLE_CACHE_TTL_MINS = "aws.glue.cache.table.ttl-mins"; +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelper.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelper.java new file mode 100644 index 000000000..5a36dc26b --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelper.java @@ -0,0 +1,133 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.metastore.AWSGlueMetastore; +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.glue.catalog.converters.GlueInputConverter; +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionError; +import com.google.common.collect.Lists; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.log4j.Logger; +import org.apache.thrift.TException; + +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static com.amazonaws.glue.catalog.util.PartitionUtils.isInvalidUserInputException; + +public final class BatchCreatePartitionsHelper { + + private static final Logger logger = Logger.getLogger(BatchCreatePartitionsHelper.class); + + private final AWSGlueMetastore glueClient; + private final String databaseName; + private final String tableName; + private final List partitions; + private final boolean ifNotExists; + private Map partitionMap; + private List partitionsFailed; + private TException firstTException; + private String catalogId; + + public BatchCreatePartitionsHelper(AWSGlueMetastore glueClient, String databaseName, String tableName, String catalogId, + List partitions, boolean ifNotExists) { + this.glueClient = glueClient; + this.databaseName = databaseName; + this.tableName = tableName; + this.catalogId = catalogId; + this.partitions = partitions; + this.ifNotExists = ifNotExists; + } + + public BatchCreatePartitionsHelper createPartitions() { + partitionMap = PartitionUtils.buildPartitionMap(partitions); + partitionsFailed = Lists.newArrayList(); + + try { + List result = + glueClient.createPartitions(databaseName, tableName, + GlueInputConverter.convertToPartitionInputs(partitionMap.values())); + processResult(result); + } catch (Exception e) { + logger.error("Exception thrown while creating partitions in DataCatalog: ", e); + firstTException = CatalogToHiveConverter.wrapInHiveException(e); + if (isInvalidUserInputException(e)) { + setAllFailed(); + } else { + checkIfPartitionsCreated(); + } + } + return this; + } + + private void setAllFailed() { + partitionsFailed = partitions; + partitionMap.clear(); + } + + private void processResult(List partitionErrors) { + if (partitionErrors == null || partitionErrors.isEmpty()) { + return; + } + + logger.error(String.format("BatchCreatePartitions failed to create %d out of %d partitions. \n", + partitionErrors.size(), partitionMap.size())); + + for (PartitionError partitionError : partitionErrors) { + Partition partitionFailed = partitionMap.remove(new PartitionKey(partitionError.getPartitionValues())); + + TException exception = CatalogToHiveConverter.errorDetailToHiveException(partitionError.getErrorDetail()); + if (ifNotExists && exception instanceof AlreadyExistsException) { + // AlreadyExistsException is allowed, so we shouldn't add the partition to partitionsFailed list + continue; + } + logger.error(exception); + if (firstTException == null) { + firstTException = exception; + } + partitionsFailed.add(partitionFailed); + } + } + + private void checkIfPartitionsCreated() { + for (Partition partition : partitions) { + if (!partitionExists(partition)) { + partitionsFailed.add(partition); + partitionMap.remove(new PartitionKey(partition)); + } + } + } + + private boolean partitionExists(Partition partition) { + + try { + Partition partitionReturned = glueClient.getPartition(databaseName, tableName, partition.getValues()); + return partitionReturned != null; //probably always true here + } catch (EntityNotFoundException e) { + // here we assume namespace and table exist. It is assured by calling "isInvalidUserInputException" method above + return false; + } catch (Exception e) { + logger.error(String.format("Get partition request %s failed. ", + StringUtils.join(partition.getValues(), "/")), e); + // partition status unknown, we assume that the partition was not created + return false; + } + } + + public TException getFirstTException() { + return firstTException; + } + + public Collection getPartitionsCreated() { + return partitionMap.values(); + } + + public List getPartitionsFailed() { + return partitionsFailed; + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelper.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelper.java new file mode 100644 index 000000000..1826c4d6d --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelper.java @@ -0,0 +1,125 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.glue.catalog.metastore.GlueMetastoreClientDelegate; +import com.amazonaws.services.glue.model.BatchDeletePartitionRequest; +import com.amazonaws.services.glue.model.BatchDeletePartitionResult; +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.ErrorDetail; +import com.amazonaws.services.glue.model.GetPartitionRequest; +import com.amazonaws.services.glue.model.GetPartitionResult; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionError; +import com.amazonaws.services.glue.AWSGlue; + +import org.apache.log4j.Logger; +import org.apache.thrift.TException; + +import java.util.Collection; +import java.util.List; +import java.util.Map; + +public final class BatchDeletePartitionsHelper { + + private static final Logger logger = Logger.getLogger(BatchDeletePartitionsHelper.class); + + private final AWSGlue client; + private final String namespaceName; + private final String tableName; + private final String catalogId; + private final List partitions; + private Map partitionMap; + private TException firstTException; + + public BatchDeletePartitionsHelper(AWSGlue client, String namespaceName, String tableName, + String catalogId, List partitions) { + this.client = client; + this.namespaceName = namespaceName; + this.tableName = tableName; + this.catalogId = catalogId; + this.partitions = partitions; + } + + public BatchDeletePartitionsHelper deletePartitions() { + partitionMap = PartitionUtils.buildPartitionMap(partitions); + + BatchDeletePartitionRequest request = new BatchDeletePartitionRequest().withDatabaseName(namespaceName) + .withTableName(tableName).withCatalogId(catalogId) + .withPartitionsToDelete(PartitionUtils.getPartitionValuesList(partitionMap)); + + try { + BatchDeletePartitionResult result = client.batchDeletePartition(request); + processResult(result); + } catch (Exception e) { + logger.error("Exception thrown while deleting partitions in DataCatalog: ", e); + firstTException = CatalogToHiveConverter.wrapInHiveException(e); + if (PartitionUtils.isInvalidUserInputException(e)) { + setAllFailed(); + } else { + checkIfPartitionsDeleted(); + } + } + return this; + } + + private void setAllFailed() { + partitionMap.clear(); + } + + private void processResult(final BatchDeletePartitionResult batchDeletePartitionsResult) { + List partitionErrors = batchDeletePartitionsResult.getErrors(); + if (partitionErrors == null || partitionErrors.isEmpty()) { + return; + } + + logger.error(String.format("BatchDeletePartitions failed to delete %d out of %d partitions. \n", + partitionErrors.size(), partitionMap.size())); + + for (PartitionError partitionError : partitionErrors) { + partitionMap.remove(new PartitionKey(partitionError.getPartitionValues())); + ErrorDetail errorDetail = partitionError.getErrorDetail(); + logger.error(errorDetail.toString()); + if (firstTException == null) { + firstTException = CatalogToHiveConverter.errorDetailToHiveException(errorDetail); + } + } + } + + private void checkIfPartitionsDeleted() { + for (Partition partition : partitions) { + if (!partitionDeleted(partition)) { + partitionMap.remove(new PartitionKey(partition)); + } + } + } + + private boolean partitionDeleted(Partition partition) { + GetPartitionRequest request = new GetPartitionRequest() + .withDatabaseName(partition.getDatabaseName()) + .withTableName(partition.getTableName()) + .withPartitionValues(partition.getValues()) + .withCatalogId(catalogId); + + try { + GetPartitionResult result = client.getPartition(request); + Partition partitionReturned = result.getPartition(); + return partitionReturned == null; //probably always false + } catch (EntityNotFoundException e) { + // here we assume namespace and table exist. It is assured by calling "isInvalidUserInputException" method above + return true; + } catch (Exception e) { + logger.error(String.format("Get partition request %s failed. ", request.toString()), e); + // Partition status unknown, we assume that the partition was not deleted + return false; + } + } + + public TException getFirstTException() { + return firstTException; + } + + public Collection getPartitionsDeleted() { + return partitionMap.values(); + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/ExpressionHelper.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/ExpressionHelper.java new file mode 100644 index 000000000..dd4340706 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/ExpressionHelper.java @@ -0,0 +1,196 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.shims.ShimsLoader; + +import com.google.common.base.Joiner; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.log4j.Logger; + +import java.util.LinkedList; +import java.util.List; +import java.util.Set; + +/** + * Utility methods for constructing the string representation of query expressions used by Catalog service + */ +public final class ExpressionHelper { + + private final static String HIVE_STRING_TYPE_NAME = "string"; + private final static String HIVE_IN_OPERATOR = "IN"; + private final static String HIVE_NOT_IN_OPERATOR = "NOT IN"; + private final static String HIVE_NOT_OPERATOR = "not"; + + // TODO "hook" into Hive logging (hive or hive.metastore) + private final static Logger logger = Logger.getLogger(ExpressionHelper.class); + + private final static List QUOTED_TYPES = ImmutableList.of("string", "char", "varchar", "date", "datetime", "timestamp"); + private final static Joiner JOINER = Joiner.on(" AND "); + + /* + * The method below is used to rewrite the hive expression tree to quote the timestamp values. + * An example of this would be hive providing us as query as follows: + * ((strCol = 'test') and (timestamp = 1969-12-31 16:02:03.456)) + * + * this will be rewritten by the method to: + * ((strCol = 'test') and (timestamp = '1969-12-31 16:02:03.456')) + * + * Notice the way the timestamp is quoted. + * + * In order to perform this operation we recursively navigate the ExpressionTree + * given to us by hive and switch the type to 'string' whenever we encounter a node type + * of type 'timestamp' + * + * When we call the getExprTree method of the modified expression tree, the timestamp values are + * properly quoted. + * + * This method also rewrites the expression string for "NOT IN" expression. + * Hive converts the expression " NOT IN ()" to "(not () IN ())". + * But in DataCatalog service, the parsing is done based on the original expression (which contains NOT IN). + * So, we need to rewrite the expression if NOT IN was used. + * */ + public static String convertHiveExpressionToCatalogExpression(byte[] exprBytes) throws MetaException { + ExprNodeGenericFuncDesc exprTree = deserializeExpr(exprBytes); + Set columnNamesInNotInExpression = Sets.newHashSet(); + fieldEscaper(exprTree.getChildren(), exprTree, columnNamesInNotInExpression); + String expression = rewriteExpressionForNotIn(exprTree.getExprString(), columnNamesInNotInExpression); + return expression; + } + + private static ExprNodeGenericFuncDesc deserializeExpr(byte[] exprBytes) throws MetaException { + ExprNodeGenericFuncDesc expr = null; + try { + expr = ShimsLoader.getHiveShims().getDeserializeExpression(exprBytes); + } catch (Exception ex) { + logger.error("Failed to deserialize the expression", ex); + throw new MetaException(ex.getMessage()); + } + if (expr == null) { + throw new MetaException("Failed to deserialize expression - ExprNodeDesc not present"); + } + return expr; + } + + //Helper method that recursively switches the type of the node, this is used + //by the convertHiveExpressionToCatalogExpression + private static void fieldEscaper(List exprNodes, ExprNodeDesc parent, Set columnNamesInNotInExpression) { + if (exprNodes == null || exprNodes.isEmpty()) { + return; + } else { + for (ExprNodeDesc nodeDesc : exprNodes) { + String nodeType = nodeDesc.getTypeString().toLowerCase(); + if (QUOTED_TYPES.contains(nodeType)) { + PrimitiveTypeInfo tInfo = new PrimitiveTypeInfo(); + tInfo.setTypeName(HIVE_STRING_TYPE_NAME); + nodeDesc.setTypeInfo(tInfo); + } + addColumnNamesOfNotInExpressionToSet(nodeDesc, parent, columnNamesInNotInExpression); + fieldEscaper(nodeDesc.getChildren(), nodeDesc, columnNamesInNotInExpression); + } + } + } + + /* + * Method to extract the names of columns that are involved in NOT IN expression. Only one column is allowed to be + * used in NOT IN expression. So, ExprNodeDesc.getCols() would return only 1 column. + * + * @param childNode + * @param parentNode + * @param columnsInNotInExpression + */ + private static void addColumnNamesOfNotInExpressionToSet(ExprNodeDesc childNode, ExprNodeDesc parentNode, Set columnsInNotInExpression) { + if (parentNode != null && childNode != null && parentNode instanceof ExprNodeGenericFuncDesc && childNode instanceof ExprNodeGenericFuncDesc) { + ExprNodeGenericFuncDesc parentFuncNode = (ExprNodeGenericFuncDesc) parentNode; + ExprNodeGenericFuncDesc childFuncNode = (ExprNodeGenericFuncDesc) childNode; + if(parentFuncNode.getGenericUDF() instanceof GenericUDFOPNot && childFuncNode.getGenericUDF() instanceof GenericUDFIn) { + // The current parent child pair represents a "NOT IN" expression. Add name of the column to the set. + columnsInNotInExpression.addAll(childFuncNode.getCols()); + } + } + } + + private static String rewriteExpressionForNotIn(String expression, Set columnsInNotInExpression){ + for (String columnName : columnsInNotInExpression) { + if (columnName != null) { + String hiveExpression = getHiveCompatibleNotInExpression(columnName); + hiveExpression = escapeParentheses(hiveExpression); + String catalogExpression = getCatalogCompatibleNotInExpression(columnName); + catalogExpression = escapeParentheses(catalogExpression); + expression = expression.replaceAll(hiveExpression, catalogExpression); + } + } + return expression; + } + + // return "not () IN (" + private static String getHiveCompatibleNotInExpression(String columnName) { + return String.format("%s (%s) %s (", HIVE_NOT_OPERATOR, columnName, HIVE_IN_OPERATOR); + } + + // return "() NOT IN (" + private static String getCatalogCompatibleNotInExpression(String columnName) { + return String.format("(%s) %s (", columnName, HIVE_NOT_IN_OPERATOR); + } + + /* + * Escape the parentheses so that they are considered literally and not as part of regular expression. In the updated + * expression , we need "\\(" as the output. So, the first four '\' generate '\\' and the last two '\' generate a '(' + */ + private static String escapeParentheses(String expression) { + expression = expression.replaceAll("\\(", "\\\\\\("); + expression = expression.replaceAll("\\)", "\\\\\\)"); + return expression; + } + + public static String buildExpressionFromPartialSpecification(org.apache.hadoop.hive.metastore.api.Table table, + List partitionValues) throws MetaException { + + List partitionKeys = table.getPartitionKeys(); + + if (partitionValues == null || partitionValues.isEmpty() ) { + return null; + } + + if (partitionKeys == null || partitionValues.size() > partitionKeys.size()) { + throw new MetaException("Incorrect number of partition values: " + partitionValues); + } + + partitionKeys = partitionKeys.subList(0, partitionValues.size()); + List predicates = new LinkedList<>(); + for (int i = 0; i < partitionValues.size(); i++) { + if (!Strings.isNullOrEmpty(partitionValues.get(i))) { + predicates.add(buildPredicate(partitionKeys.get(i), partitionValues.get(i))); + } + } + + return JOINER.join(predicates); + } + + private static String buildPredicate(org.apache.hadoop.hive.metastore.api.FieldSchema schema, String value) { + if (isQuotedType(schema.getType())) { + return String.format("(%s='%s')", schema.getName(), escapeSingleQuotes(value)); + } else { + return String.format("(%s=%s)", schema.getName(), value); + } + } + + private static String escapeSingleQuotes(String s) { + return s.replaceAll("'", "\\\\'"); + } + + private static boolean isQuotedType(String type) { + return QUOTED_TYPES.contains(type); + } + + public static String replaceDoubleQuoteWithSingleQuotes(String s) { + return s.replaceAll("\"", "\'"); + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/HiveTableValidator.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/HiveTableValidator.java new file mode 100644 index 000000000..380e6e37f --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/HiveTableValidator.java @@ -0,0 +1,67 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.InvalidInputException; +import com.amazonaws.services.glue.model.Table; + +import org.apache.hadoop.hive.metastore.TableType; + +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE; + +public enum HiveTableValidator { + + REQUIRED_PROPERTIES_VALIDATOR { + public void validate(Table table) { + String missingProperty = null; + + if(notApplicableTableType(table)) { + return; + } + + if (table.getTableType() == null) { + missingProperty = "TableType"; + } else if (table.getStorageDescriptor() == null) { + missingProperty = "StorageDescriptor"; + } else if (table.getStorageDescriptor().getInputFormat() == null) { + missingProperty = "StorageDescriptor#InputFormat"; + } else if (table.getStorageDescriptor().getOutputFormat() == null) { + missingProperty = "StorageDescriptor#OutputFormat"; + } else if (table.getStorageDescriptor().getSerdeInfo() == null) { + missingProperty = "StorageDescriptor#SerdeInfo"; + } else if (table.getStorageDescriptor().getSerdeInfo().getSerializationLibrary() == null) { + missingProperty = "StorageDescriptor#SerdeInfo#SerializationLibrary"; + } + + if (missingProperty != null) { + throw new InvalidInputException(String.format("%s cannot be null for table: %s", missingProperty, table.getName())); + } + } + }; + + public abstract void validate(Table table); + + private static boolean notApplicableTableType(Table table) { + if (isNotManagedOrExternalTable(table) || + isStorageHandlerType(table)) { + return true; + } + return false; + } + + private static boolean isNotManagedOrExternalTable(Table table) { + if (table.getTableType() != null && + TableType.valueOf(table.getTableType()) != TableType.MANAGED_TABLE && + TableType.valueOf(table.getTableType()) != TableType.EXTERNAL_TABLE) { + return true; + } + return false; + } + + private static boolean isStorageHandlerType(Table table) { + if (table.getParameters() != null && table.getParameters().containsKey(META_TABLE_STORAGE) && + isNotEmpty(table.getParameters().get(META_TABLE_STORAGE))) { + return true; + } + return false; + } +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/LoggingHelper.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/LoggingHelper.java new file mode 100644 index 000000000..80577e15b --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/LoggingHelper.java @@ -0,0 +1,36 @@ +package com.amazonaws.glue.catalog.util; + +import java.util.Collection; + +public class LoggingHelper { + + private static final int MAX_LOG_STRING_LEN = 2000; + + private LoggingHelper() { + } + + public static String concatCollectionToStringForLogging(Collection collection, String delimiter) { + if (collection == null) { + return ""; + } + if (delimiter == null) { + delimiter = ","; + } + StringBuilder bldr = new StringBuilder(); + int totalLen = 0; + int delimiterSize = delimiter.length(); + for (String str : collection) { + if (totalLen > MAX_LOG_STRING_LEN) break; + if (str.length() + totalLen > MAX_LOG_STRING_LEN) { + bldr.append(str.subSequence(0, (MAX_LOG_STRING_LEN-totalLen))); + break; + } else { + bldr.append(str); + bldr.append(delimiter); + totalLen += str.length() + delimiterSize; + } + } + return bldr.toString(); + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/MetastoreClientUtils.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/MetastoreClientUtils.java new file mode 100644 index 000000000..db6fbf4c7 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/MetastoreClientUtils.java @@ -0,0 +1,124 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.metastore.GlueMetastoreClientDelegate; +import com.amazonaws.glue.shims.AwsGlueHiveShims; +import com.amazonaws.glue.shims.ShimsLoader; +import com.google.common.collect.Maps; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Table; + +import java.util.Map; + +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; + +public final class MetastoreClientUtils { + + private static final AwsGlueHiveShims hiveShims = ShimsLoader.getHiveShims(); + + private MetastoreClientUtils() { + // static util class should not be instantiated + } + + /** + * @return boolean + * true -> if directory was able to be created. + * false -> if directory already exists. + * @throws MetaException if directory could not be created. + */ + public static boolean makeDirs(Warehouse wh, Path path) throws MetaException { + checkNotNull(wh, "Warehouse cannot be null"); + checkNotNull(path, "Path cannot be null"); + + boolean madeDir = false; + if (!wh.isDir(path)) { + if (!wh.mkdirs(path, true)) { + throw new MetaException("Unable to create path: " + path); + } + madeDir = true; + } + return madeDir; + } + + /** + * Taken from HiveMetaStore#create_table_core + * https://github.com/apache/hive/blob/rel/release-2.3.0/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java#L1370-L1383 + */ + public static void validateTableObject(Table table, Configuration conf) throws InvalidObjectException { + checkNotNull(table, "table cannot be null"); + checkNotNull(table.getSd(), "Table#StorageDescriptor cannot be null"); + + if (!hiveShims.validateTableName(table.getTableName(), conf)) { + throw new InvalidObjectException(table.getTableName() + " is not a valid object name"); + } + String validate = MetaStoreUtils.validateTblColumns(table.getSd().getCols()); + if (validate != null) { + throw new InvalidObjectException("Invalid column " + validate); + } + + if (table.getPartitionKeys() != null) { + validate = MetaStoreUtils.validateTblColumns(table.getPartitionKeys()); + if (validate != null) { + throw new InvalidObjectException("Invalid partition column " + validate); + } + } + } + + /** + * Should be used when getting table from Glue that may have been created by + * users manually or through Crawlers. Validates that table contains properties required by Hive/Spark. + * @param table + */ + public static void validateGlueTable(com.amazonaws.services.glue.model.Table table) { + checkNotNull(table, "table cannot be null"); + + for (HiveTableValidator validator : HiveTableValidator.values()) { + validator.validate(table); + } + } + + public static Map deepCopyMap(Map originalMap) { + Map deepCopy = Maps.newHashMap(); + if (originalMap == null) { + return deepCopy; + } + + for (Map.Entry entry : originalMap.entrySet()) { + deepCopy.put(entry.getKey(), entry.getValue()); + } + return deepCopy; + } + + /** + * Mimics MetaStoreUtils.isExternalTable + * Additional logic: check Table#getTableType to see if isExternalTable + */ + public static boolean isExternalTable(org.apache.hadoop.hive.metastore.api.Table table) { + if (table == null) { + return false; + } + + Map params = table.getParameters(); + String paramsExternalStr = params == null ? null : params.get("EXTERNAL"); + if (paramsExternalStr != null) { + return "TRUE".equalsIgnoreCase(paramsExternalStr); + } + + return table.getTableType() != null && EXTERNAL_TABLE.name().equalsIgnoreCase(table.getTableType()); + } + + public static String getCatalogId(Configuration conf) { + if (StringUtils.isNotEmpty(conf.get(GlueMetastoreClientDelegate.CATALOG_ID_CONF))) { + return conf.get(GlueMetastoreClientDelegate.CATALOG_ID_CONF); + } + // This case defaults to using the caller's account Id as Catalog Id. + return null; + } +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionKey.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionKey.java new file mode 100644 index 000000000..f807e8f8a --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionKey.java @@ -0,0 +1,39 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.Partition; + +import java.util.List; + +public class PartitionKey { + + private final List partitionValues; + private final int hashCode; + + public PartitionKey(Partition partition) { + this(partition.getValues()); + } + + public PartitionKey(List partitionValues) { + if (partitionValues == null) { + throw new IllegalArgumentException("Partition values cannot be null"); + } + this.partitionValues = partitionValues; + this.hashCode = partitionValues.hashCode(); + } + + @Override + public boolean equals(Object other) { + return this == other || (other != null && other instanceof PartitionKey + && this.partitionValues.equals(((PartitionKey) other).partitionValues)); + } + + @Override + public int hashCode() { + return hashCode; + } + + List getValues() { + return partitionValues; + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionUtils.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionUtils.java new file mode 100644 index 000000000..40475c946 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/catalog/util/PartitionUtils.java @@ -0,0 +1,36 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.InvalidInputException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionValueList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +public final class PartitionUtils { + + public static Map buildPartitionMap(final List partitions) { + Map partitionValuesMap = Maps.newHashMap(); + for (Partition partition : partitions) { + partitionValuesMap.put(new PartitionKey(partition), partition); + } + return partitionValuesMap; + } + + public static List getPartitionValuesList(final Map partitionMap) { + List partitionValuesList = Lists.newArrayList(); + for (Map.Entry entry : partitionMap.entrySet()) { + partitionValuesList.add(new PartitionValueList().withValues(entry.getValue().getValues())); + } + return partitionValuesList; + } + + public static boolean isInvalidUserInputException(Exception e) { + // exceptions caused by invalid requests, in which case we know all partitions creation failed + return e instanceof EntityNotFoundException || e instanceof InvalidInputException; + } + +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHive2Shims.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHive2Shims.java new file mode 100644 index 000000000..4cbb78f03 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHive2Shims.java @@ -0,0 +1,67 @@ +package com.amazonaws.glue.shims; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; + +final class AwsGlueHive2Shims + implements AwsGlueHiveShims { + + private static final String HIVE_2_VERSION = "2."; + + static boolean supportsVersion(String version) { + return version.startsWith(HIVE_2_VERSION); + } + + @Override + public ExprNodeGenericFuncDesc getDeserializeExpression(byte[] exprBytes) { + return SerializationUtilities.deserializeExpressionFromKryo(exprBytes); + } + + @Override + public byte[] getSerializeExpression(ExprNodeGenericFuncDesc expr) { + return SerializationUtilities.serializeExpressionToKryo(expr); + } + + @Override + public Path getDefaultTablePath(Database db, String tableName, Warehouse warehouse) throws MetaException { + return warehouse.getDefaultTablePath(db, tableName); + } + + @Override + public boolean validateTableName(String name, Configuration conf) { + return MetaStoreUtils.validateName(name, conf); + } + + @Override + public boolean requireCalStats( + Configuration conf, + Partition oldPart, + Partition newPart, + Table tbl, + EnvironmentContext environmentContext) { + return MetaStoreUtils.requireCalStats(conf, oldPart, newPart, tbl, environmentContext); + } + + @Override + public boolean updateTableStatsFast( + Database db, + Table tbl, + Warehouse wh, + boolean madeDir, + boolean forceRecompute, + EnvironmentContext environmentContext + ) throws MetaException { + return MetaStoreUtils.updateTableStatsFast(db, tbl, wh, madeDir, forceRecompute, environmentContext); + } + +} + diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHiveShims.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHiveShims.java new file mode 100644 index 000000000..cdfd1c362 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/AwsGlueHiveShims.java @@ -0,0 +1,29 @@ +package com.amazonaws.glue.shims; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; + +public interface AwsGlueHiveShims +{ + + ExprNodeGenericFuncDesc getDeserializeExpression(byte[] exprBytes); + + byte[] getSerializeExpression(ExprNodeGenericFuncDesc expr); + + Path getDefaultTablePath(Database db, String tableName, Warehouse warehouse) + throws MetaException; + + boolean validateTableName(String name, Configuration conf); + + boolean requireCalStats(Configuration conf, Partition oldPart, Partition newPart, Table tbl, EnvironmentContext environmentContext); + + boolean updateTableStatsFast(Database db, Table tbl, Warehouse wh, boolean madeDir, boolean forceRecompute, EnvironmentContext environmentContext) + throws MetaException; +} diff --git a/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/ShimsLoader.java b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/ShimsLoader.java new file mode 100644 index 000000000..99c300a38 --- /dev/null +++ b/waggle-dance-glue/src/main/java/com/amazonaws/glue/shims/ShimsLoader.java @@ -0,0 +1,36 @@ +package com.amazonaws.glue.shims; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.hive.common.util.HiveVersionInfo; + +public final class ShimsLoader +{ + + private static AwsGlueHiveShims hiveShims; + + public static synchronized AwsGlueHiveShims getHiveShims() { + if (hiveShims == null) { + hiveShims = loadHiveShims(); + } + return hiveShims; + } + + private static AwsGlueHiveShims loadHiveShims() { + String hiveVersion = HiveVersionInfo.getShortVersion(); + if (AwsGlueHive2Shims.supportsVersion(hiveVersion)) { + try { + return AwsGlueHive2Shims.class.newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new RuntimeException("unable to get instance of Hive 2.x shim class"); + } + } else { + throw new RuntimeException("Shim class for Hive version " + hiveVersion + " does not exist"); + } + } + + @VisibleForTesting + static synchronized void clearShimClass() { + hiveShims = null; + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/ConverterUtilsTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/ConverterUtilsTest.java new file mode 100644 index 000000000..94f6a65f7 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/ConverterUtilsTest.java @@ -0,0 +1,19 @@ +package com.amazonaws.glue.catalog.convertors; + +import com.amazonaws.glue.catalog.converters.ConverterUtils; +import com.amazonaws.glue.catalog.util.TestObjects; +import com.amazonaws.services.glue.model.Table; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class ConverterUtilsTest { + + @Test + public void testCoralTableToStringConversion() { + Table table = TestObjects.getTestTable(); + assertEquals(table, ConverterUtils.stringToCatalogTable(ConverterUtils.catalogTableToString(table))); + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/EntityConversionTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/EntityConversionTest.java new file mode 100644 index 000000000..97e6ea723 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/EntityConversionTest.java @@ -0,0 +1,203 @@ +package com.amazonaws.glue.catalog.convertors; + +import com.amazonaws.glue.catalog.converters.HiveToCatalogConverter; +import com.amazonaws.glue.catalog.util.TestObjects; +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.services.glue.model.AlreadyExistsException; +import com.amazonaws.services.glue.model.Database; +import com.amazonaws.services.glue.model.Order; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.SerDeInfo; +import com.amazonaws.services.glue.model.SkewedInfo; +import com.amazonaws.services.glue.model.Table; +import com.amazonaws.services.glue.model.UserDefinedFunction; + +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.hive.metastore.api.TableMeta; +import org.junit.Test; + +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +public class EntityConversionTest { + + private static final String TEST_DB_NAME = "testDb"; + private static final String TEST_TBL_NAME = "testTbl"; + + @Test + public void testDatabaseConversion() { + Database catalogDb = TestObjects.getTestDatabase(); + org.apache.hadoop.hive.metastore.api.Database hiveDatabase = CatalogToHiveConverter + .convertDatabase(catalogDb); + Database catalogDb2 = HiveToCatalogConverter.convertDatabase(hiveDatabase); + assertEquals(catalogDb, catalogDb2); + } + + @Test + public void testDatabaseConversionWithNullFields() { + Database catalogDb = TestObjects.getTestDatabase(); + catalogDb.setLocationUri(null); + catalogDb.setParameters(null); + org.apache.hadoop.hive.metastore.api.Database hiveDatabase = CatalogToHiveConverter + .convertDatabase(catalogDb); + assertThat(hiveDatabase.getLocationUri(), is("")); + assertNotNull(hiveDatabase.getParameters()); + } + + @Test + public void testExceptionTranslation() { + assertEquals("org.apache.hadoop.hive.metastore.api.AlreadyExistsException", + CatalogToHiveConverter.wrapInHiveException(new AlreadyExistsException("")).getClass().getName()); + } + + @Test + public void testTableConversion() { + Table catalogTable = TestObjects.getTestTable(); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertEquals(catalogTable, HiveToCatalogConverter.convertTable(hiveTable)); + } + + @Test + public void testTableConversionWithNullParameterMap() { + // Test to ensure the parameter map returned to Hive is never null. + Table catalogTable = TestObjects.getTestTable(); + catalogTable.setParameters(null); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertNotNull(hiveTable.getParameters()); + assertTrue(hiveTable.getParameters().isEmpty()); + } + + @Test + public void testPartitionConversion() { + Partition partition = TestObjects.getTestPartition(TEST_DB_NAME, TEST_TBL_NAME, ImmutableList.of("1")); + org.apache.hadoop.hive.metastore.api.Partition hivePartition = CatalogToHiveConverter.convertPartition(partition); + Partition converted = HiveToCatalogConverter.convertPartition(hivePartition); + assertEquals(partition, converted); + } + + @Test + public void testPartitionConversionWithNullParameterMap() { + // Test to ensure the parameter map returned to Hive is never null. + Partition partition = TestObjects.getTestPartition(TEST_DB_NAME, TEST_TBL_NAME, ImmutableList.of("1")); + partition.setParameters(null); + org.apache.hadoop.hive.metastore.api.Partition hivePartition = CatalogToHiveConverter.convertPartition(partition); + assertNotNull(hivePartition.getParameters()); + assertTrue(hivePartition.getParameters().isEmpty()); + } + + @Test + public void testConvertPartitions() { + Partition partition = TestObjects.getTestPartition(TEST_DB_NAME, TEST_TBL_NAME, ImmutableList.of("value1", "value2")); + org.apache.hadoop.hive.metastore.api.Partition hivePartition = CatalogToHiveConverter.convertPartition(partition); + List partitions = ImmutableList.of(partition); + assertEquals(ImmutableList.of(hivePartition), CatalogToHiveConverter.convertPartitions(partitions)); + } + + @Test + public void testConvertPartitionsEmpty() { + assertEquals(ImmutableList.of(), CatalogToHiveConverter.convertPartitions(ImmutableList.of())); + } + + @Test + public void testConvertPartitionsNull() { + assertEquals(null, CatalogToHiveConverter.convertPartitions(null)); + } + + @Test + public void testSkewedInfoConversion() { + SkewedInfo catalogSkewedInfo = TestObjects.getSkewedInfo(); + org.apache.hadoop.hive.metastore.api.SkewedInfo hiveSkewedinfo = CatalogToHiveConverter.convertSkewedInfo(catalogSkewedInfo); + assertEquals(catalogSkewedInfo, HiveToCatalogConverter.convertSkewedInfo(hiveSkewedinfo)); + assertEquals(null, HiveToCatalogConverter.convertSkewedInfo(null)); + assertEquals(null, CatalogToHiveConverter.convertSkewedInfo(null)); + } + + @Test + public void testConvertSkewedInfoNullFields() { + SkewedInfo catalogSkewedInfo = new SkewedInfo(); + org.apache.hadoop.hive.metastore.api.SkewedInfo hiveSkewedInfo = CatalogToHiveConverter.convertSkewedInfo(catalogSkewedInfo); + assertNotNull(hiveSkewedInfo.getSkewedColNames()); + assertNotNull(hiveSkewedInfo.getSkewedColValues()); + assertNotNull(hiveSkewedInfo.getSkewedColValueLocationMaps()); + } + + @Test + public void testConvertSerdeInfoNullParameter() { + SerDeInfo serDeInfo = TestObjects.getTestSerdeInfo(); + serDeInfo.setParameters(null); + assertNotNull(CatalogToHiveConverter.convertSerDeInfo(serDeInfo).getParameters()); + } + + @Test + public void testFunctionConversion() { + UserDefinedFunction catalogFunction = TestObjects.getCatalogTestFunction(); + org.apache.hadoop.hive.metastore.api.Function hiveFunction = CatalogToHiveConverter.convertFunction(TEST_DB_NAME, catalogFunction); + assertEquals(TEST_DB_NAME, hiveFunction.getDbName()); + assertEquals(catalogFunction, HiveToCatalogConverter.convertFunction(hiveFunction)); + } + + @Test + public void testConvertOrderList() { + List hiveOrderList = ImmutableList.of(TestObjects.getTestOrder()); + List catalogOrderList = HiveToCatalogConverter.convertOrderList(hiveOrderList); + + assertEquals(hiveOrderList.get(0).getCol(), catalogOrderList.get(0).getColumn()); + assertEquals(hiveOrderList.get(0).getOrder(), catalogOrderList.get(0).getSortOrder().intValue()); + } + + @Test + public void testConvertOrderListNull() { + assertNull(HiveToCatalogConverter.convertOrderList(null)); + } + + @Test + public void testTableMetaConversion() { + Table catalogTable = TestObjects.getTestTable(); + TableMeta tableMeta = CatalogToHiveConverter.convertTableMeta(catalogTable, TEST_DB_NAME); + assertEquals(catalogTable.getName(), tableMeta.getTableName()); + assertEquals(TEST_DB_NAME, tableMeta.getDbName()); + assertEquals(catalogTable.getTableType(), tableMeta.getTableType()); + } + + @Test + public void testTableConversionStorageDescriptorParameterMapNull() { + Table catalogTable = TestObjects.getTestTable(); + catalogTable.getStorageDescriptor().setParameters(null); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertNotNull(hiveTable.getSd().getParameters()); + assertTrue(hiveTable.getSd().getParameters().isEmpty()); + } + + @Test + public void testTableConversionStorageDescriptorBucketColsNull() { + Table catalogTable = TestObjects.getTestTable(); + catalogTable.getStorageDescriptor().setBucketColumns(null); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertNotNull(hiveTable.getSd().getBucketCols()); + assertTrue(hiveTable.getSd().getBucketCols().isEmpty()); + } + + @Test + public void testTableConversionStorageDescriptorSorColsNull() { + Table catalogTable = TestObjects.getTestTable(); + catalogTable.getStorageDescriptor().setSortColumns(null); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertNotNull(hiveTable.getSd().getSortCols()); + assertTrue(hiveTable.getSd().getSortCols().isEmpty()); + } + + @Test + public void testTableConversionWithNullPartitionKeys() { + Table catalogTable = TestObjects.getTestTable(); + catalogTable.setPartitionKeys(null); + org.apache.hadoop.hive.metastore.api.Table hiveTable = CatalogToHiveConverter.convertTable(catalogTable, TEST_DB_NAME); + assertNotNull(hiveTable.getPartitionKeys()); + assertTrue(hiveTable.getPartitionKeys().isEmpty()); + } +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/GlueInputConverterTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/GlueInputConverterTest.java new file mode 100644 index 000000000..2056b7f80 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/convertors/GlueInputConverterTest.java @@ -0,0 +1,132 @@ +package com.amazonaws.glue.catalog.convertors; + +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.glue.catalog.converters.GlueInputConverter; +import com.amazonaws.glue.catalog.util.TestObjects; +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.PartitionInput; +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 org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class GlueInputConverterTest { + + private Database testDB; + private Table testTable; + private Partition testPartition; + private UserDefinedFunction testFunction; + + @Before + public void setup() { + testDB = TestObjects.getTestDatabase(); + testTable = TestObjects.getTestTable(); + testPartition = TestObjects.getTestPartition(testDB.getName(), testTable.getName(), Lists.newArrayList("val1")); + testFunction = TestObjects.getCatalogTestFunction(); + } + + @Test + public void testConvertHiveDbToDatabaseInput() { + org.apache.hadoop.hive.metastore.api.Database hivedb = CatalogToHiveConverter.convertDatabase(testDB); + DatabaseInput dbInput = GlueInputConverter.convertToDatabaseInput(hivedb); + + assertEquals(testDB.getName(), dbInput.getName()); + assertEquals(testDB.getDescription(), dbInput.getDescription()); + assertEquals(testDB.getLocationUri(), dbInput.getLocationUri()); + assertEquals(testDB.getParameters(), dbInput.getParameters()); + } + + @Test + public void testConvertCatalogDbToDatabaseInput() { + DatabaseInput dbInput = GlueInputConverter.convertToDatabaseInput(testDB); + + assertEquals(testDB.getName(), dbInput.getName()); + assertEquals(testDB.getDescription(), dbInput.getDescription()); + assertEquals(testDB.getLocationUri(), dbInput.getLocationUri()); + assertEquals(testDB.getParameters(), dbInput.getParameters()); + } + + @Test + public void testConvertHiveTableToTableInput() { + org.apache.hadoop.hive.metastore.api.Table hivetbl = CatalogToHiveConverter.convertTable(testTable, testDB.getName()); + TableInput tblInput = GlueInputConverter.convertToTableInput(hivetbl); + + assertEquals(testTable.getName(), tblInput.getName()); + assertEquals(testTable.getOwner(), tblInput.getOwner()); + assertEquals(testTable.getTableType(), tblInput.getTableType()); + assertEquals(testTable.getParameters(), tblInput.getParameters()); + assertEquals(testTable.getPartitionKeys(), tblInput.getPartitionKeys()); + assertEquals(testTable.getRetention(), tblInput.getRetention()); + assertEquals(testTable.getLastAccessTime(), tblInput.getLastAccessTime()); + assertEquals(testTable.getStorageDescriptor(), tblInput.getStorageDescriptor()); + assertEquals(testTable.getViewExpandedText(), tblInput.getViewExpandedText()); + assertEquals(testTable.getViewOriginalText(), tblInput.getViewOriginalText()); + } + + @Test + public void testConvertCatalogTableToTableInput() { + TableInput tblInput = GlueInputConverter.convertToTableInput(testTable); + + assertEquals(testTable.getName(), tblInput.getName()); + assertEquals(testTable.getOwner(), tblInput.getOwner()); + assertEquals(testTable.getTableType(), tblInput.getTableType()); + assertEquals(testTable.getParameters(), tblInput.getParameters()); + assertEquals(testTable.getPartitionKeys(), tblInput.getPartitionKeys()); + assertEquals(testTable.getRetention(), tblInput.getRetention()); + assertEquals(testTable.getLastAccessTime(), tblInput.getLastAccessTime()); + assertEquals(testTable.getStorageDescriptor(), tblInput.getStorageDescriptor()); + assertEquals(testTable.getViewExpandedText(), tblInput.getViewExpandedText()); + assertEquals(testTable.getViewOriginalText(), tblInput.getViewOriginalText()); + } + + @Test + public void testConvertHivePartitionToPartitionInput() { + org.apache.hadoop.hive.metastore.api.Partition hivePartition = CatalogToHiveConverter.convertPartition(testPartition); + PartitionInput partitionInput = GlueInputConverter.convertToPartitionInput(hivePartition); + + assertEquals(testPartition.getLastAccessTime(), partitionInput.getLastAccessTime()); + assertEquals(testPartition.getParameters(), partitionInput.getParameters()); + assertEquals(testPartition.getStorageDescriptor(), partitionInput.getStorageDescriptor()); + assertEquals(testPartition.getValues(), partitionInput.getValues()); + } + + @Test + public void testConvertCatalogPartitionToPartitionInput() { + PartitionInput partitionInput = GlueInputConverter.convertToPartitionInput(testPartition); + + assertEquals(testPartition.getLastAccessTime(), partitionInput.getLastAccessTime()); + assertEquals(testPartition.getParameters(), partitionInput.getParameters()); + assertEquals(testPartition.getStorageDescriptor(), partitionInput.getStorageDescriptor()); + assertEquals(testPartition.getValues(), partitionInput.getValues()); + } + + @Test + public void testConvertHiveFunctionToFunctionInput() { + org.apache.hadoop.hive.metastore.api.Function hiveFunction = + CatalogToHiveConverter.convertFunction(testDB.getName(), testFunction); + UserDefinedFunctionInput functionInput = GlueInputConverter.convertToUserDefinedFunctionInput(hiveFunction); + + assertEquals(testFunction.getClassName(), functionInput.getClassName()); + assertEquals(testFunction.getFunctionName(), functionInput.getFunctionName()); + assertEquals(testFunction.getOwnerName(), functionInput.getOwnerName()); + assertEquals(testFunction.getOwnerType(), functionInput.getOwnerType()); + assertEquals(testFunction.getResourceUris(), functionInput.getResourceUris()); + } + + @Test + public void testConvertHiveFunctionToFunctionInputNullOwnerType() { + org.apache.hadoop.hive.metastore.api.Function hiveFunction = + CatalogToHiveConverter.convertFunction(testDB.getName(), testFunction); + hiveFunction.setOwnerType(null); + GlueInputConverter.convertToUserDefinedFunctionInput(hiveFunction); + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactoryTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactoryTest.java new file mode 100644 index 000000000..3b5578c37 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactoryTest.java @@ -0,0 +1,146 @@ +package com.amazonaws.glue.catalog.metastore; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicSessionCredentials; +import com.amazonaws.services.glue.AWSGlue; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Before; +import org.junit.Test; + +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; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class AWSGlueClientFactoryTest { + + private static final String FAKE_ACCESS_KEY = "accessKey"; + private static final String FAKE_SECRET_KEY = "secretKey"; + private static final String FAKE_SESSION_TOKEN = "sessionToken"; + + private AWSGlueClientFactory glueClientFactory; + private HiveConf hiveConf; + + @Before + public void setup() { + hiveConf = spy(new HiveConf()); + glueClientFactory = new AWSGlueClientFactory(hiveConf); + } + + @Test + public void testGlueClientConstructionWithHiveConfig() throws Exception { + System.setProperty(AWS_REGION, ""); + System.setProperty(AWS_GLUE_ENDPOINT, ""); + when(hiveConf.get(AWS_GLUE_ENDPOINT)).thenReturn("endpoint"); + when(hiveConf.get(AWS_REGION)).thenReturn("us-west-1"); + + AWSGlue glueClient = glueClientFactory.newClient(); + + assertNotNull(glueClient); + + // client reads hive conf for region & endpoint + verify(hiveConf, atLeastOnce()).get(AWS_GLUE_ENDPOINT); + verify(hiveConf, atLeastOnce()).get(AWS_REGION); + } + + @Test + public void testGlueClientContructionWithAWSConfig() throws Exception { + glueClientFactory.newClient(); + verify(hiveConf, atLeastOnce()).getInt(AWS_GLUE_MAX_RETRY, DEFAULT_MAX_RETRY); + verify(hiveConf, atLeastOnce()).getInt(AWS_GLUE_MAX_CONNECTIONS, DEFAULT_MAX_CONNECTIONS); + verify(hiveConf, atLeastOnce()).getInt(AWS_GLUE_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); + verify(hiveConf, atLeastOnce()).getInt(AWS_GLUE_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT); + } + + @Test + public void testGlueClientConstructionWithSystemProperty() throws Exception { + System.setProperty(AWS_REGION, "us-east-1"); + System.setProperty(AWS_GLUE_ENDPOINT, "endpoint"); + + AWSGlue glueClient = glueClientFactory.newClient(); + + assertNotNull(glueClient); + + // client has no interactions with the hive conf since system property is set + verify(hiveConf, never()).get(AWS_GLUE_ENDPOINT); + verify(hiveConf, never()).get(AWS_REGION); + } + + @Test + public void testClientConstructionWithSessionCredentialsProviderFactory() throws Exception { + System.setProperty("aws.region", "us-west-2"); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_ACCESS_KEY_CONF_VAR, FAKE_ACCESS_KEY); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_SECRET_KEY_CONF_VAR, FAKE_SECRET_KEY); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_SESSION_TOKEN_CONF_VAR, FAKE_SESSION_TOKEN); + + hiveConf.setStrings(AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS, + SessionCredentialsProviderFactory.class.getCanonicalName()); + + AWSGlue glueClient = glueClientFactory.newClient(); + + assertNotNull(glueClient); + + verify(hiveConf, atLeastOnce()).get(SessionCredentialsProviderFactory.AWS_ACCESS_KEY_CONF_VAR); + verify(hiveConf, atLeastOnce()).get(SessionCredentialsProviderFactory.AWS_SECRET_KEY_CONF_VAR); + verify(hiveConf, atLeastOnce()).get(SessionCredentialsProviderFactory.AWS_SESSION_TOKEN_CONF_VAR); + } + + @Test + public void testCredentialsCreatedBySessionCredentialsProviderFactory() throws Exception { + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_ACCESS_KEY_CONF_VAR, FAKE_ACCESS_KEY); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_SECRET_KEY_CONF_VAR, FAKE_SECRET_KEY); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_SESSION_TOKEN_CONF_VAR, FAKE_SESSION_TOKEN); + + SessionCredentialsProviderFactory factory = new SessionCredentialsProviderFactory(); + AWSCredentialsProvider provider = factory.buildAWSCredentialsProvider(hiveConf); + AWSCredentials credentials = provider.getCredentials(); + + assertThat(credentials, instanceOf(BasicSessionCredentials.class)); + + BasicSessionCredentials sessionCredentials = (BasicSessionCredentials) credentials; + + assertEquals(FAKE_ACCESS_KEY, sessionCredentials.getAWSAccessKeyId()); + assertEquals(FAKE_SECRET_KEY, sessionCredentials.getAWSSecretKey()); + assertEquals(FAKE_SESSION_TOKEN, sessionCredentials.getSessionToken()); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingAccessKeyWithSessionCredentialsProviderFactory() throws Exception { + SessionCredentialsProviderFactory factory = new SessionCredentialsProviderFactory(); + factory.buildAWSCredentialsProvider(hiveConf); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingSecretKey() throws Exception { + SessionCredentialsProviderFactory factory = new SessionCredentialsProviderFactory(); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_ACCESS_KEY_CONF_VAR, FAKE_ACCESS_KEY); + factory.buildAWSCredentialsProvider(hiveConf); + } + + @Test(expected = IllegalArgumentException.class) + public void testMissingSessionTokenKey() throws Exception { + SessionCredentialsProviderFactory factory = new SessionCredentialsProviderFactory(); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_ACCESS_KEY_CONF_VAR, FAKE_ACCESS_KEY); + hiveConf.setStrings(SessionCredentialsProviderFactory.AWS_SECRET_KEY_CONF_VAR, FAKE_SECRET_KEY); + factory.buildAWSCredentialsProvider(hiveConf); + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecoratorTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecoratorTest.java new file mode 100644 index 000000000..5347163e6 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecoratorTest.java @@ -0,0 +1,292 @@ +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.cache.Cache; +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.junit.Assert.assertTrue; + +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; +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 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; + +public class AWSGlueMetastoreCacheDecoratorTest { + + private AWSGlueMetastore glueMetastore; + private HiveConf hiveConf; + + private static final String DB_NAME = "db"; + private static final String TABLE_NAME = "table"; + private static final AWSGlueMetastoreCacheDecorator.TableIdentifier TABLE_IDENTIFIER = + new AWSGlueMetastoreCacheDecorator.TableIdentifier(DB_NAME, TABLE_NAME); + + @Before + public void setUp() { + glueMetastore = mock(AWSGlueMetastore.class); + hiveConf = spy(new HiveConf()); + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(true); + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(true); + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_SIZE, 0)).thenReturn(100); + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_TTL_MINS, 0)).thenReturn(100); + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_SIZE, 0)).thenReturn(100); + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_TTL_MINS, 0)).thenReturn(100); + + } + + @Test(expected = NullPointerException.class) + public void testConstructorWithNullConf() { + new AWSGlueMetastoreCacheDecorator(null, glueMetastore); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorWithInvalidTableCacheSize() { + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_SIZE, 0)).thenReturn(0); + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorWithInvalidTableCacheTtl() { + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_TTL_MINS, 0)).thenReturn(0); + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorWithInvalidDbCacheSize() { + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_SIZE, 0)).thenReturn(0); + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorWithInvalidDbCacheTtl() { + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_TTL_MINS, 0)).thenReturn(0); + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + } + + @Test + public void testGetDatabaseWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(false); + Database db = new Database(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + when(glueMetastore.getDatabase(DB_NAME)).thenReturn(db); + assertEquals(db, cacheDecorator.getDatabase(DB_NAME)); + assertNull(cacheDecorator.databaseCache); + verify(glueMetastore, times(1)).getDatabase(DB_NAME); + } + + @Test + public void testGetDatabaseWhenCacheEnabledAndCacheMiss() { + Database db = new Database(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + assertNotNull(cacheDecorator.databaseCache); + Cache dbCache = mock(Cache.class); + cacheDecorator.databaseCache = dbCache; + + when(dbCache.getIfPresent(DB_NAME)).thenReturn(null); + when(glueMetastore.getDatabase(DB_NAME)).thenReturn(db); + doNothing().when(dbCache).put(DB_NAME, db); + + assertEquals(db, cacheDecorator.getDatabase(DB_NAME)); + + verify(glueMetastore, times(1)).getDatabase(DB_NAME); + verify(dbCache, times(1)).getIfPresent(DB_NAME); + verify(dbCache, times(1)).put(DB_NAME, db); + } + + @Test + public void testGetDatabaseWhenCacheEnabledAndCacheHit() { + Database db = new Database(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + assertNotNull(cacheDecorator.databaseCache); + Cache dbCache = mock(Cache.class); + cacheDecorator.databaseCache = dbCache; + + when(dbCache.getIfPresent(DB_NAME)).thenReturn(db); + + assertEquals(db, cacheDecorator.getDatabase(DB_NAME)); + + verify(dbCache, times(1)).getIfPresent(DB_NAME); + } + + @Test + public void testUpdateDatabaseWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(false); + DatabaseInput dbInput = new DatabaseInput(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + doNothing().when(glueMetastore).updateDatabase(DB_NAME, dbInput); + cacheDecorator.updateDatabase(DB_NAME, dbInput); + assertNull(cacheDecorator.databaseCache); + verify(glueMetastore, times(1)).updateDatabase(DB_NAME, dbInput); + } + + @Test + public void testUpdateDatabaseWhenCacheEnabled() { + DatabaseInput dbInput = new DatabaseInput(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + cacheDecorator.databaseCache.put(DB_NAME, new Database()); + doNothing().when(glueMetastore).updateDatabase(DB_NAME, dbInput); + + cacheDecorator.updateDatabase(DB_NAME, dbInput); + + //db should have been removed from cache + assertNull(cacheDecorator.databaseCache.getIfPresent(DB_NAME)); + verify(glueMetastore, times(1)).updateDatabase(DB_NAME, dbInput); + } + + @Test + public void testDeleteDatabaseWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(false); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + doNothing().when(glueMetastore).deleteDatabase(DB_NAME); + cacheDecorator.deleteDatabase(DB_NAME); + assertNull(cacheDecorator.databaseCache); + verify(glueMetastore, times(1)).deleteDatabase(DB_NAME); + } + + @Test + public void testDeleteDatabaseWhenCacheEnabled() { + DatabaseInput dbInput = new DatabaseInput(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + cacheDecorator.databaseCache.put(DB_NAME, new Database()); + doNothing().when(glueMetastore).deleteDatabase(DB_NAME); + + cacheDecorator.deleteDatabase(DB_NAME); + + //db should have been removed from cache + assertNull(cacheDecorator.databaseCache.getIfPresent(DB_NAME)); + verify(glueMetastore, times(1)).deleteDatabase(DB_NAME); + } + + @Test + public void testGetTableWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(false); + Table table = new Table(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + when(glueMetastore.getTable(DB_NAME, TABLE_NAME)).thenReturn(table); + assertEquals(table, cacheDecorator.getTable(DB_NAME, TABLE_NAME)); + assertNull(cacheDecorator.tableCache); + verify(glueMetastore, times(1)).getTable(DB_NAME, TABLE_NAME); + } + + @Test + public void testGetTableWhenCacheEnabledAndCacheMiss() { + Table table = new Table(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + assertNotNull(cacheDecorator.tableCache); + Cache tableCache = mock(Cache.class); + cacheDecorator.tableCache = tableCache; + + when(tableCache.getIfPresent(TABLE_IDENTIFIER)).thenReturn(null); + when(glueMetastore.getTable(DB_NAME, TABLE_NAME)).thenReturn(table); + doNothing().when(tableCache).put(TABLE_IDENTIFIER, table); + + assertEquals(table, cacheDecorator.getTable(DB_NAME, TABLE_NAME)); + + verify(glueMetastore, times(1)).getTable(DB_NAME, TABLE_NAME); + verify(tableCache, times(1)).getIfPresent(TABLE_IDENTIFIER); + verify(tableCache, times(1)).put(TABLE_IDENTIFIER, table); + } + + @Test + public void testGetTableWhenCacheEnabledAndCacheHit() { + Table table = new Table(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + assertNotNull(cacheDecorator.tableCache); + Cache tableCache = mock(Cache.class); + cacheDecorator.tableCache = tableCache; + + when(tableCache.getIfPresent(TABLE_IDENTIFIER)).thenReturn(table); + + assertEquals(table, cacheDecorator.getTable(DB_NAME, TABLE_NAME)); + + verify(tableCache, times(1)).getIfPresent(TABLE_IDENTIFIER); + } + + @Test + public void testUpdateTableWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(false); + TableInput tableInput = new TableInput(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + doNothing().when(glueMetastore).updateTable(TABLE_NAME, tableInput); + cacheDecorator.updateTable(TABLE_NAME, tableInput); + assertNull(cacheDecorator.tableCache); + verify(glueMetastore, times(1)).updateTable(TABLE_NAME, tableInput); + } + + @Test + public void testUpdateTableWhenCacheEnabled() { + TableInput tableInput = new TableInput(); + tableInput.setName(TABLE_NAME); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + + cacheDecorator.tableCache.put(TABLE_IDENTIFIER, new Table()); + doNothing().when(glueMetastore).updateTable(DB_NAME, tableInput); + + cacheDecorator.updateTable(DB_NAME, tableInput); + + //table should have been removed from cache + assertNull(cacheDecorator.tableCache.getIfPresent(TABLE_IDENTIFIER)); + verify(glueMetastore, times(1)).updateTable(DB_NAME, tableInput); + } + + @Test + public void testDeleteTableWhenCacheDisabled() { + //disable cache + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(false); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + doNothing().when(glueMetastore).deleteTable(DB_NAME, TABLE_NAME); + cacheDecorator.deleteTable(DB_NAME, TABLE_NAME); + assertNull(cacheDecorator.tableCache); + verify(glueMetastore, times(1)).deleteTable(DB_NAME, TABLE_NAME); + } + + @Test + public void testDeleteTableWhenCacheEnabled() { + DatabaseInput dbInput = new DatabaseInput(); + AWSGlueMetastoreCacheDecorator cacheDecorator = + new AWSGlueMetastoreCacheDecorator(hiveConf, glueMetastore); + cacheDecorator.tableCache.put(TABLE_IDENTIFIER, new Table()); + doNothing().when(glueMetastore).deleteDatabase(DB_NAME); + + cacheDecorator.deleteTable(DB_NAME, TABLE_NAME); + + //table should have been removed from cache + assertNull(cacheDecorator.tableCache.getIfPresent(TABLE_IDENTIFIER)); + verify(glueMetastore, times(1)).deleteTable(DB_NAME, TABLE_NAME); + } + +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactoryTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactoryTest.java new file mode 100644 index 000000000..89f4de3ba --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactoryTest.java @@ -0,0 +1,84 @@ +package com.amazonaws.glue.catalog.metastore; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Before; +import org.junit.Test; + +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; +import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_ENDPOINT; +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 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_REGION; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.junit.Assert.assertTrue; + +public class AWSGlueMetastoreFactoryTest { + + private AWSGlueMetastoreFactory awsGlueMetastoreFactory; + private HiveConf hiveConf; + + @Before + public void setUp() { + awsGlueMetastoreFactory = new AWSGlueMetastoreFactory(); + hiveConf = spy(new HiveConf()); + + // these configs are needed for AWSGlueClient to get initialized + System.setProperty(AWS_REGION, ""); + System.setProperty(AWS_GLUE_ENDPOINT, ""); + when(hiveConf.get(AWS_GLUE_ENDPOINT)).thenReturn("endpoint"); + when(hiveConf.get(AWS_REGION)).thenReturn("us-west-1"); + + // these configs are needed for AWSGlueMetastoreCacheDecorator to get initialized + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_SIZE, 0)).thenReturn(1); + when(hiveConf.getInt(AWS_GLUE_DB_CACHE_TTL_MINS, 0)).thenReturn(1); + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_SIZE, 0)).thenReturn(1); + when(hiveConf.getInt(AWS_GLUE_TABLE_CACHE_TTL_MINS, 0)).thenReturn(1); + } + + @Test + public void testNewMetastoreWhenCacheDisabled() throws Exception { + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(false); + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(false); + assertTrue(DefaultAWSGlueMetastore.class.equals( + awsGlueMetastoreFactory.newMetastore(hiveConf).getClass())); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false); + } + + @Test + public void testNewMetastoreWhenTableCacheEnabled() throws Exception { + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(false); + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(true); + assertTrue(AWSGlueMetastoreCacheDecorator.class.equals( + awsGlueMetastoreFactory.newMetastore(hiveConf).getClass())); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false); + } + + @Test + public void testNewMetastoreWhenDBCacheEnabled() throws Exception { + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(true); + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(true); + assertTrue(AWSGlueMetastoreCacheDecorator.class.equals( + awsGlueMetastoreFactory.newMetastore(hiveConf).getClass())); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false); + } + + @Test + public void testNewMetastoreWhenAllCacheEnabled() throws Exception { + when(hiveConf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false)).thenReturn(true); + when(hiveConf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false)).thenReturn(true); + assertTrue(AWSGlueMetastoreCacheDecorator.class.equals( + awsGlueMetastoreFactory.newMetastore(hiveConf).getClass())); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false); + verify(hiveConf, atLeastOnce()).getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false); + } + +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegateTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegateTest.java new file mode 100644 index 000000000..62d19aea6 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegateTest.java @@ -0,0 +1,1667 @@ +package com.amazonaws.glue.catalog.metastore; + +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.glue.catalog.converters.GlueInputConverter; +import com.amazonaws.glue.catalog.util.TestObjects; +import com.amazonaws.glue.catalog.util.TestExecutorServiceFactory; +import com.amazonaws.services.glue.AWSGlue; +import com.amazonaws.services.glue.model.BatchCreatePartitionRequest; +import com.amazonaws.services.glue.model.BatchCreatePartitionResult; +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.DeleteDatabaseRequest; +import com.amazonaws.services.glue.model.DeletePartitionRequest; +import com.amazonaws.services.glue.model.DeletePartitionResult; +import com.amazonaws.services.glue.model.DeleteTableRequest; +import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionRequest; +import com.amazonaws.services.glue.model.EntityNotFoundException; +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.GetPartitionResult; +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.GetUserDefinedFunctionResult; +import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest; +import com.amazonaws.services.glue.model.GetUserDefinedFunctionsResult; +import com.amazonaws.services.glue.model.InternalServiceException; +import com.amazonaws.services.glue.model.InvalidInputException; +import com.amazonaws.services.glue.model.OperationTimeoutException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionInput; +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.UpdatePartitionResult; +import com.amazonaws.services.glue.model.UpdateTableRequest; +import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionRequest; +import com.amazonaws.services.glue.model.UserDefinedFunction; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import com.google.common.collect.Sets; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.Warehouse; +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.PrincipalType; +import org.apache.hadoop.hive.metastore.api.TableMeta; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.api.FieldSchema; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.Set; + +import static com.amazonaws.glue.catalog.util.TestObjects.getTestDatabase; +import static com.amazonaws.glue.catalog.util.TestObjects.getTestPartition; +import static com.amazonaws.glue.catalog.util.TestObjects.getTestTable; + +import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; +import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isIn; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class GlueMetastoreClientDelegateTest { + + private GlueMetastoreClientDelegate metastoreClientDelegate; + private GlueMetastoreClientDelegate metastoreClientDelegateCatalogId; + + private HiveConf conf; + HiveConf hiveConfCatalogId; // conf with CatalogId + private AWSGlue glueClient; + private Warehouse wh; + + private Database testDb; + private Table testTbl; + + private static final int BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE = 100; + private static final int BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE = 1000; + private static final String CATALOG_ID = "12345"; + + @Before + public void setup() throws Exception { + conf = new HiveConf(); + glueClient = mock(AWSGlue.class); + wh = mock(Warehouse.class); + metastoreClientDelegate = new GlueMetastoreClientDelegate(conf, new DefaultAWSGlueMetastore(conf, glueClient), wh); + + // Create a client delegate with CatalogId + hiveConfCatalogId = new HiveConf(); + hiveConfCatalogId.set(GlueMetastoreClientDelegate.CATALOG_ID_CONF, CATALOG_ID); + metastoreClientDelegateCatalogId = new GlueMetastoreClientDelegate(hiveConfCatalogId, new DefaultAWSGlueMetastore(hiveConfCatalogId, glueClient), wh); + + testDb = getTestDatabase(); + testTbl= getTestTable(testDb.getName()); + setupMockWarehouseForPath(new Path(testTbl.getStorageDescriptor().getLocation().toString()), false, true); + } + + private void setupMockWarehouseForPath(Path path, boolean isDir, boolean mkDir) throws Exception { + when(wh.getDnsPath(path)).thenReturn(path); + when(wh.isDir(path)).thenReturn(isDir); + when(wh.mkdirs(path, true)).thenReturn(mkDir); + } + + // ===================== Thread Executor ===================== + + @Test + public void testExecutorService() throws Exception { + Object defaultExecutorService = new DefaultExecutorServiceFactory().getExecutorService(conf); + assertEquals("Default executor service should be used", metastoreClientDelegate.getExecutorService(), defaultExecutorService); + HiveConf customConf = new HiveConf(); + customConf.set(GlueMetastoreClientDelegate.CATALOG_ID_CONF, CATALOG_ID); + customConf.setClass(GlueMetastoreClientDelegate.CUSTOM_EXECUTOR_FACTORY_CONF, TestExecutorServiceFactory.class, ExecutorServiceFactory.class); + GlueMetastoreClientDelegate customDelegate = new GlueMetastoreClientDelegate(customConf, mock(AWSGlueMetastore.class), mock(Warehouse.class)); + Object customExecutorService = new TestExecutorServiceFactory().getExecutorService(customConf); + + assertEquals("Custom executor service should be used", customDelegate.getExecutorService(), customExecutorService); + } + + // ===================== Database ===================== + + @Test + public void testCreateDatabaseWithExistingDir() throws Exception { + Path dbPath = new Path(testDb.getLocationUri()); + setupMockWarehouseForPath(dbPath, true, true); + + metastoreClientDelegate.createDatabase(CatalogToHiveConverter.convertDatabase(testDb)); + verify(glueClient, times(1)).createDatabase(any(CreateDatabaseRequest.class)); + verify(wh, times(1)).isDir(dbPath); + verify(wh, never()).mkdirs(dbPath, true); + } + + @Test + public void testCreateDatabaseWithExistingDirWthCatalogId() throws Exception { + Path dbPath = new Path(testDb.getLocationUri()); + setupMockWarehouseForPath(dbPath, true, true); + + metastoreClientDelegateCatalogId.createDatabase(CatalogToHiveConverter.convertDatabase(testDb)); + ArgumentCaptor captor = ArgumentCaptor.forClass(CreateDatabaseRequest.class); + verify(glueClient, times(1)).createDatabase(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + verify(wh, times(1)).isDir(dbPath); + verify(wh, never()).mkdirs(dbPath, true); + } + + @Test + public void testCreateDatabaseWithoutExistingDir() throws Exception { + Path dbPath = new Path(testDb.getLocationUri()); + setupMockWarehouseForPath(dbPath, false, true); + + metastoreClientDelegate.createDatabase(CatalogToHiveConverter.convertDatabase(testDb)); + verify(glueClient, times(1)).createDatabase(any(CreateDatabaseRequest.class)); + verify(wh, times(1)).isDir(dbPath); + verify(wh, times(1)).mkdirs(dbPath, true); + } + + @Test + public void testGetDatabases() throws Exception { + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(testDb)); + + List dbs = metastoreClientDelegate.getDatabases("*"); + assertEquals(testDb.getName(), Iterables.getOnlyElement(dbs)); + } + + @Test + public void testGetDatabasesWithCatalogId() throws Exception { + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(testDb)); + + List dbs = metastoreClientDelegateCatalogId.getDatabases("*"); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetDatabasesRequest.class); + verify(glueClient, times(1)).getDatabases(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + assertEquals(testDb.getName(), Iterables.getOnlyElement(dbs)); + } + + + @Test + public void testGetDatabasesNullPattern() throws Exception { + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(testDb)); + + List dbs = metastoreClientDelegate.getDatabases(null); + assertEquals(testDb.getName(), Iterables.getOnlyElement(dbs)); + } + + @Test + public void testGetDatabase() throws Exception { + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn( + new GetDatabaseResult().withDatabase(getTestDatabase())); + metastoreClientDelegate.getDatabase("db"); + verify(glueClient, atLeastOnce()).getDatabase(any(GetDatabaseRequest.class)); + } + + @Test + public void testGetDatabaseWithCatalogId() throws Exception { + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn( + new GetDatabaseResult().withDatabase(getTestDatabase())); + metastoreClientDelegateCatalogId.getDatabase("db"); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetDatabaseRequest.class); + verify(glueClient, atLeastOnce()).getDatabase(captor.capture()); + GetDatabaseRequest request = captor.getValue(); + assertEquals(CATALOG_ID, request.getCatalogId()); + assertEquals("db", request.getName()); + } + + @Test + public void testGetAllDatabases() throws Exception { + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(getTestDatabase())); + metastoreClientDelegate.getDatabases("*"); + // Ensure this gets invoked + verify(glueClient, atLeastOnce()).getDatabases(any(GetDatabasesRequest.class)); + } + + @Test + public void testGetAllDatabasesPaginated() throws Exception { + when(glueClient.getDatabases(any(GetDatabasesRequest.class))) + .thenReturn(new GetDatabasesResult().withDatabaseList(testDb).withNextToken("token")) + .thenReturn(new GetDatabasesResult().withDatabaseList(getTestDatabase())); + List databases = metastoreClientDelegate.getDatabases(".*"); + + assertEquals(2, databases.size()); + verify(glueClient, times(2)).getDatabases(any(GetDatabasesRequest.class)); + } + + @Test + public void testAlterDatabase() throws Exception { + metastoreClientDelegate.alterDatabase("db", CatalogToHiveConverter.convertDatabase(testDb)); + verify(glueClient, times(1)).updateDatabase(any(UpdateDatabaseRequest.class)); + } + + @Test + public void testAlterDatabaseWithCatalogId() throws Exception { + metastoreClientDelegateCatalogId.alterDatabase("db", CatalogToHiveConverter.convertDatabase(testDb)); + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateDatabaseRequest.class); + verify(glueClient, times(1)).updateDatabase(any(UpdateDatabaseRequest.class)); + verify(glueClient).updateDatabase(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testDropDatabaseDeleteData() throws Exception { + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn( + new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTables(any(GetTablesRequest.class))).thenReturn( + new GetTablesResult().withTableList(ImmutableList.
of())); + Path dbPath = new Path(testDb.getLocationUri()); + when(wh.deleteDir(dbPath, true)).thenReturn(true); + + metastoreClientDelegate.dropDatabase(testDb.getName(), true, false, false); + verify(glueClient, times(1)).deleteDatabase(any(DeleteDatabaseRequest.class)); + verify(wh, times(1)).deleteDir(dbPath, true); + } + + @Test + public void testDropDatabaseDeleteDataWithCatalogId() throws Exception { + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn( + new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTables(any(GetTablesRequest.class))).thenReturn( + new GetTablesResult().withTableList(ImmutableList.
of())); + Path dbPath = new Path(testDb.getLocationUri()); + when(wh.deleteDir(dbPath, true)).thenReturn(true); + + metastoreClientDelegateCatalogId.dropDatabase(testDb.getName(), true, false, false); + ArgumentCaptor captor = ArgumentCaptor.forClass(DeleteDatabaseRequest.class); + verify(glueClient, times(1)).deleteDatabase(captor.capture()); + DeleteDatabaseRequest request = captor.getValue(); + verify(wh, times(1)).deleteDir(dbPath, true); + assertEquals(CATALOG_ID, request.getCatalogId()); + assertEquals(testDb.getName(), request.getName()); + } + + @Test + public void testDropDatabaseKeepData() throws Exception { + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn( + new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTables(any(GetTablesRequest.class))).thenReturn( + new GetTablesResult().withTableList(ImmutableList.
of())); + Path dbPath = new Path(testDb.getLocationUri()); + when(wh.deleteDir(dbPath, true)).thenReturn(true); + + metastoreClientDelegate.dropDatabase(testDb.getName(), false, false, false); + verify(glueClient, times(1)).deleteDatabase(any(DeleteDatabaseRequest.class)); + verify(wh, never()).deleteDir(dbPath, true); + } + + // ======================= Table ====================== + + @Test(expected = InvalidObjectException.class) + public void testGetTableInvalidGlueTable() throws Exception { + Table tbl = getTestTable().withTableType(null); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(tbl)); + metastoreClientDelegate.getTable(testDb.getName(), tbl.getName()); + } + + @Test + public void testGetTables() throws Exception { + Table tbl2 = getTestTable(); + List tableNames = ImmutableList.of(testTbl.getName(), tbl2.getName()); + List
tableList = ImmutableList.of(testTbl, tbl2); + + when(glueClient.getTables(new GetTablesRequest().withDatabaseName(testDb.getName()).withExpression("*"))) + .thenReturn(new GetTablesResult().withTableList(tableList)); + List result = metastoreClientDelegate.getTables(testDb.getName(), "*"); + + verify(glueClient).getTables(new GetTablesRequest().withDatabaseName(testDb.getName()).withExpression("*")); + assertThat(result, is(tableNames)); + } + + @Test + public void testGetTableWithCatalogId() throws Exception { + Table tbl2 = getTestTable(); + List tableNames = ImmutableList.of(testTbl.getName(), tbl2.getName()); + List
tableList = ImmutableList.of(testTbl, tbl2); + + when(glueClient.getTables(new GetTablesRequest().withDatabaseName(testDb.getName()).withExpression("*").withCatalogId(CATALOG_ID))) + .thenReturn(new GetTablesResult().withTableList(tableList)); + List result = metastoreClientDelegateCatalogId.getTables(testDb.getName(), "*"); + + assertThat(result, is(tableNames)); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetTablesRequest.class); + verify(glueClient, times(1)).getTables(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + assertEquals(testDb.getName(), captor.getValue().getDatabaseName()); + assertEquals("*", captor.getValue().getExpression()); + } + + @Test + public void testGetTablesWithPagination() throws Exception { + Table tbl2 = getTestTable(); + List tableNames = ImmutableList.of(testTbl.getName(), tbl2.getName()); + List
tableList1 = ImmutableList.of(testTbl); + List
tableList2 = ImmutableList.of(tbl2); + + String nextToken = "1"; + when(glueClient.getTables(any(GetTablesRequest.class))) + .thenReturn(new GetTablesResult().withTableList(tableList1).withNextToken(nextToken)) + .thenReturn(new GetTablesResult().withTableList(tableList2)); + List result = metastoreClientDelegate.getTables(testDb.getName(), "*"); + + verify(glueClient, times(2)).getTables(any(GetTablesRequest.class)); + assertThat(result, is(tableNames)); + } + + @Test + public void testGetTableMeta() throws Exception { + List
tables = Lists.newArrayList(testTbl); + List tableTypes = Lists.newArrayList(TableType.MANAGED_TABLE.name()); + + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(testDb)); + when(glueClient.getTables(any(GetTablesRequest.class))).thenReturn( + new GetTablesResult().withTableList(tables)); + + List tableMetaResult = metastoreClientDelegate.getTableMeta(testDb.getName(), testTbl.getName(), tableTypes); + assertEquals(CatalogToHiveConverter.convertTableMeta(testTbl, testDb.getName()), Iterables.getOnlyElement(tableMetaResult)); + } + + @Test + public void testGetTableMetaNullEmptyTableType() throws Exception { + List
tables = Lists.newArrayList(testTbl); + List tableTypes = null; + + when(glueClient.getDatabases(any(GetDatabasesRequest.class))).thenReturn( + new GetDatabasesResult().withDatabaseList(testDb)); + when(glueClient.getTables(any(GetTablesRequest.class))).thenReturn( + new GetTablesResult().withTableList(tables)); + + List tableMetaResult = metastoreClientDelegate.getTableMeta(testDb.getName(), testTbl.getName(), tableTypes); + assertEquals(CatalogToHiveConverter.convertTableMeta(testTbl, testDb.getName()), Iterables.getOnlyElement(tableMetaResult)); + + tableTypes = Lists.newArrayList(); + tableMetaResult = metastoreClientDelegate.getTableMeta(testDb.getName(), testTbl.getName(), tableTypes); + assertEquals(CatalogToHiveConverter.convertTableMeta(testTbl, testDb.getName()), Iterables.getOnlyElement(tableMetaResult)); + } + + @Test + public void testCreateTableWithExistingDir() throws Exception { + Path tblPath = new Path(testTbl.getStorageDescriptor().getLocation()); + setupMockWarehouseForPath(tblPath, true, true); + + when(glueClient.getDatabase(new GetDatabaseRequest().withName(testDb.getName()))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTable(new GetTableRequest().withDatabaseName(testTbl.getDatabaseName()) + .withName(testTbl.getName()))).thenThrow(new EntityNotFoundException("")); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + + metastoreClientDelegate.createTable(CatalogToHiveConverter.convertTable(testTbl, testTbl.getDatabaseName())); + + verify(glueClient, times(1)).createTable(any(CreateTableRequest.class)); + verify(wh).isDir(tblPath); + verify(wh, never()).mkdirs(tblPath, true); + } + + @Test + public void testCreateTableWithExistingDirWithCatalogId() throws Exception { + Path tblPath = new Path(testTbl.getStorageDescriptor().getLocation()); + setupMockWarehouseForPath(tblPath, true, true); + + when(glueClient.getDatabase(new GetDatabaseRequest().withName(testDb.getName()).withCatalogId(CATALOG_ID))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTable(new GetTableRequest().withDatabaseName(testTbl.getDatabaseName()).withCatalogId(CATALOG_ID) + .withName(testTbl.getName()))).thenThrow(new EntityNotFoundException("")); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + + metastoreClientDelegateCatalogId.createTable(CatalogToHiveConverter.convertTable(testTbl, testTbl.getDatabaseName())); + ArgumentCaptor captor = ArgumentCaptor.forClass(CreateTableRequest.class); + verify(glueClient, times(1)).createTable(captor.capture()); + verify(wh).isDir(tblPath); + verify(wh, never()).mkdirs(tblPath, true); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testCreateTableWithoutExistingDir() throws Exception { + Path tblPath = new Path(testTbl.getStorageDescriptor().getLocation()); + setupMockWarehouseForPath(tblPath, false, true); + + when(glueClient.getDatabase(new GetDatabaseRequest().withName(testDb.getName()))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTable(new GetTableRequest().withDatabaseName(testTbl.getDatabaseName()) + .withName(testTbl.getName()))).thenThrow(new EntityNotFoundException("")); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + metastoreClientDelegate.createTable(CatalogToHiveConverter.convertTable(testTbl, testTbl.getDatabaseName())); + + verify(glueClient, times(1)).createTable(any(CreateTableRequest.class)); + verify(wh).isDir(tblPath); + verify(wh).mkdirs(tblPath, true); + } + + @Test (expected = org.apache.hadoop.hive.metastore.api.AlreadyExistsException.class) + public void testCreateTableWithExistTable() throws Exception { + setupMockWarehouseForPath(new Path(testTbl.getStorageDescriptor().getLocation()), true, false); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + metastoreClientDelegate.createTable(CatalogToHiveConverter.convertTable(testTbl, testTbl.getDatabaseName())); + } + + @Test + public void testAlterTable() throws Exception { + org.apache.hadoop.hive.metastore.api.Table newHiveTable + = CatalogToHiveConverter.convertTable(getTestTable(), testDb.getName()); + newHiveTable.setTableName(testTbl.getName()); + + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn(new GetDatabaseResult().withDatabase((testDb))); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable((testTbl))); + metastoreClientDelegateCatalogId.alterTable(testDb.getName(), testTbl.getName(), newHiveTable, null); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateTableRequest.class); + verify(glueClient, times(1)).updateTable(captor.capture()); + + TableInput expectedTableInput = GlueInputConverter.convertToTableInput(newHiveTable); + assertEquals(expectedTableInput, captor.getValue().getTableInput()); + } + + @Test(expected = UnsupportedOperationException.class) + public void testAlterTableRename() throws Exception { + org.apache.hadoop.hive.metastore.api.Table newHiveTable + = CatalogToHiveConverter.convertTable(getTestTable(), testDb.getName()); + metastoreClientDelegate.alterTable(testDb.getName(), testTbl.getName(), newHiveTable, null); + } + + @Test + public void testAlterTableSetExternalType() throws Exception { + org.apache.hadoop.hive.metastore.api.Table newHiveTable + = CatalogToHiveConverter.convertTable(getTestTable(), testDb.getName()); + newHiveTable.setTableType(MANAGED_TABLE.toString()); + newHiveTable.getParameters().put("EXTERNAL", "TRUE"); + + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn(new GetDatabaseResult().withDatabase((testDb))); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable((testTbl))); + metastoreClientDelegate.alterTable(testDb.getName(), newHiveTable.getTableName(), newHiveTable, null); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateTableRequest.class); + verify(glueClient, times(1)).updateTable(captor.capture()); + assertEquals(EXTERNAL_TABLE.toString(), captor.getValue().getTableInput().getTableType()); + } + + @Test + public void testAlterTableSetManagedType() throws Exception { + org.apache.hadoop.hive.metastore.api.Table newHiveTable + = CatalogToHiveConverter.convertTable(getTestTable(), testDb.getName()); + newHiveTable.setTableType(EXTERNAL_TABLE.toString()); + newHiveTable.getParameters().put("EXTERNAL", "FALSE"); + + when(glueClient.getDatabase(any(GetDatabaseRequest.class))).thenReturn(new GetDatabaseResult().withDatabase((testDb))); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable((testTbl))); + metastoreClientDelegate.alterTable(testDb.getName(), newHiveTable.getTableName(), newHiveTable, null); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateTableRequest.class); + verify(glueClient, times(1)).updateTable(captor.capture()); + assertEquals(MANAGED_TABLE.toString(), captor.getValue().getTableInput().getTableType()); + } + + @Test(expected=UnsupportedOperationException.class) + public void testListTableNamesByFilter() throws Exception { + metastoreClientDelegate.listTableNamesByFilter("db","filter", (short)1); + } + + @Test + public void testDropTableWithDeleteData() throws Exception { + Path tbl_path = new Path(testTbl.getStorageDescriptor().getLocation()); + List values = Lists.newArrayList("foo"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()).withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + + when(glueClient.getTable(new GetTableRequest() + .withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName()))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(glueClient.deletePartition(new DeletePartitionRequest() + .withDatabaseName(testDb.getName()).withPartitionValues(values).withTableName(testTbl.getName()))) + .thenReturn(new DeletePartitionResult()); + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenReturn(new GetPartitionsResult().withPartitions(partition)); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(testDb.getName()).withTableName(testTbl.getName()).withPartitionValues(values))) + .thenReturn(new GetPartitionResult().withPartition(partition)); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + metastoreClientDelegate.dropTable(testTbl.getDatabaseName(), testTbl.getName(), true, true, true); + + verify(glueClient).deleteTable(new DeleteTableRequest().withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName())); + verify(wh).deleteDir(tbl_path, true, true); + } + + @Test + public void testDropTableWithoutDeleteData() throws Exception { + Path tblPath = new Path(testTbl.getStorageDescriptor().getLocation()); + List values = Lists.newArrayList("foo"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()).withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + + when(glueClient.getTable(new GetTableRequest() + .withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName()))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(glueClient.deletePartition(new DeletePartitionRequest() + .withDatabaseName(testDb.getName()).withPartitionValues(values).withTableName(testTbl.getName()))) + .thenReturn(new DeletePartitionResult()); + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenReturn(new GetPartitionsResult().withPartitions(partition)); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(testDb.getName()).withTableName(testTbl.getName()).withPartitionValues(values))) + .thenReturn(new GetPartitionResult().withPartition(partition)); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + metastoreClientDelegate.dropTable(testTbl.getDatabaseName(), testTbl.getName(), false, true, true); + + verify(glueClient).deleteTable(new DeleteTableRequest().withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName())); + verify(wh, never()).deleteDir(tblPath, true, true); + } + + @Test + public void testDropExternalTableWithoutDeleteData() throws Exception { + Path tblPath = new Path(testTbl.getStorageDescriptor().getLocation()); + List values = Lists.newArrayList("foo"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()).withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + testTbl.getParameters().put("EXTERNAL", "TRUE"); + + when(glueClient.getTable(new GetTableRequest() + .withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName()))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(glueClient.deletePartition(new DeletePartitionRequest() + .withDatabaseName(testDb.getName()).withPartitionValues(values).withTableName(testTbl.getName()))) + .thenReturn(new DeletePartitionResult()); + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenReturn(new GetPartitionsResult().withPartitions(partition)); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(testDb.getName()).withTableName(testTbl.getName()).withPartitionValues(values))) + .thenReturn(new GetPartitionResult().withPartition(partition)); + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + metastoreClientDelegate.dropTable(testTbl.getDatabaseName(), testTbl.getName(), false, true, true); + + verify(glueClient).deleteTable(new DeleteTableRequest().withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName())); + verify(wh, never()).deleteDir(tblPath, true, true); + } + + @Test + public void testValidateTableAndCreateDirectoryVirtualView() throws Exception { + testTbl.setTableType(TableType.VIRTUAL_VIEW.toString()); + testTbl.getStorageDescriptor().setLocation(null); + org.apache.hadoop.hive.metastore.api.Table hiveTbl = CatalogToHiveConverter.convertTable(testTbl, testTbl.getDatabaseName()); + + when(glueClient.getDatabase(any(GetDatabaseRequest.class))) + .thenReturn(new GetDatabaseResult().withDatabase(testDb)); + when(glueClient.getTable(new GetTableRequest() + .withDatabaseName(testTbl.getDatabaseName()).withName(testTbl.getName()))) + .thenThrow(EntityNotFoundException.class); + + assertFalse(metastoreClientDelegate.validateNewTableAndCreateDirectory(hiveTbl)); + assertNull(testTbl.getStorageDescriptor().getLocation()); + verify(wh, never()).mkdirs(any(Path.class), anyBoolean()); + } + + // ======================= Partition ======================= + + @Test + public void testGetPartitionByValues() throws Exception { + List values = Lists.newArrayList("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + GetPartitionRequest request = new GetPartitionRequest() + .withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withPartitionValues(values); + when(glueClient.getPartition(request)).thenReturn(new GetPartitionResult().withPartition(partition)); + org.apache.hadoop.hive.metastore.api.Partition result = metastoreClientDelegate.getPartition(testDb.getName(), testTbl.getName(), values); + + verify(glueClient, times(1)).getPartition(request); + assertThat(result.getValues(), is(values)); + } + + @Test + public void testGetPartitionByValuesWithCatalogId() throws Exception { + List values = Lists.newArrayList("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + GetPartitionRequest request = new GetPartitionRequest() + .withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withPartitionValues(values). + withCatalogId(CATALOG_ID); + when(glueClient.getPartition(request)).thenReturn(new GetPartitionResult().withPartition(partition)); + org.apache.hadoop.hive.metastore.api.Partition result = metastoreClientDelegateCatalogId.getPartition(testDb.getName(), testTbl.getName(), values); + + ArgumentCaptor captor = ArgumentCaptor.forClass(GetPartitionRequest.class); + verify(glueClient, times(1)).getPartition(captor.capture()); + assertThat(result.getValues(), is(values)); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testGetPartitionByName() throws Exception { + String partitionName = "/a=foo/b=bar"; + List values = ImmutableList.of("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + when(glueClient.getPartition(any(GetPartitionRequest.class))) + .thenReturn(new GetPartitionResult().withPartition(partition)); + + org.apache.hadoop.hive.metastore.api.Partition result + = metastoreClientDelegate.getPartition(testDb.getName(), testTbl.getName(), partitionName); + + verify(glueClient).getPartition(any(GetPartitionRequest.class)); + assertThat(result.getValues(), is(values)); + } + + @Test(expected=NoSuchObjectException.class) + public void testGetPartitionEntityNotFound() throws Exception { + when(glueClient.getPartition(any(GetPartitionRequest.class))) + .thenThrow(new EntityNotFoundException("Test exception: partition not found")); + metastoreClientDelegate.getPartition(testDb.getName(), testTbl.getName(), "testPart"); + verify(glueClient, times(1)).getPartition(any(GetPartitionRequest.class)); + } + + @Test + public void testGetPartitionsByNames() throws Exception { + String partitionName = "/a=foo/b=bar"; + List values = ImmutableList.of("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + when(glueClient.batchGetPartition(any(BatchGetPartitionRequest.class))) + .thenReturn(new BatchGetPartitionResult().withPartitions(partition)); + + List result + = metastoreClientDelegate.getPartitionsByNames(testDb.getName(), testTbl.getName(), ImmutableList.of(partitionName)); + + verify(glueClient, times(1)).batchGetPartition(any(BatchGetPartitionRequest.class)); + assertNotNull(result); + assertThat(Iterables.getOnlyElement(result).getValues(), is(values)); + } + + @Test + public void testGetPartitionsByNamesWithCatalogId() throws Exception { + String partitionName = "/a=foo/b=bar"; + List values = ImmutableList.of("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + when(glueClient.batchGetPartition(any(BatchGetPartitionRequest.class))) + .thenReturn(new BatchGetPartitionResult().withPartitions(partition)); + + List result + = metastoreClientDelegateCatalogId.getPartitionsByNames(testDb.getName(), testTbl.getName(), ImmutableList.of(partitionName)); + + ArgumentCaptor captor = ArgumentCaptor.forClass(BatchGetPartitionRequest.class); + verify(glueClient, times(1)).batchGetPartition(captor.capture()); + assertNotNull(result); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testGetPartitionsByNamePropagateException() throws Exception { + String exceptionMessage = "Partition not found"; + when(glueClient.batchGetPartition(any(BatchGetPartitionRequest.class))) + .thenThrow(new EntityNotFoundException(exceptionMessage)); + + try { + metastoreClientDelegate.getPartitionsByNames(testDb.getName(), testTbl.getName(), ImmutableList.of("/a=foo/b=bar")); + } catch (Exception e) { + assertThat(e, instanceOf(NoSuchObjectException.class)); + assertThat(e.getMessage(), containsString(exceptionMessage)); + } + verify(glueClient, times(1)).batchGetPartition(any(BatchGetPartitionRequest.class)); + } + + @Test + public void testGetPartitionsByNameTwoPages() throws Exception { + int numPartNames = BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE + 10; + List partNames = getTestPartitionNames(numPartNames); + + when(glueClient.batchGetPartition(any(BatchGetPartitionRequest.class))) + .thenReturn(new BatchGetPartitionResult().withPartitions(ImmutableList.of())); + + metastoreClientDelegate.getPartitionsByNames(testDb.getName(), testTbl.getName(), partNames); + verify(glueClient, times(2)).batchGetPartition(any(BatchGetPartitionRequest.class)); + } + + private static List getTestPartitionNames(int numPartitions) { + List partNames = Lists.newArrayList(); + for (int i = 1; i < numPartitions; i++) { + partNames.add(String.format("a=%d", i)); + } + return partNames; + } + + @Test + public void testGetPartitions() throws Exception { + List expectedValues = Lists.newArrayList("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(expectedValues); + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenReturn(new GetPartitionsResult().withPartitions(Lists.newArrayList(partition))); + + List res = metastoreClientDelegate.getPartitions( + testDb.getName(), testTbl.getName(), null, 10); + + verify(glueClient, times(1)).getPartitions(any(GetPartitionsRequest.class)); + assertThat(res, is(not(empty()))); + List values = Iterables.getOnlyElement(res).getValues(); + assertThat(values, is(expectedValues)); + } + + @Test + public void testGetPartitionsParallel() throws Exception { + final int numSegments = 2; + HiveConf conf = new HiveConf(this.conf); + conf.setInt(GlueMetastoreClientDelegate.NUM_PARTITION_SEGMENTS_CONF, numSegments); + GlueMetastoreClientDelegate delegate = new GlueMetastoreClientDelegate(conf, new DefaultAWSGlueMetastore(conf, glueClient), wh); + + final Set> expectedValues = Sets.newHashSet(); + final List partitions = Lists.newArrayList(); + final int numPartitions = DefaultAWSGlueMetastore.GET_PARTITIONS_MAX_SIZE + 10; + final int maxPartitionsToRequest = numPartitions - 1; + + for (int i = 1; i <= numPartitions; i++) { + List partitionKeys = Arrays.asList("keyA:" + i, "keyB:" + i); + if (i <= maxPartitionsToRequest) { + expectedValues.add(partitionKeys); + } + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(partitionKeys); + partitions.add(partition); + } + + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenAnswer(new Answer() { + @Override + public GetPartitionsResult answer(InvocationOnMock invocation) { + GetPartitionsRequest request = invocation.getArgument(0, GetPartitionsRequest.class); + GetPartitionsResult result; + if (request.getSegment() == null) { + fail("Should pass in segment"); + } + switch (request.getSegment().getSegmentNumber()) { + case 0: + result = new GetPartitionsResult().withPartitions(partitions.subList(0, numPartitions / 2)); + break; + case 1: + result = new GetPartitionsResult().withPartitions(partitions.subList(numPartitions / 2, partitions.size())); + break; + default: + result = new GetPartitionsResult().withPartitions(Collections.emptyList()); + fail("Got segmentNumber >= " + numSegments); + } + return result; + } + }); + + List res = delegate.getPartitions( + testDb.getName(), testTbl.getName(), null, maxPartitionsToRequest); + + verify(glueClient, times(numSegments)) + .getPartitions(any(GetPartitionsRequest.class)); + assertThat(res, is(not(empty()))); + Iterable> values = Iterables.transform(res, + new Function>() { + public List apply(org.apache.hadoop.hive.metastore.api.Partition partition) { + return partition.getValues(); + } + }); + assertThat(Sets.newHashSet(values), is(expectedValues)); + } + + @Test(expected = MetaException.class) + public void testGetPartitionsPartialFailure() throws Exception { + List partitionKeys1 = Arrays.asList("foo1", "bar1"); + final Partition partition1 = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(partitionKeys1); + + when(glueClient.getPartitions(any(GetPartitionsRequest.class))) + .thenAnswer(new Answer() { + @Override + public GetPartitionsResult answer(InvocationOnMock invocation) { + GetPartitionsRequest request = invocation.getArgument(0, GetPartitionsRequest.class); + GetPartitionsResult result; + switch (request.getSegment().getSegmentNumber()) { + case 0: + result = new GetPartitionsResult().withPartitions(Lists.newArrayList(partition1)); + break; + default: + throw new OperationTimeoutException("timeout"); + } + return result; + } + }); + + List res = metastoreClientDelegate.getPartitions( + testDb.getName(), testTbl.getName(), null, -1); + } + + @Test(expected = IllegalArgumentException.class) + public void testTooHighGluePartitionSegments() throws MetaException { + HiveConf conf = new HiveConf(this.conf); + conf.setInt(GlueMetastoreClientDelegate.NUM_PARTITION_SEGMENTS_CONF, + DefaultAWSGlueMetastore.MAX_NUM_PARTITION_SEGMENTS + 1); + GlueMetastoreClientDelegate delegate = new GlueMetastoreClientDelegate(conf, new DefaultAWSGlueMetastore(conf, glueClient), wh); + } + + @Test + public void testDropPartitionUsingValues() throws Exception { + List values = Lists.newArrayList("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + DeletePartitionRequest request = new DeletePartitionRequest() + .withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withPartitionValues(values); + + when(glueClient.deletePartition(request)).thenReturn(new DeletePartitionResult()); + when(glueClient.getPartition(any(GetPartitionRequest.class))).thenReturn(new GetPartitionResult().withPartition(partition)); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + + metastoreClientDelegate.dropPartition(testDb.getName(), testTbl.getName(), values, false, false, false); + verify(glueClient, times(1)).deletePartition(request); + } + + @Test + public void testDropPartitionUsingValuesWithCatalogId() throws Exception { + List values = Lists.newArrayList("foo", "bar"); + Partition partition = new Partition().withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withValues(values) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + DeletePartitionRequest request = new DeletePartitionRequest() + .withDatabaseName(testDb.getName()) + .withTableName(testTbl.getName()) + .withPartitionValues(values); + + when(glueClient.deletePartition(request)).thenReturn(new DeletePartitionResult()); + when(glueClient.getPartition(any(GetPartitionRequest.class))).thenReturn(new GetPartitionResult().withPartition(partition)); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + + metastoreClientDelegateCatalogId.dropPartition(testDb.getName(), testTbl.getName(), values, false, false, false); + ArgumentCaptor captor = ArgumentCaptor.forClass(DeletePartitionRequest.class); + verify(glueClient, times(1)).deletePartition(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testAppendPartition() throws Exception { + List values = ImmutableList.of("foo"); + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + Path partLocation = new Path(testTbl.getStorageDescriptor().getLocation(), + Warehouse.makePartName(CatalogToHiveConverter.convertFieldSchemaList(testTbl.getPartitionKeys()), values)); + setupMockWarehouseForPath(partLocation, false, true); + mockBatchCreatePartitionsSucceed(); + + org.apache.hadoop.hive.metastore.api.Partition res = + metastoreClientDelegate.appendPartition(testDb.getName(), testTbl.getName(), values); + + verify(wh, times(1)).mkdirs(partLocation, true); + assertThat(res.getValues(), is(values)); + } + + @Test + public void testAddPartitionsEmpty() throws Exception { + List partitions = Lists.newArrayList(); + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, false, true); + + verify(glueClient, never()).getTable(any(GetTableRequest.class)); + verify(glueClient, never()).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + assertThat(partitionsCreated, is(empty())); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitions() throws Exception { + mockBatchCreatePartitionsSucceed(); + setupMockWarehouseForPath(new Path(testTbl.getStorageDescriptor().getLocation().toString()), false, true); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, false, true); + + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, never()).deleteDir(any(Path.class), eq(true)); + assertEquals(numPartitions, partitionsCreated.size()); + assertThat(partitionsCreated, containsInAnyOrder(partitions.toArray())); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsEmptyPartitionLocation() throws Exception { + // Case: table contains location & partition location is empty. + // Test that created partitions contains location + int numPartitions = 2; + List partitionsCreated = addPartitionsWithEmptyLocationsValid(numPartitions); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + for (org.apache.hadoop.hive.metastore.api.Partition part : partitionsCreated) { + assertThat(part.getSd().getLocation(), notNullValue()); + } + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsEmptyTableAndPartitionLocation() throws Exception { + // Case: table location is empty (VIRTUAL_VIEW) & partition location is empty. + // Test that created partitions does not contain location as these are Views. + testTbl.getStorageDescriptor().setLocation(null); + int numPartitions = 1; + List partitionsCreated = addPartitionsWithEmptyLocationsValid(numPartitions); + verify(wh, never()).mkdirs(any(Path.class), anyBoolean()); + assertThat(partitionsCreated.get(0).getSd().getLocation(), nullValue()); + assertDaemonThreadPools(); + } + + private List addPartitionsWithEmptyLocationsValid(int numPartitions) throws Exception { + List partitions = getTestPartitions(numPartitions); + for (org.apache.hadoop.hive.metastore.api.Partition partition : partitions) { + partition.getSd().setLocation(null); + } + mockBatchCreatePartitionsSucceed(); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(wh.mkdirs(any(Path.class), anyBoolean())).thenReturn(true); + + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, false, true); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, never()).deleteDir(any(Path.class), anyBoolean()); + assertEquals(numPartitions, partitionsCreated.size()); + assertThat(partitionsCreated, containsInAnyOrder(partitions.toArray())); + return partitionsCreated; + } + + @Test(expected = MetaException.class) + public void testAddPartitions_PartitionViewWithLocation() throws Exception { + // Case: table location is empty (VIRTUAL_VIEW) with partition containing location + // In Hive, this throws MetaException because it doesn't allow parititon views to have location + Table table = testTbl; + table.getStorageDescriptor().setLocation(null); + + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + + mockBatchCreatePartitionsSucceed(); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(table)); + when(wh.mkdirs(any(Path.class), anyBoolean())).thenReturn(true); + + metastoreClientDelegate.addPartitions(partitions, false, true); + + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsDoNotNeedResult() throws Exception { + mockBatchCreatePartitionsSucceed(); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, false, false); + + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, never()).deleteDir(any(Path.class), eq(true)); + assertThat(partitionsCreated, is(nullValue())); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsTwoPages() throws Exception { + mockBatchCreatePartitionsSucceed(); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = (int) (BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE * 1.2); + int expectedBatches = 2; + List partitions = getTestPartitions(numPartitions); + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, false, true); + + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(expectedBatches)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, never()).deleteDir(any(Path.class), eq(true)); + assertEquals(numPartitions, partitionsCreated.size()); + assertThat(partitionsCreated, containsInAnyOrder(partitions.toArray())); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsTwoPagesWithCatalogId() throws Exception { + mockBatchCreatePartitionsSucceed(); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = (int) (BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE * 1.2); + int expectedBatches = 2; + List partitions = getTestPartitions(numPartitions); + List partitionsCreated = + metastoreClientDelegateCatalogId.addPartitions(partitions, false, true); + ArgumentCaptor captor = ArgumentCaptor.forClass(BatchCreatePartitionRequest.class); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(expectedBatches)).batchCreatePartition(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, never()).deleteDir(any(Path.class), eq(true)); + assertEquals(numPartitions, partitionsCreated.size()); + assertThat(partitionsCreated, containsInAnyOrder(partitions.toArray())); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsFailedServiceException() throws Exception { + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + List values = partitions.get(0).getValues(); + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenReturn(new BatchCreatePartitionResult().withErrors(TestObjects.getPartitionError(values, + new InternalServiceException("exception")))); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + try { + metastoreClientDelegate.addPartitions(partitions, false, true); + fail("should throw"); + } catch (Exception e) { + assertThat(e, is(instanceOf(MetaException.class))); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(1)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsFailedAlreadyExistsException() throws Exception { + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + List values = ImmutableList.of("foo1"); + + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenReturn(new BatchCreatePartitionResult().withErrors(TestObjects.getPartitionError(values, + new com.amazonaws.services.glue.model.AlreadyExistsException("exception")))); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + try { + metastoreClientDelegate.addPartitions(partitions, false, true); + fail("Should throw"); + } catch (Exception e) { + assertThat(e, is(instanceOf(org.apache.hadoop.hive.metastore.api.AlreadyExistsException.class))); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(1)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsThrowsEntityNotFoundException() throws Exception { + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenThrow(new EntityNotFoundException("exception")); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + + try { + metastoreClientDelegate.addPartitions(partitions, false, true); + fail("Should throw"); + } catch (Exception e) { + assertThat(e, is(instanceOf(NoSuchObjectException.class))); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(numPartitions)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsThrowsExceptionSecondPage() throws Exception { + int numPartitions = 200; + int secondPageSize = numPartitions - BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE; + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenReturn(new BatchCreatePartitionResult()) + .thenThrow(new InvalidInputException("exception")); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + List partitions = getTestPartitions(numPartitions); + + try { + metastoreClientDelegate.addPartitions(partitions, false, true); + fail("Should throw"); + } catch (Exception e) { + assertThat(e, is(instanceOf(InvalidObjectException.class))); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(2)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(secondPageSize)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsIfNotExists() throws Exception { + List values = ImmutableList.of("foo1"); + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenReturn(new BatchCreatePartitionResult().withErrors(TestObjects.getPartitionError(values, + new com.amazonaws.services.glue.model.AlreadyExistsException("exception")))); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + List partitionsCreated = + metastoreClientDelegate.addPartitions(partitions, true, true); + + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, never()).deleteDir(any(Path.class), eq(true)); + assertEquals(1, partitionsCreated.size()); + assertThat(partitionsCreated.get(0), isIn(partitions)); + assertDaemonThreadPools(); + } + + @Test + public void testAddPartitionsKeysAndValuesNotMatch() throws Exception { + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + //make the partition value size inconsistent with key size + partitions.get(1).setValues(Lists.newArrayList("foo1", "bar1")); + + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + + try { + metastoreClientDelegate.addPartitions(partitions, true, true); + fail("should throw"); + } catch (IllegalArgumentException e) { + verify(wh, never()).getDnsPath(any(Path.class)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsDeleteAddedPathsWhenAddPathFail() throws Exception { + int numPartitions = 2; + List partitions = getTestPartitions(numPartitions); + + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(wh.isDir(any(Path.class))).thenReturn(false); + when(wh.mkdirs(any(Path.class), eq(true))).thenReturn(true).thenReturn(false); // succeed first, then fail + + try { + metastoreClientDelegate.addPartitions(partitions, true, true); + fail("should throw"); + } catch (MetaException e) { + verify(wh, times(numPartitions)).getDnsPath(any(Path.class)); + verify(wh, times(numPartitions)).isDir(any(Path.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(1)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + @Test + public void testAddPartitionsCallGetPartitionForInternalServiceException() throws Exception { + int numPartitions = 3; + String dbName = testDb.getName(); + String tableName = testTbl.getName(); + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List values3 = Lists.newArrayList("val3"); + Partition partition1 = TestObjects.getTestPartition(dbName, tableName, values1); + Partition partition2 = TestObjects.getTestPartition(dbName, tableName, values2); + Partition partition3 = TestObjects.getTestPartition(dbName, tableName, values3); + List partitions = Lists.newArrayList(partition1, partition2, partition3); + + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenThrow(new InternalServiceException("InternalServiceException")); + when(glueClient.getTable(any(GetTableRequest.class))) + .thenReturn(new GetTableResult().withTable(testTbl)); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(dbName) + .withTableName(tableName) + .withPartitionValues(partition1.getValues()))) + .thenReturn(new GetPartitionResult().withPartition(partition1)); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(dbName) + .withTableName(tableName) + .withPartitionValues(partition2.getValues()))) + .thenThrow(new EntityNotFoundException("EntityNotFoundException")); + when(glueClient.getPartition(new GetPartitionRequest() + .withDatabaseName(dbName) + .withTableName(tableName) + .withPartitionValues(partition3.getValues()))) + .thenThrow(new NullPointerException("NullPointerException")); + + try { + metastoreClientDelegate.addPartitions(CatalogToHiveConverter.convertPartitions(partitions), false, true); + fail("Should throw"); + } catch (Exception e) { + assertThat(e, is(instanceOf(MetaException.class))); + verify(glueClient, times(1)).getTable(any(GetTableRequest.class)); + verify(glueClient, times(1)).batchCreatePartition(any(BatchCreatePartitionRequest.class)); + verify(glueClient, times(numPartitions)).getPartition(any(GetPartitionRequest.class)); + verify(wh, times(numPartitions)).mkdirs(any(Path.class), eq(true)); + verify(wh, times(2)).deleteDir(any(Path.class), eq(true)); + assertDaemonThreadPools(); + } + } + + private void mockBatchCreatePartitionsSucceed() { + when(glueClient.batchCreatePartition(any(BatchCreatePartitionRequest.class))) + .thenReturn(new BatchCreatePartitionResult()); + } + + private List getTestPartitions(int count) { + List partitions = Lists.newArrayList(); + for (int i = 0; i < count; i++) { + List values = ImmutableList.of("foo" + i); + Partition partition = TestObjects.getTestPartition(testDb.getName(), testTbl.getName(), values); + partitions.add(CatalogToHiveConverter.convertPartition(partition)); + } + return partitions; + } + + @Test + public void testAlterPartitions() throws Exception { + List values = ImmutableList.of("foo", "bar"); + Partition partition = getTestPartition(testTbl.getDatabaseName(), testTbl.getName(), values); + org.apache.hadoop.hive.metastore.api.Partition hivePartition = CatalogToHiveConverter.convertPartition(partition); + PartitionInput input = GlueInputConverter.convertToPartitionInput(partition); + UpdatePartitionRequest request = new UpdatePartitionRequest() + .withDatabaseName(testTbl.getDatabaseName()) + .withTableName(testTbl.getName()) + .withPartitionInput(input) + .withPartitionValueList(partition.getValues()); + + when(glueClient.updatePartition(request)).thenReturn(new UpdatePartitionResult()); + metastoreClientDelegate.alterPartitions(testDb.getName(), testTbl.getName(), ImmutableList.of(hivePartition)); + + verify(glueClient, times(1)).updatePartition(any(UpdatePartitionRequest.class)); + } + + @Test + public void testAlterParititonDDLTimeUpdated() throws Exception { + List values = ImmutableList.of("foo", "bar"); + org.apache.hadoop.hive.metastore.api.Partition partition + = CatalogToHiveConverter.convertPartition(getTestPartition(testTbl.getDatabaseName(), testTbl.getName(), values)); + metastoreClientDelegate.alterPartitions(testTbl.getDatabaseName(), testTbl.getName(), Lists.newArrayList(partition)); + + ArgumentCaptor captor = ArgumentCaptor.forClass(UpdatePartitionRequest.class); + verify(glueClient, times(1)).updatePartition(captor.capture()); + assertTrue(captor.getValue().getPartitionInput().getParameters().containsKey(hive_metastoreConstants.DDL_TIME)); + } + + // =================== Roles & Privilege =================== + + @Test(expected=UnsupportedOperationException.class) + public void testGrantPublicRole() throws Exception { + metastoreClientDelegate.grantRole("public", "user", + org.apache.hadoop.hive.metastore.api.PrincipalType.USER, "grantor", + org.apache.hadoop.hive.metastore.api.PrincipalType.ROLE, true); + } + + @Test(expected=UnsupportedOperationException.class) + public void testRevokeRole() throws Exception { + metastoreClientDelegate.revokeRole("role", "user", + org.apache.hadoop.hive.metastore.api.PrincipalType.USER, true); + } + + @Test(expected=UnsupportedOperationException.class) + public void testCreateRole() throws Exception { + metastoreClientDelegate.createRole(new org.apache.hadoop.hive.metastore.api.Role( + "role", (int) (new Date().getTime() / 1000), "owner")); + } + + @Test(expected=UnsupportedOperationException.class) + public void testCreatePublicRole() throws Exception { + metastoreClientDelegate.createRole(new org.apache.hadoop.hive.metastore.api.Role( + "public", (int) (new Date().getTime() / 1000), "owner")); + } + + @Test(expected=UnsupportedOperationException.class) + public void testDropRole() throws Exception { + metastoreClientDelegate.dropRole("role"); + } + + @Test(expected=UnsupportedOperationException.class) + public void testDropPublicRole() throws Exception { + metastoreClientDelegate.dropRole("public"); + } + + @Test(expected=UnsupportedOperationException.class) + public void testDropAdminRole() throws Exception { + metastoreClientDelegate.dropRole("admin"); + } + + @Test(expected=UnsupportedOperationException.class) + public void testListRolesWithRolePrincipalType() throws Exception { + metastoreClientDelegate.listRoles("user", PrincipalType.ROLE); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGetPrincipalsInRole() throws Exception { + metastoreClientDelegate.getPrincipalsInRole( + new org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest("role")); + } + + @Test(expected=UnsupportedOperationException.class) + public void testRoleGrantsForPrincipal() throws Exception { + metastoreClientDelegate.getRoleGrantsForPrincipal( + new org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest("user", + org.apache.hadoop.hive.metastore.api.PrincipalType.USER)); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGrantRole() throws Exception { + metastoreClientDelegate.grantRole("role", "user", + org.apache.hadoop.hive.metastore.api.PrincipalType.USER, "grantor", + org.apache.hadoop.hive.metastore.api.PrincipalType.ROLE, true); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGrantPrivileges() throws Exception { + metastoreClientDelegate.grantPrivileges(TestObjects.getPrivilegeBag()); + } + + @Test(expected=UnsupportedOperationException.class) + public void testRevokePrivileges() throws Exception { + metastoreClientDelegate.revokePrivileges(TestObjects.getPrivilegeBag(), false); + } + + @Test(expected=UnsupportedOperationException.class) + public void testListPrivileges() throws Exception { + String principal = "user1"; + org.apache.hadoop.hive.metastore.api.PrincipalType principalType = + org.apache.hadoop.hive.metastore.api.PrincipalType.USER; + + metastoreClientDelegate.listPrivileges(principal, principalType, TestObjects.getHiveObjectRef()); + } + + @Test + public void testGetPrincipalPrivilegeSet() throws Exception { + String user = "user1"; + List groupList = ImmutableList.of(); + org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet privilegeSet = metastoreClientDelegate + .getPrivilegeSet(TestObjects.getHiveObjectRef(), user, groupList); + + assertThat(privilegeSet, is(nullValue())); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGrantPrivilegesThrowingMetaException() throws Exception { + metastoreClientDelegate.grantPrivileges(TestObjects.getPrivilegeBag()); + } + + // ====================== Statistics ====================== + + @Test(expected=UnsupportedOperationException.class) + public void testDeletePartitionColumnStatisticsValid() throws Exception { + String databaseName = "database-name"; + String tableName = "table-name"; + String partitionName = "A=a/B=b"; + String columnName = "column-name"; + + metastoreClientDelegate.deletePartitionColumnStatistics(databaseName, tableName, partitionName, columnName); + } + + @Test(expected=UnsupportedOperationException.class) + public void testDeleteTableColumnStatistics() throws Exception { + String databaseName = "database-name"; + String tableName = "table-name"; + String columnName = "column-name"; + + metastoreClientDelegate.deleteTableColumnStatistics(databaseName, tableName, columnName); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGetPartitionColumnStatisticsValid() throws Exception { + String databaseName = "database-name"; + String tableName = "table-name"; + List partitionNames = ImmutableList.of("A=a/B=b", "A=x/B=y"); + List columnNames = ImmutableList.of("decimal-column", "string-column"); + + metastoreClientDelegate.getPartitionColumnStatistics(databaseName, tableName, partitionNames, columnNames); + } + + @Test(expected=UnsupportedOperationException.class) + public void testGetTableColumnStatistics() throws Exception { + String databaseName = "database-name"; + String tableName = "table-name"; + List columnNames = ImmutableList.of("decimal-column", "string-column"); + + metastoreClientDelegate.getTableColumnStatistics(databaseName, tableName, columnNames); + } + + @Test(expected=UnsupportedOperationException.class) + public void testUpdatePartitionColumnStatistics() throws Exception { + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics = + TestObjects.getHivePartitionColumnStatistics(); + + metastoreClientDelegate.updatePartitionColumnStatistics(columnStatistics); + } + + @Test(expected=UnsupportedOperationException.class) + public void testUpdateTableColumnStatistics() throws Exception { + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics = + TestObjects.getHiveTableColumnStatistics(); + + metastoreClientDelegate.updateTableColumnStatistics(columnStatistics); + } + + private void assertDaemonThreadPools() { + String threadNameCreatePrefix = + GlueMetastoreClientDelegate.GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT.substring( + 0, + GlueMetastoreClientDelegate.GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT.indexOf('%')); + for (Thread thread : Thread.getAllStackTraces().keySet()) { + String threadName = thread.getName(); + if (threadName != null && threadName.startsWith(threadNameCreatePrefix)) { + assertTrue(thread.isDaemon()); + } + } + } + + //==================== Functions ===================== + + @Test + public void getFunction() throws Exception { + UserDefinedFunction udf = createUserDefinedFunction(); + when(glueClient.getUserDefinedFunction(any(GetUserDefinedFunctionRequest.class))).thenReturn( + new GetUserDefinedFunctionResult().withUserDefinedFunction(udf)); + metastoreClientDelegateCatalogId.getFunction(testDb.getName(), "test-func"); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetUserDefinedFunctionRequest.class); + verify(glueClient, times(1)).getUserDefinedFunction(captor.capture()); + GetUserDefinedFunctionRequest request = captor.getValue(); + assertEquals(CATALOG_ID, request.getCatalogId()); + assertEquals(testDb.getName(), request.getDatabaseName()); + assertEquals("test-func", request.getFunctionName()); + } + + @Test + public void getFunctions() throws Exception { + UserDefinedFunction udf1 = createUserDefinedFunction(); + UserDefinedFunction udf2 = createUserDefinedFunction(); + + List udfList = new ArrayList<>(); + udfList.add(udf1); + udfList.add(udf2); + + when(glueClient.getUserDefinedFunctions(any(GetUserDefinedFunctionsRequest.class))).thenReturn( + new GetUserDefinedFunctionsResult().withUserDefinedFunctions(udfList).withNextToken(null)); + List result = metastoreClientDelegateCatalogId.getFunctions(testDb.getName(), "test-func"); + ArgumentCaptor captor = ArgumentCaptor + .forClass(GetUserDefinedFunctionsRequest.class); + verify(glueClient, times(1)).getUserDefinedFunctions(captor.capture()); + GetUserDefinedFunctionsRequest request = captor.getValue(); + assertEquals(CATALOG_ID, request.getCatalogId()); + assertEquals(testDb.getName(), request.getDatabaseName()); + assertEquals("test-func", request.getPattern()); + assertEquals(2, result.size()); + } + + @Test + public void testCreateFunction() throws Exception { + org.apache.hadoop.hive.metastore.api.Function hiveFunction = createHiveFunction(); + metastoreClientDelegateCatalogId.createFunction(hiveFunction); + ArgumentCaptor captor = ArgumentCaptor + .forClass(CreateUserDefinedFunctionRequest.class); + verify(glueClient, times(1)).createUserDefinedFunction(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testDropFunction() throws Exception { + metastoreClientDelegateCatalogId.dropFunction(testDb.getName(), "test-func"); + ArgumentCaptor captor = ArgumentCaptor + .forClass(DeleteUserDefinedFunctionRequest.class); + verify(glueClient, times(1)).deleteUserDefinedFunction(captor.capture()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + @Test + public void testAlterFunction() throws Exception { + org.apache.hadoop.hive.metastore.api.Function hiveFunction = createHiveFunction(); + metastoreClientDelegateCatalogId.alterFunction(testDb.getName(), "test-func", createHiveFunction()); + ArgumentCaptor captor = ArgumentCaptor + .forClass(UpdateUserDefinedFunctionRequest.class); + verify(glueClient, times(1)).updateUserDefinedFunction(captor.capture()); + UpdateUserDefinedFunctionRequest request = captor.getValue(); + assertEquals(testDb.getName(), request.getDatabaseName()); + assertEquals("test-func", request.getFunctionName()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + } + + private org.apache.hadoop.hive.metastore.api.Function createHiveFunction() { + org.apache.hadoop.hive.metastore.api.Function hiveFunction = new org.apache.hadoop.hive.metastore.api.Function(); + hiveFunction.setClassName("testClass"); + hiveFunction.setFunctionName("test-func"); + hiveFunction.setOwnerName("test-owner"); + hiveFunction.setOwnerType(org.apache.hadoop.hive.metastore.api.PrincipalType.USER); + return hiveFunction; + } + + private UserDefinedFunction createUserDefinedFunction() { + UserDefinedFunction udf = new UserDefinedFunction(); + udf.setFunctionName("test-func"); + udf.setClassName("test-class"); + udf.setCreateTime(new Date()); + udf.setOwnerName("test-owner"); + udf.setOwnerType(com.amazonaws.services.glue.model.PrincipalType.USER.name()); + return udf; + } + + // ==================== Schema ===================== + @Test + public void testGetFields() throws Exception { + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + List res = metastoreClientDelegateCatalogId.getFields(testDb.getName(), testTbl.getName()); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetTableRequest.class); + verify(glueClient, times(1)).getTable(captor.capture()); + GetTableRequest request = captor.getValue(); + assertEquals(testDb.getName(), request.getDatabaseName()); + assertEquals(testTbl.getName(), request.getName()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + assertEquals(1, res.size()); + } + + @Test + public void testGetSchema() throws Exception { + when(glueClient.getTable(any(GetTableRequest.class))).thenReturn(new GetTableResult().withTable(testTbl)); + List res = metastoreClientDelegateCatalogId.getFields(testDb.getName(), testTbl.getName()); + ArgumentCaptor captor = ArgumentCaptor.forClass(GetTableRequest.class); + verify(glueClient, times(1)).getTable(captor.capture()); + GetTableRequest request = captor.getValue(); + assertEquals(testDb.getName(), request.getDatabaseName()); + assertEquals(testTbl.getName(), request.getName()); + assertEquals(CATALOG_ID, captor.getValue().getCatalogId()); + assertEquals(1, res.size()); + } +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelperTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelperTest.java new file mode 100644 index 000000000..d7cc9e9dd --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelperTest.java @@ -0,0 +1,227 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.metastore.AWSGlueMetastore; +import com.amazonaws.services.glue.AWSGlue; +import com.amazonaws.services.glue.model.AlreadyExistsException; +import com.amazonaws.services.glue.model.BatchCreatePartitionRequest; +import com.amazonaws.services.glue.model.BatchCreatePartitionResult; +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.GetPartitionRequest; +import com.amazonaws.services.glue.model.GetPartitionResult; +import com.amazonaws.services.glue.model.InternalServiceException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionError; +import com.amazonaws.services.glue.model.ResourceNumberLimitExceededException; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import java.util.Collection; +import java.util.List; + +import static com.amazonaws.glue.catalog.util.TestObjects.getPartitionError; + +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +public class BatchCreatePartitionsHelperTest { + + @Mock + private AWSGlueMetastore awsGlueMetastore; + + private BatchCreatePartitionsHelper batchCreatePartitionsHelper; + + private static final String NAMESPACE_NAME = "ns"; + private static final String TABLE_NAME = "table"; + + @Before + public void init() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testCreatePartitionsEmpty() throws Exception { + mockBatchCreateSuccess(); + + List partitions = Lists.newArrayList(); + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false) + .createPartitions(); + + assertTrue(batchCreatePartitionsHelper.getPartitionsCreated().isEmpty()); + assertNull(batchCreatePartitionsHelper.getFirstTException()); + } + + @Test + public void testCreatePartitionsSucceed() throws Exception { + mockBatchCreateSuccess(); + + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List partitions = Lists.newArrayList( + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1), + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2)); + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false) + .createPartitions(); + + assertEquals(2, batchCreatePartitionsHelper.getPartitionsCreated().size()); + assertNull(batchCreatePartitionsHelper.getFirstTException()); + for (Partition partition : partitions) { + assertTrue(batchCreatePartitionsHelper.getPartitionsCreated().contains(partition)); + } + assertEquals(0, batchCreatePartitionsHelper.getPartitionsFailed().size()); + } + + @Test + public void testCreatePartitionsThrowsException() throws Exception { + Exception e = new RuntimeException("foo"); + mockBatchCreateThrowsException(e); + + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List partitions = Lists.newArrayList( + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1), + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2)); + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false); + batchCreatePartitionsHelper.createPartitions(); + + assertNotNull(batchCreatePartitionsHelper.getFirstTException()); + assertEquals("foo", batchCreatePartitionsHelper.getFirstTException().getMessage()); + assertEquals(partitions, batchCreatePartitionsHelper.getPartitionsFailed()); + assertTrue(batchCreatePartitionsHelper.getPartitionsCreated().isEmpty()); + } + + @Test + public void testCreatePartitionsThrowsServiceExceptionAndPartitionPartiallyCreated() throws Exception { + Exception e = new InternalServiceException("foo"); + mockBatchCreateThrowsException(e); + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List values3 = Lists.newArrayList("val3"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + Partition partition3 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values3); + List partitions = Lists.newArrayList(partition1, partition2, partition3); + Mockito.when(awsGlueMetastore.getPartition(Mockito.anyString(), Mockito.anyString(), Mockito.anyList())) + .thenReturn(partition1) + .thenThrow(new EntityNotFoundException("bar")) + .thenThrow(new NullPointerException("baz")); + + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false) + .createPartitions(); + + assertThat(batchCreatePartitionsHelper.getFirstTException(), is(instanceOf(MetaException.class))); + assertThat(batchCreatePartitionsHelper.getPartitionsCreated(), hasItems(partition1)); + assertThat(batchCreatePartitionsHelper.getPartitionsCreated(), not(hasItems(partition2, partition3))); + assertThat(batchCreatePartitionsHelper.getPartitionsFailed(), hasItems(partition2, partition3)); + assertThat(batchCreatePartitionsHelper.getPartitionsFailed(), not(hasItems(partition1))); + } + + @Test + public void testCreatePartitionsDuplicateValues() throws Exception { + mockBatchCreateSuccess(); + + List values1 = Lists.newArrayList("val1"); + Partition partition = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + List partitions = Lists.newArrayList(partition, partition); + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false) + .createPartitions(); + + assertEquals(1, batchCreatePartitionsHelper.getPartitionsCreated().size()); + assertNull(batchCreatePartitionsHelper.getFirstTException()); + for (Partition p : partitions) { + assertTrue(batchCreatePartitionsHelper.getPartitionsCreated().contains(p)); + } + assertTrue(batchCreatePartitionsHelper.getPartitionsFailed().isEmpty()); + } + + @Test + public void testCreatePartitionsWithFailure() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + List partitions = Lists.newArrayList(partition1, partition2); + + PartitionError error = getPartitionError(values1, new AlreadyExistsException("foo error msg")); + mockBatchCreateWithFailures(Lists.newArrayList(error)); + + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, false) + .createPartitions(); + + assertEquals(1, batchCreatePartitionsHelper.getPartitionsCreated().size()); + assertThat(batchCreatePartitionsHelper.getPartitionsCreated(), hasItem(partition2)); + assertThat(batchCreatePartitionsHelper.getFirstTException(), + is(instanceOf(org.apache.hadoop.hive.metastore.api.AlreadyExistsException.class))); + assertThat(batchCreatePartitionsHelper.getPartitionsFailed(), hasItem(partition1)); + } + + @Test + public void testCreatePartitionsWithFailureAllowExists() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + List partitions = Lists.newArrayList(partition1, partition2); + + PartitionError error = getPartitionError(values1, new AlreadyExistsException("foo error msg")); + mockBatchCreateWithFailures(Lists.newArrayList(error)); + + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, true) + .createPartitions(); + + assertEquals(1, batchCreatePartitionsHelper.getPartitionsCreated().size()); + assertThat(batchCreatePartitionsHelper.getPartitionsCreated(), hasItem(partition2)); + assertNull(batchCreatePartitionsHelper.getFirstTException()); + assertEquals(0, batchCreatePartitionsHelper.getPartitionsFailed().size()); + } + + @Test + public void testCreatePartitionsWithFailures() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + List partitions = Lists.newArrayList(partition1, partition2); + + PartitionError error1 = getPartitionError(values1, new ResourceNumberLimitExceededException("foo error msg")); + PartitionError error2 = getPartitionError(values2, new AlreadyExistsException("foo error msg2")); + mockBatchCreateWithFailures(Lists.newArrayList(error1, error2)); + + batchCreatePartitionsHelper = new BatchCreatePartitionsHelper(awsGlueMetastore, NAMESPACE_NAME, TABLE_NAME, null, partitions, true) + .createPartitions(); + + assertEquals(0, batchCreatePartitionsHelper.getPartitionsCreated().size()); + assertTrue(batchCreatePartitionsHelper.getFirstTException() instanceof MetaException); + assertEquals(1, batchCreatePartitionsHelper.getPartitionsFailed().size()); + assertThat(batchCreatePartitionsHelper.getPartitionsFailed(), hasItem(partition1)); + } + + private void mockBatchCreateSuccess() { + Mockito.when(awsGlueMetastore.createPartitions(Mockito.anyString(), Mockito.anyString(), + Mockito.anyList())).thenReturn(null); + } + + private void mockBatchCreateWithFailures(List errors) { + Mockito.when(awsGlueMetastore.createPartitions(Mockito.anyString(), Mockito.anyString(), Mockito.anyList())) + .thenReturn(errors); + } + + private void mockBatchCreateThrowsException(Exception e) { + Mockito.when(awsGlueMetastore.createPartitions(Mockito.anyString(), Mockito.anyString(), + Mockito.anyList())).thenThrow(e); + } + +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelperTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelperTest.java new file mode 100644 index 000000000..aaeb2f165 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelperTest.java @@ -0,0 +1,213 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.AWSGlue; +import com.amazonaws.services.glue.model.BatchDeletePartitionRequest; +import com.amazonaws.services.glue.model.BatchDeletePartitionResult; +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.GetPartitionRequest; +import com.amazonaws.services.glue.model.GetPartitionResult; +import com.amazonaws.services.glue.model.InternalServiceException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionError; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.InvalidInputException; +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.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; + +import java.util.Collection; +import java.util.List; + +import static com.amazonaws.glue.catalog.util.TestObjects.getPartitionError; + +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +public class BatchDeletePartitionsHelperTest { + + @Mock + private AWSGlue client; + + private BatchDeletePartitionsHelper batchDeletePartitionsHelper; + + private static final String NAMESPACE_NAME = "ns"; + private static final String TABLE_NAME = "table"; + + @Before + public void init() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testDeletePartitionsEmpty() throws Exception { + mockBatchDeleteSuccess(); + + List partitions = Lists.newArrayList(); + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().isEmpty()); + assertNull(batchDeletePartitionsHelper.getFirstTException()); + } + + @Test + public void testDeletePartitionsSucceed() throws Exception { + mockBatchDeleteSuccess(); + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List partitions = Lists.newArrayList( + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1), + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2)); + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertEquals(2, batchDeletePartitionsHelper.getPartitionsDeleted().size()); + assertNull(batchDeletePartitionsHelper.getFirstTException()); + for (Partition partition : partitions) { + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().contains(partition)); + } + } + + @Test + public void testDeletePartitionsThrowsRuntimeException() throws Exception { + Exception e = new NullPointerException("foo"); + mockBatchDeleteThrowsException(e); + + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List partitions = Lists.newArrayList( + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1), + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2)); + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions); + + batchDeletePartitionsHelper.deletePartitions(); + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().isEmpty()); + assertNotNull(batchDeletePartitionsHelper.getFirstTException()); + assertEquals("foo", batchDeletePartitionsHelper.getFirstTException().getMessage()); + } + + @Test + public void testDeletePartitionsThrowsInvalidInputException() throws Exception { + Exception e = new com.amazonaws.services.glue.model.InvalidInputException("foo"); + mockBatchDeleteThrowsException(e); + + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List partitions = Lists.newArrayList( + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1), + TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2)); + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions); + + batchDeletePartitionsHelper.deletePartitions(); + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().isEmpty()); + assertThat(batchDeletePartitionsHelper.getFirstTException(), is(instanceOf(InvalidObjectException.class))); + } + + @Test + public void testDeletePartitionsThrowsServiceException() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + List values3 = Lists.newArrayList("val3"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + Partition partition3 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values3); + List partitions = Lists.newArrayList(partition1, partition2, partition3); + + Exception e = new InternalServiceException("foo"); + mockBatchDeleteThrowsException(e); + Mockito.when(client.getPartition(Mockito.any(GetPartitionRequest.class))) + .thenReturn(new GetPartitionResult().withPartition(partition1)) + .thenThrow(new EntityNotFoundException("bar")) + .thenThrow(new NullPointerException("baz")); + + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertThat(batchDeletePartitionsHelper.getFirstTException(), is(instanceOf(MetaException.class))); + assertThat(batchDeletePartitionsHelper.getPartitionsDeleted(), hasItems(partition2)); + assertThat(batchDeletePartitionsHelper.getPartitionsDeleted(), not(hasItems(partition1, partition3))); + } + + @Test + public void testDeletePartitionsDuplicateValues() throws Exception { + mockBatchDeleteSuccess(); + + List values1 = Lists.newArrayList("val1"); + Partition partition = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + List partitions = Lists.newArrayList(partition, partition); + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertEquals(1, batchDeletePartitionsHelper.getPartitionsDeleted().size()); + assertNull(batchDeletePartitionsHelper.getFirstTException()); + for (Partition p : partitions) { + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().contains(p)); + } + } + + @Test + public void testDeletePartitionsWithFailure() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + List partitions = Lists.newArrayList(partition1, partition2); + + PartitionError error = getPartitionError(values1, new EntityNotFoundException("foo error msg")); + mockBatchDeleteWithFailures(Lists.newArrayList(error)); + + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertEquals(1, batchDeletePartitionsHelper.getPartitionsDeleted().size()); + assertTrue(batchDeletePartitionsHelper.getPartitionsDeleted().contains(partition2)); + assertTrue(batchDeletePartitionsHelper.getFirstTException() instanceof NoSuchObjectException); + } + + @Test + public void testDeletePartitionsWithFailures() throws Exception { + List values1 = Lists.newArrayList("val1"); + List values2 = Lists.newArrayList("val2"); + Partition partition1 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values1); + Partition partition2 = TestObjects.getTestPartition(NAMESPACE_NAME, TABLE_NAME, values2); + List partitions = Lists.newArrayList(partition1, partition2); + + PartitionError error1 = getPartitionError(values1, new EntityNotFoundException("foo error msg")); + PartitionError error2 = getPartitionError(values2, new InvalidInputException("foo error msg2")); + mockBatchDeleteWithFailures(Lists.newArrayList(error1, error2)); + + batchDeletePartitionsHelper = new BatchDeletePartitionsHelper(client, NAMESPACE_NAME, TABLE_NAME, null, partitions) + .deletePartitions(); + + assertEquals(0, batchDeletePartitionsHelper.getPartitionsDeleted().size()); + assertTrue(batchDeletePartitionsHelper.getFirstTException() instanceof NoSuchObjectException); + } + + private void mockBatchDeleteSuccess() { + Mockito.when(client.batchDeletePartition(Mockito.any(BatchDeletePartitionRequest.class))) + .thenReturn(new BatchDeletePartitionResult()); + } + + private void mockBatchDeleteWithFailures(Collection errors) { + Mockito.when(client.batchDeletePartition(Mockito.any(BatchDeletePartitionRequest.class))) + .thenReturn(new BatchDeletePartitionResult().withErrors(errors)); + } + + private void mockBatchDeleteThrowsException(Exception e) { + Mockito.when(client.batchDeletePartition(Mockito.any(BatchDeletePartitionRequest.class))).thenThrow(e); + } + +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/ExprBuilder.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/ExprBuilder.java new file mode 100644 index 000000000..e7ec2d23a --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/ExprBuilder.java @@ -0,0 +1,106 @@ +package com.amazonaws.glue.catalog.util; + +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +import java.sql.Date; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.List; +import java.util.Stack; + +/** + * Utility to craft mock expression trees. Inspired by org.apache.hadoop.hive.metastore.TestMetastoreExpr + */ +public class ExprBuilder { + + private final String tblName; + private final Stack stack = new Stack<>(); + + public ExprBuilder(String tblName) { + this.tblName = tblName; + } + + public ExprNodeGenericFuncDesc build() throws Exception { + if (stack.size() != 1) { + throw new Exception("Bad test: " + stack.size()); + } + return (ExprNodeGenericFuncDesc) stack.pop(); + } + + public ExprBuilder pred(String name, int args) throws Exception { + return fn(name, TypeInfoFactory.booleanTypeInfo, args); + } + + private ExprBuilder fn(String name, TypeInfo ti, int args) throws Exception { + List children = new ArrayList<>(); + for (int i = 0; i < args; ++i) { + children.add(stack.pop()); + } + stack.push(new ExprNodeGenericFuncDesc(ti, FunctionRegistry.getFunctionInfo(name).getGenericUDF(), children)); + return this; + } + + public ExprBuilder strCol(String col) { + return colInternal(TypeInfoFactory.stringTypeInfo, col, true); + } + + public ExprBuilder timestampCol(String col) { + return colInternal(TypeInfoFactory.timestampTypeInfo, col, true); + } + + public ExprBuilder booleanCol(String col) { + return colInternal(TypeInfoFactory.booleanTypeInfo, col, true); + } + + public ExprBuilder charCol(String col) { + return colInternal(TypeInfoFactory.charTypeInfo, col, true); + } + + public ExprBuilder dateCol(String col) { + return colInternal(TypeInfoFactory.dateTypeInfo, col, true); + } + + private ExprBuilder colInternal(TypeInfo ti, String col, boolean part) { + stack.push(new ExprNodeColumnDesc(ti, col, tblName, part)); + return this; + } + + public ExprBuilder val(boolean val) { + return valInternal(TypeInfoFactory.booleanTypeInfo, val); + } + + public ExprBuilder val(String val) { + return valInternal(TypeInfoFactory.stringTypeInfo, val); + } + + public ExprBuilder vals(List vals) { + for (String s : vals) { + val(s); + } + return this; + } + + public ExprBuilder val(Timestamp val) { + return valInternal(TypeInfoFactory.timestampTypeInfo, val); + } + + public ExprBuilder val(Character val) { + return valInternal(TypeInfoFactory.charTypeInfo, val); + } + + public ExprBuilder val(Date val) { + return valInternal(TypeInfoFactory.timestampTypeInfo, val); + } + + private ExprBuilder valInternal(TypeInfo ti, Object val) { + stack.push(new ExprNodeConstantDesc(ti, val)); + return this; + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/GlueTestClientFactory.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/GlueTestClientFactory.java new file mode 100644 index 000000000..1216a2e6e --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/GlueTestClientFactory.java @@ -0,0 +1,60 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; +import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; +import com.amazonaws.glue.catalog.metastore.GlueClientFactory; +import com.amazonaws.retry.PredefinedRetryPolicies; +import com.amazonaws.retry.RetryPolicy; +import com.amazonaws.services.glue.AWSGlue; +import com.amazonaws.services.glue.AWSGlueClientBuilder; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.metastore.api.MetaException; + +public final class GlueTestClientFactory implements GlueClientFactory { + + private static final int SC_GATEWAY_TIMEOUT = 504; + + @Override + public AWSGlue newClient() throws MetaException { + AWSGlueClientBuilder glueClientBuilder = AWSGlueClientBuilder.standard() + .withClientConfiguration(createGatewayTimeoutRetryableConfiguration()) + .withCredentials(new DefaultAWSCredentialsProviderChain()); + + String endpoint = System.getProperty("endpoint"); + if (StringUtils.isNotBlank(endpoint)) { + glueClientBuilder.setEndpointConfiguration(new EndpointConfiguration(endpoint, null)); + } + + return glueClientBuilder.build(); + } + + private static ClientConfiguration createGatewayTimeoutRetryableConfiguration() { + ClientConfiguration retryableConfig = new ClientConfiguration(); + RetryPolicy.RetryCondition retryCondition = new PredefinedRetryPolicies.SDKDefaultRetryCondition() { + @Override + public boolean shouldRetry(AmazonWebServiceRequest originalRequest, AmazonClientException exception, + int retriesAttempted) { + if (super.shouldRetry(originalRequest, exception, retriesAttempted)) { + return true; + } + if (exception != null && exception instanceof AmazonServiceException) { + AmazonServiceException ase = (AmazonServiceException) exception; + if (ase.getStatusCode() == SC_GATEWAY_TIMEOUT) { + return true; + } + } + return false; + } + }; + RetryPolicy retryPolicy = new RetryPolicy(retryCondition, PredefinedRetryPolicies.DEFAULT_BACKOFF_STRATEGY, + PredefinedRetryPolicies.DEFAULT_MAX_ERROR_RETRY, true); + retryableConfig.setRetryPolicy(retryPolicy); + return retryableConfig; + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/HiveTableValidatorTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/HiveTableValidatorTest.java new file mode 100644 index 000000000..ba66d68df --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/HiveTableValidatorTest.java @@ -0,0 +1,92 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.InvalidInputException; +import com.amazonaws.services.glue.model.Table; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.apache.hadoop.hive.metastore.TableType; + +import static com.amazonaws.glue.catalog.util.HiveTableValidator.REQUIRED_PROPERTIES_VALIDATOR; +import static com.amazonaws.glue.catalog.util.TestObjects.getTestTable; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE; + +public class HiveTableValidatorTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + private static final String EXPECTED_MESSAGE = "%s cannot be null"; + + @Test + public void testRequiredProperty_TableType() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "TableType")); + Table tbl = getTestTable().withTableType(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_StorageDescriptor() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "StorageDescriptor")); + Table tbl = getTestTable().withStorageDescriptor(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_InputFormat() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "StorageDescriptor#InputFormat")); + Table tbl = getTestTable(); + tbl.getStorageDescriptor().setInputFormat(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_OutputFormat() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "StorageDescriptor#OutputFormat")); + Table tbl = getTestTable(); + tbl.getStorageDescriptor().setOutputFormat(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_SerdeInfo() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "StorageDescriptor#SerdeInfo")); + Table tbl = getTestTable(); + tbl.getStorageDescriptor().setSerdeInfo(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_SerializationLibrary() { + thrown.expect(InvalidInputException.class); + thrown.expectMessage(String.format(EXPECTED_MESSAGE, "StorageDescriptor#SerdeInfo#SerializationLibrary")); + Table tbl = getTestTable(); + tbl.getStorageDescriptor().getSerdeInfo().setSerializationLibrary(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testRequiredProperty_ValidTable() { + REQUIRED_PROPERTIES_VALIDATOR.validate(getTestTable()); + } + + @Test + public void testValidate_ViewTableType() { + Table tbl = getTestTable(); + tbl.setTableType(TableType.VIRTUAL_VIEW.name()); + tbl.getStorageDescriptor().getSerdeInfo().setSerializationLibrary(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } + + @Test + public void testValidate_ExcludeStorageHandlerType() { + Table tbl = getTestTable(); + tbl.getParameters().put(META_TABLE_STORAGE, "org.apache.hadoop.hive.dynamodb.DynamoDBStorageHandler"); + tbl.getStorageDescriptor().setInputFormat(null); + REQUIRED_PROPERTIES_VALIDATOR.validate(tbl); + } +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/LoggingHelperTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/LoggingHelperTest.java new file mode 100644 index 000000000..699cdd681 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/LoggingHelperTest.java @@ -0,0 +1,69 @@ +package com.amazonaws.glue.catalog.util; + +import com.google.common.collect.ImmutableList; +import org.junit.Test; + +import java.util.Collection; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +public class LoggingHelperTest { + + @Test + public void concatCollectionToStringForLoggingTest() { + Collection logs = ImmutableList.of("test_log_1", "test_log_2", "test_log_3"); + String delimiter = "|"; + + String result = LoggingHelper.concatCollectionToStringForLogging(logs, delimiter); + String expected = "test_log_1|test_log_2|test_log_3|"; + + assertThat(result, is(equalTo(expected))); + } + + @Test + public void concatCollectionToStringForLoggingTestWithoutCollection() { + String delimiter = "|"; + + String result = LoggingHelper.concatCollectionToStringForLogging(null, delimiter); + String expected = ""; + + assertThat(result, is(equalTo(expected))); + } + + @Test + public void concatCollectionToStringForLoggingTestWithoutDelimiter() { + Collection logs = ImmutableList.of("test_log_1", "test_log_2", "test_log_3"); + + String result = LoggingHelper.concatCollectionToStringForLogging(logs, null); + String expected = "test_log_1,test_log_2,test_log_3,"; + + assertThat(result, is(equalTo(expected))); + } + + @Test + public void concatCollectionToStringForLoggingTestWithLongerThanLimitInput() { + ImmutableList.Builder listBuilder = new ImmutableList.Builder<>(); + + final int max = 2000; + final String key = "KEY"; + final StringBuilder stringBuilder = new StringBuilder(); + for (int i = 0; i < max; i += key.length()) { + listBuilder.add(key); + stringBuilder.append(key); + } + final String overflow = "OVERFLOW"; + for (int i = 0; i < 100; i += overflow.length()) { + listBuilder.add(overflow); + } + + String result = LoggingHelper.concatCollectionToStringForLogging(listBuilder.build(), ""); + String expected = stringBuilder.toString().substring(0, max); + + assertThat(result.length(), is(equalTo(max))); + assertThat(result, is(equalTo(expected))); + assertThat(expected.indexOf(overflow), is(equalTo(-1))); + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/MetastoreClientUtilsTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/MetastoreClientUtilsTest.java new file mode 100644 index 000000000..7b4e6ffda --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/MetastoreClientUtilsTest.java @@ -0,0 +1,126 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; + +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.junit.Before; +import org.junit.Test; + +import java.util.Map; + +import static com.amazonaws.glue.catalog.util.TestObjects.getTestDatabase; +import static com.amazonaws.glue.catalog.util.TestObjects.getTestTable; +import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MetastoreClientUtilsTest { + + private Warehouse wh; + private HiveConf conf; + + private Path testPath; + private static final String TEST_LOCATION = "s3://mybucket/"; + private Database testDb; + private Table testTbl; + + @Before + public void setup() { + wh = mock(Warehouse.class); + conf = new HiveConf(); + testPath = new Path(TEST_LOCATION); + testDb = CatalogToHiveConverter.convertDatabase(getTestDatabase()); + testTbl = CatalogToHiveConverter.convertTable(getTestTable(), testDb.getName()); + } + + @Test(expected = NullPointerException.class) + public void testMakeDirsNullWh() throws Exception { + MetastoreClientUtils.makeDirs(null, testPath); + } + + @Test(expected = NullPointerException.class) + public void testMakeDirsNullPath() throws Exception { + MetastoreClientUtils.makeDirs(wh, null); + } + + @Test + public void testMakeDirsAlreadyExists() throws Exception { + when(wh.isDir(testPath)).thenReturn(true); + assertFalse(MetastoreClientUtils.makeDirs(wh, testPath)); + } + + @Test(expected = MetaException.class) + public void testMakeDirsCannotCreateDir() throws Exception { + when(wh.isDir(testPath)).thenReturn(false); + when(wh.mkdirs(testPath, true)).thenReturn(false); + MetastoreClientUtils.makeDirs(wh, testPath); + } + + @Test(expected = InvalidObjectException.class) + public void testValidateTableObjectInvalidName() throws Exception { + testTbl.setTableName("!"); + MetastoreClientUtils.validateTableObject(testTbl, conf); + } + + @Test(expected = InvalidObjectException.class) + public void testValidateTableObjectInvalidColumnName() throws Exception { + testTbl.getSd().getCols().get(0).setType("invalidtype"); + MetastoreClientUtils.validateTableObject(testTbl, conf); + } + + @Test(expected = InvalidObjectException.class) + public void testValidateTableObjectInvalidPartitionKeys() throws Exception { + testTbl.getPartitionKeys().get(0).setType("invalidtype"); + MetastoreClientUtils.validateTableObject(testTbl, conf); + } + + @Test + public void testDeepCopy() throws Exception { + Map orig = ImmutableMap.of("key", "val"); + Map deepCopy = MetastoreClientUtils.deepCopyMap(orig); + assertNotSame(deepCopy, orig); + assertEquals(deepCopy, orig); + } + + @Test + public void testIsExternalTableFalse() { + assertFalse(MetastoreClientUtils.isExternalTable(testTbl)); + } + + @Test + public void testIsExternalTableParamTrue() { + testTbl.getParameters().put("EXTERNAL", "true"); + assertTrue(MetastoreClientUtils.isExternalTable(testTbl)); + } + + @Test + public void testIsExternalTableTableTypeTrue() { + testTbl.setTableType(EXTERNAL_TABLE.name()); + testTbl.setParameters(null); + assertTrue(MetastoreClientUtils.isExternalTable(testTbl)); + } + + @Test + public void testIsExternalTableParamPriority() { + // parameters has higher priority when there is conflict + testTbl.getParameters().put("EXTERNAL", "false"); + testTbl.setTableType(EXTERNAL_TABLE.name()); + assertFalse(MetastoreClientUtils.isExternalTable(testTbl)); + } + + @Test + public void testIsExternalTableNull() { + assertFalse(MetastoreClientUtils.isExternalTable(null)); + } +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionKeyTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionKeyTest.java new file mode 100644 index 000000000..4af25e4f5 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionKeyTest.java @@ -0,0 +1,100 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionError; + +import com.google.common.collect.Lists; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class PartitionKeyTest { + + @Test(expected=IllegalArgumentException.class) + public void testNull() { + new PartitionKey((List) null); + } + + @Test + public void testEqualsDifferentTable() { + List values1 = Lists.newArrayList("value1", "value2"); + List values2 = Lists.newArrayList("value1", "value2"); + Partition partition1 = TestObjects.getTestPartition("ns", "table1", values1); + Partition partition2 = TestObjects.getTestPartition("ns", "table2", values2); + PartitionKey partitionKey1 = new PartitionKey(partition1); + PartitionKey partitionKey2 = new PartitionKey(partition2); + assertEquals(partitionKey1, partitionKey2); + assertEquals(partitionKey1.hashCode(), partitionKey2.hashCode()); + } + + @Test + public void testEqualsEmptyValue() { + List values = Lists.newArrayList(); + Partition partition1 = TestObjects.getTestPartition("ns", "table", values); + Partition partition2 = TestObjects.getTestPartition("ns", "table", values); + PartitionKey partitionKey1 = new PartitionKey(partition1); + PartitionKey partitionKey2 = new PartitionKey(partition2); + assertEquals(partitionKey1, partitionKey2); + assertEquals(partitionKey1.hashCode(), partitionKey2.hashCode()); + } + + @Test + public void testEqualsDifferentClass() { + List values1 = Lists.newArrayList("value1", "value2"); + List values2 = Lists.newLinkedList(values1); + Partition partition1 = TestObjects.getTestPartition("ns", "table", values1); + Partition partition2 = TestObjects.getTestPartition("ns", "table", values2); + PartitionKey partitionKey1 = new PartitionKey(partition1); + PartitionKey partitionKey2 = new PartitionKey(partition2); + assertEquals(partitionKey1, partitionKey2); + assertEquals(partitionKey1.hashCode(), partitionKey2.hashCode()); + } + + @Test + public void testEqualsPartitionError() { + List values1 = Lists.newArrayList("value1", "value2"); + List values2 = Lists.newArrayList("value1", "value2"); + PartitionError partitionError1 = TestObjects.getPartitionError(values1, new RuntimeException("foo")); + PartitionError partitionError2 = TestObjects.getPartitionError(values2, new Exception("foo2")); + PartitionKey partitionKey1 = new PartitionKey(partitionError1.getPartitionValues()); + PartitionKey partitionKey2 = new PartitionKey(partitionError2.getPartitionValues()); + assertEquals(partitionKey1, partitionKey2); + assertEquals(partitionKey1.hashCode(), partitionKey2.hashCode()); + } + + @Test + public void testEqualsPartitionAndPartitionError() { + List values1 = Lists.newArrayList("value1", "value2"); + List values2 = Lists.newArrayList("value1", "value2"); + Partition partition = TestObjects.getTestPartition("ns", "table", values1); + PartitionError partitionError = TestObjects.getPartitionError(values2, new RuntimeException("foo")); + PartitionKey partitionKey1 = new PartitionKey(partition); + PartitionKey partitionKey2 = new PartitionKey(partitionError.getPartitionValues()); + assertEquals(partitionKey1, partitionKey2); + assertEquals(partitionKey1.hashCode(), partitionKey2.hashCode()); + } + + @Test + public void testEqualsNull() { + List values = Lists.newArrayList("value1", "value2"); + Partition partition = TestObjects.getTestPartition("ns", "table", values); + PartitionKey partitionKey = new PartitionKey(partition); + assertFalse(partitionKey.equals(null)); + } + + @Test + public void testGetValues() { + List values = Lists.newArrayList("value1", "value2"); + Partition partition = TestObjects.getTestPartition("ns", "table", values); + PartitionKey partitionKey1 = new PartitionKey(partition); + assertEquals(Lists.newArrayList(values), partitionKey1.getValues()); + + PartitionError partitionError = TestObjects.getPartitionError(values, new RuntimeException("foo")); + PartitionKey partitionKey2 = new PartitionKey(partitionError.getPartitionValues()); + assertEquals(Lists.newArrayList(values), partitionKey2.getValues()); + } + +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionUtilsTest.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionUtilsTest.java new file mode 100644 index 000000000..600d18644 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/PartitionUtilsTest.java @@ -0,0 +1,61 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.EntityNotFoundException; +import com.amazonaws.services.glue.model.InternalServiceException; +import com.amazonaws.services.glue.model.InvalidInputException; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PartitionValueList; +import com.amazonaws.services.glue.model.ResourceNumberLimitExceededException; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +public class PartitionUtilsTest { + + @Test + public void testBuildPartitionMapAndGetPartitionValuesList() { + String namespaceName = "ns"; + String tableName = "table"; + // choose special values to make values1.hashCode() == values2.hashCode() + List values1 = Lists.newArrayList("Aa"); + List values2 = Lists.newArrayList("BB"); + Partition partition1 = TestObjects.getTestPartition(namespaceName, tableName, values1); + Partition partition2 = TestObjects.getTestPartition(namespaceName, tableName, values2); + Map partitionMap = PartitionUtils.buildPartitionMap(Lists.newArrayList(partition1, partition2)); + List partitionValuesList = PartitionUtils.getPartitionValuesList(partitionMap); + + assertEquals(2, partitionMap.size()); + Set> valuesSet = Sets.newHashSet(values1, values2); + for (PartitionKey partitionKey : partitionMap.keySet()) { + assertThat(valuesSet, hasItem(partitionKey.getValues())); + assertThat(partitionMap.get(partitionKey).getValues(), equalTo(partitionKey.getValues())); + } + + assertEquals(2, partitionValuesList.size()); + for (PartitionValueList partitionValueList : partitionValuesList) { + assertThat(valuesSet, hasItem(partitionValueList.getValues())); + } + } + + @Test + public void testIsInvalidUserInputException() { + assertTrue(PartitionUtils.isInvalidUserInputException(new InvalidInputException("foo"))); + assertTrue(PartitionUtils.isInvalidUserInputException(new EntityNotFoundException("bar"))); + assertFalse(PartitionUtils.isInvalidUserInputException(new InternalServiceException("bar2"))); + assertFalse(PartitionUtils.isInvalidUserInputException(new ResourceNumberLimitExceededException("bar3"))); + assertFalse(PartitionUtils.isInvalidUserInputException(new NullPointerException("bar4"))); + } + +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorService.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorService.java new file mode 100644 index 000000000..d6f8f28b1 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorService.java @@ -0,0 +1,11 @@ +package com.amazonaws.glue.catalog.util; + +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadFactory; + +public class TestExecutorService extends ScheduledThreadPoolExecutor { + + public TestExecutorService(int corePoolSize, ThreadFactory factory) { + super(corePoolSize, factory); + } +} \ No newline at end of file diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorServiceFactory.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorServiceFactory.java new file mode 100644 index 000000000..c55aba8d6 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestExecutorServiceFactory.java @@ -0,0 +1,16 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.glue.catalog.metastore.ExecutorServiceFactory; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.hive.conf.HiveConf; + +import java.util.concurrent.ExecutorService; + +public class TestExecutorServiceFactory implements ExecutorServiceFactory { + private static ExecutorService execService = new TestExecutorService(1, new ThreadFactoryBuilder().build()); + + @Override + public ExecutorService getExecutorService(HiveConf conf) { + return execService; + } +} diff --git a/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestObjects.java b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestObjects.java new file mode 100644 index 000000000..c8964a8b6 --- /dev/null +++ b/waggle-dance-glue/src/test/java/com/amazonaws/glue/catalog/util/TestObjects.java @@ -0,0 +1,370 @@ +package com.amazonaws.glue.catalog.util; + +import com.amazonaws.services.glue.model.Database; +import com.amazonaws.services.glue.model.ErrorDetail; +import com.amazonaws.services.glue.model.PartitionError; +import com.amazonaws.services.glue.model.UserDefinedFunction; +import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter; +import com.amazonaws.glue.catalog.converters.HiveToCatalogConverter; +import com.amazonaws.services.glue.model.Column; +import com.amazonaws.services.glue.model.Order; +import com.amazonaws.services.glue.model.Partition; +import com.amazonaws.services.glue.model.PrincipalType; +import com.amazonaws.services.glue.model.ResourceType; +import com.amazonaws.services.glue.model.ResourceUri; +import com.amazonaws.services.glue.model.SerDeInfo; +import com.amazonaws.services.glue.model.SkewedInfo; +import com.amazonaws.services.glue.model.StorageDescriptor; +import com.amazonaws.services.glue.model.Table; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData; +import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData; +import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData; +import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData; +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.LongColumnStatsData; +import org.apache.hadoop.hive.metastore.api.PrivilegeBag; +import org.apache.hadoop.hive.metastore.api.Role; +import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; + +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +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 final class TestObjects { + + private TestObjects() { + } + + /** + * + * @return a test db + */ + public static Database getTestDatabase() { + + Map parameters = Maps.newHashMap(); + parameters.put("param1", "value1"); + parameters.put("param2", "value2"); + + Database database = new Database() + .withName("test-db-" + UUID.randomUUID().toString().replaceAll("[^a-zA-Z0-9]+", "")) + .withDescription("database desc") + .withLocationUri("/db") + .withParameters(parameters); + + return database; + } + + /** + * + * @param len + * @return a random string of size len + */ + public static String getStringOfLength(final int len) { + StringBuffer sb = new StringBuffer(UUID.randomUUID().toString()); + for (int i = sb.length(); i < len; i++) { + sb.append('0'); + } + return sb.toString(); + } + + public static Table getTestTable() { + Table table = new Table(); + table.setName("testtable" + UUID.randomUUID().toString().replaceAll("[^a-zA-Z0-9]+", "").substring(0,4)); + table.setOwner("owner"); + table.setCreateTime(new Date(System.currentTimeMillis() / 1000 * 1000)); + table.setLastAccessTime(new Date(System.currentTimeMillis() / 1000 * 1000)); + table.setParameters(new HashMap()); + table.setPartitionKeys(getTestFieldList()); + table.setStorageDescriptor(getTestStorageDescriptor()); + table.setTableType("MANAGED_TABLE"); + table.setRetention(1); + table.setViewOriginalText("originalText"); + table.setViewExpandedText("expandedText"); + return table; + } + + public static Table getTestTable(String dbName) { + Table table = getTestTable(); + table.setDatabaseName(dbName); + return table; + } + + public static StorageDescriptor getTestStorageDescriptor() { + StorageDescriptor sd = new StorageDescriptor(); + List cols = new ArrayList<>(); + cols.add("sampleCols"); + sd.setBucketColumns(cols); + sd.setColumns(getTestFieldList()); + sd.setParameters(new HashMap()); + sd.setSerdeInfo(getTestSerdeInfo()); + sd.setSkewedInfo(getSkewedInfo()); + sd.setSortColumns(new ArrayList()); + sd.setInputFormat("inputFormat"); + sd.setOutputFormat("outputFormat"); + sd.setLocation("/test-table"); + sd.withSortColumns(new Order().withColumn("foo").withSortOrder(1)); + sd.setCompressed(false); + sd.setStoredAsSubDirectories(false); + sd.setNumberOfBuckets(0); + return sd; + } + + public static SerDeInfo getTestSerdeInfo() { + return new SerDeInfo() + .withName("serdeName") + .withSerializationLibrary("serdeLib") + .withParameters(new HashMap()); + } + + public static List getTestFieldList() { + List fieldList = new ArrayList<>(); + Column field = new Column() + .withComment(UUID.randomUUID().toString()) + .withName("column" + UUID.randomUUID().toString().replaceAll("[^a-zA-Z0-9]+", "")) + .withType("string"); + fieldList.add(field); + return fieldList; + } + + public static Index getTestHiveIndex(final String dbName) { + Index index = new Index(); + index.setIndexName("testIndex" + UUID.randomUUID().toString().replaceAll("[^a-zA-Z0-9]+", "")); + index.setCreateTime((int)(System.currentTimeMillis() / 1000)); + index.setLastAccessTime((int)(System.currentTimeMillis() / 1000)); + index.setDbName(dbName); + index.setDeferredRebuild(false); + index.setOrigTableName("OriginalTable"); + index.setIndexTableName("IndexTable"); + index.setIndexHandlerClass("handlerClass"); + index.setParameters(new HashMap()); + index.setSd(CatalogToHiveConverter.convertStorageDescriptor(getTestStorageDescriptor())); + + return index; + } + + public static void setIndexParametersForIndexTable(Table indexTable, String dbName, String originTableName) { + indexTable.getParameters().put(INDEX_DEFERRED_REBUILD, "FALSE"); + indexTable.getParameters().put(INDEX_HANDLER_CLASS, "handlerClass"); + indexTable.getParameters().put(INDEX_DB_NAME, dbName); + indexTable.getParameters().put(INDEX_ORIGIN_TABLE_NAME, originTableName); + indexTable.getParameters().put(INDEX_TABLE_NAME, indexTable.getName()); + } + + public static SkewedInfo getSkewedInfo() { + List skewedName = new ArrayList<>(); + List skewedValue = new ArrayList<>(); + List skewedMapKey = new ArrayList<>(); + List> skewedValueList = new ArrayList<>(); + skewedName.add(UUID.randomUUID().toString()); + skewedName.add(UUID.randomUUID().toString()); + skewedValue.add(UUID.randomUUID().toString()); + skewedValue.add(UUID.randomUUID().toString()); + skewedValueList.add(skewedValue); + skewedMapKey.add(UUID.randomUUID().toString()); + skewedMapKey.add(UUID.randomUUID().toString()); + Map skewedMap = new HashMap<>(); + skewedMap.put(HiveToCatalogConverter.convertListToString(skewedMapKey), UUID.randomUUID().toString()); + + return new SkewedInfo().withSkewedColumnValueLocationMaps(skewedMap).withSkewedColumnNames(skewedName) + .withSkewedColumnValues(HiveToCatalogConverter.convertSkewedValue(skewedValueList)); + } + + public static Partition getTestPartition(String dbName, String tblName, List values) { + return new Partition() + .withDatabaseName(dbName) + .withTableName(tblName) + .withValues(values) + .withCreationTime(new Date(System.currentTimeMillis() / 1000 * 1000)) + .withLastAccessTime(new Date(System.currentTimeMillis() / 1000 * 1000)) + .withParameters(Maps.newHashMap()) + .withStorageDescriptor(TestObjects.getTestStorageDescriptor()); + } + + public static UserDefinedFunction getCatalogTestFunction() { + List resourceUriList = Lists.newArrayList(new ResourceUri().withUri("s3://abc/def.jar") + .withResourceType(ResourceType.JAR), new ResourceUri().withUri("hdfs://ghi/jkl.jar") + .withResourceType(ResourceType.ARCHIVE)); + return new UserDefinedFunction().withFunctionName("functionname").withClassName("classname").withOwnerName("ownername") + .withCreateTime(new Date(System.currentTimeMillis() / 1000 * 1000)) + .withOwnerType(PrincipalType.USER).withResourceUris(resourceUriList); + } + + + private static ByteBuffer byteBuffer(long value) { + return ByteBuffer.wrap(BigInteger.valueOf(value).toByteArray()); + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveBinaryColumnStatsData() { + BinaryColumnStatsData statsData = new BinaryColumnStatsData(); + statsData.setAvgColLen(12.3); + statsData.setMaxColLen(45L); + statsData.setNumNulls(56L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setBinaryStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveBooleanColumnStatsData() { + BooleanColumnStatsData statsData = new BooleanColumnStatsData(); + statsData.setNumFalses(12L); + statsData.setNumNulls(34L); + statsData.setNumTrues(56L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setBooleanStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveDecimalColumnStatsData() { + DecimalColumnStatsData statsData = new DecimalColumnStatsData(); + org.apache.hadoop.hive.metastore.api.Decimal highValue = new org.apache.hadoop.hive.metastore.api.Decimal(); + highValue.setScale((short) 1); + highValue.setUnscaled(BigInteger.valueOf(1234L).toByteArray()); + statsData.setHighValue(highValue); + org.apache.hadoop.hive.metastore.api.Decimal lowValue = new org.apache.hadoop.hive.metastore.api.Decimal(); + lowValue.setScale((short) 4); + lowValue.setUnscaled(BigInteger.valueOf(5678L).toByteArray()); + statsData.setLowValue(lowValue); + statsData.setNumDVs(12L); + statsData.setNumNulls(56L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setDecimalStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveDoubleColumnStatsData() { + DoubleColumnStatsData statsData = new DoubleColumnStatsData(); + statsData.setHighValue(9999.9); + statsData.setLowValue(-1111.1); + statsData.setNumDVs(123L); + statsData.setNumNulls(456L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setDoubleStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveLongColumnStatsData() { + LongColumnStatsData statsData = new LongColumnStatsData(); + statsData.setHighValue(9999L); + statsData.setLowValue(-1111L); + statsData.setNumDVs(123L); + statsData.setNumNulls(456L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setLongStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatisticsData getHiveStringColumnStatsData() { + StringColumnStatsData statsData = new StringColumnStatsData(); + statsData.setAvgColLen(123.4); + statsData.setMaxColLen(567L); + statsData.setNumDVs(89L); + statsData.setNumNulls(13L); + org.apache.hadoop.hive.metastore.api.ColumnStatisticsData statsWrapper = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsData(); + statsWrapper.setStringStats(statsData); + return statsWrapper; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatistics getHiveTableColumnStatistics() { + org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc columnStatisticsDesc = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc(); + columnStatisticsDesc.setDbName("database-name"); + columnStatisticsDesc.setTableName("table-name"); + columnStatisticsDesc.setIsTblLevel(true); + columnStatisticsDesc.setLastAnalyzed(12345); + + org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj decimalObj = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj(); + decimalObj.setColName("decimal-column"); + decimalObj.setColType("decimal(9,6)"); + decimalObj.setStatsData(TestObjects.getHiveDecimalColumnStatsData()); + + org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj longObj = + new org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj(); + longObj.setColName("long-column"); + longObj.setColType("integer"); + longObj.setStatsData(TestObjects.getHiveLongColumnStatsData()); + + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics = + new org.apache.hadoop.hive.metastore.api.ColumnStatistics(); + columnStatistics.setStatsDesc(columnStatisticsDesc); + columnStatistics.setStatsObj(Arrays.asList(decimalObj, longObj)); + + return columnStatistics; + } + + public static org.apache.hadoop.hive.metastore.api.ColumnStatistics getHivePartitionColumnStatistics() { + org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics = getHiveTableColumnStatistics(); + columnStatistics.getStatsDesc().setIsTblLevel(false); + columnStatistics.getStatsDesc().setPartName("A=a/B=b"); + + return columnStatistics; + } + + public static PartitionError getPartitionError(List values, Exception exception) { + return new PartitionError() + .withPartitionValues(values) + .withErrorDetail(new ErrorDetail() + .withErrorCode(exception.getClass().getSimpleName()) + .withErrorMessage(exception.getMessage())); + } + + public static HiveObjectRef getHiveObjectRef() { + HiveObjectRef obj = new HiveObjectRef(); + obj.setObjectType(HiveObjectType.TABLE); + obj.setDbName("default"); + obj.setObjectName("foo"); + return obj; + } + + public static PrivilegeBag getPrivilegeBag() { + PrivilegeBag bag = new PrivilegeBag(); + HiveObjectPrivilege hivePrivilege = new HiveObjectPrivilege(); + hivePrivilege.setPrincipalName("user1"); + hivePrivilege.setPrincipalType(org.apache.hadoop.hive.metastore.api.PrincipalType.USER); + org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo grantInfo = new org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo(); + grantInfo.setGrantor("user2"); + grantInfo.setGrantorType(org.apache.hadoop.hive.metastore.api.PrincipalType.USER); + hivePrivilege.setGrantInfo(grantInfo); + bag.setPrivileges(Lists.newArrayList(hivePrivilege)); + return bag; + } + + public static org.apache.hadoop.hive.metastore.api.Order getTestOrder() { + org.apache.hadoop.hive.metastore.api.Order order = new org.apache.hadoop.hive.metastore.api.Order(); + order.setCol("foo"); + order.setOrder(1); + return order; + } + + public static Role getTestRole() { + Role role = new Role(); + role.setRoleName("test-role"); + role.setOwnerName("owner"); + return role; + } + +} \ No newline at end of file