diff --git a/flink-catalog-aws/flink-catalog-aws-glue/pom.xml b/flink-catalog-aws/flink-catalog-aws-glue/pom.xml new file mode 100644 index 00000000..db6abb45 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/pom.xml @@ -0,0 +1,96 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-catalog-aws-parent + 4.4-SNAPSHOT + + + flink-catalog-aws-glue + Flink : Catalog : AWS : Glue + + + jar + + + + + org.apache.flink + flink-table-api-java + ${flink.version} + provided + + + + org.apache.flink + flink-connector-aws-base + ${project.version} + + + + software.amazon.awssdk + glue + + + + software.amazon.awssdk + apache-client + + + + + + org.apache.flink + flink-architecture-tests-test + test + + + + org.apache.flink + flink-table-common + ${flink.version} + test-jar + test + + + + org.apache.flink + flink-table-api-java + ${flink.version} + test-jar + test + + + + org.projectlombok + lombok + 1.18.22 + test + + + + + diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java new file mode 100644 index 00000000..629dfdab --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java @@ -0,0 +1,1128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.aws.util.AWSClientUtil; +import org.apache.flink.connector.aws.util.AWSGeneralUtil; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionImpl; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.operator.GlueDatabaseOperator; +import org.apache.flink.table.catalog.glue.operator.GlueFunctionOperator; +import org.apache.flink.table.catalog.glue.operator.GluePartitionOperator; +import org.apache.flink.table.catalog.glue.operator.GlueTableOperator; +import org.apache.flink.table.catalog.glue.util.GlueUtils; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.GetTablesResponse; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.Partition; +import software.amazon.awssdk.services.glue.model.Table; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; + +/** Glue catalog implementation that uses AWS Glue Data Catalog as persistence at backend. */ +@PublicEvolving +public class GlueCatalog extends AbstractCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(GlueCatalog.class); + + /** instance of GlueOperator to facilitate glue related actions. */ + public GlueDatabaseOperator glueDatabaseOperator; + + public GlueTableOperator glueTableOperator; + public GluePartitionOperator gluePartitionOperator; + public GlueFunctionOperator glueFunctionOperator; + + public GlueClient glueClient; + + /** Default database name if not passed as part of catalog. */ + public static final String DEFAULT_DB = "default"; + + public GlueCatalog( + String catalogName, + String databaseName, + ReadableConfig catalogConfig, + Properties glueClientProperties) { + super(catalogName, databaseName); + checkNotNull(catalogConfig, "Catalog config cannot be null."); + String glueCatalogId = + String.valueOf( + catalogConfig.getOptional(GlueCatalogOptions.GLUE_CATALOG_ID).orElse(null)); + glueClient = createClient(glueClientProperties); + this.glueDatabaseOperator = new GlueDatabaseOperator(getName(), glueClient, glueCatalogId); + this.glueTableOperator = new GlueTableOperator(getName(), glueClient, glueCatalogId); + this.gluePartitionOperator = + new GluePartitionOperator(getName(), glueClient, glueCatalogId); + this.glueFunctionOperator = new GlueFunctionOperator(getName(), glueClient, glueCatalogId); + } + + private static GlueClient createClient(Properties glueClientProperties) { + return AWSClientUtil.createAwsSyncClient( + glueClientProperties, + AWSGeneralUtil.createSyncHttpClient( + glueClientProperties, ApacheHttpClient.builder()), + GlueClient.builder(), + GlueCatalogConstants.BASE_GLUE_USER_AGENT_PREFIX_FORMAT, + GlueCatalogConstants.GLUE_CLIENT_USER_AGENT_PREFIX); + } + + @VisibleForTesting + public GlueCatalog( + String catalogName, + String databaseName, + GlueClient glueClient, + GlueDatabaseOperator glueDatabaseOperator, + GlueTableOperator glueTableOperator, + GluePartitionOperator gluePartitionOperator, + GlueFunctionOperator glueFunctionOperator) { + super(catalogName, databaseName); + this.glueClient = glueClient; + this.glueDatabaseOperator = glueDatabaseOperator; + this.glueTableOperator = glueTableOperator; + this.gluePartitionOperator = gluePartitionOperator; + this.glueFunctionOperator = glueFunctionOperator; + } + + /** + * Open the catalog. Used for any required preparation in initialization phase. + * + * @throws CatalogException in case of any runtime exception + */ + @Override + public void open() throws CatalogException {} + + /** + * Close the catalog when it is no longer needed and release any resource that it might be + * holding. + * + * @throws CatalogException in case of any runtime exception + */ + @Override + public void close() throws CatalogException { + try { + glueClient.close(); + } catch (Exception e) { + throw new CatalogException("Glue Client is not closed properly!", e); + } + } + + // ------ databases ------ + + /** + * Create a database. + * + * @param databaseName Name of the database to be created + * @param database The database definition + * @param ignoreIfExists Flag to specify behavior when a database with the given name already + * exists: if set to false, throw a DatabaseAlreadyExistException, if set to true, do + * nothing. + * @throws DatabaseAlreadyExistException if the given database already exists and ignoreIfExists + * is false + * @throws CatalogException in case of any runtime exception + */ + @Override + public void createDatabase( + String databaseName, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + checkNotNull(database, "Database cannot be null."); + databaseName = GlueUtils.getGlueConventionalName(databaseName); + if (databaseExists(databaseName) && !ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), databaseName); + } else { + glueDatabaseOperator.createGlueDatabase(databaseName, database); + LOG.info("Created Database {}.", databaseName); + } + } + + /** + * Drop a database. + * + * @param databaseName Name of the database to be dropped. + * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to + * false, throw an exception, if set to true, do nothing. + * @param cascade Flag to specify behavior when the database contains table or function: if set + * to true, delete all tables and functions in the database and then delete the database, if + * set to false, throw an exception. + * @throws DatabaseNotExistException if the given database does not exist + * @throws DatabaseNotEmptyException if the given database is not empty and isRestrict is true + * @throws CatalogException in case of any runtime exception + */ + @Override + public void dropDatabase(String databaseName, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + databaseName = GlueUtils.getGlueConventionalName(databaseName); + if (databaseExists(databaseName)) { + if (cascade) { + List tables = listTables(databaseName); + if (!tables.isEmpty()) { + glueDatabaseOperator.deleteTablesFromDatabase(databaseName, tables); + LOG.info("{} Tables deleted from Database {}.", tables.size(), databaseName); + } + List functions = listFunctions(databaseName); + if (!functions.isEmpty()) { + glueDatabaseOperator.deleteFunctionsFromDatabase(databaseName, functions); + LOG.info( + "{} Functions deleted from Database {}.", + functions.size(), + databaseName); + } + } + if (!isDatabaseEmpty(databaseName)) { + throw new DatabaseNotEmptyException(getName(), databaseName); + } + glueDatabaseOperator.dropGlueDatabase(databaseName); + LOG.info("Dropped Database: {}.", databaseName); + } else if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + /** + * Modify existing database. + * + * @param name Name of the database to be modified + * @param newDatabase The new database definition + * @param ignoreIfNotExists Flag to specify behavior when the given database does not exist: if + * set to false, throw an exception, if set to true, do nothing. + * @throws DatabaseNotExistException if the given database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(name), + "Database name cannot be null or empty."); + checkNotNull(newDatabase, "Database cannot be null."); + name = GlueUtils.getGlueConventionalName(name); + try { + CatalogDatabase existingDatabase = glueDatabaseOperator.getDatabase(name); + if (existingDatabase != null) { + if (existingDatabase.getClass() != newDatabase.getClass()) { + throw new CatalogException( + String.format( + "Database types don't match. Existing database is '%s' and new database is '%s'.", + existingDatabase.getClass().getName(), + newDatabase.getClass().getName())); + } + glueDatabaseOperator.updateGlueDatabase(name, newDatabase); + } + } catch (DatabaseNotExistException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + /** + * Get list of databases in catalog. + * + * @return a list of the names of all databases + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listDatabases() throws CatalogException { + return glueDatabaseOperator.listGlueDatabases(); + } + + /** + * Get a database from this catalog. + * + * @param databaseName Name of the database + * @return The requested database + * @throws DatabaseNotExistException if the database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + databaseName = GlueUtils.getGlueConventionalName(databaseName); + return glueDatabaseOperator.getDatabase(databaseName); + } + + /** + * Check if a database exists in this catalog. + * + * @param databaseName Name of the database + * @return true if the given database exists in the catalog false otherwise + * @throws CatalogException in case of any runtime exception + */ + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + try { + return getDatabase(databaseName) != null; + } catch (DatabaseNotExistException e) { + return false; + } + } + + /** + * Check if database is empty. i.e. it should not contain 1. table 2. functions + * + * @param databaseName name of database. + * @return boolean True/False based on the content of database. + * @throws CatalogException Any Exception thrown due to glue error + */ + public boolean isDatabaseEmpty(String databaseName) throws CatalogException { + checkArgument( + !isNullOrWhitespaceOnly(databaseName), "Database name cannot be null or empty."); + GlueUtils.validate(databaseName); + GetTablesRequest tablesRequest = + GetTablesRequest.builder() + .catalogId(glueTableOperator.getGlueCatalogId()) + .databaseName(databaseName) + .maxResults(1) + .build(); + GetUserDefinedFunctionsRequest functionsRequest = + GetUserDefinedFunctionsRequest.builder() + .databaseName(databaseName) + .catalogId(glueFunctionOperator.getGlueCatalogId()) + .maxResults(1) + .build(); + try { + GetTablesResponse tableResponse = glueClient.getTables(tablesRequest); + GetUserDefinedFunctionsResponse functionResponse = + glueClient.getUserDefinedFunctions(functionsRequest); + if (tableResponse.sdkHttpResponse().isSuccessful() + && functionResponse.sdkHttpResponse().isSuccessful()) { + return tableResponse.tableList().isEmpty() + && functionResponse.userDefinedFunctions().isEmpty(); + } else { + String errorMessage = + String.format( + "Error checking if database '%s' is empty. Glue API requests failed with the following IDs:\n" + + "1. GetTables: %s\n" + + "2. GetUserDefinedFunctions: %s\n" + + "Please check the Glue service logs for more details.", + databaseName, + tableResponse.responseMetadata().requestId(), + functionResponse.responseMetadata().requestId()); + throw new CatalogException(errorMessage); + } + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + // ------ tables ------ + + /** + * Creates a new table or view. + * + *

The framework will make sure to call this method with fully validated {@link + * ResolvedCatalogTable} or {@link ResolvedCatalogView}. Those instances are easy to serialize + * for a durable catalog implementation. + * + * @param tablePath path of the table or view to be created + * @param table the table definition + * @param ignoreIfExists flag to specify behavior when a table or view already exists at the + * given path: if set to false, it throws a TableAlreadyExistException, if set to true, do + * nothing. + * @throws TableAlreadyExistException if table already exists and ignoreIfExists is false + * @throws DatabaseNotExistException if the database in tablePath doesn't exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(table, "table cannot be null"); + checkArgument(table instanceof ResolvedCatalogBaseTable, "table should be resolved"); + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + if (tableExists(tablePath)) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath); + } + } else { + glueTableOperator.createGlueTable(tablePath, table); + } + } + + /** + * Modifies an existing table or view. Note that the new and old {@link CatalogBaseTable} must + * be of the same kind. For example, this doesn't allow altering a regular table to partitioned + * table, or altering a view to a table, and vice versa. + * + *

The framework will make sure to call this method with fully validated {@link + * ResolvedCatalogTable} or {@link ResolvedCatalogView}. Those instances are easy to serialize + * for a durable catalog implementation. + * + * @param tablePath path of the table or view to be modified + * @param newTable the new table definition + * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if + * set to false, throw an exception, if set to true, do nothing. + * @throws TableNotExistException if the table does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + checkNotNull(newTable, "Table cannot be null."); + CatalogBaseTable existingTable = getTable(tablePath); + if (existingTable != null) { + if (existingTable.getTableKind() != newTable.getTableKind()) { + throw new CatalogException( + String.format( + "Table types don't match. Existing table is '%s' and new table is '%s'.", + existingTable.getTableKind(), newTable.getTableKind())); + } + glueTableOperator.alterGlueTable(tablePath, newTable); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } + + // ------ tables and views ------ + + /** + * Drop a table or view. + * + * @param tablePath Path of the table or view to be dropped + * @param ignoreIfNotExists Flag to specify behavior when the table or view does not exist: if + * set to false, throw an exception, if set to true, do nothing. + * @throws TableNotExistException if the table or view does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + if (tableExists(tablePath)) { + glueTableOperator.dropGlueTable(tablePath); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } + + /** + * Rename an existing table or view. + * + * @param tablePath Path of the table or view to be renamed + * @param newTableName the new name of the table or view + * @param ignoreIfNotExists Flag to specify behavior when the table or view does not exist: if + * set to false, throw an exception, if set to true, do nothing. + * @throws TableNotExistException if the table does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + + checkNotNull(tablePath, "TablePath cannot be null"); + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(newTableName), + "Table name cannot be null or empty."); + + if (tableExists(tablePath)) { + ObjectPath newTablePath = new ObjectPath(tablePath.getDatabaseName(), newTableName); + if (tableExists(newTablePath)) { + throw new TableAlreadyExistException(getName(), newTablePath); + } + glueTableOperator.renameGlueTable(tablePath, newTablePath); + } else if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } + + /** + * Get names of all tables and views under this database. An empty list is returned if none + * exists. + * + * @param databaseName fully qualified database name. + * @return a list of the names of all tables and views in this database + * @throws DatabaseNotExistException if the database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + List tableAndViewList = + glueTableOperator.getGlueTableList( + databaseName, CatalogBaseTable.TableKind.TABLE.name()); + tableAndViewList.addAll(listViews(databaseName)); + return tableAndViewList; + } + + /** + * Get names of all views under this database. An empty list is returned if none exists. + * + * @param databaseName the name of the given database + * @return a list of the names of all views in the given database + * @throws DatabaseNotExistException if the database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty"); + + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + return glueTableOperator.getGlueTableList( + databaseName, CatalogBaseTable.TableKind.VIEW.name()); + } + + /** + * Returns a {@link CatalogTable} or {@link CatalogView} identified by the given {@link + * ObjectPath}. The framework will resolve the metadata objects when necessary. + * + * @param tablePath Path of the table or view + * @return The requested table or view + * @throws TableNotExistException if the target does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + return glueTableOperator.getCatalogBaseTableFromGlueTable( + glueTableOperator.getGlueTable(tablePath)); + } + + /** + * Check if a table or view exists in this catalog. + * + * @param tablePath Path of the table or view + * @return true if the given table exists in the catalog false otherwise + * @throws CatalogException in case of any runtime exception + */ + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + return databaseExists(tablePath.getDatabaseName()) + && glueTableOperator.glueTableExists(tablePath); + } + + // ------ functions ------ + /** + * Create a function. Function name should be handled in a case-insensitive way. + * + * @param path path of the function + * @param function the function to be created + * @param ignoreIfExists flag to specify behavior if a function with the given name already + * exists: if set to false, it throws a FunctionAlreadyExistException, if set to true, + * nothing happens. + * @throws FunctionAlreadyExistException if the function already exist + * @throws DatabaseNotExistException if the given database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void createFunction(ObjectPath path, CatalogFunction function, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + checkNotNull(path, "Function path cannot be null."); + checkNotNull(function, "Catalog Function cannot be null."); + ObjectPath functionPath = normalize(path); + if (!databaseExists(functionPath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), functionPath.getDatabaseName()); + } + if (!functionExists(functionPath)) { + glueFunctionOperator.createGlueFunction(functionPath, function); + } else { + if (!ignoreIfExists) { + throw new FunctionAlreadyExistException(getName(), functionPath); + } + } + } + + public ObjectPath normalize(ObjectPath path) { + return new ObjectPath( + path.getDatabaseName(), FunctionIdentifier.normalizeName(path.getObjectName())); + } + + /** + * Modify an existing function. Function name should be handled in a case-insensitive way. + * + * @param path path of the function + * @param newFunction the function to be modified + * @param ignoreIfNotExists flag to specify behavior if the function does not exist: if set to + * false, throw an exception if set to true, nothing happens + * @throws FunctionNotExistException if the function does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterFunction( + ObjectPath path, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + checkNotNull(path, "Function path cannot be null."); + checkNotNull(newFunction, "Catalog Function cannot be null."); + ObjectPath functionPath = normalize(path); + CatalogFunction existingFunction = getFunction(functionPath); + if (existingFunction != null) { + if (existingFunction.getClass() != newFunction.getClass()) { + throw new CatalogException( + String.format( + "Function types don't match. Existing function is '%s' and new function is '%s'.", + existingFunction.getClass().getName(), + newFunction.getClass().getName())); + } + glueFunctionOperator.alterGlueFunction(functionPath, newFunction); + } else if (!ignoreIfNotExists) { + throw new FunctionNotExistException(getName(), functionPath); + } + } + + /** + * Drop a function. Function name should be handled in a case-insensitive way. + * + * @param path path of the function to be dropped + * @param ignoreIfNotExists flag to specify behavior if the function does not exist: if set to + * false, throw an exception if set to true, nothing happens + * @throws FunctionNotExistException if the function does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void dropFunction(ObjectPath path, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + checkNotNull(path, "Function path cannot be null."); + ObjectPath functionPath = normalize(path); + if (functionExists(functionPath)) { + glueFunctionOperator.dropGlueFunction(functionPath); + } else if (!ignoreIfNotExists) { + throw new FunctionNotExistException(getName(), functionPath); + } + } + + /** + * List the names of all functions in the given database. An empty list is returned if none is + * registered. + * + * @param databaseName name of the database. + * @return a list of the names of the functions in this database + * @throws DatabaseNotExistException if the database does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listFunctions(String databaseName) + throws DatabaseNotExistException, CatalogException { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(databaseName), + "Database name cannot be null or empty."); + databaseName = GlueUtils.getGlueConventionalName(databaseName); + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + return glueFunctionOperator.listGlueFunctions(databaseName); + } + + /** + * Get the function. Function name should be handled in a case-insensitive way. + * + * @param path path of the function + * @return the requested function + * @throws FunctionNotExistException if the function does not exist in the catalog + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogFunction getFunction(ObjectPath path) + throws FunctionNotExistException, CatalogException { + checkNotNull(path, "Function path cannot be null."); + ObjectPath functionPath = normalize(path); + if (!functionExists(functionPath)) { + throw new FunctionNotExistException(getName(), functionPath); + } else { + return glueFunctionOperator.getGlueFunction(functionPath); + } + } + + /** + * Check whether a function exists or not. Function name should be handled in a case-insensitive + * way. + * + * @param path path of the function + * @return true if the function exists in the catalog false otherwise + * @throws CatalogException in case of any runtime exception + */ + @Override + public boolean functionExists(ObjectPath path) throws CatalogException { + checkNotNull(path, "Function path cannot be null."); + ObjectPath functionPath = normalize(path); + return databaseExists(functionPath.getDatabaseName()) + && glueFunctionOperator.glueFunctionExists(functionPath); + } + + // ------ partitions ------ + /** + * Create a partition. + * + * @param tablePath path of the table. + * @param partitionSpec partition spec of the partition + * @param partition the partition to add. + * @param ignoreIfExists flag to specify behavior if a table with the given name already exists: + * if set to false, it throws a TableAlreadyExistException, if set to true, nothing happens. + * @throws TableNotExistException thrown if the target table does not exist + * @throws TableNotPartitionedException thrown if the target table is not partitioned + * @throws PartitionSpecInvalidException thrown if the given partition spec is invalid + * @throws PartitionAlreadyExistsException thrown if the target partition already exists + * @throws CatalogException in case of any runtime exception + */ + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + checkNotNull(partitionSpec, "PartitionSpec cannot be null."); + checkNotNull(partition, "Partition cannot be null."); + Table glueTable = glueTableOperator.getGlueTable(tablePath); + gluePartitionOperator.ensurePartitionedTable(tablePath, glueTable); + if (!partitionExists(tablePath, partitionSpec)) { + gluePartitionOperator.createGluePartition(glueTable, partitionSpec, partition); + } else { + if (!ignoreIfExists) { + throw new PartitionAlreadyExistsException(getName(), tablePath, partitionSpec); + } + } + } + + /** + * Get CatalogPartitionSpec of all partitions of the table. + * + * @param tablePath path of the table + * @return a list of CatalogPartitionSpec of the table + * @throws TableNotExistException thrown if the table does not exist in the catalog + * @throws TableNotPartitionedException thrown if the table is not partitioned + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + if (isPartitionedTable(tablePath)) { + return gluePartitionOperator.listPartitions(tablePath); + } + throw new TableNotPartitionedException(getName(), tablePath); + } + + public boolean isPartitionedTable(ObjectPath tablePath) { + CatalogBaseTable table; + try { + table = getTable(tablePath); + if (table instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) table; + return catalogTable.isPartitioned(); + } + return false; + } catch (TableNotExistException e) { + throw new CatalogException(tablePath.getFullName() + " Table doesn't Exists.", e); + } + } + + /** + * Get CatalogPartitionSpec of all partitions that is under the given CatalogPartitionSpec in + * the table. + * + * @param tablePath path of the table + * @param partitionSpec the partition spec to list + * @return a list of CatalogPartitionSpec that is under the given CatalogPartitionSpec in the + * table + * @throws TableNotExistException thrown if the table does not exist in the catalog + * @throws TableNotPartitionedException thrown if the table is not partitioned + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + checkNotNull(partitionSpec, "Partition spec cannot be null."); + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + if (!isPartitionedTable(tablePath)) { + throw new TableNotPartitionedException(getName(), tablePath); + } + return gluePartitionOperator.listPartitions(tablePath, partitionSpec); + } + + /** + * Get CatalogPartitionSpec of partitions by expression filters in the table. + * + *

NOTE: For FieldReferenceExpression, the field index is based on schema of this table + * instead of partition columns only. + * + *

The passed in predicates have been translated in conjunctive form. + * + *

If catalog does not support this interface at present, throw an {@link + * UnsupportedOperationException} directly. If the catalog does not have a valid filter, throw + * the {@link UnsupportedOperationException} directly. Planner will fall back to get all + * partitions and filter by itself. + * + * @param tablePath path of the table + * @param filters filters to push down filter to catalog + * @return a list of CatalogPartitionSpec that is under the given CatalogPartitionSpec in the + * table + * @throws TableNotExistException thrown if the table does not exist in the catalog + * @throws TableNotPartitionedException thrown if the table is not partitioned + * @throws CatalogException in case of any runtime exception + */ + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + if (!isPartitionedTable(tablePath)) { + throw new TableNotPartitionedException(getName(), tablePath); + } + return gluePartitionOperator.listGluePartitionsByFilter(tablePath, filters); + } + + /** + * Get a partition of the given table. The given partition spec keys and values need to be + * matched exactly for a result. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of partition to get + * @return the requested partition + * @throws PartitionNotExistException thrown if the partition doesn't exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + checkNotNull(partitionSpec, "CatalogPartitionSpec cannot be null."); + Table glueTable; + try { + glueTable = glueTableOperator.getGlueTable(tablePath); + } catch (TableNotExistException e) { + throw new CatalogException("Table doesn't exist in Glue Data Catalog", e); + } + Partition gluePartition = gluePartitionOperator.getGluePartition(glueTable, partitionSpec); + if (gluePartition == null) { + throw new PartitionNotExistException(getName(), tablePath, partitionSpec); + } + Map catalogPartitionProperties = + new HashMap<>(gluePartition.storageDescriptor().parameters()); + String comment = catalogPartitionProperties.remove(GlueCatalogConstants.COMMENT); + return new CatalogPartitionImpl(catalogPartitionProperties, comment); + } + + /** + * Check whether a partition exists or not. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition to check + * @throws CatalogException in case of any runtime exception + */ + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + checkNotNull(tablePath, "TablePath cannot be null"); + if (!databaseExists(tablePath.getDatabaseName())) { + throw new CatalogException(tablePath.getDatabaseName() + " Database doesn't exists."); + } + try { + Table glueTable = glueTableOperator.getGlueTable(tablePath); + return gluePartitionOperator.gluePartitionExists(tablePath, glueTable, partitionSpec); + } catch (TableNotExistException e) { + throw new CatalogException( + tablePath.getFullName() + " Table doesn't Exists in Glue Data Catalog.", e); + } + } + + /** + * Drop a partition. + * + * @param tablePath path of the table. + * @param partitionSpec partition spec of the partition to drop + * @param ignoreIfNotExists flag to specify behavior if the database does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws PartitionNotExistException thrown if the target partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + checkNotNull(partitionSpec, "PartitionSpec cannot be null."); + if (partitionExists(tablePath, partitionSpec)) { + Table glueTable; + try { + glueTable = glueTableOperator.getGlueTable(tablePath); + } catch (TableNotExistException e) { + throw new CatalogException(tablePath.getFullName() + " Table doesn't exists.", e); + } + gluePartitionOperator.dropGluePartition(tablePath, partitionSpec, glueTable); + } else if (!ignoreIfNotExists) { + throw new PartitionNotExistException(getName(), tablePath, partitionSpec); + } + } + + /** + * Alter a partition. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition + * @param newPartition new partition to replace the old one + * @param ignoreIfNotExists flag to specify behavior if the database does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws PartitionNotExistException thrown if the target partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + checkNotNull(tablePath, "TablePath cannot be null."); + checkNotNull(partitionSpec, "CatalogPartitionSpec cannot be null."); + checkNotNull(newPartition, "New partition cannot be null."); + CatalogPartition existingPartition = getPartition(tablePath, partitionSpec); + if (existingPartition != null) { + try { + Table glueTable = glueTableOperator.getGlueTable(tablePath); + gluePartitionOperator.alterGluePartition( + tablePath, glueTable, partitionSpec, newPartition); + } catch (TableNotExistException e) { + throw new CatalogException("Table Not Found in Glue data catalog", e); + } catch (PartitionSpecInvalidException e) { + throw new CatalogException("Invalid Partition Spec", e); + } + } else if (!ignoreIfNotExists) { + throw new PartitionNotExistException(getName(), tablePath, partitionSpec); + } + } + + /** + * Get the statistics of a table. + * + * @param tablePath path of the table + * @return statistics of the given table + * @throws TableNotExistException if the table does not exist in the catalog + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + /** + * Get the column statistics of a table. + * + * @param tablePath path of the table + * @return column statistics of the given table + * @throws TableNotExistException if the table does not exist in the catalog + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + /** + * Get the statistics of a partition. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition + * @return statistics of the given partition + * @throws PartitionNotExistException if the partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + /** + * Get the column statistics of a partition. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition + * @return column statistics of the given partition + * @throws PartitionNotExistException if the partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + /** + * Update the statistics of a table. + * + * @param tablePath path of the table + * @param tableStatistics new statistics to update + * @param ignoreIfNotExists flag to specify behavior if the table does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws TableNotExistException if the table does not exist in the catalog + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException("Operation with Statistics not supported."); + } + + /** + * Update the column statistics of a table. + * + * @param tablePath path of the table + * @param columnStatistics new column statistics to update + * @param ignoreIfNotExists flag to specify behavior if the table does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws TableNotExistException if the table does not exist in the catalog + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + throw new UnsupportedOperationException("Operation with Statistics not supported."); + } + + /** + * Update the statistics of a table partition. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition + * @param partitionStatistics new statistics to update + * @param ignoreIfNotExists flag to specify behavior if the partition does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws PartitionNotExistException if the partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("Operation with Statistics not supported."); + } + + /** + * Update the column statistics of a table partition. + * + * @param tablePath path of the table + * @param partitionSpec partition spec of the partition @@param columnStatistics new column + * statistics to update + * @param columnStatistics column related statistics + * @param ignoreIfNotExists flag to specify behavior if the partition does not exist: if set to + * false, throw an exception, if set to true, nothing happens. + * @throws PartitionNotExistException if the partition does not exist + * @throws CatalogException in case of any runtime exception + */ + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException("Operation with Statistics not supported."); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java new file mode 100644 index 00000000..254557be --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.connector.aws.config.AWSConfigConstants; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.catalog.glue.constants.AWSGlueConfigConstants; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; + +import software.amazon.awssdk.regions.Region; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.table.catalog.glue.GlueCatalog.DEFAULT_DB; + +/** Collection of {@link ConfigOption} used in GlueCatalog. */ +@Internal +public class GlueCatalogOptions extends CommonCatalogOptions { + + public static final String IDENTIFIER = "glue"; + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue(DEFAULT_DB); + + public static final ConfigOption INPUT_FORMAT = + ConfigOptions.key(GlueCatalogConstants.TABLE_INPUT_FORMAT) + .stringType() + .noDefaultValue(); + + public static final ConfigOption OUTPUT_FORMAT = + ConfigOptions.key(GlueCatalogConstants.TABLE_OUTPUT_FORMAT) + .stringType() + .noDefaultValue(); + + public static final ConfigOption GLUE_CATALOG_ENDPOINT = + ConfigOptions.key(AWSGlueConfigConstants.GLUE_CATALOG_ENDPOINT) + .stringType() + .noDefaultValue(); + + public static final ConfigOption GLUE_CATALOG_ID = + ConfigOptions.key(AWSGlueConfigConstants.GLUE_CATALOG_ID).stringType().noDefaultValue(); + + public static final ConfigOption GLUE_ACCOUNT_ID = + ConfigOptions.key(AWSGlueConfigConstants.GLUE_ACCOUNT_ID).stringType().noDefaultValue(); + + public static final ConfigOption CREDENTIAL_PROVIDER = + ConfigOptions.key(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER) + .stringType() + .defaultValue(String.valueOf(AWSConfigConstants.CredentialProvider.AUTO)); + + public static final ConfigOption HTTP_CLIENT_TYPE = + ConfigOptions.key(AWSConfigConstants.HTTP_CLIENT_TYPE) + .stringType() + .defaultValue(AWSConfigConstants.CLIENT_TYPE_APACHE); + + public static final ConfigOption REGION = + ConfigOptions.key(AWSConfigConstants.AWS_REGION) + .stringType() + .defaultValue(Region.US_WEST_1.toString()); + + public static Set> getAllConfigOptions() { + Set> configOptions = new HashSet<>(); + configOptions.add(INPUT_FORMAT); + configOptions.add(OUTPUT_FORMAT); + configOptions.add(GLUE_CATALOG_ENDPOINT); + configOptions.add(GLUE_ACCOUNT_ID); + configOptions.add(GLUE_CATALOG_ID); + configOptions.add(DEFAULT_DATABASE); + configOptions.add(HTTP_CLIENT_TYPE); + configOptions.add(REGION); + configOptions.add(CREDENTIAL_PROVIDER); + return configOptions; + } + + public static Set> getRequiredConfigOptions() { + return new HashSet<>(); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java new file mode 100644 index 00000000..5eeee652 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.AbstractDataType; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarCharType; + +/** + * The {@code TypeMapper} class provides utility methods to map Flink's {@link LogicalType} to AWS. + * Glue data types and vice versa. + * + *

This class supports conversion between Flink's logical types and Glue data types, handling + * both primitive types and complex types such as arrays, maps, and rows. The mapping allows for + * seamless integration between Flink and AWS Glue, enabling Flink to read from and write to Glue + * tables with the appropriate data types. + * + *

For complex types like arrays, maps, and rows, the conversion is handled recursively, ensuring + * that nested types are also converted accurately. + * + *

This class currently supports the following mappings: + * + *

    + *
  • Flink {@code IntType} -> Glue {@code int} + *
  • Flink {@code BigIntType} -> Glue {@code bigint} + *
  • Flink {@code VarCharType} -> Glue {@code string} + *
  • Flink {@code BooleanType} -> Glue {@code boolean} + *
  • Flink {@code DecimalType} -> Glue {@code decimal} + *
  • Flink {@code FloatType} -> Glue {@code float} + *
  • Flink {@code DoubleType} -> Glue {@code double} + *
  • Flink {@code DateType} -> Glue {@code date} + *
  • Flink {@code TimestampType} -> Glue {@code timestamp} + *
  • Flink {@code ArrayType} -> Glue {@code array} + *
  • Flink {@code MapType} -> Glue {@code map} + *
  • Flink {@code RowType} -> Glue {@code struct} + *
+ * + *

Note: Struct type handling in {@code glueTypeToFlinkType} is currently not supported and will + * throw an {@link UnsupportedOperationException}. + * + * @see org.apache.flink.table.types.logical.LogicalType + * @see org.apache.flink.table.api.DataTypes + * @see org.apache.flink.table.catalog.CatalogTable + * @see org.apache.flink.table.catalog.ResolvedCatalogTable + */ +public class TypeMapper { + + /** + * Maps a given Flink {@link LogicalType} to its corresponding AWS Glue data type as a string. + * + * @param logicalType the Flink logical type to be mapped + * @return the corresponding AWS Glue data type as a string + * @throws UnsupportedOperationException if the Flink type is not supported + */ + public static String mapFlinkTypeToGlueType(LogicalType logicalType) { + if (logicalType instanceof IntType) { + return "int"; + } else if (logicalType instanceof BigIntType) { + return "bigint"; + } else if (logicalType instanceof VarCharType) { + return "string"; + } else if (logicalType instanceof BooleanType) { + return "boolean"; + } else if (logicalType instanceof DecimalType) { + return "decimal"; + } else if (logicalType instanceof FloatType) { + return "float"; + } else if (logicalType instanceof DoubleType) { + return "double"; + } else if (logicalType instanceof DateType) { + return "date"; + } else if (logicalType instanceof TimestampType) { + return "timestamp"; + } else if (logicalType instanceof ArrayType) { + ArrayType arrayType = (ArrayType) logicalType; + String elementType = mapFlinkTypeToGlueType(arrayType.getElementType()); + return "array<" + elementType + ">"; + } else if (logicalType instanceof MapType) { + MapType mapType = (MapType) logicalType; + String keyType = mapFlinkTypeToGlueType(mapType.getKeyType()); + String valueType = mapFlinkTypeToGlueType(mapType.getValueType()); + return "map<" + keyType + "," + valueType + ">"; + } else if (logicalType instanceof RowType) { + RowType rowType = (RowType) logicalType; + StringBuilder structType = new StringBuilder("struct<"); + for (RowType.RowField field : rowType.getFields()) { + structType + .append(field.getName()) + .append(":") + .append(mapFlinkTypeToGlueType(field.getType())) + .append(","); + } + // Remove the trailing comma and close the struct definition + structType.setLength(structType.length() - 1); + structType.append(">"); + return structType.toString(); + } else { + throw new UnsupportedOperationException("Unsupported Flink type: " + logicalType); + } + } + + /** + * Maps a given AWS Glue data type as a string to its corresponding Flink {@link + * AbstractDataType}. + * + * @param glueType the AWS Glue data type as a string + * @return the corresponding Flink data type + * @throws UnsupportedOperationException if the Glue type is not supported + */ + public static AbstractDataType glueTypeToFlinkType(String glueType) { + if (glueType.equals("int")) { + return DataTypes.INT(); + } else if (glueType.equals("bigint")) { + return DataTypes.BIGINT(); + } else if (glueType.equals("string")) { + return DataTypes.STRING(); + } else if (glueType.equals("boolean")) { + return DataTypes.BOOLEAN(); + } else if (glueType.equals("decimal")) { + return DataTypes.DECIMAL(10, 0); + } else if (glueType.equals("float")) { + return DataTypes.FLOAT(); + } else if (glueType.equals("double")) { + return DataTypes.DOUBLE(); + } else if (glueType.equals("date")) { + return DataTypes.DATE(); + } else if (glueType.equals("timestamp")) { + return DataTypes.TIMESTAMP(5); + } else if (glueType.startsWith("array")) { + String elementType = glueType.substring(6, glueType.length() - 1); + return DataTypes.ARRAY(glueTypeToFlinkType(elementType)); + } else if (glueType.startsWith("map")) { + // Example: map -> DataTypes.MAP(DataTypes.STRING(), + // DataTypes.STRING()) + int commaIndex = glueType.indexOf(","); + String keyType = glueType.substring(4, commaIndex); + String valueType = glueType.substring(commaIndex + 1, glueType.length() - 1); + return DataTypes.MAP(glueTypeToFlinkType(keyType), glueTypeToFlinkType(valueType)); + } else { + // Handle struct type if necessary + // For this case, custom parsing might be required based on struct definition + throw new UnsupportedOperationException("Struct type not yet supported"); + } + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java new file mode 100644 index 00000000..e2c992a5 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.constants; + +import org.apache.flink.annotation.PublicEvolving; + +/** Configuration keys for AWS Glue Data Catalog service usage. */ +@PublicEvolving +public class AWSGlueConfigConstants { + + /** + * Configure an alternative endpoint of the Glue service for GlueCatalog to access. + * + *

This could be used to use GlueCatalog with any glue-compatible metastore service that has + * a different endpoint + */ + public static final String GLUE_CATALOG_ENDPOINT = "aws.glue.endpoint"; + + /** + * The ID of the Glue Data Catalog where the tables reside. If none is provided, Glue + * automatically uses the caller's AWS account ID by default. + * + *

For more details, see ... + */ + public static final String GLUE_CATALOG_ID = "aws.glue.id"; + + /** + * The account ID used in a Glue resource ARN, e.g. + * arn:aws:glue:us-east-1:1000000000000:table/db1/table1 + */ + public static final String GLUE_ACCOUNT_ID = "aws.glue.account-id"; +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java new file mode 100644 index 00000000..61ee8370 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.constants; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.glue.GlueCatalog; + +import java.util.regex.Pattern; + +/** Constants and Defined Values used for {@link GlueCatalog}. */ +@Internal +public class GlueCatalogConstants { + public static final String COMMENT = "comment"; + public static final String DEFAULT_SEPARATOR = ":"; + public static final String LOCATION_SEPARATOR = "/"; + public static final String LOCATION_URI = "path"; + public static final String AND = "and"; + public static final String NEXT_LINE = "\n"; + public static final String SPACE = " "; + + public static final String TABLE_OWNER = "owner"; + public static final String TABLE_INPUT_FORMAT = "table.input.format"; + public static final String TABLE_OUTPUT_FORMAT = "table.output.format"; + + public static final String FLINK_SCALA_FUNCTION_PREFIX = "flink:scala:"; + public static final String FLINK_PYTHON_FUNCTION_PREFIX = "flink:python:"; + public static final String FLINK_JAVA_FUNCTION_PREFIX = "flink:java:"; + + public static final String FLINK_CATALOG = "FLINK_CATALOG"; + + public static final Pattern GLUE_DB_PATTERN = Pattern.compile("^[a-z0-9_]{1,255}$"); + public static final String GLUE_EXCEPTION_MSG_IDENTIFIER = "GLUE EXCEPTION"; + public static final String TABLE_NOT_EXISTS_IDENTIFIER = "TABLE DOESN'T EXIST"; + public static final String DEFAULT_PARTITION_NAME = "__GLUE_DEFAULT_PARTITION__"; + + public static final int UDF_CLASS_NAME_SIZE = 3; + + public static final String BASE_GLUE_USER_AGENT_PREFIX_FORMAT = + "Apache Flink %s (%s) Glue Catalog"; + + /** Glue Catalog identifier for user agent prefix. */ + public static final String GLUE_CLIENT_USER_AGENT_PREFIX = "aws.glue.client.user-agent-prefix"; + + public static final String IS_PERSISTED = "isPersisted"; + public static final String EXPLAIN_EXTRAS = "explainExtras"; + public static final String IS_PHYSICAL = "isPhysical"; +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java new file mode 100644 index 00000000..c5eb68be --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.factory; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.glue.GlueCatalog; +import org.apache.flink.table.catalog.glue.GlueCatalogOptions; +import org.apache.flink.table.catalog.glue.util.GlueCatalogOptionsUtils; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +/** Catalog factory for {@link GlueCatalog}. */ +@PublicEvolving +public class GlueCatalogFactory implements CatalogFactory { + + private static final Logger LOG = LoggerFactory.getLogger(GlueCatalogFactory.class); + + @Override + public String factoryIdentifier() { + return GlueCatalogOptions.IDENTIFIER; + } + + @Override + public Set> optionalOptions() { + Set> allConfigs = GlueCatalogOptions.getAllConfigOptions(); + allConfigs.removeAll(GlueCatalogOptions.getRequiredConfigOptions()); + return allConfigs; + } + + @Override + public Set> requiredOptions() { + return GlueCatalogOptions.getRequiredConfigOptions(); + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = + FactoryUtil.createCatalogFactoryHelper(this, context); + GlueCatalogOptionsUtils optionsUtils = + new GlueCatalogOptionsUtils(context.getOptions(), context.getConfiguration()); + helper.validateExcept(optionsUtils.getNonValidatedPrefixes().toArray(new String[0])); + if (LOG.isDebugEnabled()) { + LOG.debug( + context.getOptions().entrySet().stream() + .map(entry -> entry.getKey() + "-> " + entry.getValue()) + .collect(Collectors.joining("\n"))); + } + Properties glueCatalogValidatedProperties = optionsUtils.getValidatedConfigurations(); + return new GlueCatalog( + context.getName(), + helper.getOptions().get(GlueCatalogOptions.DEFAULT_DATABASE), + context.getConfiguration(), + glueCatalogValidatedProperties); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java new file mode 100644 index 00000000..710ce3f3 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.util.GlueUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.BatchDeleteTableRequest; +import software.amazon.awssdk.services.glue.model.BatchDeleteTableResponse; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateDatabaseResponse; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.DatabaseInput; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseResponse; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabaseResponse; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetDatabasesResponse; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseResponse; + +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** Utilities for Glue catalog Database related operations. */ +@Internal +public class GlueDatabaseOperator extends GlueOperator { + + private static final Logger LOG = LoggerFactory.getLogger(GlueDatabaseOperator.class); + + public GlueDatabaseOperator(String catalogName, GlueClient glueClient, String glueCatalogId) { + super(catalogName, glueClient, glueCatalogId); + } + + /** + * List all databases present in glue data catalog service. + * + * @return fully qualified list of database name. + */ + public List listGlueDatabases() throws CatalogException { + try { + GetDatabasesRequest.Builder databasesRequestBuilder = + GetDatabasesRequest.builder().catalogId(getGlueCatalogId()); + GetDatabasesResponse response = + glueClient.getDatabases(databasesRequestBuilder.build()); + GlueUtils.validateGlueResponse(response); + List databaseList = + response.databaseList().stream() + .map(Database::name) + .collect(Collectors.toList()); + String dbResultNextToken = response.nextToken(); + while (Optional.ofNullable(dbResultNextToken).isPresent()) { + databasesRequestBuilder.nextToken(dbResultNextToken); + response = glueClient.getDatabases(databasesRequestBuilder.build()); + GlueUtils.validateGlueResponse(response); + databaseList.addAll( + response.databaseList().stream() + .map(Database::name) + .collect(Collectors.toList())); + dbResultNextToken = response.nextToken(); + } + return databaseList; + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Create database in glue data catalog service. + * + * @param databaseName fully qualified name of database. + * @param database Instance of {@link CatalogDatabase}. + * @throws CatalogException on unexpected error happens. + * @throws DatabaseAlreadyExistException when database exists already in glue data catalog. + */ + public void createGlueDatabase(String databaseName, CatalogDatabase database) + throws CatalogException, DatabaseAlreadyExistException { + GlueUtils.validate(databaseName); + Map properties = new HashMap<>(database.getProperties()); + DatabaseInput databaseInput = + DatabaseInput.builder() + .name(databaseName) + .description(database.getComment()) + .parameters(properties) + .build(); + CreateDatabaseRequest.Builder requestBuilder = + CreateDatabaseRequest.builder() + .databaseInput(databaseInput) + .catalogId(getGlueCatalogId()); + try { + CreateDatabaseResponse response = glueClient.createDatabase(requestBuilder.build()); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + } catch (EntityNotFoundException e) { + throw new CatalogException(catalogName, e); + } catch (AlreadyExistsException e) { + throw new DatabaseAlreadyExistException(catalogName, databaseName, e); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Delete a database from Glue data catalog service only when database is empty. + * + * @param databaseName fully qualified name of database. + * @throws CatalogException on unexpected error happens. + * @throws DatabaseNotExistException when database doesn't exists in glue catalog. + */ + public void dropGlueDatabase(String databaseName) + throws CatalogException, DatabaseNotExistException { + GlueUtils.validate(databaseName); + DeleteDatabaseRequest deleteDatabaseRequest = + DeleteDatabaseRequest.builder() + .name(databaseName) + .catalogId(getGlueCatalogId()) + .build(); + try { + DeleteDatabaseResponse response = glueClient.deleteDatabase(deleteDatabaseRequest); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + } catch (EntityNotFoundException e) { + throw new DatabaseNotExistException(catalogName, databaseName); + } catch (GlueException e) { + throw new CatalogException(catalogName, e); + } + } + + /** + * Delete list of table in database from glue data catalog service. + * + * @param databaseName fully qualified name of database. + * @param tables List of table to remove from database. + * @throws CatalogException on unexpected Exception thrown. + */ + public void deleteTablesFromDatabase(String databaseName, Collection tables) + throws CatalogException { + GlueUtils.validate(databaseName); + BatchDeleteTableRequest batchTableRequest = + BatchDeleteTableRequest.builder() + .databaseName(databaseName) + .catalogId(getGlueCatalogId()) + .tablesToDelete(tables) + .build(); + try { + BatchDeleteTableResponse response = glueClient.batchDeleteTable(batchTableRequest); + if (response.hasErrors()) { + String errorMsg = + String.format( + "Glue Table errors:- %s", + response.errors().stream() + .map( + e -> + "Table: " + + e.tableName() + + "\tErrorDetail: " + + e.errorDetail().errorMessage()) + .collect(Collectors.joining("\n"))); + LOG.error(errorMsg); + throw new CatalogException(errorMsg); + } + GlueUtils.validateGlueResponse(response); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Delete list of user defined function associated with Database from glue data catalog service. + * + * @param databaseName fully qualified name of database. + * @param functions List of functions to remove from database. + * @throws CatalogException on unexpected Exception thrown. + */ + public void deleteFunctionsFromDatabase(String databaseName, Collection functions) + throws CatalogException { + GlueUtils.validate(databaseName); + DeleteUserDefinedFunctionRequest.Builder requestBuilder = + DeleteUserDefinedFunctionRequest.builder() + .databaseName(databaseName) + .catalogId(getGlueCatalogId()); + DeleteUserDefinedFunctionResponse response; + for (String functionName : functions) { + requestBuilder.functionName(functionName); + try { + response = glueClient.deleteUserDefinedFunction(requestBuilder.build()); + } catch (GlueException e) { + LOG.error( + "Error deleting function {} in database: {}\n{}", + functionName, + databaseName, + e); + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + } + } + + /** + * Get {@link CatalogDatabase} instance using the information from glue data-catalog service. + * + * @param databaseName fully qualified name of database. + * @return Instance of {@link CatalogDatabase } . + * @throws DatabaseNotExistException when database doesn't exists in Glue data catalog Service. + * @throws CatalogException when any unknown error occurs in Execution. + */ + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + GlueUtils.validate(databaseName); + GetDatabaseRequest getDatabaseRequest = + GetDatabaseRequest.builder() + .name(databaseName) + .catalogId(getGlueCatalogId()) + .build(); + try { + GetDatabaseResponse response = glueClient.getDatabase(getDatabaseRequest); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + return GlueUtils.getCatalogDatabase(response.database()); + } catch (EntityNotFoundException e) { + throw new DatabaseNotExistException(catalogName, databaseName); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Update Database in Glue Metastore. + * + * @param databaseName Database name. + * @param newDatabase instance of {@link CatalogDatabase}. + * @throws CatalogException in case of Errors. + */ + public void updateGlueDatabase(String databaseName, CatalogDatabase newDatabase) + throws CatalogException { + GlueUtils.validate(databaseName); + Map newProperties = new HashMap<>(newDatabase.getProperties()); + DatabaseInput databaseInput = + DatabaseInput.builder() + .parameters(newProperties) + .description(newDatabase.getComment()) + .name(databaseName) + .build(); + + UpdateDatabaseRequest updateRequest = + UpdateDatabaseRequest.builder() + .databaseInput(databaseInput) + .name(databaseName) + .catalogId(getGlueCatalogId()) + .build(); + UpdateDatabaseResponse response = glueClient.updateDatabase(updateRequest); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + LOG.info("Updated Database: {}", databaseName); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java new file mode 100644 index 00000000..8fa32143 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java @@ -0,0 +1,262 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.util.GlueUtils; +import org.apache.flink.table.resource.ResourceUri; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.PrincipalType; +import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.UserDefinedFunction; +import software.amazon.awssdk.services.glue.model.UserDefinedFunctionInput; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** Utilities for Glue catalog Function related operations. */ +@Internal +public class GlueFunctionOperator extends GlueOperator { + + private static final Logger LOG = LoggerFactory.getLogger(GlueFunctionOperator.class); + + public GlueFunctionOperator(String catalogName, GlueClient glueClient, String glueCatalogId) { + super(catalogName, glueClient, glueCatalogId); + } + + /** + * Create a function. Function name should be handled in a case-insensitive way. + * + * @param functionPath path of the function + * @param function Flink function to be created + * @throws CatalogException in case of any runtime exception + */ + public void createGlueFunction(ObjectPath functionPath, CatalogFunction function) + throws CatalogException, FunctionAlreadyExistException { + UserDefinedFunctionInput functionInput = createFunctionInput(functionPath, function); + CreateUserDefinedFunctionRequest.Builder createUDFRequest = + CreateUserDefinedFunctionRequest.builder() + .databaseName(functionPath.getDatabaseName()) + .catalogId(getGlueCatalogId()) + .functionInput(functionInput); + try { + CreateUserDefinedFunctionResponse response = + glueClient.createUserDefinedFunction(createUDFRequest.build()); + GlueUtils.validateGlueResponse(response); + LOG.info("Created Function: {}", functionPath.getFullName()); + } catch (AlreadyExistsException e) { + LOG.error( + String.format( + "%s already Exists. Function language of type: %s. \n%s", + functionPath.getFullName(), function.getFunctionLanguage(), e)); + throw new FunctionAlreadyExistException(catalogName, functionPath, e); + } catch (GlueException e) { + LOG.error("Error creating glue function: {}\n{}", functionPath.getFullName(), e); + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Get the user defined function from glue Catalog. Function name should be handled in a + * case-insensitive way. + * + * @param functionPath path of the function + * @return the requested function + * @throws CatalogException in case of any runtime exception + */ + public CatalogFunction getGlueFunction(ObjectPath functionPath) { + GetUserDefinedFunctionRequest request = + GetUserDefinedFunctionRequest.builder() + .databaseName(functionPath.getDatabaseName()) + .catalogId(getGlueCatalogId()) + .functionName(functionPath.getObjectName()) + .build(); + GetUserDefinedFunctionResponse response = glueClient.getUserDefinedFunction(request); + GlueUtils.validateGlueResponse(response); + UserDefinedFunction udf = response.userDefinedFunction(); + List resourceUris = + udf.resourceUris().stream() + .map( + resourceUri -> + new org.apache.flink.table.resource.ResourceUri( + org.apache.flink.table.resource.ResourceType + .valueOf(resourceUri.resourceType().name()), + resourceUri.uri())) + .collect(Collectors.toList()); + return new CatalogFunctionImpl( + GlueUtils.getCatalogFunctionClassName(udf), + GlueUtils.getFunctionalLanguage(udf), + resourceUris); + } + + public List listGlueFunctions(String databaseName) { + GetUserDefinedFunctionsRequest.Builder functionsRequest = + GetUserDefinedFunctionsRequest.builder() + .databaseName(databaseName) + .catalogId(getGlueCatalogId()); + List glueFunctions; + try { + GetUserDefinedFunctionsResponse functionsResponse = + glueClient.getUserDefinedFunctions(functionsRequest.build()); + String token = functionsResponse.nextToken(); + glueFunctions = + functionsResponse.userDefinedFunctions().stream() + .map(UserDefinedFunction::functionName) + .collect(Collectors.toCollection(LinkedList::new)); + while (Optional.ofNullable(token).isPresent()) { + functionsRequest.nextToken(token); + functionsResponse = glueClient.getUserDefinedFunctions(functionsRequest.build()); + glueFunctions.addAll( + functionsResponse.userDefinedFunctions().stream() + .map(UserDefinedFunction::functionName) + .collect(Collectors.toCollection(LinkedList::new))); + token = functionsResponse.nextToken(); + } + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + return glueFunctions; + } + + public boolean glueFunctionExists(ObjectPath functionPath) { + GetUserDefinedFunctionRequest request = + GetUserDefinedFunctionRequest.builder() + .functionName(functionPath.getObjectName()) + .databaseName(functionPath.getDatabaseName()) + .catalogId(getGlueCatalogId()) + .build(); + + try { + GetUserDefinedFunctionResponse response = glueClient.getUserDefinedFunction(request); + GlueUtils.validateGlueResponse(response); + return response.userDefinedFunction() != null; + } catch (EntityNotFoundException e) { + return false; + } catch (GlueException e) { + LOG.error(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Modify an existing function. Function name should be handled in a case-insensitive way. + * + * @param functionPath path of function. + * @param newFunction modified function. + * @throws CatalogException on runtime errors. + */ + public void alterGlueFunction(ObjectPath functionPath, CatalogFunction newFunction) + throws CatalogException { + + UserDefinedFunctionInput functionInput = createFunctionInput(functionPath, newFunction); + + UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest = + UpdateUserDefinedFunctionRequest.builder() + .functionName(functionPath.getObjectName()) + .databaseName(functionPath.getDatabaseName()) + .catalogId(getGlueCatalogId()) + .functionInput(functionInput) + .build(); + UpdateUserDefinedFunctionResponse response = + glueClient.updateUserDefinedFunction(updateUserDefinedFunctionRequest); + GlueUtils.validateGlueResponse(response); + LOG.info("Altered Function: {}", functionPath.getFullName()); + } + + /** + * Drop / Delete UserDefinedFunction from glue data catalog. + * + * @param functionPath fully qualified function path + * @throws CatalogException In case of Unexpected errors. + */ + public void dropGlueFunction(ObjectPath functionPath) throws CatalogException { + DeleteUserDefinedFunctionRequest request = + DeleteUserDefinedFunctionRequest.builder() + .catalogId(getGlueCatalogId()) + .functionName(functionPath.getObjectName()) + .databaseName(functionPath.getDatabaseName()) + .build(); + DeleteUserDefinedFunctionResponse response = glueClient.deleteUserDefinedFunction(request); + GlueUtils.validateGlueResponse(response); + LOG.info("Dropped Function: {}", functionPath.getFullName()); + } + + /** + * Utility method to Create UserDefinedFunctionInput instance. + * + * @param functionPath fully qualified for function path. + * @param function Catalog Function instance. + * @return User defined function input instance for Glue. + * @throws UnsupportedOperationException in case of unsupported operation encountered. + */ + public static UserDefinedFunctionInput createFunctionInput( + final ObjectPath functionPath, final CatalogFunction function) + throws UnsupportedOperationException { + Collection resourceUris = + new LinkedList<>(); + for (org.apache.flink.table.resource.ResourceUri resourceUri : + function.getFunctionResources()) { + switch (resourceUri.getResourceType()) { + case JAR: + case FILE: + case ARCHIVE: + resourceUris.add( + software.amazon.awssdk.services.glue.model.ResourceUri.builder() + .resourceType(resourceUri.getResourceType().name()) + .uri(resourceUri.getUri()) + .build()); + break; + default: + throw new UnsupportedOperationException( + "GlueCatalog supports only creating resources JAR/FILE or ARCHIVE."); + } + } + return UserDefinedFunctionInput.builder() + .functionName(functionPath.getObjectName()) + .className(GlueUtils.getGlueFunctionClassName(function)) + .ownerType(PrincipalType.USER) + .ownerName(GlueCatalogConstants.FLINK_CATALOG) + .resourceUris(resourceUris) + .build(); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java new file mode 100644 index 00000000..be77913d --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.StringUtils; + +import software.amazon.awssdk.services.glue.GlueClient; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Glue related operation. Important Note : * ... + */ +@Internal +public abstract class GlueOperator { + + public final String glueCatalogId; + + protected final GlueClient glueClient; + + public final String catalogName; + + public GlueOperator(String catalogName, GlueClient glueClient, String glueCatalogId) { + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(catalogName), + "catalogName name cannot be null or empty."); + checkNotNull(glueClient, "GlueClient Instance cannot be Null."); + checkArgument( + !StringUtils.isNullOrWhitespaceOnly(glueCatalogId), + "glue Catalog Id name cannot be null or empty."); + this.catalogName = catalogName; + this.glueClient = glueClient; + this.glueCatalogId = glueCatalogId; + } + + public String getGlueCatalogId() { + return glueCatalogId; + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java new file mode 100644 index 00000000..e612c3fa --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java @@ -0,0 +1,484 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.util.GlueUtils; +import org.apache.flink.table.expressions.Expression; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.Column; +import software.amazon.awssdk.services.glue.model.CreatePartitionRequest; +import software.amazon.awssdk.services.glue.model.CreatePartitionResponse; +import software.amazon.awssdk.services.glue.model.DeletePartitionRequest; +import software.amazon.awssdk.services.glue.model.DeletePartitionResponse; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionResponse; +import software.amazon.awssdk.services.glue.model.GetPartitionsRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionsResponse; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.Partition; +import software.amazon.awssdk.services.glue.model.PartitionInput; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdatePartitionResponse; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; + +/** Utilities for Glue catalog Partition related operations. */ +@Internal +public class GluePartitionOperator extends GlueOperator { + + private static final Logger LOG = LoggerFactory.getLogger(GluePartitionOperator.class); + + public GluePartitionOperator(String catalogName, GlueClient glueClient, String glueCatalogId) { + super(catalogName, glueClient, glueCatalogId); + } + + /** + * Create partition in glue data catalog service. + * + * @param glueTable glue table + * @param partitionSpec partition spec + * @param catalogPartition partition to add. + * @throws CatalogException when partition is unable to be created. + * @throws PartitionSpecInvalidException when partition specification is invalid. + */ + public void createGluePartition( + final Table glueTable, + final CatalogPartitionSpec partitionSpec, + final CatalogPartition catalogPartition) + throws CatalogException, PartitionSpecInvalidException { + Map catalogPartitionProperties = + new HashMap<>(catalogPartition.getProperties()); + String comment = catalogPartition.getComment(); + Map partitionSpecProperties = + new HashMap<>(partitionSpec.getPartitionSpec()); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Partition Keys retrieved from glue table: {}", + glueTable.partitionKeys().stream() + .map(Column::name) + .collect(Collectors.toList())); + } + List partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys()); + List partitionValues = + getOrderedFullPartitionValues( + partitionSpec, + partitionColumns, + new ObjectPath(glueTable.databaseName(), glueTable.name())); + if (LOG.isDebugEnabled()) { + LOG.debug("Partition Values are: {}", String.join(", ", partitionValues)); + } + for (int i = 0; i < partitionColumns.size(); i++) { + if (isNullOrWhitespaceOnly(partitionValues.get(i))) { + throw new PartitionSpecInvalidException( + catalogName, + partitionColumns, + new ObjectPath(glueTable.databaseName(), glueTable.name()), + partitionSpec); + } + } + StorageDescriptor.Builder storageDescriptor = glueTable.storageDescriptor().toBuilder(); + storageDescriptor.parameters(partitionSpecProperties); + catalogPartitionProperties.put(GlueCatalogConstants.COMMENT, comment); + PartitionInput.Builder partitionInput = + PartitionInput.builder() + .parameters(catalogPartitionProperties) + .lastAccessTime(Instant.now()) + .storageDescriptor(storageDescriptor.build()) + .values(partitionValues); + CreatePartitionRequest createPartitionRequest = + CreatePartitionRequest.builder() + .partitionInput(partitionInput.build()) + .catalogId(getGlueCatalogId()) + .databaseName(glueTable.databaseName()) + .tableName(glueTable.name()) + .build(); + try { + CreatePartitionResponse response = glueClient.createPartition(createPartitionRequest); + GlueUtils.validateGlueResponse(response); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Alter Partition in glue data catalog service. + * + * @param tablePath contains database name and table name. + * @param partitionSpec Existing partition information. + * @param newPartition Partition information with new changes. + * @throws CatalogException Exception in failure. + */ + public void alterGluePartition( + final ObjectPath tablePath, + final Table glueTable, + final CatalogPartitionSpec partitionSpec, + final CatalogPartition newPartition) + throws CatalogException, PartitionSpecInvalidException { + Map partitionSpecProperties = + new HashMap<>(partitionSpec.getPartitionSpec()); + Map newPartitionProperties = new HashMap<>(newPartition.getProperties()); + String comment = newPartition.getComment(); + List partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys()); + List partitionValues = + getOrderedFullPartitionValues( + partitionSpec, + partitionColumns, + new ObjectPath(glueTable.databaseName(), glueTable.name())); + StorageDescriptor.Builder storageDescriptor = glueTable.storageDescriptor().toBuilder(); + storageDescriptor.parameters(partitionSpecProperties); + newPartitionProperties.put(GlueCatalogConstants.COMMENT, comment); + PartitionInput.Builder partitionInput = + PartitionInput.builder() + .lastAccessTime(Instant.now()) + .parameters(newPartitionProperties) + .storageDescriptor(storageDescriptor.build()) + .values(partitionValues); + UpdatePartitionRequest.Builder updatePartitionRequest = + UpdatePartitionRequest.builder() + .partitionInput(partitionInput.build()) + .databaseName(tablePath.getDatabaseName()) + .catalogId(getGlueCatalogId()) + .tableName(tablePath.getObjectName()) + .partitionValueList(partitionValues); + UpdatePartitionResponse response = + glueClient.updatePartition(updatePartitionRequest.build()); + GlueUtils.validateGlueResponse(response); + } + + /** + * Drop partition from Glue data catalog service. + * + * @param tablePath fully qualified table path + * @param partitionSpec partition spec details + * @throws CatalogException in case of unknown errors + */ + public void dropGluePartition( + final ObjectPath tablePath, + final CatalogPartitionSpec partitionSpec, + final Table glueTable) + throws CatalogException { + try { + List partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys()); + DeletePartitionRequest deletePartitionRequest = + DeletePartitionRequest.builder() + .catalogId(getGlueCatalogId()) + .databaseName(tablePath.getDatabaseName()) + .tableName(tablePath.getObjectName()) + .partitionValues( + getOrderedFullPartitionValues( + partitionSpec, partitionColumns, tablePath)) + .build(); + DeletePartitionResponse response = glueClient.deletePartition(deletePartitionRequest); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + } catch (PartitionSpecInvalidException e) { + throw new CatalogException("Invalid Partition Spec", e); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Retrieve partition from glue data catalog service. + * + * @param glueTable Instance of {@link Table} from glue data Catalog. + * @param partitionSpec instance of {@link CatalogPartitionSpec} containing details of partition + * @return Instance of {@link Partition} matching the given partitionSpec. + * @throws PartitionNotExistException when partition doesn't exist in Glue data catalog. + */ + public Partition getGluePartition( + final Table glueTable, final CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException { + ObjectPath tablePath = new ObjectPath(glueTable.databaseName(), glueTable.name()); + try { + List partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys()); + List partitionValues = + getOrderedFullPartitionValues(partitionSpec, partitionColumns, tablePath); + LOG.info("Partition values are: {}", String.join(", ", partitionValues)); + GetPartitionRequest request = + GetPartitionRequest.builder() + .catalogId(getGlueCatalogId()) + .databaseName(glueTable.databaseName()) + .tableName(glueTable.name()) + .partitionValues(partitionValues) + .build(); + GetPartitionResponse response = glueClient.getPartition(request); + GlueUtils.validateGlueResponse(response); + Partition partition = response.partition(); + if (LOG.isDebugEnabled()) { + LOG.debug( + "(catalogPartition properties) Partition Parameters: {}", + partition.parameters().entrySet().stream() + .map(e -> e.getKey() + " - " + e.getValue()) + .collect(Collectors.joining(", "))); + LOG.debug( + "(PartitionSpec properties) Partition Parameters: {}", + partition.storageDescriptor().parameters().entrySet().stream() + .map(e -> e.getKey() + " - " + e.getValue()) + .collect(Collectors.joining(", "))); + LOG.debug(GlueUtils.getDebugLog(response)); + } + if (partition.hasValues()) { + return partition; + } + } catch (EntityNotFoundException e) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec); + } catch (PartitionSpecInvalidException e) { + throw new CatalogException("PartitionSpec Invalid ", e); + } + return null; + } + + /** + * check Partition exists in glue data catalog. + * + * @param tablePath Fully Qualified tablePath. + * @param partitionSpec Instance of {@link CatalogPartitionSpec}. + * @return weather partition exists ? + * @throws CatalogException in case of unknown errors. + */ + public boolean gluePartitionExists( + final ObjectPath tablePath, + final Table glueTable, + final CatalogPartitionSpec partitionSpec) + throws CatalogException { + try { + List partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys()); + List partitionValues = + getOrderedFullPartitionValues(partitionSpec, partitionColumns, tablePath); + GetPartitionRequest request = + GetPartitionRequest.builder() + .catalogId(getGlueCatalogId()) + .databaseName(tablePath.getDatabaseName()) + .tableName(tablePath.getObjectName()) + .partitionValues(partitionValues) + .build(); + GetPartitionResponse response = glueClient.getPartition(request); + GlueUtils.validateGlueResponse(response); + return response.partition() + .storageDescriptor() + .parameters() + .keySet() + .containsAll(partitionSpec.getPartitionSpec().keySet()); + } catch (EntityNotFoundException e) { + LOG.warn(String.format("%s is not found", partitionSpec.getPartitionSpec())); + } catch (GlueException e) { + throw new CatalogException(catalogName, e); + } catch (PartitionSpecInvalidException e) { + throw new CatalogException("PartitionSpec Invalid ", e); + } + return false; + } + + /** + * Get List of CatalogPartitionSpec from glue data catalog associated with Table. + * + * @param tablePath fully qualified table path. + * @return List of PartitionSpec + */ + public List listPartitions(ObjectPath tablePath) { + + GetPartitionsRequest.Builder getPartitionRequest = + GetPartitionsRequest.builder() + .catalogId(getGlueCatalogId()) + .databaseName(tablePath.getDatabaseName()) + .tableName(tablePath.getObjectName()); + try { + GetPartitionsResponse response = glueClient.getPartitions(getPartitionRequest.build()); + GlueUtils.validateGlueResponse(response); + List finalPartitionsList = + response.partitions().stream() + .map(this::getCatalogPartitionSpec) + .collect(Collectors.toList()); + String partitionsResultNextToken = response.nextToken(); + while (Optional.ofNullable(partitionsResultNextToken).isPresent()) { + getPartitionRequest.nextToken(partitionsResultNextToken); + response = glueClient.getPartitions(getPartitionRequest.build()); + finalPartitionsList.addAll( + response.partitions().stream() + .map(this::getCatalogPartitionSpec) + .collect(Collectors.toList())); + partitionsResultNextToken = response.nextToken(); + } + return finalPartitionsList; + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Get list of CatalogPartitionSpec from glue data catalog service associated with table and + * matches given CatalogPartitionSpec. + * + * @param tablePath Fully qualified table Path. + * @param partitionSpec Partition spec . + * @return List of CatalogPartitionSpec. + */ + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) { + List partitionSpecList = listPartitions(tablePath); + return partitionSpecList.stream() + .filter( + currPartSpec -> + currPartSpec + .getPartitionSpec() + .entrySet() + .containsAll(partitionSpec.getPartitionSpec().entrySet())) + .collect(Collectors.toList()); + } + + /** + * Get List of CatalogPartitionSpec of partition by expression filters from glue data catalog + * associated with the given Table. + * + * @param tablePath Fully Qualified Table Path. + * @param filters List of Filters. + * @return List of Partition Spec + */ + public List listGluePartitionsByFilter( + ObjectPath tablePath, List filters) { + String expression = + filters.stream() + .map(GlueUtils::getExpressionString) + .collect( + Collectors.joining( + GlueCatalogConstants.SPACE + + GlueCatalogConstants.AND + + GlueCatalogConstants.SPACE)); + try { + GetPartitionsRequest.Builder getPartitionsRequest = + GetPartitionsRequest.builder() + .databaseName(tablePath.getDatabaseName()) + .tableName(tablePath.getObjectName()) + .catalogId(getGlueCatalogId()) + .expression(expression); + GetPartitionsResponse response = glueClient.getPartitions(getPartitionsRequest.build()); + List catalogPartitionSpecList = + response.partitions().stream() + .map(this::getCatalogPartitionSpec) + .collect(Collectors.toList()); + GlueUtils.validateGlueResponse(response); + String nextToken = response.nextToken(); + while (Optional.ofNullable(nextToken).isPresent()) { + getPartitionsRequest.nextToken(nextToken); + response = glueClient.getPartitions(getPartitionsRequest.build()); + GlueUtils.validateGlueResponse(response); + catalogPartitionSpecList.addAll( + response.partitions().stream() + .map(this::getCatalogPartitionSpec) + .collect(Collectors.toList())); + nextToken = response.nextToken(); + } + return catalogPartitionSpecList; + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Get a list of ordered partition values by re-arranging them based on the given list of + * partition keys. If the partition value is null, it'll be converted into default partition + * name. + * + * @param partitionSpec a partition spec. + * @param partitionKeys a list of partition keys. + * @param tablePath path of the table to which the partition belongs. + * @return A list of partition values ordered according to partitionKeys. + * @throws PartitionSpecInvalidException thrown if partitionSpec and partitionKeys have + * different sizes, or any key in partitionKeys doesn't exist in partitionSpec. + */ + private List getOrderedFullPartitionValues( + CatalogPartitionSpec partitionSpec, List partitionKeys, ObjectPath tablePath) + throws PartitionSpecInvalidException { + Map spec = partitionSpec.getPartitionSpec(); + if (spec.size() != partitionKeys.size()) { + throw new PartitionSpecInvalidException( + catalogName, partitionKeys, tablePath, partitionSpec); + } + + List values = new ArrayList<>(spec.size()); + for (String key : partitionKeys) { + if (!spec.containsKey(key)) { + throw new PartitionSpecInvalidException( + catalogName, partitionKeys, tablePath, partitionSpec); + } else { + String value = spec.get(key); + if (value == null) { + value = GlueCatalogConstants.DEFAULT_PARTITION_NAME; + } + values.add(value); + } + } + + return values; + } + + /** + * Validate and ensure Table is Partitioned. + * + * @param tablePath Fully Qualified TablePath. + * @param glueTable Instance of {@link Table} from glue data catalog. + * @throws TableNotPartitionedException In case of table is not partitioned. + */ + public void ensurePartitionedTable(ObjectPath tablePath, Table glueTable) + throws TableNotPartitionedException { + if (!glueTable.hasPartitionKeys()) { + throw new TableNotPartitionedException(catalogName, tablePath); + } + } + + /** + * Derive {@link CatalogPartitionSpec} instance from {@link Partition}. + * + * @param partition Glue Partition instance + * @return {@link CatalogPartitionSpec} instance + */ + private CatalogPartitionSpec getCatalogPartitionSpec(Partition partition) { + Map params = new HashMap<>(partition.storageDescriptor().parameters()); + return new CatalogPartitionSpec(params); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java new file mode 100644 index 00000000..ac1877d0 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.util.GlueUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.Column; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.CreateTableResponse; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableResponse; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTableResponse; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.GetTablesResponse; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableResponse; + +import java.time.Instant; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Utilities for Glue Table related operations. */ +@Internal +public class GlueTableOperator extends GlueOperator { + + private static final Logger LOG = LoggerFactory.getLogger(GlueTableOperator.class); + + public GlueTableOperator(String catalogName, GlueClient glueClient, String glueCatalogId) { + super(catalogName, glueClient, glueCatalogId); + } + + /** + * Create table in glue data catalog service. + * + * @param tablePath Fully qualified name of table. {@link ObjectPath} + * @param table instance of {@link CatalogBaseTable} containing table related information. + * @throws CatalogException on unexpected error happens. + */ + public void createGlueTable(final ObjectPath tablePath, final CatalogBaseTable table) + throws CatalogException { + + checkNotNull(tablePath, "tablePath cannot be null"); + checkNotNull(table, "table cannot be null"); + checkArgument(table instanceof ResolvedCatalogBaseTable, "table should be resolved"); + + final Map tableProperties = new HashMap<>(table.getOptions()); + String tableOwner = GlueUtils.extractTableOwner(tableProperties); + List glueTableColumns = GlueUtils.getGlueColumnsFromCatalogTable(table); + StorageDescriptor.Builder storageDescriptorBuilder = + StorageDescriptor.builder() + .inputFormat(GlueUtils.extractInputFormat(tableProperties)) + .outputFormat(GlueUtils.extractOutputFormat(tableProperties)); + + TableInput.Builder tableInputBuilder = + TableInput.builder() + .name(tablePath.getObjectName()) + .description(table.getComment()) + .tableType(table.getTableKind().name()) + .lastAccessTime(Instant.now()) + .owner(tableOwner) + .viewExpandedText(GlueUtils.getExpandedQuery(table)) + .viewOriginalText(GlueUtils.getOriginalQuery(table)); + + CreateTableRequest.Builder requestBuilder = + CreateTableRequest.builder() + .catalogId(getGlueCatalogId()) + .databaseName(tablePath.getDatabaseName()); + + if (table instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) table; + if (catalogTable.isPartitioned()) { + LOG.info("table is partitioned"); + Collection partitionKeys = + GlueUtils.getPartitionKeys(catalogTable, glueTableColumns); + tableInputBuilder.partitionKeys(partitionKeys); + } + } + + try { + storageDescriptorBuilder.columns(glueTableColumns); + tableInputBuilder.storageDescriptor(storageDescriptorBuilder.build()); + tableInputBuilder.parameters(tableProperties); + requestBuilder.tableInput(tableInputBuilder.build()); + CreateTableResponse response = glueClient.createTable(requestBuilder.build()); + GlueUtils.validateGlueResponse(response); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + LOG.info("Created Table: {}", tablePath.getFullName()); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Update Table in glue data catalog service. + * + * @param tablePath fully Qualified Table Path. + * @param newTable instance of {@link CatalogBaseTable} containing information for table. + * @throws CatalogException Glue related exception. + */ + public void alterGlueTable(ObjectPath tablePath, CatalogBaseTable newTable) + throws CatalogException { + + Map tableProperties = new HashMap<>(newTable.getOptions()); + String tableOwner = GlueUtils.extractTableOwner(tableProperties); + List glueColumns = GlueUtils.getGlueColumnsFromCatalogTable(newTable); + + StorageDescriptor.Builder storageDescriptorBuilder = + StorageDescriptor.builder() + .inputFormat(GlueUtils.extractInputFormat(tableProperties)) + .outputFormat(GlueUtils.extractOutputFormat(tableProperties)) + .parameters(tableProperties) + .columns(glueColumns); + + TableInput.Builder tableInputBuilder = + TableInput.builder() + .name(tablePath.getObjectName()) + .description(newTable.getComment()) + .tableType(newTable.getTableKind().name()) + .lastAccessTime(Instant.now()) + .owner(tableOwner); + + UpdateTableRequest.Builder requestBuilder = + UpdateTableRequest.builder() + .tableInput(tableInputBuilder.build()) + .catalogId(getGlueCatalogId()) + .databaseName(tablePath.getDatabaseName()); + + if (newTable instanceof CatalogTable) { + CatalogTable catalogTable = (CatalogTable) newTable; + if (catalogTable.isPartitioned()) { + tableInputBuilder.partitionKeys( + GlueUtils.getPartitionKeys(catalogTable, glueColumns)); + } + } + + tableInputBuilder.storageDescriptor(storageDescriptorBuilder.build()); + requestBuilder.tableInput(tableInputBuilder.build()); + + try { + UpdateTableResponse response = glueClient.updateTable(requestBuilder.build()); + if (LOG.isDebugEnabled()) { + LOG.debug(GlueUtils.getDebugLog(response)); + } + GlueUtils.validateGlueResponse(response); + LOG.info("Updated Table: {}", tablePath.getFullName()); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Get List of name of table/view in database based on type identifier. An empty list is + * returned if database doesn't contain any table/view. + * + * @param databaseName fully qualified database name. + * @param type type identifier. + * @return a list of table/view name in database based on type identifier. + * @throws CatalogException in case of any runtime exception. + */ + public List getGlueTableList(String databaseName, String type) throws CatalogException { + GetTablesRequest.Builder tablesRequestBuilder = + GetTablesRequest.builder().databaseName(databaseName).catalogId(getGlueCatalogId()); + GetTablesResponse response = glueClient.getTables(tablesRequestBuilder.build()); + GlueUtils.validateGlueResponse(response); + List finalTableList = + response.tableList().stream() + .filter(table -> table.tableType().equalsIgnoreCase(type)) + .map(Table::name) + .collect(Collectors.toList()); + String tableResultNextToken = response.nextToken(); + while (Optional.ofNullable(tableResultNextToken).isPresent()) { + tablesRequestBuilder.nextToken(tableResultNextToken); + response = glueClient.getTables(tablesRequestBuilder.build()); + GlueUtils.validateGlueResponse(response); + finalTableList.addAll( + response.tableList().stream() + .filter(table -> table.tableType().equalsIgnoreCase(type)) + .map(Table::name) + .collect(Collectors.toList())); + tableResultNextToken = response.nextToken(); + } + return finalTableList; + } + + /** + * Returns {@link Table} instance identified by the given {@link ObjectPath}. + * + * @param tablePath Path of the table or view. + * @return The requested table. Glue encapsulates whether table or view in its attribute called + * type. + * @throws TableNotExistException if the target does not exist + * @throws CatalogException in case of any runtime exception + */ + public Table getGlueTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + + checkNotNull(tablePath, "TablePath cannot be Null"); + + GetTableRequest tablesRequest = + GetTableRequest.builder() + .databaseName(tablePath.getDatabaseName()) + .name(tablePath.getObjectName()) + .catalogId(getGlueCatalogId()) + .build(); + try { + GetTableResponse response = glueClient.getTable(tablesRequest); + GlueUtils.validateGlueResponse(response); + return response.table(); + } catch (EntityNotFoundException e) { + throw new TableNotExistException(catalogName, tablePath, e); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Check if a table or view exists in glue data catalog service. + * + * @param tablePath Path of the table or view + * @return true if the given table exists in the catalog false otherwise + * @throws CatalogException in case of any runtime exception + */ + public boolean glueTableExists(ObjectPath tablePath) throws CatalogException { + try { + Table glueTable = getGlueTable(tablePath); + return glueTable != null && glueTable.name().equals(tablePath.getObjectName()); + } catch (TableNotExistException e) { + return false; + } catch (CatalogException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Drop table/view from glue data catalog service. + * + * @param tablePath fully qualified Table Path. + * @throws CatalogException on runtime errors. + */ + public void dropGlueTable(ObjectPath tablePath) throws CatalogException { + DeleteTableRequest.Builder tableRequestBuilder = + DeleteTableRequest.builder() + .databaseName(tablePath.getDatabaseName()) + .name(tablePath.getObjectName()) + .catalogId(getGlueCatalogId()); + try { + DeleteTableResponse response = glueClient.deleteTable(tableRequestBuilder.build()); + GlueUtils.validateGlueResponse(response); + LOG.info("Dropped Table: {}", tablePath.getObjectName()); + } catch (GlueException e) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e); + } + } + + /** + * Create {@link CatalogTable} instance from {@link Table} instance. + * + * @param glueTable Instance of Table from glue Data catalog. + * @return {@link CatalogTable} instance. + */ + public CatalogBaseTable getCatalogBaseTableFromGlueTable(Table glueTable) { + + checkNotNull(glueTable, "Glue Table cannot be null"); + Schema schemaInfo = GlueUtils.getSchemaFromGlueTable(glueTable); + List partitionKeys = + glueTable.partitionKeys().stream().map(Column::name).collect(Collectors.toList()); + Map properties = new HashMap<>(glueTable.parameters()); + + if (glueTable.owner() != null) { + properties.put(GlueCatalogConstants.TABLE_OWNER, glueTable.owner()); + } + + if (glueTable.storageDescriptor().hasParameters()) { + properties.putAll(glueTable.storageDescriptor().parameters()); + } + + if (glueTable.storageDescriptor().inputFormat() != null) { + properties.put( + GlueCatalogConstants.TABLE_INPUT_FORMAT, + glueTable.storageDescriptor().inputFormat()); + } + + if (glueTable.storageDescriptor().outputFormat() != null) { + properties.put( + GlueCatalogConstants.TABLE_OUTPUT_FORMAT, + glueTable.storageDescriptor().outputFormat()); + } + + if (glueTable.tableType().equals(CatalogBaseTable.TableKind.TABLE.name())) { + return CatalogTable.of(schemaInfo, glueTable.description(), partitionKeys, properties); + } else if (glueTable.tableType().equals(CatalogBaseTable.TableKind.VIEW.name())) { + return CatalogView.of( + schemaInfo, + glueTable.description(), + glueTable.viewOriginalText(), + glueTable.viewExpandedText(), + properties); + + } else { + throw new CatalogException( + String.format( + "Unknown TableType: %s from Glue Catalog.", glueTable.tableType())); + } + } + + /** + * Glue doesn't Support renaming of table by default. Rename glue table. Glue catalog don't + * support renaming table. For renaming in Flink, it has to be done in 3 step. 1. fetch existing + * table info from glue 2. Create a table with new-name and use properties of existing table 3. + * Delete existing table Note: This above steps are not Atomic in nature. + * + *

Associated issue :- ... + * + * @param oldTablePath old table name + * @param newTablePath new renamed table + */ + public void renameGlueTable(ObjectPath oldTablePath, ObjectPath newTablePath) + throws CatalogException, TableNotExistException { + throw new UnsupportedOperationException( + "Rename Table Operation in Glue Data Catalog is not Supported."); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java new file mode 100644 index 00000000..587d7535 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.aws.config.AWSConfigConstants; +import org.apache.flink.connector.aws.table.util.AWSOptionUtils; +import org.apache.flink.connector.aws.table.util.HttpClientOptionUtils; +import org.apache.flink.connector.base.table.options.ConfigurationValidator; +import org.apache.flink.connector.base.table.options.TableOptionsUtils; +import org.apache.flink.table.catalog.glue.GlueCatalogOptions; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** Option Handler for Glue Catalog. */ +@Internal +public class GlueCatalogOptionsUtils implements TableOptionsUtils, ConfigurationValidator { + + /** Allowed Http Client Types. */ + private static final String[] ALLOWED_GLUE_HTTP_CLIENTS = + new String[] { + AWSConfigConstants.CLIENT_TYPE_URLCONNECTION, AWSConfigConstants.CLIENT_TYPE_APACHE + }; + + private final AWSOptionUtils awsOptionUtils; + private final HttpClientOptionUtils httpClientOptionUtils; + private final ReadableConfig tableConfig; + + public GlueCatalogOptionsUtils( + Map resolvedOptions, ReadableConfig tableConfig) { + this.awsOptionUtils = new AWSOptionUtils(resolvedOptions); + this.httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, resolvedOptions); + this.tableConfig = tableConfig; + } + + @Override + public Properties getValidatedConfigurations() { + Properties validatedConfigs = new Properties(); + validatedConfigs.putAll(awsOptionUtils.getValidatedConfigurations()); + validatedConfigs.putAll(httpClientOptionUtils.getValidatedConfigurations()); + + for (ConfigOption option : GlueCatalogOptions.getAllConfigOptions()) { + if (tableConfig.getOptional(option).isPresent()) { + validatedConfigs.put(option.key(), tableConfig.getOptional(option).get()); + } + } + return validatedConfigs; + } + + @Override + public Map getProcessedResolvedOptions() { + Map processedOptions = awsOptionUtils.getProcessedResolvedOptions(); + processedOptions.putAll(httpClientOptionUtils.getProcessedResolvedOptions()); + return processedOptions; + } + + @Override + public List getNonValidatedPrefixes() { + // Glue Specific Options are handled by FactoryHelper + return Arrays.asList( + AWSOptionUtils.AWS_PROPERTIES_PREFIX, HttpClientOptionUtils.CLIENT_PREFIX); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java new file mode 100644 index 00000000..073ba748 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java @@ -0,0 +1,416 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.glue.GlueCatalogOptions; +import org.apache.flink.table.catalog.glue.TypeMapper; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.types.AbstractDataType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.glue.model.Column; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.GlueResponse; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.UserDefinedFunction; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.BooleanUtils.FALSE; +import static org.apache.commons.lang3.BooleanUtils.TRUE; +import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.EXPLAIN_EXTRAS; +import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.IS_PERSISTED; +import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.IS_PHYSICAL; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; + +/** Utilities related glue Operation. */ +@Internal +public class GlueUtils { + + private static final Logger LOG = LoggerFactory.getLogger(GlueUtils.class); + + /** + * Glue supports lowercase naming convention. + * + * @param name fully qualified name. + * @return modified name according to glue convention. + */ + public static String getGlueConventionalName(String name) { + return name.toLowerCase(Locale.ROOT); + } + + /** + * Extract database location from properties and remove location from properties. fallback to + * create default location if not present + * + * @param databaseProperties database properties. + * @param databaseName fully qualified name for database. + * @param catalogPath catalog path. + * @return location for database. + */ + public static String extractDatabaseLocation( + final Map databaseProperties, + final String databaseName, + final String catalogPath) { + if (databaseProperties.containsKey(GlueCatalogConstants.LOCATION_URI)) { + return databaseProperties.remove(GlueCatalogConstants.LOCATION_URI); + } else { + LOG.info("No location URI Set. Using Catalog Path as default"); + return catalogPath + GlueCatalogConstants.LOCATION_SEPARATOR + databaseName; + } + } + + /** + * Extract table location from table properties and remove location from properties. fallback to + * create default location if not present + * + * @param tableProperties table properties. + * @param tablePath fully qualified object for table. + * @param catalogPath catalog path. + * @return location for table. + */ + public static String extractTableLocation( + final Map tableProperties, + final ObjectPath tablePath, + final String catalogPath) { + if (tableProperties.containsKey(GlueCatalogConstants.LOCATION_URI)) { + return tableProperties.remove(GlueCatalogConstants.LOCATION_URI); + } else { + return catalogPath + + GlueCatalogConstants.LOCATION_SEPARATOR + + tablePath.getDatabaseName() + + GlueCatalogConstants.LOCATION_SEPARATOR + + tablePath.getObjectName(); + } + } + + /** + * Build CatalogDatabase instance using information from glue Database instance. + * + * @param glueDatabase {@link Database } + * @return {@link CatalogDatabase } instance. + */ + public static CatalogDatabase getCatalogDatabase(final Database glueDatabase) { + Map properties = new HashMap<>(glueDatabase.parameters()); + return new CatalogDatabaseImpl(properties, glueDatabase.description()); + } + + /** + * A Glue database name cannot be longer than 255 characters. The only acceptable characters are + * lowercase letters, numbers, and the underscore character. More details: ... + * + * @param name name + */ + public static void validate(String name) { + checkArgument( + name != null && name.matches(GlueCatalogConstants.GLUE_DB_PATTERN.pattern()), + "Database name does not comply with the Glue naming convention. " + + "Check here https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html"); + } + + /** validate response from client call. */ + public static void validateGlueResponse(GlueResponse response) { + if (response != null && !response.sdkHttpResponse().isSuccessful()) { + throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER); + } + } + + /** + * @param udf Instance of UserDefinedFunction + * @return ClassName for function + */ + public static String getCatalogFunctionClassName(final UserDefinedFunction udf) { + validateUDFClassName(udf.className()); + String[] splitName = udf.className().split(GlueCatalogConstants.DEFAULT_SEPARATOR); + return splitName[splitName.length - 1]; + } + + /** + * Validates UDF class name from glue. + * + * @param name name of UDF. + */ + private static void validateUDFClassName(final String name) { + checkArgument(!isNullOrWhitespaceOnly(name)); + + if (name.split(GlueCatalogConstants.DEFAULT_SEPARATOR).length + != GlueCatalogConstants.UDF_CLASS_NAME_SIZE) { + throw new ValidationException("Improper ClassName: " + name); + } + } + + /** + * Derive functionalLanguage from glue function name. Glue doesn't have any attribute to save + * the functionalLanguage Name. Thus, storing FunctionalLanguage in the name itself. + * + * @param glueFunction Function name from glue. + * @return Identifier for FunctionalLanguage. + */ + public static FunctionLanguage getFunctionalLanguage(final UserDefinedFunction glueFunction) { + if (glueFunction.className().startsWith(GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX)) { + return FunctionLanguage.JAVA; + } else if (glueFunction + .className() + .startsWith(GlueCatalogConstants.FLINK_PYTHON_FUNCTION_PREFIX)) { + return FunctionLanguage.PYTHON; + } else if (glueFunction + .className() + .startsWith(GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX)) { + return FunctionLanguage.SCALA; + } else { + throw new CatalogException( + "Invalid Functional Language for className: " + glueFunction.className()); + } + } + + /** + * Get expanded Query from CatalogBaseTable. + * + * @param table Instance of catalogBaseTable. + * @return expandedQuery for Glue Table. + */ + public static String getExpandedQuery(CatalogBaseTable table) { + // https://issues.apache.org/jira/browse/FLINK-31961 + return ""; + } + + /** + * Get Original Query from CatalogBaseTable. + * + * @param table Instance of CatalogBaseTable. + * @return OriginalQuery for Glue Table. + */ + public static String getOriginalQuery(CatalogBaseTable table) { + // https://issues.apache.org/jira/browse/FLINK-31961 + return ""; + } + + /** + * Extract table owner name and remove from properties. + * + * @param properties Map of properties. + * @return fully qualified owner name. + */ + public static String extractTableOwner(Map properties) { + return properties.containsKey(GlueCatalogConstants.TABLE_OWNER) + ? properties.remove(GlueCatalogConstants.TABLE_OWNER) + : null; + } + + /** + * Derive Instance of Glue Column from {@link CatalogBaseTable}. + * + * @param flinkColumn Instance of {@link org.apache.flink.table.catalog.Column}. + * @throws CatalogException Throws exception in case of failure. + */ + public static Column getGlueColumn(org.apache.flink.table.catalog.Column flinkColumn) + throws CatalogException { + return Column.builder() + .comment(flinkColumn.asSummaryString()) + .type(TypeMapper.mapFlinkTypeToGlueType(flinkColumn.getDataType().getLogicalType())) + .name(flinkColumn.getName()) + .parameters(buildGlueColumnParams(flinkColumn)) + .build(); + } + + public static Map buildGlueColumnParams( + org.apache.flink.table.catalog.Column column) { + Map params = new HashMap<>(); + params.put(IS_PERSISTED, column.isPersisted() ? TRUE : FALSE); + params.put(EXPLAIN_EXTRAS, column.explainExtras().orElse(null)); + params.put(IS_PHYSICAL, column.isPhysical() ? TRUE : FALSE); + return params; + } + + /** + * Build set of {@link Column} associated with table. + * + * @param table instance of {@link CatalogBaseTable}. + * @return List of Column + */ + public static List getGlueColumnsFromCatalogTable(final CatalogBaseTable table) { + ResolvedCatalogBaseTable resolvedTable = (ResolvedCatalogBaseTable) table; + return resolvedTable.getResolvedSchema().getColumns().stream() + .map(GlueUtils::getGlueColumn) + .collect(Collectors.toList()); + } + + /** + * Extract InputFormat from properties if present and remove inputFormat from properties. + * fallback to default format if not present + * + * @param tableProperties Key/Value properties + * @return input Format. + */ + public static String extractInputFormat(final Map tableProperties) { + return tableProperties.containsKey(GlueCatalogConstants.TABLE_INPUT_FORMAT) + ? tableProperties.remove(GlueCatalogConstants.TABLE_INPUT_FORMAT) + : GlueCatalogOptions.INPUT_FORMAT.defaultValue(); + } + + /** + * Extract OutputFormat from properties if present and remove outputFormat from properties. + * fallback to default format if not present + * + * @param tableProperties Key/Value properties + * @return output Format. + */ + public static String extractOutputFormat(Map tableProperties) { + return tableProperties.containsKey(GlueCatalogConstants.TABLE_OUTPUT_FORMAT) + ? tableProperties.remove(GlueCatalogConstants.TABLE_OUTPUT_FORMAT) + : GlueCatalogOptions.OUTPUT_FORMAT.defaultValue(); + } + + /** + * Get list of filtered columns which are partition columns. + * + * @param catalogTable {@link CatalogTable} instance. + * @param columns List of all column in table. + * @return List of column marked as partition key. + */ + public static Collection getPartitionKeys( + CatalogTable catalogTable, Collection columns) { + Set partitionKeys = new HashSet<>(catalogTable.getPartitionKeys()); + return columns.stream() + .filter(column -> partitionKeys.contains(column.name())) + .collect(Collectors.toList()); + } + + public static String getDebugLog(final GlueResponse response) { + return String.format( + "Glue response : status = %s \n" + "Details = %s \nMetadataResponse = %s", + response.sdkHttpResponse().isSuccessful(), + response.sdkHttpResponse().toString(), + response.responseMetadata()); + } + + /** + * Derive {@link Schema} from Glue {@link Table}. + * + * @param glueTable Instance of {@link Table} + * @return {@link Schema} of table. + */ + public static Schema getSchemaFromGlueTable(Table glueTable) { + List columns = glueTable.storageDescriptor().columns(); + Schema.Builder schemaBuilder = Schema.newBuilder(); + for (Column column : columns) { + String columnName = column.name(); + String columnType = column.type().toLowerCase(); + AbstractDataType flinkDataType = TypeMapper.glueTypeToFlinkType(columnType); + schemaBuilder.column(columnName, flinkDataType); + } + return schemaBuilder.build(); + } + + /** + * Get column names from List of {@link Column}. + * + * @param columns List of {@link Column}. + * @return Names of all Columns. + */ + public static List getColumnNames(final List columns) { + return columns.stream().map(Column::name).collect(Collectors.toList()); + } + + /** + * Function ClassName pattern to be kept in Glue Data Catalog. + * + * @param function Catalog Function. + * @return function class name. + */ + public static String getGlueFunctionClassName(CatalogFunction function) { + switch (function.getFunctionLanguage()) { + case JAVA: + return GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX + function.getClassName(); + case SCALA: + return GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX + function.getClassName(); + case PYTHON: + return GlueCatalogConstants.FLINK_PYTHON_FUNCTION_PREFIX + function.getClassName(); + default: + throw new UnsupportedOperationException( + "GlueCatalog supports only creating: " + + Arrays.stream(FunctionLanguage.values()) + .map(FunctionLanguage::name) + .collect( + Collectors.joining( + GlueCatalogConstants.NEXT_LINE))); + } + } + + /** + * Derive the expression string from given {@link Expression}. + * + * @param expression Instance of {@link Expression}. + * @return Derived String from {@link Expression}. + */ + public static String getExpressionString(Expression expression) { + return getExpressionString(expression, new StringBuilder()); + } + + /** + * Recursively derive the expression string from given {@link Expression}. + * + * @param expression Instance of {@link Expression}. + * @param sb Used to build the derived expression string during recursion. + * @return Derived String from {@link Expression}. + */ + private static String getExpressionString(Expression expression, StringBuilder sb) { + for (Expression childExpression : expression.getChildren()) { + if (childExpression.getChildren() != null && !childExpression.getChildren().isEmpty()) { + getExpressionString(childExpression, sb); + } + } + + // If the StringBuilder is not empty, append "AND " + if (sb.length() > 0) { + sb.append(GlueCatalogConstants.SPACE) + .append(GlueCatalogConstants.AND) + .append(GlueCatalogConstants.SPACE); + } + + // Append the current expression summary + sb.append(expression.asSummaryString()); + + return sb.toString(); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..3f91d1a2 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE @@ -0,0 +1,7 @@ +flink-catalog-aws-glue +Copyright 2014-2023 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..d7d504eb --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.table.catalog.glue.factory.GlueCatalogFactory \ No newline at end of file diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java new file mode 100644 index 00000000..0be1eea6 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java @@ -0,0 +1,623 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import lombok.Data; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.GlueServiceClientConfiguration; +import software.amazon.awssdk.services.glue.model.AlreadyExistsException; +import software.amazon.awssdk.services.glue.model.BatchDeleteTableRequest; +import software.amazon.awssdk.services.glue.model.BatchDeleteTableResponse; +import software.amazon.awssdk.services.glue.model.ConcurrentModificationException; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreateDatabaseResponse; +import software.amazon.awssdk.services.glue.model.CreatePartitionRequest; +import software.amazon.awssdk.services.glue.model.CreatePartitionResponse; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.CreateTableResponse; +import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest; +import software.amazon.awssdk.services.glue.model.DeleteDatabaseResponse; +import software.amazon.awssdk.services.glue.model.DeletePartitionRequest; +import software.amazon.awssdk.services.glue.model.DeletePartitionResponse; +import software.amazon.awssdk.services.glue.model.DeleteTableRequest; +import software.amazon.awssdk.services.glue.model.DeleteTableResponse; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.EntityNotFoundException; +import software.amazon.awssdk.services.glue.model.GetDatabaseRequest; +import software.amazon.awssdk.services.glue.model.GetDatabaseResponse; +import software.amazon.awssdk.services.glue.model.GetDatabasesRequest; +import software.amazon.awssdk.services.glue.model.GetDatabasesResponse; +import software.amazon.awssdk.services.glue.model.GetPartitionRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionResponse; +import software.amazon.awssdk.services.glue.model.GetPartitionsRequest; +import software.amazon.awssdk.services.glue.model.GetPartitionsResponse; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTableResponse; +import software.amazon.awssdk.services.glue.model.GetTablesRequest; +import software.amazon.awssdk.services.glue.model.GetTablesResponse; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest; +import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse; +import software.amazon.awssdk.services.glue.model.GlueEncryptionException; +import software.amazon.awssdk.services.glue.model.GlueException; +import software.amazon.awssdk.services.glue.model.InternalServiceException; +import software.amazon.awssdk.services.glue.model.InvalidInputException; +import software.amazon.awssdk.services.glue.model.InvalidStateException; +import software.amazon.awssdk.services.glue.model.OperationTimeoutException; +import software.amazon.awssdk.services.glue.model.Partition; +import software.amazon.awssdk.services.glue.model.ResourceNotReadyException; +import software.amazon.awssdk.services.glue.model.ResourceNumberLimitExceededException; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseResponse; +import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdatePartitionResponse; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableResponse; +import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionResponse; +import software.amazon.awssdk.services.glue.model.UserDefinedFunction; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.dummySdkHttpResponse; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getFullyQualifiedName; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getPartitionFromCreatePartitionRequest; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getTableFromCreateTableRequest; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getTableFromUpdateTableRequest; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getUDFFromCreateUserDefinedFunctionRequest; + +/** Dummy Glue client for Test. */ +@Data +public class DummyGlueClient implements GlueClient { + + public Map databaseMap; + + public Map tableMap; + + public Map userDefinedFunctionMap; + + public Map> partitionMap; + + @Override + public UpdateUserDefinedFunctionResponse updateUserDefinedFunction( + UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + String functionName = + getFullyQualifiedName( + updateUserDefinedFunctionRequest.databaseName(), + updateUserDefinedFunctionRequest.functionName()); + if (!databaseMap.containsKey(updateUserDefinedFunctionRequest.databaseName()) + || !userDefinedFunctionMap.containsKey(functionName)) { + throw EntityNotFoundException.builder().build(); + } + UserDefinedFunction udf = userDefinedFunctionMap.get(functionName); + UserDefinedFunction updatedUDF = + udf.toBuilder() + .catalogId(updateUserDefinedFunctionRequest.catalogId()) + .functionName(updateUserDefinedFunctionRequest.functionName()) + .databaseName(updateUserDefinedFunctionRequest.databaseName()) + .ownerName(updateUserDefinedFunctionRequest.functionInput().ownerName()) + .ownerType(updateUserDefinedFunctionRequest.functionInput().ownerType()) + .className(updateUserDefinedFunctionRequest.functionInput().className()) + .resourceUris( + updateUserDefinedFunctionRequest.functionInput().resourceUris()) + .build(); + userDefinedFunctionMap.put(functionName, updatedUDF); + return (UpdateUserDefinedFunctionResponse) + UpdateUserDefinedFunctionResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + public DummyGlueClient() { + databaseMap = new HashMap<>(); + tableMap = new HashMap<>(); + userDefinedFunctionMap = new HashMap<>(); + partitionMap = new HashMap<>(); + } + + @Override + public String serviceName() { + return "Glue"; + } + + @Override + public void close() {} + + @Override + public GlueServiceClientConfiguration serviceClientConfiguration() { + return null; + } + + @Override + public CreateDatabaseResponse createDatabase(CreateDatabaseRequest createDatabaseRequest) + throws InvalidInputException, AlreadyExistsException, + ResourceNumberLimitExceededException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, + ConcurrentModificationException, AwsServiceException, SdkClientException, + GlueException { + CreateDatabaseResponse.Builder responseBuilder = CreateDatabaseResponse.builder(); + + if (databaseMap.containsKey(createDatabaseRequest.databaseInput().name())) { + throw AlreadyExistsException.builder().build(); + } + databaseMap.put( + createDatabaseRequest.databaseInput().name(), + GlueCatalogTestUtils.getDatabaseFromCreateDatabaseRequest(createDatabaseRequest)); + return (CreateDatabaseResponse) + responseBuilder.sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public UpdateDatabaseResponse updateDatabase(UpdateDatabaseRequest updateDatabaseRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, + ConcurrentModificationException, AwsServiceException, SdkClientException, + GlueException { + + if (!databaseMap.containsKey(updateDatabaseRequest.name())) { + throw EntityNotFoundException.builder().build(); + } + databaseMap.remove(updateDatabaseRequest.name()); + databaseMap.put( + updateDatabaseRequest.name(), + GlueCatalogTestUtils.getDatabaseFromUpdateDatabaseRequest(updateDatabaseRequest)); + + return (UpdateDatabaseResponse) + UpdateDatabaseResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public GetDatabaseResponse getDatabase(GetDatabaseRequest getDatabaseRequest) + throws InvalidInputException, EntityNotFoundException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + + GetDatabaseResponse.Builder responseBuilder = + (GetDatabaseResponse.Builder) + GetDatabaseResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)); + + if (!databaseMap.containsKey(getDatabaseRequest.name())) { + throw EntityNotFoundException.builder().build(); + } + return responseBuilder.database(databaseMap.get(getDatabaseRequest.name())).build(); + } + + @Override + public GetDatabasesResponse getDatabases(GetDatabasesRequest getDatabasesRequest) + throws InvalidInputException, InternalServiceException, OperationTimeoutException, + GlueEncryptionException, AwsServiceException, SdkClientException, + GlueException { + return (GetDatabasesResponse) + GetDatabasesResponse.builder() + .databaseList(databaseMap.values()) + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public DeleteDatabaseResponse deleteDatabase(DeleteDatabaseRequest deleteDatabaseRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, ConcurrentModificationException, AwsServiceException, + SdkClientException, GlueException { + + if (databaseMap.containsKey(deleteDatabaseRequest.name())) { + databaseMap.remove(deleteDatabaseRequest.name()); + return (DeleteDatabaseResponse) + DeleteDatabaseResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + throw EntityNotFoundException.builder().build(); + } + + @Override + public CreateTableResponse createTable(CreateTableRequest createTableRequest) + throws AlreadyExistsException, InvalidInputException, EntityNotFoundException, + ResourceNumberLimitExceededException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, + ConcurrentModificationException, ResourceNotReadyException, AwsServiceException, + SdkClientException, GlueException { + + Table table = getTableFromCreateTableRequest(createTableRequest); + String tableName = + getFullyQualifiedName( + createTableRequest.databaseName(), createTableRequest.tableInput().name()); + if (tableMap.containsKey(tableName)) { + throw AlreadyExistsException.builder().build(); + } + + tableMap.put(tableName, table); + return (CreateTableResponse) + CreateTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public UpdateTableResponse updateTable(UpdateTableRequest updateTableRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, ConcurrentModificationException, + ResourceNumberLimitExceededException, GlueEncryptionException, + ResourceNotReadyException, AwsServiceException, SdkClientException, + GlueException { + + String tableName = + getFullyQualifiedName( + updateTableRequest.databaseName(), updateTableRequest.tableInput().name()); + if (!databaseMap.containsKey(updateTableRequest.databaseName()) + || !tableMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + + tableMap.put(tableName, getTableFromUpdateTableRequest(updateTableRequest)); + return (UpdateTableResponse) + UpdateTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public GetTableResponse getTable(GetTableRequest getTableRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, ResourceNotReadyException, + AwsServiceException, SdkClientException, GlueException { + + String tableName = + getFullyQualifiedName(getTableRequest.databaseName(), getTableRequest.name()); + + if (!tableMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + + Table table = tableMap.get(tableName); + return (GetTableResponse) + GetTableResponse.builder() + .table(table) + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public GetTablesResponse getTables(GetTablesRequest getTablesRequest) + throws EntityNotFoundException, InvalidInputException, OperationTimeoutException, + InternalServiceException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + String databaseName = getTablesRequest.databaseName(); + + if (!databaseMap.containsKey(databaseName)) { + throw EntityNotFoundException.builder().build(); + } + + List tables = + tableMap.entrySet().stream() + .filter(e -> e.getKey().startsWith(databaseName)) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + return (GetTablesResponse) + GetTablesResponse.builder() + .tableList(tables) + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public DeleteTableResponse deleteTable(DeleteTableRequest deleteTableRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, ConcurrentModificationException, + ResourceNotReadyException, AwsServiceException, SdkClientException, + GlueException { + + String tableName = + getFullyQualifiedName(deleteTableRequest.databaseName(), deleteTableRequest.name()); + if (!databaseMap.containsKey(deleteTableRequest.databaseName()) + || !tableMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + + tableMap.remove(tableName); + return (DeleteTableResponse) + DeleteTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public BatchDeleteTableResponse batchDeleteTable( + BatchDeleteTableRequest batchDeleteTableRequest) + throws InvalidInputException, EntityNotFoundException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, ResourceNotReadyException, + AwsServiceException, SdkClientException, GlueException { + + if (!databaseMap.containsKey(batchDeleteTableRequest.databaseName())) { + throw EntityNotFoundException.builder().build(); + } + for (Map.Entry entry : tableMap.entrySet()) { + if (entry.getKey().startsWith(batchDeleteTableRequest.databaseName())) { + tableMap.remove(entry.getKey()); + } + } + return (BatchDeleteTableResponse) + BatchDeleteTableResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + // -- partition + @Override + public CreatePartitionResponse createPartition(CreatePartitionRequest createPartitionRequest) + throws InvalidInputException, AlreadyExistsException, + ResourceNumberLimitExceededException, InternalServiceException, + EntityNotFoundException, OperationTimeoutException, GlueEncryptionException, + AwsServiceException, SdkClientException, GlueException { + Partition partition = getPartitionFromCreatePartitionRequest(createPartitionRequest); + String tableName = + getFullyQualifiedName( + createPartitionRequest.databaseName(), createPartitionRequest.tableName()); + List partitionList = partitionMap.getOrDefault(tableName, new ArrayList<>()); + String partValues = String.join(":", partition.values()); + for (Partition part : partitionList) { + if (String.join(":", part.values()).equals(partValues)) { + throw AlreadyExistsException.builder().build(); + } + } + + partitionList.add(partition); + partitionMap.put(tableName, partitionList); + return (CreatePartitionResponse) + CreatePartitionResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public UpdatePartitionResponse updatePartition(UpdatePartitionRequest updatePartitionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + + String tableName = + getFullyQualifiedName( + updatePartitionRequest.databaseName(), updatePartitionRequest.tableName()); + if (!partitionMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + List partitionList = partitionMap.get(tableName); + String values = String.join(":", updatePartitionRequest.partitionInput().values()); + for (int i = 0; i < partitionList.size(); i++) { + if (values.equals(String.join(":", partitionList.get(i).values()))) { + partitionList.remove(i); + } + } + partitionList.add( + GlueCatalogTestUtils.getPartitionFromUpdatePartitionRequest( + updatePartitionRequest)); + partitionMap.put(tableName, partitionList); + return (UpdatePartitionResponse) + UpdatePartitionResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public GetPartitionResponse getPartition(GetPartitionRequest getPartitionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + String tableName = + getFullyQualifiedName( + getPartitionRequest.databaseName(), getPartitionRequest.tableName()); + if (!partitionMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + List partitionList = partitionMap.get(tableName); + String partitionValues = String.join(":", getPartitionRequest.partitionValues()); + for (Partition partition : partitionList) { + if (partitionValues.equals(String.join(":", partition.values()))) { + return (GetPartitionResponse) + GetPartitionResponse.builder() + .partition(partition) + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + } + return (GetPartitionResponse) + GetPartitionResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public DeletePartitionResponse deletePartition(DeletePartitionRequest deletePartitionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, AwsServiceException, SdkClientException, + GlueException { + + String tableName = + getFullyQualifiedName( + deletePartitionRequest.databaseName(), deletePartitionRequest.tableName()); + + if (!databaseMap.containsKey(deletePartitionRequest.databaseName()) + || !tableMap.containsKey(tableName) + || !partitionMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + + List partitions = partitionMap.get(tableName); + int pos = 0; + for (Partition partition : partitions) { + if (matchValues(partition.values(), deletePartitionRequest.partitionValues())) { + break; + } + pos++; + } + if (pos < partitions.size()) { + partitions.remove(pos); + partitionMap.remove(tableName); + partitionMap.put(tableName, partitions); + } + return (DeletePartitionResponse) + DeletePartitionResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + private boolean matchValues(List gluePartValues, List partValues) { + Set gluePartitionValueSet = new HashSet<>(gluePartValues); + int count = 0; + for (String partVal : partValues) { + if (gluePartitionValueSet.contains(partVal)) { + count++; + } + } + + return count == partValues.size(); + } + + @Override + public GetPartitionsResponse getPartitions(GetPartitionsRequest getPartitionsRequest) + throws EntityNotFoundException, InvalidInputException, OperationTimeoutException, + InternalServiceException, GlueEncryptionException, InvalidStateException, + ResourceNotReadyException, AwsServiceException, SdkClientException, + GlueException { + + String tableName = + getFullyQualifiedName( + getPartitionsRequest.databaseName(), getPartitionsRequest.tableName()); + if (!databaseMap.containsKey(getPartitionsRequest.databaseName()) + || !tableMap.containsKey(tableName)) { + throw EntityNotFoundException.builder().build(); + } + + return (GetPartitionsResponse) + GetPartitionsResponse.builder() + .partitions(partitionMap.getOrDefault(tableName, new ArrayList<>())) + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + // -- functions + @Override + public CreateUserDefinedFunctionResponse createUserDefinedFunction( + CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest) + throws AlreadyExistsException, InvalidInputException, InternalServiceException, + EntityNotFoundException, OperationTimeoutException, + ResourceNumberLimitExceededException, GlueEncryptionException, + AwsServiceException, SdkClientException, GlueException { + if (!databaseMap.containsKey(createUserDefinedFunctionRequest.databaseName())) { + throw EntityNotFoundException.builder().build(); + } + String functionName = + getFullyQualifiedName( + createUserDefinedFunctionRequest.databaseName(), + createUserDefinedFunctionRequest.functionInput().functionName()); + if (userDefinedFunctionMap.containsKey(functionName)) { + throw AlreadyExistsException.builder().build(); + } + UserDefinedFunction udf = + getUDFFromCreateUserDefinedFunctionRequest(createUserDefinedFunctionRequest); + userDefinedFunctionMap.put(functionName, udf); + return (CreateUserDefinedFunctionResponse) + CreateUserDefinedFunctionResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)) + .build(); + } + + @Override + public GetUserDefinedFunctionResponse getUserDefinedFunction( + GetUserDefinedFunctionRequest getUserDefinedFunctionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + if (!databaseMap.containsKey(getUserDefinedFunctionRequest.databaseName())) { + throw EntityNotFoundException.builder().build(); + } + String functionName = + getFullyQualifiedName( + getUserDefinedFunctionRequest.databaseName(), + getUserDefinedFunctionRequest.functionName()); + GetUserDefinedFunctionResponse.Builder response = GetUserDefinedFunctionResponse.builder(); + if (userDefinedFunctionMap.containsKey(functionName)) { + response.userDefinedFunction(userDefinedFunctionMap.get(functionName)); + } + + return (GetUserDefinedFunctionResponse) + response.sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } + + @Override + public GetUserDefinedFunctionsResponse getUserDefinedFunctions( + GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest) + throws EntityNotFoundException, InvalidInputException, OperationTimeoutException, + InternalServiceException, GlueEncryptionException, AwsServiceException, + SdkClientException, GlueException { + + GetUserDefinedFunctionsResponse.Builder response = + (GetUserDefinedFunctionsResponse.Builder) + GetUserDefinedFunctionsResponse.builder() + .sdkHttpResponse(dummySdkHttpResponse(200)); + if (!databaseMap.containsKey(getUserDefinedFunctionsRequest.databaseName())) { + throw EntityNotFoundException.builder().build(); + } + + List udfs = + userDefinedFunctionMap.entrySet().stream() + .filter( + e -> + e.getKey() + .startsWith( + getUserDefinedFunctionsRequest + .databaseName())) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + return response.userDefinedFunctions(udfs).build(); + } + + @Override + public DeleteUserDefinedFunctionResponse deleteUserDefinedFunction( + DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest) + throws EntityNotFoundException, InvalidInputException, InternalServiceException, + OperationTimeoutException, AwsServiceException, SdkClientException, + GlueException { + + String functionName = + getFullyQualifiedName( + deleteUserDefinedFunctionRequest.databaseName(), + deleteUserDefinedFunctionRequest.functionName()); + + if (!databaseMap.containsKey(deleteUserDefinedFunctionRequest.databaseName()) + || !userDefinedFunctionMap.containsKey(functionName)) { + throw EntityNotFoundException.builder().build(); + } + + DeleteUserDefinedFunctionResponse.Builder response = + DeleteUserDefinedFunctionResponse.builder(); + userDefinedFunctionMap.remove(functionName); + return (DeleteUserDefinedFunctionResponse) + response.sdkHttpResponse(dummySdkHttpResponse(200)).build(); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java new file mode 100644 index 00000000..9974c517 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.configuration.ConfigOption; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Set; + +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.CREDENTIAL_PROVIDER; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.DEFAULT_DATABASE; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_ACCOUNT_ID; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ENDPOINT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ID; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.HTTP_CLIENT_TYPE; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.INPUT_FORMAT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.OUTPUT_FORMAT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.REGION; + +class GlueCatalogOptionsTest { + + @Test + public void testGetAllConfigOptions() { + Set> allConfigOptions = GlueCatalogOptions.getAllConfigOptions(); + Assertions.assertEquals(9, allConfigOptions.size()); + Assertions.assertTrue(allConfigOptions.contains(INPUT_FORMAT)); + Assertions.assertTrue(allConfigOptions.contains(OUTPUT_FORMAT)); + Assertions.assertTrue(allConfigOptions.contains(GLUE_CATALOG_ENDPOINT)); + Assertions.assertTrue(allConfigOptions.contains(GLUE_ACCOUNT_ID)); + Assertions.assertTrue(allConfigOptions.contains(GLUE_CATALOG_ID)); + Assertions.assertTrue(allConfigOptions.contains(DEFAULT_DATABASE)); + Assertions.assertTrue(allConfigOptions.contains(HTTP_CLIENT_TYPE)); + Assertions.assertTrue(allConfigOptions.contains(REGION)); + Assertions.assertTrue(allConfigOptions.contains(CREDENTIAL_PROVIDER)); + } + + @Test + public void testGetRequiredConfigOptions() { + Set> requiredOptions = GlueCatalogOptions.getRequiredConfigOptions(); + Assertions.assertEquals(0, requiredOptions.size()); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java new file mode 100644 index 00000000..2af1ad82 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java @@ -0,0 +1,998 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionImpl; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.catalog.glue.operator.GlueDatabaseOperator; +import org.apache.flink.table.catalog.glue.operator.GlueFunctionOperator; +import org.apache.flink.table.catalog.glue.operator.GluePartitionOperator; +import org.apache.flink.table.catalog.glue.operator.GlueTableOperator; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.types.DataType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COLUMN_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COLUMN_2; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COMMENT; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_2; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_DESCRIPTION; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.FUNCTION_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_2; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_3; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_4; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_5; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.VIEW_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.VIEW_2; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDatabaseParams; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogDatabase; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogTable; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogTableWithPartition; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyTableParams; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getPartitionSpecParams; + +class GlueCatalogTest { + + public static final String WAREHOUSE_PATH = "s3://bucket"; + private static final String CATALOG_NAME = "glue"; + private static DummyGlueClient glue; + private static GlueCatalog glueCatalog; + + @BeforeAll + static void setUp() { + glue = new DummyGlueClient(); + String glueCatalogId = "dummy-catalog-Id"; + GlueDatabaseOperator glueDatabaseOperator = + new GlueDatabaseOperator(CATALOG_NAME, glue, glueCatalogId); + GlueTableOperator glueTableOperator = + new GlueTableOperator(CATALOG_NAME, glue, glueCatalogId); + GluePartitionOperator gluePartitionOperator = + new GluePartitionOperator(CATALOG_NAME, glue, glueCatalogId); + GlueFunctionOperator glueFunctionOperator = + new GlueFunctionOperator(CATALOG_NAME, glue, glueCatalogId); + glueCatalog = + new GlueCatalog( + CATALOG_NAME, + GlueCatalog.DEFAULT_DB, + glue, + glueDatabaseOperator, + glueTableOperator, + gluePartitionOperator, + glueFunctionOperator); + } + + @BeforeEach + public void clear() { + glue.setDatabaseMap(new HashMap<>()); + glue.setTableMap(new HashMap<>()); + glue.setPartitionMap(new HashMap<>()); + glue.setUserDefinedFunctionMap(new HashMap<>()); + } + + // ------ Database + @Test + void testCreateDatabase() throws DatabaseNotExistException { + + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1)); + CatalogDatabase catalogDatabase = + new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false)); + CatalogDatabase database = glueCatalog.getDatabase(DATABASE_1); + Assertions.assertNotNull(database); + Assertions.assertNotNull(database.getProperties()); + Assertions.assertNotNull(database.getComment()); + Assertions.assertEquals(DATABASE_DESCRIPTION, database.getComment()); + Assertions.assertThrows( + DatabaseAlreadyExistException.class, + () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false)); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_2, catalogDatabase, true)); + } + + @Test + void testAlterDatabase() throws DatabaseNotExistException, DatabaseAlreadyExistException { + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1)); + CatalogDatabase catalogDatabase = + new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + CatalogDatabase database = glueCatalog.getDatabase(DATABASE_1); + Assertions.assertNotNull(database); + Assertions.assertNotNull(database.getProperties()); + Assertions.assertNotNull(database.getComment()); + Assertions.assertEquals(DATABASE_DESCRIPTION, database.getComment()); + + Assertions.assertThrows( + DatabaseNotExistException.class, + () -> glueCatalog.alterDatabase(DATABASE_2, database, false)); + Assertions.assertDoesNotThrow(() -> glueCatalog.alterDatabase(DATABASE_2, database, true)); + + Map properties = catalogDatabase.getProperties(); + properties.put("newKey", "val"); + CatalogDatabase newCatalogDatabase = catalogDatabase.copy(properties); + Assertions.assertDoesNotThrow( + () -> glueCatalog.alterDatabase(DATABASE_1, newCatalogDatabase, false)); + CatalogDatabase database1 = glueCatalog.getDatabase(DATABASE_1); + Assertions.assertNotNull(database1); + Assertions.assertNotNull(database1.getProperties()); + Assertions.assertEquals(database1.getProperties(), properties); + Assertions.assertNotNull(database1.getComment()); + Assertions.assertEquals(DATABASE_DESCRIPTION, database1.getComment()); + } + + @Test + void testDatabaseExists() + throws DatabaseAlreadyExistException, DatabaseNotEmptyException, + DatabaseNotExistException { + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + CatalogDatabase catalogDatabase = + new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1)); + glueCatalog.dropDatabase(DATABASE_1, true, true); + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1)); + glueCatalog.dropDatabase(DATABASE_1, false, false); + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1)); + glueCatalog.dropDatabase(DATABASE_1, true, false); + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1)); + glueCatalog.dropDatabase(DATABASE_1, false, true); + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + } + + @Test + void testDropDatabase() throws DatabaseAlreadyExistException { + + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, false)); + + Assertions.assertThrows( + DatabaseNotExistException.class, + () -> glueCatalog.dropDatabase(DATABASE_2, false, true)); + + Assertions.assertThrows( + DatabaseNotExistException.class, + () -> glueCatalog.dropDatabase(DATABASE_2, false, false)); + + CatalogDatabase catalogDatabase = + new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, true)); + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1)); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, false)); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, true)); + glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, false)); + } + + @Test + void testListDatabases() { + Assertions.assertEquals(new ArrayList<>(), glueCatalog.listDatabases()); + List expectedDatabasesList = Arrays.asList(DATABASE_1, DATABASE_2); + CatalogDatabase catalogDatabase = + new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false)); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_2, catalogDatabase, false)); + Assertions.assertEquals(expectedDatabasesList, glueCatalog.listDatabases()); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, false)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_2, false, false)); + } + + @Test + void testGetDatabase() throws DatabaseNotExistException { + + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1)); + Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_2)); + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1)); + createDatabase(DATABASE_1); + CatalogDatabase db = glueCatalog.getDatabase(DATABASE_1); + Assertions.assertEquals(getDummyCatalogDatabase().getComment(), db.getComment()); + Assertions.assertEquals(getDatabaseParams(), db.getProperties()); + } + + @Test + void testIsDatabaseEmpty() + throws TableAlreadyExistException, DatabaseNotExistException, + FunctionAlreadyExistException { + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_1, getDummyCatalogDatabase(), false)); + Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1)); + + // create a table for the database + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + glueCatalog.createTable(tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false); + Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false)); + Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1)); + + // create userDefinedFunctions for the database + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + Assertions.assertDoesNotThrow( + () -> + glueCatalog.createFunction( + functionPath, + GlueCatalogTestUtils.getDummyCatalogFunction(), + false)); + Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false)); + Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1)); + + // both table and userDefinedFunction are present + glueCatalog.createTable(tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false); + glueCatalog.createFunction( + functionPath, GlueCatalogTestUtils.getDummyCatalogFunction(), false); + Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false)); + Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false)); + Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1)); + } + + // ------ Table + @Test + public void testCreateTable() throws TableNotExistException { + + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + Assertions.assertThrows( + DatabaseNotExistException.class, + () -> glueCatalog.createTable(tablePath, getDummyCatalogTable(), false)); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(DATABASE_1, getDummyCatalogDatabase(), false)); + Assertions.assertDoesNotThrow( + () -> + glueCatalog.createTable( + tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false)); + CatalogBaseTable table = glueCatalog.getTable(tablePath); + Assertions.assertEquals( + table.getUnresolvedSchema().getColumns().size(), + getDummyCatalogTable().getUnresolvedSchema().getColumns().size()); + Assertions.assertEquals(table.getTableKind(), getDummyCatalogTable().getTableKind()); + ObjectPath tablePath2 = new ObjectPath(DATABASE_1, TABLE_2); + CatalogBaseTable catalogBaseTable = getDummyCatalogTableWithPartition(); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createTable(tablePath2, catalogBaseTable, false)); + table = glueCatalog.getTable(tablePath2); + Assertions.assertEquals( + table.getUnresolvedSchema().getColumns().get(0).getName(), + catalogBaseTable.getUnresolvedSchema().getColumns().get(0).getName()); + Assertions.assertEquals( + table.getUnresolvedSchema().getColumns().get(1).getName(), + catalogBaseTable.getUnresolvedSchema().getColumns().get(1).getName()); + Assertions.assertEquals(table.getTableKind(), catalogBaseTable.getTableKind()); + } + + @Test + public void testCreateView() throws TableNotExistException { + ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_1); + Assertions.assertThrows(TableNotExistException.class, () -> glueCatalog.getTable(viewPath)); + createDatabase(viewPath.getDatabaseName()); + createView(viewPath); + CatalogBaseTable view = glueCatalog.getTable(viewPath); + Assertions.assertNotNull(view); + Assertions.assertEquals(getDummyTableParams(), view.getOptions()); + Assertions.assertEquals(CatalogTable.TableKind.VIEW.name(), view.getTableKind().name()); + ObjectPath tablePath = new ObjectPath(DATABASE_2, TABLE_2); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + CatalogBaseTable table = glueCatalog.getTable(tablePath); + Assertions.assertNotNull(table); + Assertions.assertEquals(getDummyTableParams(), table.getOptions()); + Assertions.assertEquals(CatalogTable.TableKind.TABLE.name(), table.getTableKind().name()); + } + + @Test + public void testGetTable() throws TableNotExistException { + Assertions.assertThrows( + IllegalArgumentException.class, + () -> glueCatalog.getTable(new ObjectPath(null, null))); + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + Assertions.assertThrows( + TableNotExistException.class, () -> glueCatalog.getTable(tablePath)); + createDatabase(tablePath.getDatabaseName()); + Assertions.assertThrows( + TableNotExistException.class, () -> glueCatalog.getTable(tablePath)); + createTable(tablePath); + CatalogBaseTable table = glueCatalog.getTable(tablePath); + Assertions.assertNotNull(table); + Assertions.assertEquals( + CatalogBaseTable.TableKind.TABLE.name(), table.getTableKind().name()); + Assertions.assertEquals(getDummyTableParams(), table.getOptions()); + } + + @Test + public void testGetView() throws TableNotExistException { + + ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_1); + createDatabase(viewPath.getDatabaseName()); + createView(viewPath); + CatalogBaseTable view = glueCatalog.getTable(viewPath); + Assertions.assertNotNull(view); + Assertions.assertEquals(CatalogBaseTable.TableKind.VIEW.name(), view.getTableKind().name()); + Assertions.assertEquals(getDummyTableParams(), view.getOptions()); + } + + @Test + public void testTableExists() { + + ObjectPath tablePath = new ObjectPath(DATABASE_2, TABLE_1); + Assertions.assertThrows(NullPointerException.class, () -> glueCatalog.getTable(null)); + Assertions.assertThrows( + TableNotExistException.class, () -> glueCatalog.getTable(tablePath)); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + Assertions.assertDoesNotThrow(() -> glueCatalog.getTable(tablePath)); + Assertions.assertThrows( + TableNotExistException.class, + () -> glueCatalog.getTable(new ObjectPath(DATABASE_2, TABLE_2))); + Assertions.assertThrows( + TableNotExistException.class, + () -> glueCatalog.getTable(new ObjectPath(DATABASE_1, TABLE_2))); + Assertions.assertTrue(glueCatalog.tableExists(tablePath)); + Assertions.assertFalse(glueCatalog.tableExists(new ObjectPath(DATABASE_1, TABLE_1))); + } + + @Test + public void testListTables() throws DatabaseNotExistException { + createDatabase(DATABASE_1); + createTable(new ObjectPath(DATABASE_1, TABLE_1)); + createTable(new ObjectPath(DATABASE_1, TABLE_2)); + createTable(new ObjectPath(DATABASE_1, TABLE_3)); + createTable(new ObjectPath(DATABASE_1, TABLE_4)); + createTable(new ObjectPath(DATABASE_1, TABLE_5)); + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2)); + Assertions.assertEquals(5, glueCatalog.listTables(DATABASE_1).size()); + Assertions.assertEquals( + Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5), + glueCatalog.listTables(DATABASE_1)); + createView(new ObjectPath(DATABASE_1, VIEW_1)); + Assertions.assertEquals(6, glueCatalog.listTables(DATABASE_1).size()); + Assertions.assertEquals( + Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1), + glueCatalog.listTables(DATABASE_1)); + } + + @Test + public void testListTablesWithCombinationOfDifferentTableKind() + throws DatabaseNotExistException { + createDatabase(DATABASE_1); + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2)); + Assertions.assertDoesNotThrow(() -> glueCatalog.listTables(DATABASE_1)); + createTable(new ObjectPath(DATABASE_1, TABLE_1)); + createTable(new ObjectPath(DATABASE_1, TABLE_2)); + createTable(new ObjectPath(DATABASE_1, TABLE_3)); + createView(new ObjectPath(DATABASE_1, VIEW_2)); + createTable(new ObjectPath(DATABASE_1, TABLE_4)); + createTable(new ObjectPath(DATABASE_1, TABLE_5)); + createView(new ObjectPath(DATABASE_1, VIEW_1)); + Assertions.assertEquals(7, glueCatalog.listTables(DATABASE_1).size()); + Assertions.assertEquals( + Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1, VIEW_2), + glueCatalog.listTables(DATABASE_1)); + } + + @Test + public void testListView() throws DatabaseNotExistException { + createDatabase(DATABASE_1); + Assertions.assertThrows( + DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2)); + Assertions.assertDoesNotThrow(() -> glueCatalog.listTables(DATABASE_1)); + createTable(new ObjectPath(DATABASE_1, TABLE_1)); + createTable(new ObjectPath(DATABASE_1, TABLE_2)); + createTable(new ObjectPath(DATABASE_1, TABLE_3)); + createView(new ObjectPath(DATABASE_1, VIEW_2)); + createTable(new ObjectPath(DATABASE_1, TABLE_4)); + createTable(new ObjectPath(DATABASE_1, TABLE_5)); + createView(new ObjectPath(DATABASE_1, VIEW_1)); + Assertions.assertEquals(2, glueCatalog.listViews(DATABASE_1).size()); + Assertions.assertNotSame( + Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1, VIEW_2), + glueCatalog.listViews(DATABASE_1)); + Assertions.assertEquals(Arrays.asList(VIEW_1, VIEW_2), glueCatalog.listViews(DATABASE_1)); + Assertions.assertNotSame( + Arrays.asList(TABLE_1, TABLE_2, VIEW_1, VIEW_2), + glueCatalog.listViews(DATABASE_1), + "Should not contain any identifier of type table"); + } + + @Test + public void testAlterTable() throws TableNotExistException { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + Assertions.assertThrows( + NullPointerException.class, () -> glueCatalog.alterTable(tablePath, null, false)); + createTable(tablePath); + Assertions.assertDoesNotThrow(() -> glueCatalog.getTable(tablePath)); + CatalogBaseTable table = glueCatalog.getTable(tablePath); + Assertions.assertNotNull(table); + Assertions.assertEquals( + table.getTableKind().name(), CatalogBaseTable.TableKind.TABLE.name()); + Assertions.assertEquals(table.getOptions(), getDummyTableParams()); + Assertions.assertNotNull(table.getUnresolvedSchema()); + Map modifiedOptions = table.getOptions(); + modifiedOptions.put("newKey", "newValue"); + Schema schema = table.getUnresolvedSchema(); + Assertions.assertNotNull(schema); + + Schema modifiedSchema = + Schema.newBuilder().fromSchema(schema).column("col3", DataTypes.STRING()).build(); + List dataTypes = + Arrays.asList(DataTypes.STRING(), DataTypes.STRING(), DataTypes.STRING()); + ResolvedSchema resolvedSchema = + ResolvedSchema.physical( + modifiedSchema.getColumns().stream() + .map(Schema.UnresolvedColumn::getName) + .collect(Collectors.toList()), + dataTypes); + ResolvedCatalogTable table1 = + new ResolvedCatalogTable( + CatalogTable.of( + modifiedSchema, + "Changed Comment", + new ArrayList<>(), + modifiedOptions), + resolvedSchema); + Assertions.assertDoesNotThrow(() -> glueCatalog.alterTable(tablePath, table1, false)); + CatalogBaseTable retrievedTable = glueCatalog.getTable(tablePath); + Assertions.assertEquals(modifiedOptions, retrievedTable.getOptions()); + Assertions.assertEquals( + modifiedSchema.getColumns().size(), + retrievedTable.getUnresolvedSchema().getColumns().size()); + } + + @Test + public void testDropTable() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2); + ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_2); + createDatabase(tablePath.getDatabaseName()); + Assertions.assertThrows( + TableNotExistException.class, () -> glueCatalog.dropTable(tablePath, false)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, true)); + createTable(tablePath); + createView(viewPath); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false)); + Assertions.assertThrows( + TableNotExistException.class, () -> glueCatalog.getTable(tablePath)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, true)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(viewPath, false)); + Assertions.assertThrows(TableNotExistException.class, () -> glueCatalog.getTable(viewPath)); + } + + @Test + public void testRenameTable() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2); + ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_2); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + createView(viewPath); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.renameTable(tablePath, TABLE_4, false)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.renameTable(viewPath, VIEW_1, false)); + } + + // ------- Function + @Test + public void testCreateFunction() { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + createDatabase(functionPath.getDatabaseName()); + Assertions.assertFalse(glueCatalog.functionExists(functionPath)); + createFunction(functionPath, FunctionLanguage.JAVA, "TestClass"); + Assertions.assertTrue(glueCatalog.functionExists(functionPath)); + } + + @Test + public void testNormalize() { + ObjectPath functionPath = new ObjectPath(DATABASE_1, "Function-1"); + ObjectPath normalizeFunctionPath = glueCatalog.normalize(functionPath); + Assertions.assertNotNull(normalizeFunctionPath); + Assertions.assertEquals(DATABASE_1, normalizeFunctionPath.getDatabaseName()); + Assertions.assertEquals("function-1", normalizeFunctionPath.getObjectName()); + } + + @Test + public void testAlterFunction() { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + createDatabase(functionPath.getDatabaseName()); + Assertions.assertFalse(glueCatalog.functionExists(functionPath)); + CatalogFunction catalogFunction = + new CatalogFunctionImpl("ClassName", FunctionLanguage.JAVA); + Assertions.assertThrows( + FunctionNotExistException.class, + () -> glueCatalog.alterFunction(functionPath, catalogFunction, true)); + + createFunction(functionPath, FunctionLanguage.JAVA, "TestClass"); + Assertions.assertTrue(glueCatalog.functionExists(functionPath)); + Assertions.assertDoesNotThrow( + () -> glueCatalog.alterFunction(functionPath, catalogFunction, false)); + } + + @Test + public void testDropFunction() { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + createDatabase(functionPath.getDatabaseName()); + Assertions.assertThrows( + FunctionNotExistException.class, + () -> glueCatalog.dropFunction(functionPath, false)); + createFunction(functionPath, FunctionLanguage.JAVA, "TestClass"); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false)); + Assertions.assertThrows( + FunctionNotExistException.class, + () -> glueCatalog.dropFunction(functionPath, false)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, true)); + } + + @Test + public void testListFunctions() throws DatabaseNotExistException { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + String className = GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX + "TestClass"; + createDatabase(DATABASE_1); + createFunction(functionPath, FunctionLanguage.SCALA, className); + Assertions.assertDoesNotThrow(() -> glueCatalog.listFunctions(DATABASE_1)); + List udfList = glueCatalog.listFunctions(DATABASE_1); + Assertions.assertNotNull(udfList); + Assertions.assertEquals(1, udfList.size()); + } + + @Test + public void testGetFunction() throws FunctionNotExistException { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + createDatabase(functionPath.getDatabaseName()); + String className = GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX + "TestClass"; + createFunction(functionPath, FunctionLanguage.JAVA, className); + Assertions.assertThrows( + ValidationException.class, () -> glueCatalog.getFunction(functionPath)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false)); + createFunction(functionPath, FunctionLanguage.JAVA, "TestClass"); + CatalogFunction catalogFunction = glueCatalog.getFunction(functionPath); + Assertions.assertNotNull(catalogFunction); + Assertions.assertEquals(FunctionLanguage.JAVA, catalogFunction.getFunctionLanguage()); + Assertions.assertEquals(3, catalogFunction.getFunctionResources().size()); + Assertions.assertEquals("TestClass", catalogFunction.getClassName()); + } + + @Test + public void testFunctionExists() { + ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1); + Assertions.assertFalse(glueCatalog.functionExists(functionPath)); + createDatabase(functionPath.getDatabaseName()); + Assertions.assertFalse(glueCatalog.functionExists(functionPath)); + createFunction(functionPath, FunctionLanguage.JAVA, "TestClass"); + Assertions.assertTrue(glueCatalog.functionExists(functionPath)); + } + + // ------ Partition + @Test + public void testCreatePartition() throws PartitionNotExistException { + + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + CatalogPartition catalogPartition = + new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT); + Assertions.assertDoesNotThrow( + () -> + glueCatalog.createPartition( + tablePath, partitionSpec, catalogPartition, false)); + + CatalogPartition partition = glueCatalog.getPartition(tablePath, partitionSpec); + Assertions.assertNotNull(partition); + Assertions.assertEquals(getPartitionSpecParams(), partition.getProperties()); + + Assertions.assertThrows( + NullPointerException.class, + () -> glueCatalog.createPartition(null, partitionSpec, catalogPartition, false)); + + Assertions.assertThrows( + NullPointerException.class, + () -> glueCatalog.createPartition(tablePath, null, catalogPartition, false)); + + Assertions.assertThrows( + NullPointerException.class, + () -> glueCatalog.createPartition(tablePath, partitionSpec, null, false)); + + Assertions.assertThrows( + CatalogException.class, + () -> + glueCatalog.getPartition( + tablePath, new CatalogPartitionSpec(new HashMap<>()))); + + Assertions.assertThrows( + NullPointerException.class, + () -> glueCatalog.getPartition(tablePath, new CatalogPartitionSpec(null))); + + Assertions.assertThrows( + CatalogException.class, + () -> + glueCatalog.createPartition( + tablePath, + new CatalogPartitionSpec(new HashMap<>()), + catalogPartition, + false)); + + Assertions.assertThrows( + PartitionAlreadyExistsException.class, + () -> + glueCatalog.createPartition( + tablePath, + partitionSpec, + new CatalogPartitionImpl(new HashMap<>(), COMMENT), + false)); + } + + @Test + public void testListPartitions() + throws TableNotPartitionedException, TableNotExistException, + PartitionSpecInvalidException { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + Assertions.assertEquals( + 0, + glueCatalog + .listPartitions(tablePath, new CatalogPartitionSpec(new HashMap<>())) + .size()); + createPartition(tablePath); + Assertions.assertEquals( + 1, + glueCatalog + .listPartitions(tablePath, new CatalogPartitionSpec(new HashMap<>())) + .size()); + Map partSpec = new HashMap<>(); + partSpec.put(COLUMN_1, "v1"); + partSpec.put(COLUMN_2, "v2"); + Assertions.assertEquals( + new CatalogPartitionSpec(partSpec), + glueCatalog + .listPartitions( + tablePath, new CatalogPartitionSpec(getPartitionSpecParams())) + .get(0)); + } + + @Test + public void testIsPartitionedTable() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + createNonPartitionedTable(tablePath); + Assertions.assertFalse(glueCatalog.isPartitionedTable(tablePath)); + Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false)); + createTable(tablePath); + createPartition(tablePath); + Assertions.assertTrue(glueCatalog.isPartitionedTable(tablePath)); + } + + @Test + public void testListPartitionsByFilter() + throws TableNotPartitionedException, TableNotExistException { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + createPartition(tablePath); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + Assertions.assertDoesNotThrow(() -> glueCatalog.getPartition(tablePath, partitionSpec)); + List expressions = new ArrayList<>(); + Assertions.assertDoesNotThrow( + () -> glueCatalog.listPartitionsByFilter(tablePath, expressions)); + List partitionSpecs = + glueCatalog.listPartitionsByFilter(tablePath, expressions); + Assertions.assertNotNull(partitionSpecs); + Assertions.assertEquals(1, partitionSpecs.size()); + Assertions.assertEquals(getPartitionSpecParams(), partitionSpecs.get(0).getPartitionSpec()); + } + + @Test + public void testDropPartition() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + Assertions.assertThrows( + CatalogException.class, + () -> glueCatalog.dropPartition(tablePath, partitionSpec, true)); + createDatabase(tablePath.getDatabaseName()); + Assertions.assertThrows( + CatalogException.class, + () -> glueCatalog.dropPartition(tablePath, partitionSpec, true)); + createTable(tablePath); + createPartition(tablePath); + Assertions.assertDoesNotThrow( + () -> glueCatalog.dropPartition(tablePath, partitionSpec, true)); + Assertions.assertThrows( + CatalogException.class, + () -> + glueCatalog.dropPartition( + tablePath, new CatalogPartitionSpec(new HashMap<>()), true)); + } + + @Test + public void testAlterPartition() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + createPartition(tablePath); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + CatalogPartition newPartition = new CatalogPartitionImpl(getPartitionSpecParams(), COMMENT); + + Assertions.assertDoesNotThrow( + () -> glueCatalog.alterPartition(tablePath, partitionSpec, newPartition, false)); + + Assertions.assertDoesNotThrow( + () -> glueCatalog.alterPartition(tablePath, partitionSpec, newPartition, true)); + + Map partitionSpecProperties = getPartitionSpecParams(); + partitionSpecProperties.put("test", "v3"); + + Assertions.assertThrows( + CatalogException.class, + () -> + glueCatalog.alterPartition( + tablePath, + new CatalogPartitionSpec(partitionSpecProperties), + newPartition, + false)); + + ObjectPath tablePath1 = new ObjectPath(DATABASE_1, TABLE_2); + createNonPartitionedTable(tablePath1); + // since table is not partition , test should throw Catalog Exception + + Assertions.assertThrows( + CatalogException.class, + () -> + glueCatalog.alterPartition( + tablePath, + new CatalogPartitionSpec(new HashMap<>()), + new CatalogPartitionImpl(new HashMap<>(), COMMENT), + false)); + + Assertions.assertThrows( + NullPointerException.class, + () -> + glueCatalog.alterPartition( + tablePath, + new CatalogPartitionSpec(null), + new CatalogPartitionImpl(new HashMap<>(), COMMENT), + false)); + Assertions.assertThrows( + NullPointerException.class, + () -> + glueCatalog.alterPartition( + tablePath, + new CatalogPartitionSpec(new HashMap<>()), + new CatalogPartitionImpl(null, COMMENT), + false)); + } + + @Test + public void testGetPartition() throws PartitionNotExistException { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + createPartition(tablePath); + CatalogPartitionSpec partitionSpec = + new CatalogPartitionSpec(GlueCatalogTestUtils.getPartitionSpecParams()); + CatalogPartition catalogPartition = + new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT); + Assertions.assertNotNull(catalogPartition); + + Assertions.assertDoesNotThrow(() -> glueCatalog.getPartition(tablePath, partitionSpec)); + CatalogPartition partition = glueCatalog.getPartition(tablePath, partitionSpec); + Assertions.assertNotNull(partition); + Assertions.assertNull(partition.getComment()); + Assertions.assertEquals( + GlueCatalogTestUtils.getPartitionSpecParams(), partition.getProperties()); + } + + @Test + public void testPartitionExists() { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + Assertions.assertThrows( + CatalogException.class, + () -> glueCatalog.partitionExists(tablePath, partitionSpec)); + createDatabase(tablePath.getDatabaseName()); + createTable(tablePath); + Assertions.assertFalse(glueCatalog.partitionExists(tablePath, partitionSpec)); + createPartition(tablePath); + Assertions.assertTrue(glueCatalog.partitionExists(tablePath, partitionSpec)); + CatalogPartitionSpec partitionSpecWithNoPartition = + new CatalogPartitionSpec(new HashMap<>()); + Assertions.assertThrows( + CatalogException.class, + () -> glueCatalog.partitionExists(tablePath, partitionSpecWithNoPartition)); + Map data = new HashMap<>(); + data.put("col2", "zz1"); + + CatalogPartitionSpec partSpecWithPartitionNotExist = new CatalogPartitionSpec(data); + Assertions.assertThrows( + CatalogException.class, + () -> glueCatalog.partitionExists(tablePath, partSpecWithPartitionNotExist)); + } + + // ---- stats + + @Test + public void testAllStatisticsOperationNotSupported() + throws PartitionNotExistException, TableNotExistException { + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + CatalogColumnStatistics columnStatistics = new CatalogColumnStatistics(new HashMap<>()); + CatalogTableStatistics catalogTableStatistics = + new CatalogTableStatistics(0L, 0, 0L, 0L, new HashMap<>()); + + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> + glueCatalog.alterPartitionColumnStatistics( + tablePath, partitionSpec, columnStatistics, true)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> + glueCatalog.alterPartitionColumnStatistics( + tablePath, partitionSpec, columnStatistics, false)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> + glueCatalog.alterPartitionStatistics( + tablePath, partitionSpec, catalogTableStatistics, true)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> + glueCatalog.alterPartitionStatistics( + tablePath, partitionSpec, catalogTableStatistics, false)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.alterTableColumnStatistics(tablePath, columnStatistics, true)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.alterTableColumnStatistics(tablePath, columnStatistics, false)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.alterTableStatistics(tablePath, catalogTableStatistics, true)); + Assertions.assertThrows( + UnsupportedOperationException.class, + () -> glueCatalog.alterTableStatistics(tablePath, catalogTableStatistics, false)); + Assertions.assertEquals( + glueCatalog.getPartitionColumnStatistics(tablePath, partitionSpec), + CatalogColumnStatistics.UNKNOWN); + Assertions.assertEquals( + glueCatalog.getPartitionStatistics(tablePath, partitionSpec), + CatalogTableStatistics.UNKNOWN); + Assertions.assertEquals( + glueCatalog.getTableColumnStatistics(tablePath), CatalogColumnStatistics.UNKNOWN); + Assertions.assertEquals( + glueCatalog.getTableStatistics(tablePath), CatalogTableStatistics.UNKNOWN); + } + + private void createDatabase(String databaseName) { + Assertions.assertDoesNotThrow( + () -> glueCatalog.createDatabase(databaseName, getDummyCatalogDatabase(), false)); + } + + private void createTable(ObjectPath tablePath) { + CatalogBaseTable baseTable = getDummyCatalogTableWithPartition(); + Assertions.assertDoesNotThrow(() -> glueCatalog.createTable(tablePath, baseTable, true)); + } + + private void createNonPartitionedTable(ObjectPath tablePath) { + CatalogBaseTable baseTable = getDummyCatalogTable(); + Assertions.assertDoesNotThrow(() -> glueCatalog.createTable(tablePath, baseTable, true)); + } + + private void createView(ObjectPath tablePath) { + Column column1 = Column.physical(COLUMN_1, DataTypes.STRING()); + Column column2 = Column.physical(COLUMN_2, DataTypes.STRING()); + ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2)); + CatalogView catalogView = + CatalogView.of( + Schema.newBuilder() + .column(COLUMN_1, DataTypes.STRING()) + .column(COLUMN_2, DataTypes.STRING()) + .build(), + COMMENT, + "", + "", + getDummyTableParams()); + + ResolvedCatalogView resolvedCatalogView = new ResolvedCatalogView(catalogView, schema); + Assertions.assertDoesNotThrow( + () -> glueCatalog.createTable(tablePath, resolvedCatalogView, true)); + } + + private void createFunction( + ObjectPath functionPath, FunctionLanguage language, String className) { + CatalogFunction catalogFunction = + new CatalogFunctionImpl( + className, language, GlueCatalogTestUtils.dummyFlinkResourceUri()); + + Assertions.assertDoesNotThrow( + () -> glueCatalog.createFunction(functionPath, catalogFunction, true)); + } + + private void createPartition(ObjectPath tablePath) { + CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams()); + CatalogPartition catalogPartition = + new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT); + Assertions.assertDoesNotThrow( + () -> + glueCatalog.createPartition( + tablePath, partitionSpec, catalogPartition, false)); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java new file mode 100644 index 00000000..d2103cf9 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; + +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.CreatePartitionRequest; +import software.amazon.awssdk.services.glue.model.CreateTableRequest; +import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.Partition; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest; +import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; +import software.amazon.awssdk.services.glue.model.UserDefinedFunction; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.catalog.glue.GlueCatalogTest.WAREHOUSE_PATH; + +/** Contains Utilities for Glue Catalog Tests. */ +public class GlueCatalogTestUtils { + + public static final String DATABASE_DESCRIPTION = "Test database"; + public static final String DATABASE_1 = "db1"; + public static final String DATABASE_2 = "db2"; + public static final String TABLE_1 = "t1"; + public static final String TABLE_2 = "t2"; + public static final String TABLE_3 = "t3"; + public static final String TABLE_4 = "t4"; + public static final String TABLE_5 = "t5"; + public static final String VIEW_1 = "v1"; + public static final String VIEW_2 = "v2"; + public static final String COLUMN_1 = "name"; + public static final String COLUMN_2 = "age"; + public static final String COMMENT = "comment"; + public static final String EXPANDED_TEXT = "TEST EXPANDED_TEXT"; + public static final String ORIGINAL_TEXT = "TEST ORIGINAL_TEXT"; + public static final String FUNCTION_1 = "f1"; + + public static Map getDatabaseParams() { + return new HashMap() { + { + put("key", "value"); + put("location-uri", WAREHOUSE_PATH); + } + }; + } + + public static Map getDummyTableParams() { + return new HashMap() { + { + put("tableParam1", "v1"); + put("tableParam2", "v2"); + put("tableParams3", "v3"); + put("tableParams4", "v4"); + } + }; + } + + public static Map getPartitionSpecParams() { + return new HashMap() { + { + put(COLUMN_1, "v1"); + put(COLUMN_2, "v2"); + } + }; + } + + /** + * Parameter related to partition. + * + * @return Partition Properties + */ + public static Map getCatalogPartitionParams() { + return new HashMap() { + { + put("k1", "v1"); + put("k2", "v2"); + } + }; + } + + public static SdkHttpResponse dummySdkHttpResponse(int statusCode) { + return SdkHttpResponse.builder().statusCode(statusCode).build(); + } + + public static Database getDatabaseFromCreateDatabaseRequest(CreateDatabaseRequest request) { + return Database.builder() + .catalogId(request.catalogId()) + .name(request.databaseInput().name()) + .parameters(request.databaseInput().parameters()) + .description(request.databaseInput().description()) + .locationUri(request.databaseInput().locationUri()) + .build(); + } + + public static Table getTableFromCreateTableRequest(CreateTableRequest request) { + return Table.builder() + .catalogId(request.catalogId()) + .databaseName(request.databaseName()) + .name(request.tableInput().name()) + .parameters(request.tableInput().parameters()) + .createdBy(request.tableInput().owner()) + .description(request.tableInput().description()) + .createTime(Instant.now()) + .partitionKeys(request.tableInput().partitionKeys()) + .storageDescriptor(request.tableInput().storageDescriptor()) + .tableType(request.tableInput().tableType()) + .updateTime(Instant.now()) + .viewExpandedText(request.tableInput().viewExpandedText()) + .viewOriginalText(request.tableInput().viewOriginalText()) + .build(); + } + + public static Table getTableFromUpdateTableRequest(UpdateTableRequest request) { + return Table.builder() + .catalogId(request.catalogId()) + .databaseName(request.databaseName()) + .name(request.tableInput().name()) + .parameters(request.tableInput().parameters()) + .createdBy(request.tableInput().owner()) + .description(request.tableInput().description()) + .createTime(Instant.now()) + .partitionKeys(request.tableInput().partitionKeys()) + .storageDescriptor(request.tableInput().storageDescriptor()) + .tableType(request.tableInput().tableType()) + .updateTime(Instant.now()) + .viewExpandedText(request.tableInput().viewExpandedText()) + .viewOriginalText(request.tableInput().viewOriginalText()) + .build(); + } + + public static String getFullyQualifiedName(String databaseName, String tableName) { + return databaseName + "." + tableName; + } + + public static Partition getPartitionFromCreatePartitionRequest(CreatePartitionRequest request) { + return Partition.builder() + .databaseName(request.databaseName()) + .parameters(request.partitionInput().parameters()) + .tableName(request.tableName()) + .storageDescriptor(request.partitionInput().storageDescriptor()) + .values(request.partitionInput().values()) + .build(); + } + + public static Partition getPartitionFromUpdatePartitionRequest(UpdatePartitionRequest request) { + return Partition.builder() + .storageDescriptor(request.partitionInput().storageDescriptor()) + .tableName(request.tableName()) + .databaseName(request.databaseName()) + .parameters(request.partitionInput().parameters()) + .values(request.partitionInput().values()) + .build(); + } + + public static CatalogDatabase getDummyCatalogDatabase() { + return new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION); + } + + public static UserDefinedFunction getUDFFromCreateUserDefinedFunctionRequest( + CreateUserDefinedFunctionRequest request) { + return UserDefinedFunction.builder() + .functionName(request.functionInput().functionName()) + .databaseName(request.databaseName()) + .className(request.functionInput().className()) + .resourceUris(request.functionInput().resourceUris()) + .build(); + } + + public static List dummyFlinkResourceUri() { + List resourceUris = new ArrayList<>(); + resourceUris.add( + new org.apache.flink.table.resource.ResourceUri( + org.apache.flink.table.resource.ResourceType.JAR, "URI-JAR")); + resourceUris.add( + new org.apache.flink.table.resource.ResourceUri( + org.apache.flink.table.resource.ResourceType.FILE, "URI-FILE")); + resourceUris.add( + new org.apache.flink.table.resource.ResourceUri( + org.apache.flink.table.resource.ResourceType.ARCHIVE, "URI-ARCHIVE")); + return resourceUris; + } + + public static Database getDatabaseFromUpdateDatabaseRequest( + UpdateDatabaseRequest updateDatabaseRequest) { + return Database.builder() + .catalogId(updateDatabaseRequest.catalogId()) + .name(updateDatabaseRequest.name()) + .locationUri(updateDatabaseRequest.databaseInput().locationUri()) + .description(updateDatabaseRequest.databaseInput().description()) + .parameters(updateDatabaseRequest.databaseInput().parameters()) + .build(); + } + + public static ResolvedCatalogTable getDummyCatalogTable() { + Column column1 = Column.physical(COLUMN_1, DataTypes.STRING()); + Column column2 = Column.physical(COLUMN_2, DataTypes.STRING()); + ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2)); + CatalogTable catalogTable = + CatalogTable.of( + Schema.newBuilder() + .column(COLUMN_1, DataTypes.STRING()) + .column(COLUMN_2, DataTypes.STRING()) + .build(), + COMMENT, + new ArrayList<>(), + getDummyTableParams()); + return new ResolvedCatalogTable(catalogTable, schema); + } + + public static CatalogBaseTable getDummyCatalogTableWithPartition() { + Column column1 = Column.physical(COLUMN_1, DataTypes.STRING()); + Column column2 = Column.physical(COLUMN_2, DataTypes.STRING()); + ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2)); + CatalogTable catalogTable = + CatalogTable.of( + Schema.newBuilder() + .column(COLUMN_1, DataTypes.STRING()) + .column(COLUMN_2, DataTypes.STRING()) + .build(), + COMMENT, + Arrays.asList(COLUMN_1, COLUMN_2), + getDummyTableParams()); + return new ResolvedCatalogTable(catalogTable, schema); + } + + public static CatalogFunction getDummyCatalogFunction() { + return new CatalogFunctionImpl("Test Function", FunctionLanguage.JAVA); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java new file mode 100644 index 00000000..c159d619 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java @@ -0,0 +1,91 @@ +package org.apache.flink.table.catalog.glue; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarCharType; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Unit tests for the {@link TypeMapper} class. */ +public class TypeMapperTest { + + @Test + public void testMapFlinkTypeToGlueType_Primitives() { + assertEquals("int", TypeMapper.mapFlinkTypeToGlueType(new IntType())); + assertEquals("bigint", TypeMapper.mapFlinkTypeToGlueType(new BigIntType())); + assertEquals("string", TypeMapper.mapFlinkTypeToGlueType(new VarCharType(255))); + assertEquals("boolean", TypeMapper.mapFlinkTypeToGlueType(new BooleanType())); + assertEquals("decimal", TypeMapper.mapFlinkTypeToGlueType(new DecimalType(10, 0))); + assertEquals("float", TypeMapper.mapFlinkTypeToGlueType(new FloatType())); + assertEquals("double", TypeMapper.mapFlinkTypeToGlueType(new DoubleType())); + assertEquals("date", TypeMapper.mapFlinkTypeToGlueType(new DateType())); + assertEquals("timestamp", TypeMapper.mapFlinkTypeToGlueType(new TimestampType(5))); + } + + @Test + public void testMapFlinkTypeToGlueType_Array() { + LogicalType arrayType = new ArrayType(new VarCharType(255)); + assertEquals("array", TypeMapper.mapFlinkTypeToGlueType(arrayType)); + } + + @Test + public void testMapFlinkTypeToGlueType_Map() { + LogicalType mapType = new MapType(new VarCharType(255), new IntType()); + assertEquals("map", TypeMapper.mapFlinkTypeToGlueType(mapType)); + } + + @Test + public void testMapFlinkTypeToGlueType_Row() { + RowType rowType = + RowType.of( + new LogicalType[] {new VarCharType(255), new IntType()}, + new String[] {"name", "age"}); + assertEquals("struct", TypeMapper.mapFlinkTypeToGlueType(rowType)); + } + + @Test + public void testGlueTypeToFlinkType_Primitives() { + assertEquals(DataTypes.INT(), TypeMapper.glueTypeToFlinkType("int")); + assertEquals(DataTypes.BIGINT(), TypeMapper.glueTypeToFlinkType("bigint")); + assertEquals(DataTypes.STRING(), TypeMapper.glueTypeToFlinkType("string")); + assertEquals(DataTypes.BOOLEAN(), TypeMapper.glueTypeToFlinkType("boolean")); + assertEquals(DataTypes.DECIMAL(10, 0), TypeMapper.glueTypeToFlinkType("decimal")); + assertEquals(DataTypes.FLOAT(), TypeMapper.glueTypeToFlinkType("float")); + assertEquals(DataTypes.DOUBLE(), TypeMapper.glueTypeToFlinkType("double")); + assertEquals(DataTypes.DATE(), TypeMapper.glueTypeToFlinkType("date")); + assertEquals(DataTypes.TIMESTAMP(5), TypeMapper.glueTypeToFlinkType("timestamp")); + } + + @Test + public void testGlueTypeToFlinkType_Array() { + LogicalType arrayType = new ArrayType(new VarCharType(255)); + assertEquals("array", TypeMapper.mapFlinkTypeToGlueType(arrayType)); + } + + @Test + public void testGlueTypeToFlinkType_Map() { + LogicalType mapType = new MapType(new VarCharType(255), new IntType()); + assertEquals("map", TypeMapper.mapFlinkTypeToGlueType(mapType)); + } + + @Test + public void testGlueTypeToFlinkType_Unsupported() { + assertThrows( + UnsupportedOperationException.class, + () -> TypeMapper.glueTypeToFlinkType("struct")); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java new file mode 100644 index 00000000..cc060306 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.constants; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.lang.reflect.Field; +import java.util.HashSet; +import java.util.Set; + +class AWSGlueConfigConstantsTest { + + @Test + void testNoDuplicateConstants() { + Class glueConfigConstant = AWSGlueConfigConstants.class; + Set constants = new HashSet<>(); + for (Field field : glueConfigConstant.getDeclaredFields()) { + String constantValue; + try { + field.setAccessible(true); + constantValue = (String) field.get(null); + } catch (IllegalAccessException e) { + throw new AssertionError("Error accessing constant field: " + field.getName(), e); + } + Assertions.assertFalse( + constants.contains(constantValue), + "Duplicate constant found: " + constantValue); + constants.add(constantValue); + } + Assertions.assertEquals( + constants.size(), + glueConfigConstant.getDeclaredFields().length, + "Duplicate Constant Found."); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java new file mode 100644 index 00000000..e52e98a0 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.factory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.catalog.glue.GlueCatalogOptions; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_REGION; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.CREDENTIAL_PROVIDER; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.DEFAULT_DATABASE; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_ACCOUNT_ID; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ENDPOINT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ID; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.HTTP_CLIENT_TYPE; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.INPUT_FORMAT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.OUTPUT_FORMAT; +import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.REGION; + +class GlueCatalogFactoryTest extends TestLogger { + + public static GlueCatalogFactory factory; + + @BeforeAll + public static void setup() { + factory = new GlueCatalogFactory(); + } + + @Test + public void testFactoryIdentifier() { + Assertions.assertEquals(GlueCatalogOptions.IDENTIFIER, factory.factoryIdentifier()); + } + + @Test + public void testOptionalOptions() { + Set> configs = factory.optionalOptions(); + Assertions.assertNotNull(configs); + Assertions.assertEquals(9, configs.size()); + Assertions.assertTrue(configs.contains(INPUT_FORMAT)); + Assertions.assertTrue(configs.contains(OUTPUT_FORMAT)); + Assertions.assertTrue(configs.contains(GLUE_CATALOG_ENDPOINT)); + Assertions.assertTrue(configs.contains(GLUE_ACCOUNT_ID)); + Assertions.assertTrue(configs.contains(GLUE_CATALOG_ID)); + Assertions.assertTrue(configs.contains(DEFAULT_DATABASE)); + Assertions.assertTrue(configs.contains(HTTP_CLIENT_TYPE)); + Assertions.assertTrue(configs.contains(REGION)); + Assertions.assertTrue(configs.contains(CREDENTIAL_PROVIDER)); + } + + @Test + public void testGetRequiredOptions() { + Set> configs = factory.requiredOptions(); + Assertions.assertNotNull(configs); + Assertions.assertEquals(new HashSet<>(), configs); + } + + @Test + public void testCreateCatalog() { + Map options = new HashMap<>(); + ReadableConfig configs = new Configuration(); + CatalogFactory.Context context = + new FactoryUtil.DefaultCatalogContext( + "TestContext", options, configs, ClassLoader.getSystemClassLoader()); + Assertions.assertThrows(NullPointerException.class, () -> factory.createCatalog(context)); + options.put(AWS_REGION, "us-east-1"); + Assertions.assertDoesNotThrow(() -> factory.createCatalog(context)); + } +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java new file mode 100644 index 00000000..7daeb9de --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.util; + +import org.junit.jupiter.api.Test; + +class GlueCatalogOptionsUtilsTest { + + @Test + void testGetValidatedConfigurations() {} + + @Test + void testGetProcessedResolvedOptions() {} + + @Test + void testGetNonValidatedPrefixes() {} +} diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java new file mode 100644 index 00000000..d667aad2 --- /dev/null +++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog.glue.util; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.services.glue.model.Database; +import software.amazon.awssdk.services.glue.model.UserDefinedFunction; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_1; +import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_1; + +/** Test methods in GlueUtils . */ +public class GlueUtilsTest { + + private static final String WAREHOUSE_PATH = "s3://bucket"; + + @Test + public void testGetGlueConventionalName() { + String name = "MyName"; + Assertions.assertEquals("myname", GlueUtils.getGlueConventionalName(name)); + String name1 = "Mtx@ndfv"; + Assertions.assertThrows(IllegalArgumentException.class, () -> GlueUtils.validate(name1)); + } + + @Test + public void testExtractDatabaseLocation() { + HashMap propertiesWithLocationUri = + new HashMap() { + { + put(GlueCatalogConstants.LOCATION_URI, "s3://some-path/myDb/"); + put("k1", "v1"); + } + }; + + String location = + GlueUtils.extractDatabaseLocation( + propertiesWithLocationUri, DATABASE_1, WAREHOUSE_PATH); + Assertions.assertEquals("s3://some-path/myDb/", location); + + String newLocation = + GlueUtils.extractDatabaseLocation( + propertiesWithLocationUri, DATABASE_1, WAREHOUSE_PATH); + Assertions.assertNotEquals("s3://some-path/myDb/", newLocation); + Assertions.assertEquals( + WAREHOUSE_PATH + GlueCatalogConstants.LOCATION_SEPARATOR + DATABASE_1, newLocation); + newLocation = + GlueUtils.extractDatabaseLocation(new HashMap<>(), DATABASE_1, WAREHOUSE_PATH); + Assertions.assertEquals( + newLocation, WAREHOUSE_PATH + GlueCatalogConstants.LOCATION_SEPARATOR + DATABASE_1); + } + + @Test + public void testExtractTableLocation() { + Map propertiesWithLocationUri = + new HashMap() { + { + put(GlueCatalogConstants.LOCATION_URI, "s3://some-path/myDb/myTable/"); + put("k1", "v1"); + } + }; + ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1); + String location = + GlueUtils.extractTableLocation( + propertiesWithLocationUri, tablePath, WAREHOUSE_PATH); + Assertions.assertEquals("s3://some-path/myDb/myTable/", location); + + String newLocation = + GlueUtils.extractTableLocation( + propertiesWithLocationUri, tablePath, WAREHOUSE_PATH); + Assertions.assertNotEquals("s3://some-path/myDb/myTable", newLocation); + Assertions.assertEquals( + WAREHOUSE_PATH + + GlueCatalogConstants.LOCATION_SEPARATOR + + DATABASE_1 + + GlueCatalogConstants.LOCATION_SEPARATOR + + TABLE_1, + newLocation); + } + + @Test + public void testGetCatalogDatabase() { + Map params = + new HashMap() { + { + put("k1", "v1"); + put("k2", "v2"); + } + }; + String description = "Test description"; + Database database = Database.builder().parameters(params).description(description).build(); + CatalogDatabase catalogDatabase = GlueUtils.getCatalogDatabase(database); + Assertions.assertInstanceOf(CatalogDatabase.class, catalogDatabase); + Assertions.assertEquals(catalogDatabase.getProperties(), params); + Assertions.assertEquals(catalogDatabase.getDescription().orElse(null), description); + } + + @Test + public void testGetCatalogFunctionClassName() { + UserDefinedFunction.Builder udfBuilder = + UserDefinedFunction.builder().functionName("Dummy").databaseName(DATABASE_1); + UserDefinedFunction udf1 = udfBuilder.className("org.test.Class").build(); + Assertions.assertThrows( + org.apache.flink.table.api.ValidationException.class, + () -> GlueUtils.getCatalogFunctionClassName(udf1)); + String className = GlueUtils.getGlueFunctionClassName(new CatalogFunctionImpl("TestClass")); + UserDefinedFunction udf2 = udfBuilder.className(className).build(); + Assertions.assertDoesNotThrow(() -> GlueUtils.getCatalogFunctionClassName(udf2)); + } + + @Test + public void testGetFunctionalLanguage() { + UserDefinedFunction.Builder udfBuilder = + UserDefinedFunction.builder().functionName("Dummy").databaseName(DATABASE_1); + Assertions.assertThrows( + CatalogException.class, + () -> + GlueUtils.getFunctionalLanguage( + udfBuilder.className("org.test.Class").build())); + String className = GlueUtils.getGlueFunctionClassName(new CatalogFunctionImpl("TestClass")); + UserDefinedFunction udf1 = udfBuilder.className(className).build(); + FunctionLanguage functionLanguage = GlueUtils.getFunctionalLanguage(udf1); + Assertions.assertEquals(functionLanguage, FunctionLanguage.JAVA); + } + + @Test + public void testExtractTableOwner() { + Map properties = + new HashMap() { + { + put("k1", "v1"); + put("k2", "v2"); + } + }; + + Assertions.assertNull(GlueUtils.extractTableOwner(properties)); + properties.put(GlueCatalogConstants.TABLE_OWNER, "testOwner"); + Assertions.assertEquals(GlueUtils.extractTableOwner(properties), "testOwner"); + } + + @Test + public void testExpressionString() { + Expression expression = ResolvedExpressionMock.of(DataTypes.INT(), "column1"); + Assertions.assertEquals("column1", GlueUtils.getExpressionString(expression)); + } +} diff --git a/flink-catalog-aws/pom.xml b/flink-catalog-aws/pom.xml new file mode 100644 index 00000000..ce2f0b7f --- /dev/null +++ b/flink-catalog-aws/pom.xml @@ -0,0 +1,38 @@ + + + + + 4.0.0 + + + flink-connector-aws + org.apache.flink + 4.4-SNAPSHOT + + + flink-catalog-aws-parent + Flink : Catalog : AWS : Parent + pom + + + flink-catalog-aws-glue + + + \ No newline at end of file diff --git a/flink-connector-aws-base/pom.xml b/flink-connector-aws-base/pom.xml index f06cc690..41a0ce2c 100644 --- a/flink-connector-aws-base/pom.xml +++ b/flink-connector-aws-base/pom.xml @@ -94,6 +94,10 @@ under the License. flink-architecture-tests-test test + + software.amazon.awssdk + url-connection-client + diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java index aea2bfe7..392ff6b4 100644 --- a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java +++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java @@ -154,6 +154,54 @@ public enum CredentialProvider { /** Read Request timeout for {@link SdkAsyncHttpClient}. */ public static final String HTTP_CLIENT_READ_TIMEOUT_MILLIS = "aws.http-client.read-timeout"; + /** + * The type of {@link software.amazon.awssdk.http.SdkHttpClient}. If set, all AWS clients will + * use this specified HTTP client. If not set, HTTP_CLIENT_TYPE_DEFAULT will be used. For + * specific types supported, see HTTP_CLIENT_TYPE_* defined below. + */ + public static final String HTTP_CLIENT_TYPE = "http-client.type"; + + // ---- glue configs + + /** + * Used to configure the connection timeout in milliseconds for {@link + * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when + * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE + * + *

For more details, see ... + */ + public static final String HTTP_CLIENT_CONNECTION_TIMEOUT_MS = + "http-client.connection-timeout-ms"; + + /** + * Used to configure the max connections number for {@link + * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when + * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE + * + *

For more details, see ... + */ + public static final String HTTP_CLIENT_APACHE_MAX_CONNECTIONS = + "http-client.apache.max-connections"; + + /** + * Used to configure the socket timeout in milliseconds for {@link + * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when + * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE + * + *

For more details, see ... + */ + public static final String HTTP_CLIENT_SOCKET_TIMEOUT_MS = "http-client.socket-timeout-ms"; + + public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection"; + + /** + * {@link software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client. + */ + public static final String CLIENT_TYPE_APACHE = "apache"; + public static String accessKeyId(String prefix) { return prefix + ".basic.accesskeyid"; } diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java new file mode 100644 index 00000000..d3a6bc0f --- /dev/null +++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.aws.table.util; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.aws.config.AWSConfigConstants; +import org.apache.flink.connector.base.table.options.ConfigurationValidator; +import org.apache.flink.connector.base.table.options.TableOptionsUtils; +import org.apache.flink.connector.base.table.util.ConfigurationValidatorUtil; + +import software.amazon.awssdk.http.Protocol; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** Class for handling AWS HTTP Client config options. */ +@PublicEvolving +public class HttpClientOptionUtils implements TableOptionsUtils, ConfigurationValidator { + public static final String CLIENT_PREFIX = "http-client."; + private static final String CLIENT_HTTP_PROTOCOL_VERSION_OPTION = "protocol.version"; + private static final String CLIENT_HTTP_MAX_CONNECTION_TIMEOUT_MS = "connection-timeout-ms"; + private static final String CLIENT_HTTP_MAX_SOCKET_TIMEOUT_MS = "socket-timeout-ms"; + private static final String APACHE_MAX_CONNECTIONS = "apache.max-connections"; + + private final List allowedClientTypes; + private final Map resolvedOptions; + + public HttpClientOptionUtils(String[] allowedClientTypes, Map resolvedOptions) { + this.allowedClientTypes = Arrays.asList(allowedClientTypes); + this.resolvedOptions = resolvedOptions; + } + + @Override + public Properties getValidatedConfigurations() { + Properties clientConfigurations = new Properties(); + clientConfigurations.putAll(getProcessedResolvedOptions()); + validateClientType(clientConfigurations); + validateConfigurations(clientConfigurations); + return clientConfigurations; + } + + @Override + public Map getProcessedResolvedOptions() { + Map mappedResolvedOptions = new HashMap<>(); + for (String key : resolvedOptions.keySet()) { + if (key.startsWith(CLIENT_PREFIX)) { + mappedResolvedOptions.put(translateClientKeys(key), resolvedOptions.get(key)); + } + } + return mappedResolvedOptions; + } + + @Override + public List getNonValidatedPrefixes() { + return Collections.singletonList(CLIENT_PREFIX); + } + + private static String translateClientKeys(String key) { + String truncatedKey = key.substring(CLIENT_PREFIX.length()); + switch (truncatedKey) { + case CLIENT_HTTP_PROTOCOL_VERSION_OPTION: + return AWSConfigConstants.HTTP_PROTOCOL_VERSION; + case CLIENT_HTTP_MAX_CONNECTION_TIMEOUT_MS: + return AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS; + case CLIENT_HTTP_MAX_SOCKET_TIMEOUT_MS: + return AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS; + case APACHE_MAX_CONNECTIONS: + return AWSConfigConstants.HTTP_CLIENT_APACHE_MAX_CONNECTIONS; + default: + return "aws.http-client." + truncatedKey; + } + } + + private void validateConfigurations(Properties config) { + ConfigurationValidatorUtil.validateOptionalPositiveIntProperty( + config, + AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS, + "Invalid value given for HTTP connection timeout. Must be positive integer."); + ConfigurationValidatorUtil.validateOptionalPositiveIntProperty( + config, + AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS, + "Invalid value given for HTTP socket read timeout. Must be positive integer."); + ConfigurationValidatorUtil.validateOptionalPositiveIntProperty( + config, + AWSConfigConstants.HTTP_CLIENT_APACHE_MAX_CONNECTIONS, + "Invalid value for max number of Connection. Must be positive integer."); + ConfigurationValidatorUtil.validateOptionalPositiveIntProperty( + config, + AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY, + "Invalid value given for HTTP client max concurrency. Must be positive integer."); + validateOptionalHttpProtocolProperty(config); + } + + private void validateClientType(Properties config) { + if (config.containsKey(AWSConfigConstants.HTTP_CLIENT_TYPE) + && !allowedClientTypes.contains( + config.getProperty(AWSConfigConstants.HTTP_CLIENT_TYPE))) { + throw new IllegalArgumentException("Invalid Http Client Type."); + } + } + + private void validateOptionalHttpProtocolProperty(Properties config) { + if (config.containsKey(AWSConfigConstants.HTTP_PROTOCOL_VERSION)) { + try { + Protocol.valueOf(config.getProperty(AWSConfigConstants.HTTP_PROTOCOL_VERSION)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Invalid value given for HTTP protocol. Must be HTTP1_1 or HTTP2."); + } + } + } +} diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java index cea82483..4a7d0402 100644 --- a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java +++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java @@ -40,6 +40,7 @@ import software.amazon.awssdk.http.async.SdkAsyncHttpClient; import software.amazon.awssdk.http.nio.netty.Http2Configuration; import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.profiles.ProfileFile; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sts.StsClient; @@ -267,7 +268,7 @@ private static AwsCredentialsProvider getAssumeRoleCredentialProvider( configProps.getProperty( AWSConfigConstants.externalId(configPrefix))) .build()) - .stsClient(stsClientBuilder.build()) + .stsClient(stsClientBuilder.httpClientBuilder(ApacheHttpClient.builder()).build()) .build(); } @@ -355,12 +356,52 @@ public static SdkAsyncHttpClient createAsyncHttpClient( return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS)); } + public static SdkHttpClient createSyncHttpClient( + final Properties configProperties, final ApacheHttpClient.Builder httpClientBuilder) { + final AttributeMap.Builder clientConfiguration = AttributeMap.builder(); + + clientConfiguration.put(SdkHttpConfigurationOption.TCP_KEEPALIVE, true); + Optional.ofNullable( + configProperties.getProperty( + SdkHttpConfigurationOption.TCP_KEEPALIVE.name())) + .map(Boolean::parseBoolean) + .ifPresent( + booleanValue -> + clientConfiguration.put( + SdkHttpConfigurationOption.TCP_KEEPALIVE, booleanValue)); + Optional.ofNullable( + configProperties.getProperty( + AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS)) + .map(Long::parseLong) + .ifPresent( + longValue -> + clientConfiguration.put( + SdkHttpConfigurationOption.CONNECTION_TIMEOUT, + Duration.ofMillis(longValue))); + Optional.ofNullable( + configProperties.getProperty( + AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS)) + .map(Long::parseLong) + .ifPresent( + longValue -> + clientConfiguration.put( + SdkHttpConfigurationOption.READ_TIMEOUT, + Duration.ofMillis(longValue))); + + return createSyncHttpClient(clientConfiguration.build(), httpClientBuilder); + } + public static SdkHttpClient createSyncHttpClient( final AttributeMap config, final ApacheHttpClient.Builder httpClientBuilder) { httpClientBuilder.connectionAcquisitionTimeout(CONNECTION_ACQUISITION_TIMEOUT); return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS)); } + public static SdkHttpClient createSyncHttpClient( + final AttributeMap config, final UrlConnectionHttpClient.Builder httpClientBuilder) { + return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS)); + } + /** * Extract region from resource ARN. * diff --git a/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java b/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java new file mode 100644 index 00000000..29ba3897 --- /dev/null +++ b/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.aws.table.util; + +import org.apache.flink.connector.aws.config.AWSConfigConstants; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +class HttpClientOptionUtilsTest { + + private static final String[] ALLOWED_GLUE_HTTP_CLIENTS = + new String[] { + AWSConfigConstants.CLIENT_TYPE_URLCONNECTION, AWSConfigConstants.CLIENT_TYPE_APACHE + }; + + @Test + public void testGoodHttpClientOptionsMapping() { + HttpClientOptionUtils httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, getDefaultClientOptions()); + + Map expectedConfigurations = getDefaultExpectedClientOptions(); + Map actualConfigurations = + httpClientOptionUtils.getProcessedResolvedOptions(); + + Assertions.assertEquals(expectedConfigurations, actualConfigurations); + } + + @Test + void testHttpClientOptionsUtilsFilteringNonPrefixedOptions() { + Map defaultClientOptions = getDefaultClientOptions(); + defaultClientOptions.put("aws.not.http-client.dummy.option", "someValue"); + + HttpClientOptionUtils httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultClientOptions); + + Map expectedConfigurations = getDefaultExpectedClientOptions(); + Map actualConfigurations = + httpClientOptionUtils.getProcessedResolvedOptions(); + + Assertions.assertEquals(expectedConfigurations, actualConfigurations); + } + + @Test + void testHttpClientOptionsUtilsExtractingCorrectConfiguration() { + HttpClientOptionUtils httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, getDefaultClientOptions()); + + Properties expectedConfigurations = getDefaultExpectedClientConfigs(); + Properties actualConfigurations = httpClientOptionUtils.getValidatedConfigurations(); + + Assertions.assertEquals(expectedConfigurations, actualConfigurations); + } + + @Test + void testHttpClientOptionsUtilsFailOnInvalidMaxConcurrency() { + Map defaultClientOptions = getDefaultClientOptions(); + defaultClientOptions.put("http-client.max-concurrency", "invalid-integer"); + + HttpClientOptionUtils httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultClientOptions); + + Assertions.assertThrows( + IllegalArgumentException.class, httpClientOptionUtils::getValidatedConfigurations); + } + + @Test + void testHttpClientOptionsUtilsFailOnInvalidHttpProtocol() { + Map defaultProperties = getDefaultClientOptions(); + defaultProperties.put("http-client.protocol.version", "invalid-http-protocol"); + + HttpClientOptionUtils httpClientOptionUtils = + new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultProperties); + + Assertions.assertThrows( + IllegalArgumentException.class, httpClientOptionUtils::getValidatedConfigurations); + } + + private static Map getDefaultClientOptions() { + Map defaultGlueClientOptions = new HashMap(); + defaultGlueClientOptions.put("region", "us-east-1"); + defaultGlueClientOptions.put("http-client.max-concurrency", "10000"); + defaultGlueClientOptions.put("http-client.protocol.version", "HTTP2"); + return defaultGlueClientOptions; + } + + private static Map getDefaultExpectedClientOptions() { + Map defaultExpectedGlueClientConfigurations = new HashMap(); + defaultExpectedGlueClientConfigurations.put( + AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY, "10000"); + defaultExpectedGlueClientConfigurations.put( + AWSConfigConstants.HTTP_PROTOCOL_VERSION, "HTTP2"); + return defaultExpectedGlueClientConfigurations; + } + + private static Properties getDefaultExpectedClientConfigs() { + Properties defaultExpectedGlueClientConfigurations = new Properties(); + defaultExpectedGlueClientConfigurations.put( + AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY, "10000"); + defaultExpectedGlueClientConfigurations.put( + AWSConfigConstants.HTTP_PROTOCOL_VERSION, "HTTP2"); + return defaultExpectedGlueClientConfigurations; + } +} diff --git a/pom.xml b/pom.xml index 4c69059e..f1d20347 100644 --- a/pom.xml +++ b/pom.xml @@ -80,10 +80,13 @@ under the License. flink-connector-aws-base + flink-connector-aws flink-formats-aws flink-python + flink-catalog-aws flink-connector-aws-e2e-tests + @@ -495,4 +498,4 @@ under the License. - + \ No newline at end of file