JingsongLi commented on code in PR #1548:
URL: https://github.com/apache/incubator-paimon/pull/1548#discussion_r1267492173
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalogFactory.java:
##########
@@ -67,11 +67,13 @@ public static FlinkCatalog createCatalog(
return new FlinkCatalog(
CatalogFactory.createCatalog(context, classLoader),
catalogName,
- context.options().get(DEFAULT_DATABASE));
+ context.options().get(DEFAULT_DATABASE),
+ classLoader);
}
public static FlinkCatalog createCatalog(String catalogName, Catalog
catalog) {
- return new FlinkCatalog(catalog, catalogName,
Catalog.DEFAULT_DATABASE);
+ return new FlinkCatalog(
+ catalog, catalogName, Catalog.DEFAULT_DATABASE,
ClassLoader.getSystemClassLoader());
Review Comment:
system? Can we just use FlinkCatalog.class.getClassloader?
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java:
##########
@@ -237,21 +255,66 @@ public void createTable(ObjectPath tablePath,
CatalogBaseTable table, boolean ig
+ " You can create TEMPORARY table instead if you
want to create the table of other connector.");
}
+ Identifier identifier = toIdentifier(tablePath);
+ Map<String, String> logSystemOptions = registerLogSystem(identifier,
options);
// remove table path
String specific = options.remove(PATH.key());
- if (specific != null) {
+ if (specific != null || !logSystemOptions.isEmpty()) {
+ options.putAll(logSystemOptions);
catalogTable = catalogTable.copy(options);
}
+ boolean unRegisterLogSystem = false;
try {
catalog.createTable(
- toIdentifier(tablePath),
- FlinkCatalog.fromCatalogTable(catalogTable),
- ignoreIfExists);
+ identifier, FlinkCatalog.fromCatalogTable(catalogTable),
ignoreIfExists);
} catch (Catalog.TableAlreadyExistException e) {
+ unRegisterLogSystem = true;
throw new TableAlreadyExistException(getName(), tablePath);
} catch (Catalog.DatabaseNotExistException e) {
+ unRegisterLogSystem = true;
throw new DatabaseNotExistException(getName(), e.database());
+ } finally {
+ if (unRegisterLogSystem && !logSystemOptions.isEmpty()) {
+ unRegisterLogSystem(identifier, options);
+ }
+ }
+ }
+
+ private Map<String, String> registerLogSystem(
Review Comment:
Can this be in separate class? Maybe just in `LogStoreRegister`, a static
method?
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java:
##########
@@ -237,21 +255,66 @@ public void createTable(ObjectPath tablePath,
CatalogBaseTable table, boolean ig
+ " You can create TEMPORARY table instead if you
want to create the table of other connector.");
}
+ Identifier identifier = toIdentifier(tablePath);
+ Map<String, String> logSystemOptions = registerLogSystem(identifier,
options);
// remove table path
String specific = options.remove(PATH.key());
- if (specific != null) {
+ if (specific != null || !logSystemOptions.isEmpty()) {
+ options.putAll(logSystemOptions);
catalogTable = catalogTable.copy(options);
}
+ boolean unRegisterLogSystem = false;
try {
catalog.createTable(
- toIdentifier(tablePath),
- FlinkCatalog.fromCatalogTable(catalogTable),
- ignoreIfExists);
+ identifier, FlinkCatalog.fromCatalogTable(catalogTable),
ignoreIfExists);
} catch (Catalog.TableAlreadyExistException e) {
+ unRegisterLogSystem = true;
throw new TableAlreadyExistException(getName(), tablePath);
} catch (Catalog.DatabaseNotExistException e) {
+ unRegisterLogSystem = true;
throw new DatabaseNotExistException(getName(), e.database());
+ } finally {
+ if (unRegisterLogSystem && !logSystemOptions.isEmpty()) {
+ unRegisterLogSystem(identifier, options);
+ }
+ }
+ }
+
+ private Map<String, String> registerLogSystem(
+ Identifier identifier, Map<String, String> options) {
+ Options tableOptions = Options.fromMap(options);
+ if (tableOptions.get(LOG_SYSTEM_AUTO_REGISTER)) {
+ String logStore = tableOptions.get(LOG_SYSTEM);
+ checkArgument(
+ !tableOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE),
+ String.format(
+ "%s must be configured when you use log system
register.",
+ LOG_SYSTEM.key()));
+ if (catalog.tableExists(identifier)) {
+ return Collections.emptyMap();
+ }
+ LogStoreTableFactory registerFactory =
+ FactoryUtil.discoverFactory(classLoader,
LogStoreTableFactory.class, logStore);
+ LogStoreRegister logStoreRegister =
registerFactory.createRegister(tableOptions);
Review Comment:
- createRegister pass an options.
- registerTopic(identifier, options) need an options.
Is it repeated here?
##########
paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java:
##########
@@ -212,8 +222,16 @@ public boolean tableExists(ObjectPath tablePath) throws
CatalogException {
@Override
public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
throws TableNotExistException, CatalogException {
+ Identifier identifier = toIdentifier(tablePath);
+ Table table = null;
try {
+ if (catalog.tableExists(identifier)) {
Review Comment:
This brings some overhead for normal tables.
Can we just have a
`if we need to unregister -> a separate branch`.
--
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]