You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "nlu90 (via GitHub)" <gi...@apache.org> on 2023/03/15 22:33:19 UTC

[GitHub] [flink-connector-pulsar] nlu90 commented on a diff in pull request #35: [FLINK-31427][Table] Initial Catalog implementation with a new config model and schema conversion.

nlu90 commented on code in PR #35:
URL: https://github.com/apache/flink-connector-pulsar/pull/35#discussion_r1137779253


##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't

Review Comment:
   some syntax issues in the comment. e.g. `but the its name...`, `we will created...`
   
   



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link
+ * TopicDomain#persistent} topic to the Flink table. We don't support {@link
+ * TopicDomain#non_persistent} topics here.
+ *
+ * <p>Each topic will be mapped to a table under a database using the topic's tenant and namespace
+ * named like {@code tenant/namespace}. The mapped table has the same name as the local name of the
+ * original topic. For example, the topic {@code persistent://public/default/some} will be mapped to
+ * {@code some} table under the {@code public/default} database. This allows users to easily query
+ * from existing Pulsar topics without explicitly creating the table. It automatically determines
+ * the Flink format to use based on the stored Pulsar schema in the Pulsar topic.
+ *
+ * <p>This mapping has some limitations, such as users can't designate the watermark and thus can't
+ * use window aggregate functions for the topics that aren't created by catalog.
+ */
+@PublicEvolving
+@SuppressWarnings("java:S1192")
+public class PulsarCatalog extends AbstractCatalog {
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class);
+
+    private final CatalogConfiguration configuration;
+
+    private PulsarCatalogClient client;
+
+    public PulsarCatalog(String catalogName, CatalogConfiguration configuration) {
+        super(catalogName, configuration.getDefaultDatabase());
+
+        PulsarConfigBuilder builder = new PulsarConfigBuilder(configuration);
+
+        // Set the required options for supporting the local catalog.
+        builder.setIfMissing(PULSAR_SERVICE_URL, LOCAL_PULSAR_SERVICE_URL);
+        builder.setIfMissing(PULSAR_ADMIN_URL, LOCAL_PULSAR_ADMIN_URL);
+
+        // We may create the CatalogConfiguration twice when using the PulsarCatalogFactory.
+        // But we truly add the config validation when you want to manually create the Pulsar
+        // catalog.
+        this.configuration = builder.build(CATALOG_VALIDATOR, CatalogConfiguration::new);
+    }
+
+    @Override
+    public Optional<Factory> getFactory() {
+        // We will add PulsarDynamicTableFactory support here in the upcoming PRs.
+        return Optional.empty();
+    }
+
+    @Override
+    public void open() throws CatalogException {
+        // Create the catalog client.
+        if (client == null) {
+            try {
+                this.client = new PulsarCatalogClient(getName(), configuration);
+            } catch (PulsarClientException e) {
+                String message =
+                        "Failed to create the client in catalog "
+                                + getName()
+                                + ", config is: "
+                                + configuration;
+                throw new CatalogException(message, e);
+            }
+        }
+
+        // Create the flink managed tenant.
+        try {
+            client.initializeManagedTenant();
+        } catch (PulsarAdminException e) {
+            String managedTenant = configuration.getManagedTenant();
+            String message =
+                    "Failed to initialize the Flink managed tenant: "
+                            + managedTenant
+                            + " in catalog: "
+                            + getName()
+                            + " with config: "
+                            + configuration;
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void close() throws CatalogException {
+        if (client != null) {
+            client.close();
+            LOG.debug("Successfully close the catalog client.");
+        }
+    }
+
+    // ------ databases ------
+
+    @Override
+    public List<String> listDatabases() throws CatalogException {
+        try {
+            return client.listDatabases();
+        } catch (PulsarAdminException e) {
+            String message = "Failed to list the databases in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.getDatabase(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to query the given database: "
+                            + databaseName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public boolean databaseExists(String databaseName) throws CatalogException {
+        try {
+            return client.databaseExists(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if database: "
+                            + databaseName
+                            + " exists in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+            throws DatabaseAlreadyExistException, CatalogException {
+        try {
+            client.createDatabase(name, database, ignoreIfExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to create database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+            throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+        try {
+            client.dropDatabase(name, ignoreIfNotExists, cascade);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to drop database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            client.alterDatabase(name, newDatabase, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to alter database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    // ------ tables and views ------
+
+    @Override
+    public List<String> listTables(String name) throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.listTables(name);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to list tables of database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public List<String> listViews(String name) throws CatalogException {
+        throw new UnsupportedOperationException("Pulsar catalog don't support view");
+    }
+
+    @Override
+    public CatalogBaseTable getTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return null;
+    }
+
+    @Override
+    public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        try {
+            return client.tableExists(tablePath);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if table: " + tablePath + " exists in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+
+        try {
+            client.dropTable(tablePath, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to delete table: " + tablePath + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)

Review Comment:
   will table renaming cause underlying topic name to be changed?
   If it is the case, then it's dangerous. We need to be careful to support this operation.
   



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link
+ * TopicDomain#persistent} topic to the Flink table. We don't support {@link
+ * TopicDomain#non_persistent} topics here.
+ *
+ * <p>Each topic will be mapped to a table under a database using the topic's tenant and namespace
+ * named like {@code tenant/namespace}. The mapped table has the same name as the local name of the
+ * original topic. For example, the topic {@code persistent://public/default/some} will be mapped to
+ * {@code some} table under the {@code public/default} database. This allows users to easily query
+ * from existing Pulsar topics without explicitly creating the table. It automatically determines
+ * the Flink format to use based on the stored Pulsar schema in the Pulsar topic.
+ *
+ * <p>This mapping has some limitations, such as users can't designate the watermark and thus can't
+ * use window aggregate functions for the topics that aren't created by catalog.
+ */
+@PublicEvolving
+@SuppressWarnings("java:S1192")
+public class PulsarCatalog extends AbstractCatalog {
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class);
+
+    private final CatalogConfiguration configuration;
+
+    private PulsarCatalogClient client;
+
+    public PulsarCatalog(String catalogName, CatalogConfiguration configuration) {
+        super(catalogName, configuration.getDefaultDatabase());
+
+        PulsarConfigBuilder builder = new PulsarConfigBuilder(configuration);
+
+        // Set the required options for supporting the local catalog.
+        builder.setIfMissing(PULSAR_SERVICE_URL, LOCAL_PULSAR_SERVICE_URL);
+        builder.setIfMissing(PULSAR_ADMIN_URL, LOCAL_PULSAR_ADMIN_URL);
+
+        // We may create the CatalogConfiguration twice when using the PulsarCatalogFactory.
+        // But we truly add the config validation when you want to manually create the Pulsar
+        // catalog.
+        this.configuration = builder.build(CATALOG_VALIDATOR, CatalogConfiguration::new);
+    }
+
+    @Override
+    public Optional<Factory> getFactory() {
+        // We will add PulsarDynamicTableFactory support here in the upcoming PRs.
+        return Optional.empty();
+    }
+
+    @Override
+    public void open() throws CatalogException {
+        // Create the catalog client.
+        if (client == null) {
+            try {
+                this.client = new PulsarCatalogClient(getName(), configuration);
+            } catch (PulsarClientException e) {
+                String message =
+                        "Failed to create the client in catalog "
+                                + getName()
+                                + ", config is: "
+                                + configuration;
+                throw new CatalogException(message, e);
+            }
+        }
+
+        // Create the flink managed tenant.
+        try {
+            client.initializeManagedTenant();
+        } catch (PulsarAdminException e) {
+            String managedTenant = configuration.getManagedTenant();
+            String message =
+                    "Failed to initialize the Flink managed tenant: "
+                            + managedTenant
+                            + " in catalog: "
+                            + getName()
+                            + " with config: "
+                            + configuration;
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void close() throws CatalogException {
+        if (client != null) {
+            client.close();
+            LOG.debug("Successfully close the catalog client.");
+        }
+    }
+
+    // ------ databases ------
+
+    @Override
+    public List<String> listDatabases() throws CatalogException {
+        try {
+            return client.listDatabases();
+        } catch (PulsarAdminException e) {
+            String message = "Failed to list the databases in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.getDatabase(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to query the given database: "
+                            + databaseName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public boolean databaseExists(String databaseName) throws CatalogException {
+        try {
+            return client.databaseExists(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if database: "
+                            + databaseName
+                            + " exists in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+            throws DatabaseAlreadyExistException, CatalogException {
+        try {
+            client.createDatabase(name, database, ignoreIfExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to create database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+            throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+        try {
+            client.dropDatabase(name, ignoreIfNotExists, cascade);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to drop database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            client.alterDatabase(name, newDatabase, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to alter database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    // ------ tables and views ------
+
+    @Override
+    public List<String> listTables(String name) throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.listTables(name);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to list tables of database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public List<String> listViews(String name) throws CatalogException {
+        throw new UnsupportedOperationException("Pulsar catalog don't support view");
+    }
+
+    @Override
+    public CatalogBaseTable getTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return null;

Review Comment:
   +1.
   
   getTable should return table metadata. (schema etc.) 



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/client/TableNameMapper.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.pulsar.table.catalog.client;
+
+import org.apache.pulsar.client.admin.PulsarAdmin;
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException;
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.schema.SchemaInfo;
+import org.apache.pulsar.common.schema.SchemaType;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.NAMING_MAPPING_TOPIC;
+import static org.apache.pulsar.common.naming.TopicDomain.persistent;
+
+/**
+ * This is a table name mapping for the renamed tables. The topic in Pulsar doesn't support
+ * renaming. So we create an internal topic under any namespaces for recording its table name

Review Comment:
   Instead of spamming all namespaces across the pulsar cluster, can you just put this name mapping table in one place, such as the managed tenant?



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/config/CatalogConfiguration.java:
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.pulsar.table.catalog.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration;
+import org.apache.flink.connector.pulsar.common.config.PulsarOptions;
+import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalog;
+import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions;
+
+import org.apache.pulsar.common.naming.NamespaceName;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.schema.SchemaType;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.regex.Pattern;
+
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.isSystemServiceNamespace;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.ALLOWED_SCHEMA_TYPES;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.NAMING_MAPPING_TOPIC;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_DEFAULT_DATABASE;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_DEFAULT_PARTITION_SIZE;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_DEFAULT_SCHEMA_TYPE;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_HIDDEN_TENANT_PATTERN;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_MANAGED_TENANT;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.PULSAR_CATALOG_VISIBLE_TENANT_PATTERN;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.pulsar.common.naming.SystemTopicNames.isSystemTopic;
+import static org.apache.pulsar.common.naming.TopicDomain.persistent;
+
+/**
+ * The configured class for the Pulsar catalog. It's required when user wants to manually create a
+ * {@link PulsarCatalog}. You can use the following code to create this.
+ *
+ * <p>All the allowed config options for this class are defined in {@link PulsarCatalogOptions} and
+ * {@link PulsarOptions}.
+ *
+ * <pre><code>
+ * Configuration config = new Configuration();
+ * config.set(PulsarOptions.PULSAR_SERVICE_URL, "pulsar://example.com:6650");
+ * config.set(PulsarOptions.PULSAR_ADMIN_URL, "http://example.com:8080");
+ *
+ * CatalogConfiguration configuration = new CatalogConfiguration(config);
+ * </code></pre>
+ */
+@PublicEvolving
+public final class CatalogConfiguration extends PulsarConfiguration {
+    private static final long serialVersionUID = -8289928767236331716L;
+
+    private final String defaultDatabase;
+    private final String managedTenant;
+    @Nullable private final Pattern hiddenTenantPattern;
+    private final Pattern visibleTenantPattern;
+    private final SchemaType defaultSchemaType;
+    private final int defaultPartitionSize;
+
+    public CatalogConfiguration(Configuration configuration) {
+        super(configuration);
+
+        this.defaultDatabase = get(PULSAR_CATALOG_DEFAULT_DATABASE);
+        this.managedTenant = get(PULSAR_CATALOG_MANAGED_TENANT);
+        this.hiddenTenantPattern = get(PULSAR_CATALOG_HIDDEN_TENANT_PATTERN, Pattern::compile);
+        this.visibleTenantPattern = get(PULSAR_CATALOG_VISIBLE_TENANT_PATTERN, Pattern::compile);

Review Comment:
   these two configurations are two sides of one coin, so only need one.



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link
+ * TopicDomain#persistent} topic to the Flink table. We don't support {@link
+ * TopicDomain#non_persistent} topics here.
+ *
+ * <p>Each topic will be mapped to a table under a database using the topic's tenant and namespace
+ * named like {@code tenant/namespace}. The mapped table has the same name as the local name of the
+ * original topic. For example, the topic {@code persistent://public/default/some} will be mapped to
+ * {@code some} table under the {@code public/default} database. This allows users to easily query
+ * from existing Pulsar topics without explicitly creating the table. It automatically determines
+ * the Flink format to use based on the stored Pulsar schema in the Pulsar topic.
+ *
+ * <p>This mapping has some limitations, such as users can't designate the watermark and thus can't
+ * use window aggregate functions for the topics that aren't created by catalog.
+ */
+@PublicEvolving
+@SuppressWarnings("java:S1192")
+public class PulsarCatalog extends AbstractCatalog {
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class);
+
+    private final CatalogConfiguration configuration;
+
+    private PulsarCatalogClient client;
+
+    public PulsarCatalog(String catalogName, CatalogConfiguration configuration) {
+        super(catalogName, configuration.getDefaultDatabase());
+
+        PulsarConfigBuilder builder = new PulsarConfigBuilder(configuration);
+
+        // Set the required options for supporting the local catalog.
+        builder.setIfMissing(PULSAR_SERVICE_URL, LOCAL_PULSAR_SERVICE_URL);
+        builder.setIfMissing(PULSAR_ADMIN_URL, LOCAL_PULSAR_ADMIN_URL);
+
+        // We may create the CatalogConfiguration twice when using the PulsarCatalogFactory.
+        // But we truly add the config validation when you want to manually create the Pulsar
+        // catalog.
+        this.configuration = builder.build(CATALOG_VALIDATOR, CatalogConfiguration::new);
+    }
+
+    @Override
+    public Optional<Factory> getFactory() {
+        // We will add PulsarDynamicTableFactory support here in the upcoming PRs.
+        return Optional.empty();
+    }
+
+    @Override
+    public void open() throws CatalogException {
+        // Create the catalog client.
+        if (client == null) {
+            try {
+                this.client = new PulsarCatalogClient(getName(), configuration);
+            } catch (PulsarClientException e) {
+                String message =
+                        "Failed to create the client in catalog "
+                                + getName()
+                                + ", config is: "
+                                + configuration;
+                throw new CatalogException(message, e);
+            }
+        }
+
+        // Create the flink managed tenant.
+        try {
+            client.initializeManagedTenant();
+        } catch (PulsarAdminException e) {

Review Comment:
   If the exception is the tenant already initialized, you should continue instead of throwing an exception



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link

Review Comment:
   There's a situation one flink table may refer to multiple pulsar topics using a regex pattern in the topic name.



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogOptions.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.docs.ConfigGroup;
+import org.apache.flink.annotation.docs.ConfigGroups;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.description.Description;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet;
+
+import org.apache.pulsar.common.schema.SchemaType;
+
+import java.util.Set;
+
+import static org.apache.flink.configuration.description.TextElement.code;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.CATALOG_CONFIG_PREFIX;
+import static org.apache.flink.table.catalog.CommonCatalogOptions.DEFAULT_DATABASE_KEY;
+import static org.apache.pulsar.common.schema.SchemaType.AVRO;
+import static org.apache.pulsar.common.schema.SchemaType.JSON;
+
+/**
+ * Config options that is used to configure a {@link PulsarCatalog}. These config options are
+ * specific to catalog only. Other runtime configurations can be found in {@link
+ * org.apache.flink.connector.pulsar.common.config.PulsarOptions}.
+ */
+@PublicEvolving
+@ConfigGroups(groups = {@ConfigGroup(name = "PulsarCatalog", keyPrefix = CATALOG_CONFIG_PREFIX)})
+public final class PulsarCatalogOptions {
+
+    // Pulsar catalog name.
+    public static final String IDENTIFIER = "pulsar";
+    public static final String LOCAL_IDENTIFIER = "local-pulsar";
+    // Pulsar catalog config prefix.
+    public static final String CATALOG_CONFIG_PREFIX = "pulsar.catalog.";
+    // Default Pulsar connection endpoints.
+    public static final String LOCAL_PULSAR_SERVICE_URL = "pulsar://127.0.0.1:6650";
+    public static final String LOCAL_PULSAR_ADMIN_URL = "http://127.0.0.1:8080";
+    // Managed topic for naming mapping.
+    public static final String NAMING_MAPPING_TOPIC = "__flink_table_name_mapping";
+    // Allowed schema types for tables.
+    public static final Set<SchemaType> ALLOWED_SCHEMA_TYPES = ImmutableSet.of(JSON, AVRO);
+
+    private PulsarCatalogOptions() {
+        // This is a constant class
+    }
+
+    public static final ConfigOption<String> PULSAR_CATALOG_DEFAULT_DATABASE =
+            ConfigOptions.key(CATALOG_CONFIG_PREFIX + "defaultDatabase")
+                    .stringType()
+                    .defaultValue("public/default")
+                    .withFallbackKeys(DEFAULT_DATABASE_KEY)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "This is the default database name to use when you enable the Pulsar Catalog.")
+                                    .text(
+                                            " We will use %s by default which will query all the tables under the % tenant and %s namespace.",
+                                            code("public/default"), code("public"), code("default"))
+                                    .text(
+                                            " You can also config this option by using common table config name %s",
+                                            code(DEFAULT_DATABASE_KEY))
+                                    .build());
+
+    public static final ConfigOption<String> PULSAR_CATALOG_MANAGED_TENANT =
+            ConfigOptions.key(CATALOG_CONFIG_PREFIX + "managedTenant")
+                    .stringType()
+                    .defaultValue("__flink_catalog_managed")
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "If you create a database without the tenant information, we will create it under this tenant.")
+                                    .text(
+                                            " This tenant will be auto created when you enable the PulsarCatalog. Remember you admin token must has the authority to create it.")
+                                    .text(
+                                            " Or you can manually create a tenant in Pulsar for serving as the catalog managed tenant, and provide it here.")
+                                    .build());
+
+    public static final ConfigOption<String> PULSAR_CATALOG_HIDDEN_TENANT_PATTERN =
+            ConfigOptions.key(CATALOG_CONFIG_PREFIX + "hiddenTenantPattern")

Review Comment:
   One pattern cannot cover all tenant cases.
   I suggest to support a allowlist or blocklist instead.



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link
+ * TopicDomain#persistent} topic to the Flink table. We don't support {@link
+ * TopicDomain#non_persistent} topics here.
+ *
+ * <p>Each topic will be mapped to a table under a database using the topic's tenant and namespace
+ * named like {@code tenant/namespace}. The mapped table has the same name as the local name of the
+ * original topic. For example, the topic {@code persistent://public/default/some} will be mapped to
+ * {@code some} table under the {@code public/default} database. This allows users to easily query
+ * from existing Pulsar topics without explicitly creating the table. It automatically determines
+ * the Flink format to use based on the stored Pulsar schema in the Pulsar topic.
+ *
+ * <p>This mapping has some limitations, such as users can't designate the watermark and thus can't
+ * use window aggregate functions for the topics that aren't created by catalog.
+ */
+@PublicEvolving
+@SuppressWarnings("java:S1192")
+public class PulsarCatalog extends AbstractCatalog {
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class);
+
+    private final CatalogConfiguration configuration;
+
+    private PulsarCatalogClient client;
+
+    public PulsarCatalog(String catalogName, CatalogConfiguration configuration) {
+        super(catalogName, configuration.getDefaultDatabase());
+
+        PulsarConfigBuilder builder = new PulsarConfigBuilder(configuration);
+
+        // Set the required options for supporting the local catalog.
+        builder.setIfMissing(PULSAR_SERVICE_URL, LOCAL_PULSAR_SERVICE_URL);
+        builder.setIfMissing(PULSAR_ADMIN_URL, LOCAL_PULSAR_ADMIN_URL);
+
+        // We may create the CatalogConfiguration twice when using the PulsarCatalogFactory.
+        // But we truly add the config validation when you want to manually create the Pulsar
+        // catalog.
+        this.configuration = builder.build(CATALOG_VALIDATOR, CatalogConfiguration::new);
+    }
+
+    @Override
+    public Optional<Factory> getFactory() {
+        // We will add PulsarDynamicTableFactory support here in the upcoming PRs.
+        return Optional.empty();
+    }
+
+    @Override
+    public void open() throws CatalogException {
+        // Create the catalog client.
+        if (client == null) {
+            try {
+                this.client = new PulsarCatalogClient(getName(), configuration);
+            } catch (PulsarClientException e) {
+                String message =
+                        "Failed to create the client in catalog "
+                                + getName()
+                                + ", config is: "
+                                + configuration;
+                throw new CatalogException(message, e);
+            }
+        }
+
+        // Create the flink managed tenant.
+        try {
+            client.initializeManagedTenant();
+        } catch (PulsarAdminException e) {
+            String managedTenant = configuration.getManagedTenant();
+            String message =
+                    "Failed to initialize the Flink managed tenant: "
+                            + managedTenant
+                            + " in catalog: "
+                            + getName()
+                            + " with config: "
+                            + configuration;
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void close() throws CatalogException {
+        if (client != null) {
+            client.close();
+            LOG.debug("Successfully close the catalog client.");
+        }
+    }
+
+    // ------ databases ------
+
+    @Override
+    public List<String> listDatabases() throws CatalogException {
+        try {
+            return client.listDatabases();
+        } catch (PulsarAdminException e) {
+            String message = "Failed to list the databases in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.getDatabase(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to query the given database: "
+                            + databaseName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public boolean databaseExists(String databaseName) throws CatalogException {
+        try {
+            return client.databaseExists(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if database: "
+                            + databaseName
+                            + " exists in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+            throws DatabaseAlreadyExistException, CatalogException {
+        try {
+            client.createDatabase(name, database, ignoreIfExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to create database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+            throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+        try {
+            client.dropDatabase(name, ignoreIfNotExists, cascade);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to drop database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            client.alterDatabase(name, newDatabase, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to alter database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    // ------ tables and views ------
+
+    @Override
+    public List<String> listTables(String name) throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.listTables(name);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to list tables of database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public List<String> listViews(String name) throws CatalogException {
+        throw new UnsupportedOperationException("Pulsar catalog don't support view");
+    }
+
+    @Override
+    public CatalogBaseTable getTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return null;
+    }
+
+    @Override
+    public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        try {
+            return client.tableExists(tablePath);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if table: " + tablePath + " exists in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+
+        try {
+            client.dropTable(tablePath, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to delete table: " + tablePath + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+            throws TableNotExistException, TableAlreadyExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        checkArgument(
+                !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty");
+
+        try {
+            client.renameTable(tablePath, newTableName, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to rename table: "
+                            + tablePath
+                            + " to new name: "
+                            + newTableName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @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");
+
+        if (table instanceof ResolvedCatalogTable) {
+            try {
+                client.createTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists);
+            } catch (PulsarAdminException e) {

Review Comment:
   what if it already exists.



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java:
##########
@@ -0,0 +1,522 @@
+/*
+ * 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.pulsar.table.catalog;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder;
+import org.apache.flink.connector.pulsar.table.catalog.client.PulsarCatalogClient;
+import org.apache.flink.connector.pulsar.table.catalog.config.CatalogConfiguration;
+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.CatalogPartitionSpec;
+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.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.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+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.factories.Factory;
+
+import org.apache.pulsar.client.admin.PulsarAdminException;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_ADMIN_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogOptions.LOCAL_PULSAR_SERVICE_URL;
+import static org.apache.flink.connector.pulsar.table.catalog.config.PulsarCatalogConfigUtils.CATALOG_VALIDATOR;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/**
+ * Catalog implementation which uses Pulsar to store Flink created databases/tables, exposes the
+ * Pulsar's namespace as the Flink databases and exposes the Pulsar's topics as the Flink tables.
+ *
+ * <h2>Database Mapping</h2>
+ *
+ * <p>{@link PulsarCatalog} offers two kinds of databases.
+ *
+ * <ul>
+ *   <li><strong>Managed Databases</strong><br>
+ *       A managed database refers to a database created by using Flink but the its name doesn't
+ *       contain tenant information.<br>
+ *       We will created the corresponding namespace under the tenant configured by {@link
+ *       PulsarCatalogOptions#PULSAR_CATALOG_MANAGED_TENANT}.
+ *   <li><strong>Pulsar Databases</strong><br>
+ *       A Pulsar databases refers to an existing namespace that wasn't a system namespace nor under
+ *       the Flink managed tenant in Pulsar. Each namespace will be mapped to a database using the
+ *       tenant and namespace name like {@code tenant/namespace}.
+ * </ul>
+ *
+ * <h2>Table Mapping</h2>
+ *
+ * <p>A table refers to a Pulsar topic, using a 1-to-1 mapping from the Pulsar's {@link
+ * TopicDomain#persistent} topic to the Flink table. We don't support {@link
+ * TopicDomain#non_persistent} topics here.
+ *
+ * <p>Each topic will be mapped to a table under a database using the topic's tenant and namespace
+ * named like {@code tenant/namespace}. The mapped table has the same name as the local name of the
+ * original topic. For example, the topic {@code persistent://public/default/some} will be mapped to
+ * {@code some} table under the {@code public/default} database. This allows users to easily query
+ * from existing Pulsar topics without explicitly creating the table. It automatically determines
+ * the Flink format to use based on the stored Pulsar schema in the Pulsar topic.
+ *
+ * <p>This mapping has some limitations, such as users can't designate the watermark and thus can't
+ * use window aggregate functions for the topics that aren't created by catalog.
+ */
+@PublicEvolving
+@SuppressWarnings("java:S1192")
+public class PulsarCatalog extends AbstractCatalog {
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class);
+
+    private final CatalogConfiguration configuration;
+
+    private PulsarCatalogClient client;
+
+    public PulsarCatalog(String catalogName, CatalogConfiguration configuration) {
+        super(catalogName, configuration.getDefaultDatabase());
+
+        PulsarConfigBuilder builder = new PulsarConfigBuilder(configuration);
+
+        // Set the required options for supporting the local catalog.
+        builder.setIfMissing(PULSAR_SERVICE_URL, LOCAL_PULSAR_SERVICE_URL);
+        builder.setIfMissing(PULSAR_ADMIN_URL, LOCAL_PULSAR_ADMIN_URL);
+
+        // We may create the CatalogConfiguration twice when using the PulsarCatalogFactory.
+        // But we truly add the config validation when you want to manually create the Pulsar
+        // catalog.
+        this.configuration = builder.build(CATALOG_VALIDATOR, CatalogConfiguration::new);
+    }
+
+    @Override
+    public Optional<Factory> getFactory() {
+        // We will add PulsarDynamicTableFactory support here in the upcoming PRs.
+        return Optional.empty();
+    }
+
+    @Override
+    public void open() throws CatalogException {
+        // Create the catalog client.
+        if (client == null) {
+            try {
+                this.client = new PulsarCatalogClient(getName(), configuration);
+            } catch (PulsarClientException e) {
+                String message =
+                        "Failed to create the client in catalog "
+                                + getName()
+                                + ", config is: "
+                                + configuration;
+                throw new CatalogException(message, e);
+            }
+        }
+
+        // Create the flink managed tenant.
+        try {
+            client.initializeManagedTenant();
+        } catch (PulsarAdminException e) {
+            String managedTenant = configuration.getManagedTenant();
+            String message =
+                    "Failed to initialize the Flink managed tenant: "
+                            + managedTenant
+                            + " in catalog: "
+                            + getName()
+                            + " with config: "
+                            + configuration;
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void close() throws CatalogException {
+        if (client != null) {
+            client.close();
+            LOG.debug("Successfully close the catalog client.");
+        }
+    }
+
+    // ------ databases ------
+
+    @Override
+    public List<String> listDatabases() throws CatalogException {
+        try {
+            return client.listDatabases();
+        } catch (PulsarAdminException e) {
+            String message = "Failed to list the databases in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.getDatabase(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to query the given database: "
+                            + databaseName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public boolean databaseExists(String databaseName) throws CatalogException {
+        try {
+            return client.databaseExists(databaseName);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if database: "
+                            + databaseName
+                            + " exists in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists)
+            throws DatabaseAlreadyExistException, CatalogException {
+        try {
+            client.createDatabase(name, database, ignoreIfExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to create database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade)
+            throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+        try {
+            client.dropDatabase(name, ignoreIfNotExists, cascade);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to drop database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+            throws DatabaseNotExistException, CatalogException {
+        try {
+            client.alterDatabase(name, newDatabase, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to alter database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    // ------ tables and views ------
+
+    @Override
+    public List<String> listTables(String name) throws DatabaseNotExistException, CatalogException {
+        try {
+            return client.listTables(name);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to list tables of database: " + name + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public List<String> listViews(String name) throws CatalogException {
+        throw new UnsupportedOperationException("Pulsar catalog don't support view");
+    }
+
+    @Override
+    public CatalogBaseTable getTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return null;
+    }
+
+    @Override
+    public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        try {
+            return client.tableExists(tablePath);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to check if table: " + tablePath + " exists in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+
+        try {
+            client.dropTable(tablePath, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message = "Failed to delete table: " + tablePath + " in catalog: " + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @Override
+    public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+            throws TableNotExistException, TableAlreadyExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        checkArgument(
+                !isNullOrWhitespaceOnly(newTableName), "newTableName cannot be null or empty");
+
+        try {
+            client.renameTable(tablePath, newTableName, ignoreIfNotExists);
+        } catch (PulsarAdminException e) {
+            String message =
+                    "Failed to rename table: "
+                            + tablePath
+                            + " to new name: "
+                            + newTableName
+                            + " in catalog: "
+                            + getName();
+            throw new CatalogException(message, e);
+        }
+    }
+
+    @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");
+
+        if (table instanceof ResolvedCatalogTable) {
+            try {
+                client.createTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists);
+            } catch (PulsarAdminException e) {
+                String message =
+                        "Failed to create table: " + tablePath + " in catalog: " + getName();
+                throw new CatalogException(message, e);
+            }
+        } else if (table instanceof ResolvedCatalogView) {
+            throw new UnsupportedOperationException("Pulsar catalog don't support view");
+        } else {
+            throw new UnsupportedOperationException(
+                    "We don't support such kind of table: " + table.getClass());
+        }
+    }
+
+    @Override
+    public void alterTable(
+            ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        checkNotNull(newTable, "newCatalogTable cannot be null");
+
+        if (newTable instanceof ResolvedCatalogTable) {
+            try {
+                client.alterTable(tablePath, (ResolvedCatalogTable) newTable, ignoreIfNotExists);
+            } catch (PulsarAdminException e) {
+                String message =
+                        "Failed to alter table: " + tablePath + " in catalog: " + getName();
+                throw new CatalogException(message, e);
+            }
+        } else if (newTable instanceof ResolvedCatalogView) {
+            throw new UnsupportedOperationException("Pulsar catalog don't support view");
+        } else {
+            throw new UnsupportedOperationException(
+                    "We don't support such kind of table: " + newTable.getClass());
+        }
+    }
+
+    @Override
+    public boolean supportsManagedTable() {
+        return true;
+    }
+
+    // ------------------------------------------------------------------------
+    // Unsupported catalog operations for Pulsar
+    // ------------------------------------------------------------------------
+
+    // ------ partitions ------
+
+    @Override
+    public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath) throws CatalogException {
+        throw new UnsupportedOperationException("Pulsar catalog don't support partition for now");

Review Comment:
   topic partition -> table partition?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org