dawidwys commented on a change in pull request #12826:
URL: https://github.com/apache/flink/pull/12826#discussion_r452106967



##########
File path: 
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogITCase.java
##########
@@ -61,9 +70,54 @@ public void testDropCatalog() {
                assertFalse(tableEnv.getCatalog(name).isPresent());
        }
 
+       @Test
+       public void testCreateCatalogFromUserClassLoader() throws Exception {
+               final String className = "UserCatalogFactory";
+               URLClassLoader classLoader = 
ClassLoaderUtils.withRoot(temporaryFolder.newFolder())
+                       
.addResource("META-INF/services/org.apache.flink.table.factories.TableFactory", 
"UserCatalogFactory")
+                       .addClass(
+                               className,
+                               "import 
org.apache.flink.table.catalog.GenericInMemoryCatalog;\n" +
+                                       "import 
org.apache.flink.table.factories.CatalogFactory;\n" +
+                                       "import java.util.Collections;\n" +
+                                       "import 
org.apache.flink.table.catalog.Catalog;\n" +
+                                       "import java.util.HashMap;\n" +
+                                       "import java.util.List;\n" +
+                                       "import java.util.Map;\n" +
+                                       "\tpublic class UserCatalogFactory 
implements CatalogFactory {\n" +
+                                       "\t\t@Override\n" +
+                                       "\t\tpublic Catalog createCatalog(\n" +
+                                       "\t\t\t\tString name,\n" +
+                                       "\t\t\t\tMap<String, String> 
properties) {\n" +
+                                       "\t\t\treturn new 
GenericInMemoryCatalog(name);\n" +
+                                       "\t\t}\n" +
+                                       "\n" +
+                                       "\t\t@Override\n" +
+                                       "\t\tpublic Map<String, String> 
requiredContext() {\n" +
+                                       "\t\t\tHashMap<String, String> hashMap 
= new HashMap<>();\n" +
+                                       "\t\t\thashMap.put(\"type\", 
\"userCatalog\");\n" +
+                                       "\t\t\treturn hashMap;\n" +
+                                       "\t\t}\n" +
+                                       "\n" +
+                                       "\t\t@Override\n" +
+                                       "\t\tpublic List<String> 
supportedProperties() {\n" +
+                                       "\t\t\treturn 
Collections.emptyList();\n" +
+                                       "\t\t}\n" +
+                                       "\t}"
+                       ).build();
+
+               try (TemporaryClassLoaderContext context = 
TemporaryClassLoaderContext.of(classLoader)) {

Review comment:
       I don't think so. The problem is the line in 
`SqlToOperationConverter:557`.
   ```
                final CatalogFactory factory =
                        TableFactoryService.find(CatalogFactory.class, 
properties, this.getClass().getClassLoader());
   ```
    It is using the class classloader, not the context classloader. I could've 
just changed that to use the context classloader, but I found it wrong that the 
catalog is instantiated in `SqlToOperationConverter`. Moreover it would not use 
the classloader provided in `EnvironmentSettings` in the future.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to