FANNG1 commented on code in PR #10560: URL: https://github.com/apache/gravitino/pull/10560#discussion_r3010182825
########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoSessionCatalogStore.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.gravitino.flink.connector.store; + +import static org.apache.gravitino.flink.connector.utils.FactoryUtils.isBuiltInCatalog; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import org.apache.flink.table.catalog.AbstractCatalogStore; +import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.catalog.GenericInMemoryCatalogStore; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.util.Preconditions; + +/** + * A catalog store that combines a session-scoped in-memory {@link GenericInMemoryCatalogStore} with + * a persistent {@link GravitinoCatalogStore}. + * + * <p>Catalogs for built-in catalog types are stored only in the in-memory store, while all other + * catalogs are stored in the Gravitino-backed store. When retrieving, listing, or removing + * catalogs, entries in the in-memory store take precedence over entries in the Gravitino-backed + * store. + * + * <p>This store is intended to be used per Flink session, keeping transient catalogs in memory + * while delegating persistent catalogs to Apache Gravitino. + */ +public class GravitinoSessionCatalogStore extends AbstractCatalogStore { + private final GenericInMemoryCatalogStore memoryCatalogStore; + private final GravitinoCatalogStore gravitinoCatalogStore; + + public GravitinoSessionCatalogStore( + GravitinoCatalogStore gravitinoCatalogStore, GenericInMemoryCatalogStore memoryCatalogStore) { + this.gravitinoCatalogStore = + Preconditions.checkNotNull(gravitinoCatalogStore, "CatalogStore cannot be null"); + this.memoryCatalogStore = + Preconditions.checkNotNull(memoryCatalogStore, "MemoryCatalogStore cannot be null"); + } + + @Override + public void storeCatalog(String catalogName, CatalogDescriptor descriptor) + throws CatalogException { + String catalogType = descriptor.getConfiguration().get(CommonCatalogOptions.CATALOG_TYPE); + if (isBuiltInCatalog(catalogType)) { + gravitinoCatalogStore.storeCatalog(catalogName, descriptor); + } else { + memoryCatalogStore.storeCatalog(catalogName, descriptor); + } + } + /** + * Removes the specified catalog. + * + * @param catalogName name of the catalog to remove + * @param ignoreIfNotExists if true, ignore when the catalog does not exist + * @throws CatalogException if the catalog cannot be removed + */ + @Override + public void removeCatalog(String catalogName, boolean ignoreIfNotExists) throws CatalogException { + if (memoryCatalogStore.contains(catalogName)) { + memoryCatalogStore.removeCatalog(catalogName, ignoreIfNotExists); + } else { + gravitinoCatalogStore.removeCatalog(catalogName, ignoreIfNotExists); + } + } + + /** + * 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 throw a CatalogException when the Catalog cannot be created. + */ + @Override + public Optional<CatalogDescriptor> getCatalog(String catalogName) throws CatalogException { + if (memoryCatalogStore.contains(catalogName)) { + Optional<CatalogDescriptor> descriptor = memoryCatalogStore.getCatalog(catalogName); + if (descriptor.isPresent()) { Review Comment: The two `catch` blocks in `listCatalogs()` have identical bodies and can be merged: ```java } catch (CatalogException | RuntimeException e) { throw new CatalogException("Failed to list catalog.", e); } ``` ########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStoreFactory.java: ########## @@ -43,10 +45,20 @@ Review Comment: `createCatalogStore()` creates a new `GravitinoCatalogStore(catalogManager)` on every invocation, while `memoryCatalogStore` is lazily initialised and reused. This means every returned `GravitinoSessionCatalogStore` wraps a *different* `gravitinoCatalogStore` instance while sharing the same `memoryCatalogStore`, which leads to inconsistent lifecycles. Consider promoting `GravitinoCatalogStore` to a factory-level field and initialising it once in `open()`, mirroring the treatment of `memoryCatalogStore`. ########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoSessionCatalogStore.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.gravitino.flink.connector.store; + +import static org.apache.gravitino.flink.connector.utils.FactoryUtils.isBuiltInCatalog; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import org.apache.flink.table.catalog.AbstractCatalogStore; +import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.catalog.GenericInMemoryCatalogStore; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.util.Preconditions; + +/** + * A catalog store that combines a session-scoped in-memory {@link GenericInMemoryCatalogStore} with + * a persistent {@link GravitinoCatalogStore}. + * + * <p>Catalogs for built-in catalog types are stored only in the in-memory store, while all other + * catalogs are stored in the Gravitino-backed store. When retrieving, listing, or removing + * catalogs, entries in the in-memory store take precedence over entries in the Gravitino-backed + * store. + * + * <p>This store is intended to be used per Flink session, keeping transient catalogs in memory + * while delegating persistent catalogs to Apache Gravitino. + */ +public class GravitinoSessionCatalogStore extends AbstractCatalogStore { + private final GenericInMemoryCatalogStore memoryCatalogStore; + private final GravitinoCatalogStore gravitinoCatalogStore; + + public GravitinoSessionCatalogStore( + GravitinoCatalogStore gravitinoCatalogStore, GenericInMemoryCatalogStore memoryCatalogStore) { + this.gravitinoCatalogStore = + Preconditions.checkNotNull(gravitinoCatalogStore, "CatalogStore cannot be null"); + this.memoryCatalogStore = + Preconditions.checkNotNull(memoryCatalogStore, "MemoryCatalogStore cannot be null"); + } + + @Override + public void storeCatalog(String catalogName, CatalogDescriptor descriptor) + throws CatalogException { + String catalogType = descriptor.getConfiguration().get(CommonCatalogOptions.CATALOG_TYPE); + if (isBuiltInCatalog(catalogType)) { + gravitinoCatalogStore.storeCatalog(catalogName, descriptor); + } else { + memoryCatalogStore.storeCatalog(catalogName, descriptor); + } + } + /** + * Removes the specified catalog. + * + * @param catalogName name of the catalog to remove + * @param ignoreIfNotExists if true, ignore when the catalog does not exist + * @throws CatalogException if the catalog cannot be removed + */ + @Override + public void removeCatalog(String catalogName, boolean ignoreIfNotExists) throws CatalogException { + if (memoryCatalogStore.contains(catalogName)) { + memoryCatalogStore.removeCatalog(catalogName, ignoreIfNotExists); + } else { + gravitinoCatalogStore.removeCatalog(catalogName, ignoreIfNotExists); + } + } + Review Comment: `getCatalog` calls `contains()` and then `getCatalog()`, which is a TOCTOU (check-then-act) race: in a concurrent scenario the catalog could be removed between the two calls. The logic can be simplified and made safer: ```java Optional<CatalogDescriptor> descriptor = memoryCatalogStore.getCatalog(catalogName); return descriptor.isPresent() ? descriptor : gravitinoCatalogStore.getCatalog(catalogName); ``` ########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoSessionCatalogStore.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.gravitino.flink.connector.store; + +import static org.apache.gravitino.flink.connector.utils.FactoryUtils.isBuiltInCatalog; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import org.apache.flink.table.catalog.AbstractCatalogStore; +import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.catalog.GenericInMemoryCatalogStore; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.util.Preconditions; + +/** + * A catalog store that combines a session-scoped in-memory {@link GenericInMemoryCatalogStore} with + * a persistent {@link GravitinoCatalogStore}. + * + * <p>Catalogs for built-in catalog types are stored only in the in-memory store, while all other + * catalogs are stored in the Gravitino-backed store. When retrieving, listing, or removing + * catalogs, entries in the in-memory store take precedence over entries in the Gravitino-backed + * store. + * + * <p>This store is intended to be used per Flink session, keeping transient catalogs in memory + * while delegating persistent catalogs to Apache Gravitino. + */ +public class GravitinoSessionCatalogStore extends AbstractCatalogStore { + private final GenericInMemoryCatalogStore memoryCatalogStore; + private final GravitinoCatalogStore gravitinoCatalogStore; + + public GravitinoSessionCatalogStore( + GravitinoCatalogStore gravitinoCatalogStore, GenericInMemoryCatalogStore memoryCatalogStore) { + this.gravitinoCatalogStore = + Preconditions.checkNotNull(gravitinoCatalogStore, "CatalogStore cannot be null"); + this.memoryCatalogStore = + Preconditions.checkNotNull(memoryCatalogStore, "MemoryCatalogStore cannot be null"); + } + + @Override + public void storeCatalog(String catalogName, CatalogDescriptor descriptor) + throws CatalogException { + String catalogType = descriptor.getConfiguration().get(CommonCatalogOptions.CATALOG_TYPE); + if (isBuiltInCatalog(catalogType)) { + gravitinoCatalogStore.storeCatalog(catalogName, descriptor); Review Comment: `descriptor.getConfiguration().get(CommonCatalogOptions.CATALOG_TYPE)` can return `null` when the catalog type property is absent. In that case `isBuiltInCatalog(null)` returns `false`, silently routing the catalog to the in-memory store instead of failing with a clear error. Please add an explicit null-check and throw a `CatalogException` with a descriptive message. ########## flink-connector/flink/src/test/java/org/apache/gravitino/flink/connector/store/TestGravitinoSessionCatalogStore.java: ########## @@ -0,0 +1,225 @@ +/* + * 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.gravitino.flink.connector.store; + +import static org.apache.gravitino.flink.connector.utils.FactoryUtils.isBuiltInCatalog; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableSet; +import java.util.Optional; +import java.util.Set; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.CommonCatalogOptions; +import org.apache.flink.table.catalog.GenericInMemoryCatalogStore; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.junit.Before; +import org.junit.Test; + Review Comment: This test class uses JUnit 4 (`org.junit.Test`, `@Before`), while the rest of the module (e.g. `TestGravitinoCatalogStoreFactory`) uses JUnit 5 (`org.junit.jupiter.api.*`). The build is configured with `useJUnitPlatform()`, so **without the JUnit Vintage engine these tests may never actually execute**, making the claimed coverage illusory. Please migrate to JUnit 5 for consistency. ########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/store/GravitinoCatalogStoreFactoryOptions.java: ########## @@ -40,7 +40,12 @@ private GravitinoCatalogStoreFactoryOptions() {} .stringType() .noDefaultValue() .withDescription("The name of Gravitino metalake"); - + public static final ConfigOption<Boolean> GRAVITINO_SUPPORT_SESSION_CATALOG = + ConfigOptions.key("gravitino.support.session.catalog") + .booleanType() + .defaultValue(false) + .withDescription( Review Comment: Missing blank line after the `withDescription(...)` closing `);`. All other constant definitions in this file are separated by a blank line — please keep the style consistent. ########## flink-connector/flink/src/main/java/org/apache/gravitino/flink/connector/utils/FactoryUtils.java: ########## @@ -36,6 +41,19 @@ private FactoryUtils() {} private static final Logger LOG = LoggerFactory.getLogger(FactoryUtils.class); + /** The set of catalog type identifiers managed by Gravitino. */ + private static final ImmutableSet<String> GRAVITINO_CATALOG_TYPES = + ImmutableSet.of( + GravitinoHiveCatalogFactoryOptions.IDENTIFIER, + GravitinoIcebergCatalogFactoryOptions.IDENTIFIER, + GravitinoJdbcCatalogFactoryOptions.MYSQL_IDENTIFIER, Review Comment: `isBuiltInCatalog` is misleading — it actually checks whether a type belongs to the set of *Gravitino-managed* catalog identifiers, not Flink built-in types. Consider renaming it to `isGravitinoManagedCatalogType` (or `isGravitinoCatalogType`) to avoid confusion for future readers. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
