twalthr commented on a change in pull request #18088:
URL: https://github.com/apache/flink/pull/18088#discussion_r768775141



##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
##########
@@ -686,7 +701,14 @@ public void createTemporaryTable(
                         }
                         return v;
                     } else {
-                        final CatalogBaseTable resolvedTable = 
resolveCatalogBaseTable(table);
+                        ResolvedCatalogBaseTable<?> resolvedTable =
+                                managedTableListener.notifyTableCreation(
+                                        
getCatalog(objectIdentifier.getCatalogName()).orElse(null),
+                                        objectIdentifier,
+                                        resolveCatalogBaseTable(table),
+                                        true,

Review comment:
       aren't the properties "temporary" and "catalog = null" related? I don't 
think we need the temporary flag. we can still add it on the internal API if 
necessary.

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/ManagedTableListener.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.Internal;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.descriptors.ConnectorDescriptorValidator;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.StringUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static 
org.apache.flink.table.catalog.CatalogBaseTable.TableKind.MANAGED;
+import static 
org.apache.flink.table.factories.ManagedTableFactory.discoverManagedTableFactory;
+
+/** The listener for managed table operations. */
+@Internal
+public class ManagedTableListener {
+
+    private final ClassLoader classLoader;
+
+    private final ReadableConfig config;
+
+    public ManagedTableListener(ClassLoader classLoader, ReadableConfig 
config) {
+        this.classLoader = classLoader;
+        this.config = config;
+    }
+
+    /** Notify for creating managed table. */
+    public ResolvedCatalogBaseTable<?> notifyTableCreation(
+            @Nullable Catalog catalog,
+            ObjectIdentifier identifier,
+            ResolvedCatalogBaseTable<?> table,
+            boolean isTemporary,
+            boolean ignoreIfExists) {
+        if (isNewlyManagedTable(catalog, table)) {
+            ResolvedCatalogTable managedTable = createManagedTable(identifier, 
table, isTemporary);
+            discoverManagedTableFactory(classLoader)
+                    .onCreateTable(
+                            createTableFactoryContext(identifier, 
managedTable, isTemporary),
+                            ignoreIfExists);
+            return managedTable;
+        }
+        return table;
+    }
+
+    /** Notify for dropping managed table. */
+    public void notifyTableDrop(
+            ObjectIdentifier identifier,
+            ResolvedCatalogBaseTable<?> table,
+            boolean isTemporary,
+            boolean ignoreIfNotExists) {
+        if (table.getTableKind() == MANAGED) {
+            discoverManagedTableFactory(classLoader)
+                    .onDropTable(
+                            createTableFactoryContext(
+                                    identifier, (ResolvedCatalogTable) table, 
isTemporary),
+                            ignoreIfNotExists);
+        }
+    }
+
+    private boolean isNewlyManagedTable(
+            @Nullable Catalog catalog, ResolvedCatalogBaseTable<?> table) {
+        if (catalog == null || !catalog.supportsManagedTable()) {
+            // catalog not support managed table
+            return false;
+        }
+
+        if (table.getTableKind() == CatalogBaseTable.TableKind.VIEW) {
+            // view is not managed table
+            return false;
+        }
+
+        if (!StringUtils.isNullOrWhitespaceOnly(

Review comment:
       we call `table.getOptions()` very often here. how about calling it once 
in a try-catch and than just check for the remaining cases?

##########
File path: 
flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TestManagedTableFactory.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.api;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.catalog.ObjectIdentifier;
+import org.apache.flink.table.factories.ManagedTableFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** A test {@link ManagedTableFactory}. */
+public class TestManagedTableFactory implements ManagedTableFactory {
+
+    public static final String ENRICHED_KEY = "ENRICHED_KEY";
+
+    public static final String ENRICHED_VALUE = "ENRICHED_VALUE";
+
+    public static final Map<ObjectIdentifier, AtomicReference<Map<String, 
String>>> MANAGED_TABLES =
+            new ConcurrentHashMap<>();
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        return new HashSet<>();
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        return new HashSet<>();
+    }
+
+    @Override
+    public Map<String, String> enrichOptions(Context context) {
+        Map<String, String> newOptions = new 
HashMap<>(context.getCatalogTable().getOptions());
+        if (MANAGED_TABLES.containsKey(context.getObjectIdentifier())) {
+            newOptions.put(ENRICHED_KEY, ENRICHED_VALUE);
+        }
+        return newOptions;
+    }
+
+    @Override
+    public void onCreateTable(Context context, boolean ignoreIfExists) {
+        MANAGED_TABLES.compute(
+                context.getObjectIdentifier(),
+                (k, v) -> {
+                    if (v != null) {
+                        if (v.get() == null) {
+                            v.set(context.getCatalogTable().toProperties());

Review comment:
       isn't calling `getOptions()` enough, do we really need to serialize the 
schema as well?

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogPropertiesUtil.java
##########
@@ -79,6 +80,10 @@
                 properties.put(COMMENT, comment);
             }
 
+            if (resolvedTable.getTableKind() == TableKind.MANAGED) {

Review comment:
       why do we need this? can't we derive the "managed" property from the 
options? looking for a missing `connector` should already be enough? I think we 
can assume that `connector.type` is not used by now.

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
##########
@@ -656,8 +663,16 @@ public ObjectIdentifier 
qualifyIdentifier(UnresolvedIdentifier identifier) {
     public void createTable(
             CatalogBaseTable table, ObjectIdentifier objectIdentifier, boolean 
ignoreIfExists) {
         execute(
-                (catalog, path) ->
-                        catalog.createTable(path, 
resolveCatalogBaseTable(table), ignoreIfExists),
+                (catalog, path) -> {
+                    ResolvedCatalogBaseTable<?> resolvedTable =
+                            managedTableListener.notifyTableCreation(
+                                    catalog,
+                                    objectIdentifier,
+                                    resolveCatalogBaseTable(table),

Review comment:
       nit: introduce a local variable here and below, it avoids too much 
nesting and makes it easier to set breakpoints during debugging

##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DefaultManagedTable.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.Internal;
+import org.apache.flink.table.api.Schema;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.Map;
+
+/** Default implementation of a managed {@link CatalogTable}. */
+@Internal
+public class DefaultManagedTable extends DefaultCatalogTable {

Review comment:
       let's drop this class and enrich `DefaultCatalogTable` with an 
additional constructor argument. it is internal anyway and implementers should 
not do `instanceof` checks against this class. 

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java
##########
@@ -86,6 +86,7 @@ private FlinkStatistic getStatistic(
         final ResolvedCatalogBaseTable<?> resolvedBaseTable = 
lookupResult.getResolvedTable();
         switch (resolvedBaseTable.getTableKind()) {
             case TABLE:
+            case MANAGED:

Review comment:
       I thought about my previous comment around a special `TableKind` again. 
I fear we are mixing metadata encoded in table kind and metadata encoded in 
options.




-- 
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-unsubscr...@flink.apache.org

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


Reply via email to