xuefuz commented on a change in pull request #8007: [FLINK-11474][table] Add ReadableCatalog, ReadableWritableCatalog, and other … URL: https://github.com/apache/flink/pull/8007#discussion_r270959880
########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalog.java ########## @@ -0,0 +1,278 @@ +/* + * 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; + +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.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * A generic catalog implementation that holds all meta objects in memory. + */ +public class GenericInMemoryCatalog implements ReadableWritableCatalog { + + public static final String DEFAULT_DB = "default"; + + private String defaultDatabaseName = DEFAULT_DB; + + private final String catalogName; + private final Map<String, CatalogDatabase> databases; + private final Map<ObjectPath, CommonTable> tables; + + public GenericInMemoryCatalog(String name) { + Preconditions.checkArgument(!StringUtils.isNullOrWhitespaceOnly(name), "name cannot be null or empty"); + + this.catalogName = name; + this.databases = new LinkedHashMap<>(); + this.databases.put(DEFAULT_DB, new GenericCatalogDatabase()); + this.tables = new LinkedHashMap<>(); + } + + @Override + public String getDefaultDatabaseName() { + return defaultDatabaseName; + } + + @Override + public void setDefaultDatabaseName(String databaseName) { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName)); + + defaultDatabaseName = databaseName; + } + + @Override + public void open() { + + } + + @Override + public void close() { + + } + + // ------ databases ------ + + @Override + public void createDatabase(String databaseName, CatalogDatabase db, boolean ignoreIfExists) + throws DatabaseAlreadyExistException { + if (databaseExists(databaseName)) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(catalogName, databaseName); + } + } else { + databases.put(databaseName, db.copy()); + } + } + + @Override + public void dropDatabase(String dbName, boolean ignoreIfNotExists) throws DatabaseNotExistException { + if (databases.containsKey(dbName)) { + + // Make sure the database is empty + if (isDatabaseEmpty(dbName)) { + databases.remove(dbName); + } else { + throw new DatabaseNotEmptyException(catalogName, dbName); + } + } else if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(catalogName, dbName); + } + } + + private boolean isDatabaseEmpty(String databaseName) { + return tables.keySet().stream().noneMatch(op -> op.getDatabaseName().equals(databaseName)); + // TODO: also check function when function is added. + } + + @Override + public void alterDatabase(String databaseName, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException { + if (databaseExists(databaseName)) { + databases.put(databaseName, newDatabase.copy()); + } else if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(catalogName, databaseName); + } + } + + @Override + public List<String> listDatabases() { + return new ArrayList<>(databases.keySet()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(catalogName, databaseName); + } else { + return databases.get(databaseName).copy(); + } + } + + @Override + public boolean databaseExists(String dbName) { + return databases.containsKey(dbName); + } + + @Override + public void renameDatabase(String name, String newName, boolean ignoreIfNotExists) throws DatabaseNotExistException { Review comment: Yeah. I guess there isn't right/wrong answer. This was added based on other review comment. It seems okay to me to keep this in place for completeness. For catalogs that cannot support it, an unsupported op ex may be thrown. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services