This is an automated email from the ASF dual-hosted git repository.
leonard pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 288a4982c34 [FLINK-32428][table] Introduce base interfaces for
CatalogStore
288a4982c34 is described below
commit 288a4982c3473fcc08be52a6641d77c4ed2cdb5b
Author: Feng Jin <[email protected]>
AuthorDate: Thu Jul 20 18:11:36 2023 +0800
[FLINK-32428][table] Introduce base interfaces for CatalogStore
This closes #22937.
---
.../table/catalog/GenericInMemoryCatalogStore.java | 79 +++++++++++
.../GenericInMemoryCatalogStoreFactory.java | 54 ++++++++
.../GenericInMemoryCatalogStoreFactoryOptions.java | 28 ++++
.../org.apache.flink.table.factories.Factory | 1 +
.../GenericInMemoryCatalogStoreFactoryTest.java | 52 +++++++
.../catalog/GenericInMemoryCatalogStoreTest.java | 71 ++++++++++
.../flink/table/catalog/AbstractCatalogStore.java | 49 +++++++
.../flink/table/catalog/CatalogDescriptor.java | 65 +++++++++
.../apache/flink/table/catalog/CatalogStore.java | 95 +++++++++++++
.../flink/table/factories/CatalogStoreFactory.java | 152 +++++++++++++++++++++
.../apache/flink/table/factories/FactoryUtil.java | 35 +++++
.../flink/table/factories/FactoryUtilTest.java | 16 +++
.../table/factories/TestCatalogStoreFactory.java | 107 +++++++++++++++
.../org.apache.flink.table.factories.Factory | 1 +
14 files changed, 805 insertions(+)
diff --git
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStore.java
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStore.java
new file mode 100644
index 00000000000..db5bad70804
--- /dev/null
+++
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStore.java
@@ -0,0 +1,79 @@
+/*
+ * 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.CatalogException;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static java.lang.String.format;
+
+/** A generic catalog store implementation that store all catalog
configuration in memory. */
+public class GenericInMemoryCatalogStore extends AbstractCatalogStore {
+
+ private final Map<String, CatalogDescriptor> descriptors;
+
+ public GenericInMemoryCatalogStore() {
+ descriptors = new HashMap<>();
+ }
+
+ @Override
+ public void storeCatalog(String catalogName, CatalogDescriptor catalog)
+ throws CatalogException {
+ checkOpenState();
+ if (descriptors.containsKey(catalogName)) {
+ throw new CatalogException(
+ format("Catalog %s already exists in the catalog store.",
catalogName));
+ }
+ descriptors.put(catalogName, catalog);
+ }
+
+ @Override
+ public void removeCatalog(String catalogName, boolean ignoreIfNotExists)
+ throws CatalogException {
+ checkOpenState();
+ if (descriptors.containsKey(catalogName)) {
+ descriptors.remove(catalogName);
+ } else if (!ignoreIfNotExists) {
+ throw new CatalogException(
+ format("Catalog %s does not exist in the catalog store.",
catalogName));
+ }
+ }
+
+ @Override
+ public Optional<CatalogDescriptor> getCatalog(String catalogName) {
+ checkOpenState();
+ return Optional.ofNullable(descriptors.get(catalogName));
+ }
+
+ @Override
+ public Set<String> listCatalogs() {
+ checkOpenState();
+ return descriptors.keySet();
+ }
+
+ @Override
+ public boolean contains(String catalogName) {
+ checkOpenState();
+ return descriptors.containsKey(catalogName);
+ }
+}
diff --git
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactory.java
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactory.java
new file mode 100644
index 00000000000..37166d6c4c0
--- /dev/null
+++
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactory.java
@@ -0,0 +1,54 @@
+/*
+ * 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.configuration.ConfigOption;
+import org.apache.flink.table.factories.CatalogStoreFactory;
+
+import java.util.Collections;
+import java.util.Set;
+
+/** Catalog store factory for {@link GenericInMemoryCatalogStore}. */
+public class GenericInMemoryCatalogStoreFactory implements CatalogStoreFactory
{
+ @Override
+ public CatalogStore createCatalogStore(Context context) {
+ return new GenericInMemoryCatalogStore();
+ }
+
+ @Override
+ public void open(Context context) {}
+
+ @Override
+ public void close() {}
+
+ @Override
+ public String factoryIdentifier() {
+ return GenericInMemoryCatalogStoreFactoryOptions.IDENTIFIER;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ return Collections.emptySet();
+ }
+}
diff --git
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryOptions.java
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryOptions.java
new file mode 100644
index 00000000000..431df872eed
--- /dev/null
+++
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryOptions.java
@@ -0,0 +1,28 @@
+/*
+ * 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.configuration.ConfigOption;
+
+/** {@link ConfigOption}s for {@link GenericInMemoryCatalogStore}. */
+public class GenericInMemoryCatalogStoreFactoryOptions {
+ public static final String IDENTIFIER = "generic_in_memory";
+
+ private GenericInMemoryCatalogStoreFactoryOptions() {}
+}
diff --git
a/flink-table/flink-table-api-java/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
b/flink-table/flink-table-api-java/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 81b51d431f1..62028acbd66 100644
---
a/flink-table/flink-table-api-java/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++
b/flink-table/flink-table-api-java/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -14,3 +14,4 @@
# limitations under the License.
org.apache.flink.table.catalog.GenericInMemoryCatalogFactory
+org.apache.flink.table.catalog.GenericInMemoryCatalogStoreFactory
diff --git
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryTest.java
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryTest.java
new file mode 100644
index 00000000000..6bf82d7d5a4
--- /dev/null
+++
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreFactoryTest.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;
+
+import org.apache.flink.table.factories.CatalogStoreFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.FactoryUtil.DefaultCatalogStoreContext;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+/** Test for {@link GenericInMemoryCatalogStoreFactory}. */
+public class GenericInMemoryCatalogStoreFactoryTest {
+
+ @Test
+ void testCatalogStoreInit() {
+ String factoryIdentifier =
GenericInMemoryCatalogStoreFactoryOptions.IDENTIFIER;
+ Map<String, String> options = new HashMap<>();
+ ClassLoader classLoader =
Thread.currentThread().getContextClassLoader();
+ final DefaultCatalogStoreContext discoveryContext =
+ new DefaultCatalogStoreContext(options, null, classLoader);
+ final CatalogStoreFactory factory =
+ FactoryUtil.discoverFactory(
+ classLoader, CatalogStoreFactory.class,
factoryIdentifier);
+ factory.open(discoveryContext);
+
+ CatalogStore catalogStore =
factory.createCatalogStore(discoveryContext);
+ assertThat(catalogStore instanceof
GenericInMemoryCatalogStore).isTrue();
+
+ factory.close();
+ }
+}
diff --git
a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreTest.java
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreTest.java
new file mode 100644
index 00000000000..c8437c7b247
--- /dev/null
+++
b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/GenericInMemoryCatalogStoreTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+
+/** Test for {@link GenericInMemoryCatalogStore}. */
+public class GenericInMemoryCatalogStoreTest {
+
+ @Test
+ void testStoreAndGet() {
+ CatalogStore catalogStore = new GenericInMemoryCatalogStore();
+ catalogStore.open();
+
+ catalogStore.storeCatalog(
+ "catalog1", CatalogDescriptor.of("catalog1", new
Configuration()));
+ assertThat(catalogStore.getCatalog("catalog1").isPresent()).isTrue();
+ assertThat(catalogStore.contains("catalog1")).isTrue();
+
+ catalogStore.removeCatalog("catalog1", true);
+ assertThat(catalogStore.contains("catalog1")).isFalse();
+
+ assertThatThrownBy(() -> catalogStore.removeCatalog("catalog1", false))
+ .isInstanceOf(CatalogException.class)
+ .hasMessageContaining("Catalog catalog1 does not exist in the
catalog store.");
+
+ catalogStore.close();
+
+ assertThatThrownBy(
+ () ->
+ catalogStore.storeCatalog(
+ "catalog1",
+ CatalogDescriptor.of("catalog1", new
Configuration())))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("CatalogStore is not opened yet.");
+
+ assertThatThrownBy(() -> catalogStore.removeCatalog("catalog1", false))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("CatalogStore is not opened yet.");
+
+ assertThatThrownBy(() -> catalogStore.contains("catalog1"))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("CatalogStore is not opened yet.");
+
+ assertThatThrownBy(() -> catalogStore.listCatalogs())
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessageContaining("CatalogStore is not opened yet.");
+ }
+}
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/AbstractCatalogStore.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/AbstractCatalogStore.java
new file mode 100644
index 00000000000..ab2a2884f8a
--- /dev/null
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/AbstractCatalogStore.java
@@ -0,0 +1,49 @@
+/*
+ * 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.util.Preconditions;
+
+/** The AbstractCatalogStore class is an abstract base class for implementing
a catalog store. */
+public abstract class AbstractCatalogStore implements CatalogStore {
+
+ /** Catalog store state. */
+ protected boolean isOpen;
+
+ /** Opens the catalog store. */
+ @Override
+ public void open() {
+ isOpen = true;
+ }
+
+ /** Closes the catalog store. */
+ @Override
+ public void close() {
+ isOpen = false;
+ }
+
+ /**
+ * Checks whether the catalog store is currently open.
+ *
+ * @throws IllegalStateException if the store is closed
+ */
+ protected void checkOpenState() {
+ Preconditions.checkState(isOpen, "CatalogStore is not opened yet.");
+ }
+}
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogDescriptor.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogDescriptor.java
new file mode 100644
index 00000000000..4daf5d1410e
--- /dev/null
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogDescriptor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * Describes a {@link Catalog} with the catalog name and configuration.
+ *
+ * <p>A {@link CatalogDescriptor} is a template for creating a {@link Catalog}
instance. It closely
+ * resembles the "CREATE CATALOG" SQL DDL statement, containing catalog name
and catalog
+ * configuration. A {@link CatalogDescriptor} could be stored to {@link
CatalogStore}.
+ *
+ * <p>This can be used to register a catalog in the Table API, see {@link
+ * TableEnvironment#createCatalog(String, CatalogDescriptor)}.
+ */
+@PublicEvolving
+public class CatalogDescriptor {
+
+ /* Catalog name */
+ private final String catalogName;
+
+ /* The configuration used to discover and construct the catalog. */
+ private final Configuration configuration;
+
+ public String getCatalogName() {
+ return catalogName;
+ }
+
+ public Configuration getConfiguration() {
+ return configuration;
+ }
+
+ private CatalogDescriptor(String catalogName, Configuration configuration)
{
+ this.catalogName = catalogName;
+ this.configuration = configuration;
+ }
+
+ /**
+ * Creates an instance of this interface.
+ *
+ * @param catalogName the name of the catalog
+ * @param configuration the configuration of the catalog
+ */
+ public static CatalogDescriptor of(String catalogName, Configuration
configuration) {
+ return new CatalogDescriptor(catalogName, configuration);
+ }
+}
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogStore.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogStore.java
new file mode 100644
index 00000000000..af35927f01f
--- /dev/null
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogStore.java
@@ -0,0 +1,95 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Represents the storage where persists all {@link Catalog}s.
+ *
+ * <p>All catalogs can be lazy initialized with the {@link CatalogStore}.
+ *
+ * <p>It can be used in {@code CatalogManager} to retrieve, save and remove
catalog in {@link
+ * CatalogDescriptor} format at the external storage system.
+ */
+@PublicEvolving
+public interface CatalogStore {
+
+ /**
+ * Stores a catalog under the given catalog name. The catalog name must be
unique.
+ *
+ * @param catalogName the given catalog name under which to store the
given catalog
+ * @param catalog catalog descriptor to store
+ * @throws CatalogException throw when registration failed
+ */
+ void storeCatalog(String catalogName, CatalogDescriptor catalog) throws
CatalogException;
+
+ /**
+ * Remove a catalog with the given catalog name.
+ *
+ * @param catalogName the given catalog name under which to remove the
given catalog
+ * @param ignoreIfNotExists whether throw an exception when the catalog
does not exist
+ * @throws CatalogException throw when the removal operation failed
+ */
+ void removeCatalog(String catalogName, boolean ignoreIfNotExists) throws
CatalogException;
+
+ /**
+ * Get a catalog by name.
+ *
+ * @param catalogName name of the catalog to retrieve
+ * @return the requested catalog or empty if the catalog does not exist
+ * @throws CatalogException in case of any runtime exception
+ */
+ Optional<CatalogDescriptor> getCatalog(String catalogName) throws
CatalogException;
+
+ /**
+ * Retrieves the names of all registered catalogs.
+ *
+ * @return the names of registered catalogs
+ * @throws CatalogException in case of any runtime exception
+ */
+ Set<String> listCatalogs() throws CatalogException;
+
+ /**
+ * Return whether the catalog exists in the catalog store.
+ *
+ * @param catalogName the name of catalog
+ * @throws CatalogException in case of any runtime exception
+ */
+ boolean contains(String catalogName) throws CatalogException;
+
+ /**
+ * Open the catalog store. Used for any required preparation in
initialization phase.
+ *
+ * @throws CatalogException in case of any runtime exception
+ */
+ void open() throws CatalogException;
+
+ /**
+ * Close the catalog store when it is no longer needed and release any
resource that it might be
+ * holding.
+ *
+ * @throws CatalogException in case of any runtime exception
+ */
+ void close() throws CatalogException;
+}
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/CatalogStoreFactory.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/CatalogStoreFactory.java
new file mode 100644
index 00000000000..ad70ffb557f
--- /dev/null
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/CatalogStoreFactory.java
@@ -0,0 +1,152 @@
+/*
+ * 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.factories;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.catalog.CatalogStore;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import java.util.Map;
+
+/**
+ * A factory to create configured catalog store instances based on
string-based properties. See also
+ * {@link Factory} for more information.
+ *
+ * <p>This factory is specifically designed for the Flink SQL gateway
scenario, where different
+ * catalog stores need to be created for different sessions.
+ *
+ * <p>If the CatalogStore is implemented using JDBC, this factory can be used
to create a JDBC
+ * connection pool in the open method. This connection pool can then be reused
for subsequent
+ * catalog store creations.
+ *
+ * <p>The following examples implementation of CatalogStoreFactory using jdbc.
+ *
+ * <pre>{@code
+ * public class JdbcCatalogStore implements CatalogStore {
+ *
+ * private JdbcConnectionPool jdbcConnectionPool;
+ * public JdbcCatalogStore(JdbcConnectionPool jdbcConnectionPool) {
+ * this.jdbcConnectionPool = jdbcConnectionPool;
+ * }
+ * ...
+ * }
+ *
+ * public class JdbcCatalogStoreFactory implements CatalogStoreFactory {
+ *
+ * private JdbcConnectionPool jdbcConnectionPool;
+ *
+ * @Override
+ * public CatalogStore createCatalogStore(Context context) {
+ * return new JdbcCatalogStore(jdbcConnectionPool);
+ * }
+ *
+ * @Override
+ * public void open(Context context) throws CatalogException {
+ * // initialize the thread pool using options from context
+ * jdbcConnectionPool = initializeJdbcConnectionPool(context);
+ * }
+ *
+ * @Override
+ * public void close() {
+ * // release the connection thread pool.
+ * releaseConnectionPool(jdbcConnectionPool);
+ * }
+ * ...
+ * }
+ * }</pre>
+ *
+ * <p>The usage of the Flink SQL gateway is as follows. It's just an example
and may not be the
+ * final implementation.
+ *
+ * <pre>{@code
+ * // initialize CatalogStoreFactory when initialize the SessionManager
+ * public class SessionManagerImpl implements SessionManager {
+ * public SessionManagerImpl(DefaultContext defaultContext) {
+ * this.catalogStoreFactory = createCatalogStore();
+ * }
+ *
+ * @Override
+ * public void start() {
+ * // initialize the CatalogStoreFactory
+ * this.catalogStoreFactory(buildCatalogStoreContext());
+ * }
+ *
+ * @Override
+ * public synchronized Session openSession(SessionEnvironment
environment) {
+ * // Create a new catalog store for this session.
+ * CatalogStore catalogStore =
this.catalogStoreFactory.createCatalogStore(buildCatalogStoreContext());
+ *
+ * // Create a new CatalogManager using catalog store.
+ * }
+ *
+ * @Override
+ * public void stop() {
+ * // Close the CatalogStoreFactory when stopping the SessionManager.
+ * this.catalogStoreFactory.close();
+ * }
+ * }
+ * }</pre>
+ */
+@PublicEvolving
+public interface CatalogStoreFactory extends Factory {
+
+ /** Creates a {@link CatalogStore} instance from context information. */
+ CatalogStore createCatalogStore(Context context);
+
+ /**
+ * Initialize the CatalogStoreFactory.
+ *
+ * <p>For the use case of Flink SQL gateway, the open method will be
called when starting
+ * SessionManager. It initializes common resources, such as a connection
pool, for various
+ * catalog stores.
+ */
+ void open(Context context) throws CatalogException;
+
+ /**
+ * Close the CatalogStoreFactory.
+ *
+ * <p>For the use case of Flink SQL gateway, the close method will be
called when closing
+ * SessionManager. It releases common resources, such as connection pool,
after closing all
+ * catalog stores.
+ */
+ void close() throws CatalogException;
+
+ /** Context provided when a catalog store is created. */
+ @PublicEvolving
+ interface Context {
+
+ /**
+ * Returns the options with which the catalog store is created.
+ *
+ * <p>An implementation should perform validation of these options.
+ */
+ Map<String, String> getOptions();
+
+ /** Gives read-only access to the configuration of the current
session. */
+ ReadableConfig getConfiguration();
+
+ /**
+ * Returns the class loader of the current session.
+ *
+ * <p>The class loader is in particular useful for discovering further
(nested) factories.
+ */
+ ClassLoader getClassLoader();
+ }
+}
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
index be4a32744be..3d96c14619d 100644
---
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FactoryUtil.java
@@ -1364,6 +1364,41 @@ public final class FactoryUtil {
}
}
+ /** Default implementation of {@link CatalogStoreFactory.Context}. */
+ @Internal
+ public static class DefaultCatalogStoreContext implements
CatalogStoreFactory.Context {
+
+ private Map<String, String> options;
+
+ private ReadableConfig configuration;
+
+ private ClassLoader classLoader;
+
+ public DefaultCatalogStoreContext(
+ Map<String, String> options,
+ ReadableConfig configuration,
+ ClassLoader classLoader) {
+ this.options = options;
+ this.configuration = configuration;
+ this.classLoader = classLoader;
+ }
+
+ @Override
+ public Map<String, String> getOptions() {
+ return options;
+ }
+
+ @Override
+ public ReadableConfig getConfiguration() {
+ return configuration;
+ }
+
+ @Override
+ public ClassLoader getClassLoader() {
+ return classLoader;
+ }
+ }
+
/** Default implementation of {@link ModuleFactory.Context}. */
@Internal
public static class DefaultModuleContext implements ModuleFactory.Context {
diff --git
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/FactoryUtilTest.java
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/FactoryUtilTest.java
index 216cdfb5e26..5dcd3a90eea 100644
---
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/FactoryUtilTest.java
+++
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/FactoryUtilTest.java
@@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.testutils.FlinkAssertions;
import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogStore;
import org.apache.flink.table.catalog.CommonCatalogOptions;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
@@ -414,6 +415,21 @@ class FactoryUtilTest {
"Unsupported options found for
'test-catalog'"));
}
+ @Test
+ void testCreateCatalogStore() {
+ final Map<String, String> options = new HashMap<>();
+ ClassLoader classLoader =
Thread.currentThread().getContextClassLoader();
+ final FactoryUtil.DefaultCatalogStoreContext discoveryContext =
+ new FactoryUtil.DefaultCatalogStoreContext(options, null,
classLoader);
+ final CatalogStoreFactory factory =
+ FactoryUtil.discoverFactory(
+ classLoader, CatalogStoreFactory.class,
TestCatalogStoreFactory.IDENTIFIER);
+ factory.open(discoveryContext);
+ CatalogStore catalogStore =
factory.createCatalogStore(discoveryContext);
+
+
assertThat(catalogStore).isInstanceOf(TestCatalogStoreFactory.TestCatalogStore.class);
+ }
+
@Test
void testFactoryHelperWithDeprecatedOptions() {
final Map<String, String> options = new HashMap<>();
diff --git
a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestCatalogStoreFactory.java
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestCatalogStoreFactory.java
new file mode 100644
index 00000000000..955e960f2b3
--- /dev/null
+++
b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/TestCatalogStoreFactory.java
@@ -0,0 +1,107 @@
+/*
+ * 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.factories;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.catalog.CatalogDescriptor;
+import org.apache.flink.table.catalog.CatalogStore;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/** Test catalog store factory for catalog store discovery tests. */
+public class TestCatalogStoreFactory implements CatalogStoreFactory {
+
+ public static final String IDENTIFIER = "test-catalog-store";
+
+ @Override
+ public CatalogStore createCatalogStore(Context context) {
+ return new TestCatalogStore(context.getOptions());
+ }
+
+ @Override
+ public void open(Context context) throws CatalogException {}
+
+ @Override
+ public void close() throws CatalogException {}
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Collections.emptySet();
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ return Collections.emptySet();
+ }
+
+ /** Test catalog store for discovery testing. */
+ public static class TestCatalogStore implements CatalogStore {
+ private final Map<String, String> options;
+
+ public TestCatalogStore(Map<String, String> options) {
+ this.options = options;
+ }
+
+ @Override
+ public void storeCatalog(String catalogName, CatalogDescriptor catalog)
+ throws CatalogException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void removeCatalog(String catalogName, boolean
ignoreIfNotExists)
+ throws CatalogException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Optional<CatalogDescriptor> getCatalog(String catalogName) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Set<String> listCatalogs() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean contains(String catalogName) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void open() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void close() {
+ throw new UnsupportedOperationException();
+ }
+ }
+}
diff --git
a/flink-table/flink-table-common/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
b/flink-table/flink-table-common/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
index c9696613aba..22b140e3e7a 100644
---
a/flink-table/flink-table-common/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++
b/flink-table/flink-table-common/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -22,5 +22,6 @@
org.apache.flink.table.factories.TestDynamicTableSourceOnlyFactory
org.apache.flink.table.factories.TestConflictingDynamicTableFactory1
org.apache.flink.table.factories.TestConflictingDynamicTableFactory2
org.apache.flink.table.factories.TestCatalogFactory
+org.apache.flink.table.factories.TestCatalogStoreFactory
org.apache.flink.table.factories.TestManagedTableFactory
org.apache.flink.table.factories.module.DummyModuleFactory