This is an automated email from the ASF dual-hosted git repository.

yuxia pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/fluss.git


The following commit(s) were added to refs/heads/main by this push:
     new 21a1d89c [lake] Ensure LakeStoragePlugin uses the plugin classloader 
for all operations (#1224)
21a1d89c is described below

commit 21a1d89cb0e33f2db4f1f9d3fde16731bb1f32e3
Author: Junbo Wang <[email protected]>
AuthorDate: Thu Jul 3 11:09:55 2025 +0800

    [lake] Ensure LakeStoragePlugin uses the plugin classloader for all 
operations (#1224)
---
 .../lake/lakestorage/LakeStoragePluginSetUp.java   |  19 +--
 .../lake/lakestorage/PluginLakeStorageWrapper.java | 120 +++++++++++++++++
 .../fluss/lake/lakestorage/LakeStorageTest.java    | 142 +++++++++++++++++++++
 .../fluss/flink/tiering/LakeTieringJobBuilder.java |   9 +-
 .../server/coordinator/CoordinatorServer.java      |   8 +-
 5 files changed, 273 insertions(+), 25 deletions(-)

diff --git 
a/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/LakeStoragePluginSetUp.java
 
b/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/LakeStoragePluginSetUp.java
index a33f1328..8f226eed 100644
--- 
a/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/LakeStoragePluginSetUp.java
+++ 
b/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/LakeStoragePluginSetUp.java
@@ -17,9 +17,6 @@
 
 package com.alibaba.fluss.lake.lakestorage;
 
-import com.alibaba.fluss.config.ConfigOptions;
-import com.alibaba.fluss.config.Configuration;
-import com.alibaba.fluss.metadata.DataLakeFormat;
 import com.alibaba.fluss.plugin.PluginManager;
 import com.alibaba.fluss.shaded.guava32.com.google.common.collect.Iterators;
 
@@ -35,29 +32,23 @@ import java.util.ServiceLoader;
  */
 public class LakeStoragePluginSetUp {
 
-    @Nullable
-    public static LakeStoragePlugin fromConfiguration(
-            final Configuration configuration, @Nullable final PluginManager 
pluginManager) {
-        DataLakeFormat dataLakeFormat = 
configuration.get(ConfigOptions.DATALAKE_FORMAT);
-        if (dataLakeFormat == null) {
-            return null;
-        }
-        String dataLakeIdentifier = dataLakeFormat.toString();
+    public static LakeStoragePlugin fromDataLakeFormat(
+            final String dataLakeFormat, @Nullable final PluginManager 
pluginManager) {
         // now, load lake storage plugin
         Iterator<LakeStoragePlugin> lakeStoragePluginIterator =
                 getAllLakeStoragePlugins(pluginManager);
 
         while (lakeStoragePluginIterator.hasNext()) {
             LakeStoragePlugin lakeStoragePlugin = 
lakeStoragePluginIterator.next();
-            if (Objects.equals(lakeStoragePlugin.identifier(), 
dataLakeIdentifier)) {
-                return lakeStoragePlugin;
+            if (Objects.equals(lakeStoragePlugin.identifier(), 
dataLakeFormat)) {
+                return PluginLakeStorageWrapper.of(lakeStoragePlugin);
             }
         }
 
         // if come here, means we haven't found LakeStoragePlugin match the 
configured
         // datalake, throw exception
         throw new UnsupportedOperationException(
-                "No LakeStoragePlugin can be found for datalake format: " + 
dataLakeIdentifier);
+                "No LakeStoragePlugin can be found for datalake format: " + 
dataLakeFormat);
     }
 
     private static Iterator<LakeStoragePlugin> getAllLakeStoragePlugins(
diff --git 
a/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/PluginLakeStorageWrapper.java
 
b/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/PluginLakeStorageWrapper.java
new file mode 100644
index 00000000..fd17fe20
--- /dev/null
+++ 
b/fluss-common/src/main/java/com/alibaba/fluss/lake/lakestorage/PluginLakeStorageWrapper.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 com.alibaba.fluss.lake.lakestorage;
+
+import com.alibaba.fluss.config.Configuration;
+import com.alibaba.fluss.exception.TableAlreadyExistException;
+import com.alibaba.fluss.lake.writer.LakeTieringFactory;
+import com.alibaba.fluss.metadata.TableDescriptor;
+import com.alibaba.fluss.metadata.TablePath;
+import com.alibaba.fluss.utils.TemporaryClassLoaderContext;
+import com.alibaba.fluss.utils.WrappingProxy;
+
+/**
+ * A wrapper around {@link LakeStoragePlugin} that ensures the plugin 
classloader is used for all
+ * {@link LakeCatalog} operations.
+ */
+public class PluginLakeStorageWrapper implements LakeStoragePlugin {
+    private final LakeStoragePlugin inner;
+    private final ClassLoader loader;
+
+    private PluginLakeStorageWrapper(final LakeStoragePlugin inner, final 
ClassLoader loader) {
+        this.inner = inner;
+        this.loader = loader;
+    }
+
+    public static PluginLakeStorageWrapper of(final LakeStoragePlugin inner) {
+        return new PluginLakeStorageWrapper(inner, 
inner.getClass().getClassLoader());
+    }
+
+    @Override
+    public ClassLoader getClassLoader() {
+        return inner.getClassLoader();
+    }
+
+    @Override
+    public String identifier() {
+        return inner.identifier();
+    }
+
+    @Override
+    public LakeStorage createLakeStorage(Configuration configuration) {
+        try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(loader)) {
+            return new 
ClassLoaderFixingLakeStorage(inner.createLakeStorage(configuration), loader);
+        }
+    }
+
+    static class ClassLoaderFixingLakeCatalog implements LakeCatalog, 
WrappingProxy<LakeCatalog> {
+
+        private final LakeCatalog inner;
+        private final ClassLoader loader;
+
+        private ClassLoaderFixingLakeCatalog(final LakeCatalog inner, final 
ClassLoader loader) {
+            this.inner = inner;
+            this.loader = loader;
+        }
+
+        @Override
+        public void createTable(TablePath tablePath, TableDescriptor 
tableDescriptor)
+                throws TableAlreadyExistException {
+            try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(loader)) {
+                inner.createTable(tablePath, tableDescriptor);
+            }
+        }
+
+        @Override
+        public void close() throws Exception {
+            try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(loader)) {
+                inner.close();
+            }
+        }
+
+        @Override
+        public LakeCatalog getWrappedDelegate() {
+            return inner;
+        }
+    }
+
+    static class ClassLoaderFixingLakeStorage implements LakeStorage, 
WrappingProxy<LakeStorage> {
+
+        private final LakeStorage inner;
+        private final ClassLoader loader;
+
+        private ClassLoaderFixingLakeStorage(final LakeStorage inner, final 
ClassLoader loader) {
+            this.inner = inner;
+            this.loader = loader;
+        }
+
+        @Override
+        public LakeStorage getWrappedDelegate() {
+            return inner;
+        }
+
+        @Override
+        public LakeTieringFactory<?, ?> createLakeTieringFactory() {
+            return inner.createLakeTieringFactory();
+        }
+
+        @Override
+        public LakeCatalog createLakeCatalog() {
+            try (TemporaryClassLoaderContext ignored = 
TemporaryClassLoaderContext.of(loader)) {
+                return new 
ClassLoaderFixingLakeCatalog(inner.createLakeCatalog(), loader);
+            }
+        }
+    }
+}
diff --git 
a/fluss-common/src/test/java/com/alibaba/fluss/lake/lakestorage/LakeStorageTest.java
 
b/fluss-common/src/test/java/com/alibaba/fluss/lake/lakestorage/LakeStorageTest.java
new file mode 100644
index 00000000..4c349233
--- /dev/null
+++ 
b/fluss-common/src/test/java/com/alibaba/fluss/lake/lakestorage/LakeStorageTest.java
@@ -0,0 +1,142 @@
+/*
+ * Copyright (c) 2025 Alibaba Group Holding Ltd.
+ *
+ * Licensed 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 com.alibaba.fluss.lake.lakestorage;
+
+import com.alibaba.fluss.config.Configuration;
+import com.alibaba.fluss.exception.TableAlreadyExistException;
+import com.alibaba.fluss.lake.writer.LakeTieringFactory;
+import com.alibaba.fluss.metadata.TableDescriptor;
+import com.alibaba.fluss.metadata.TablePath;
+import com.alibaba.fluss.plugin.PluginManager;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for the {@link LakeStorage} base class. */
+class LakeStorageTest {
+    private static final String TEST_LAKE_PLUGIN_FORMAT = "test-plugin";
+
+    @Test
+    void testInvalidPlugin() throws Exception {
+        final Map<Class<?>, Iterator<?>> lakeStoragePlugins = new HashMap<>();
+        lakeStoragePlugins.put(
+                LakeStoragePlugin.class,
+                Collections.singletonList(new 
LakeStorageTest.TestPluginLakeStoragePlugin())
+                        .iterator());
+
+        assertThatThrownBy(
+                        () ->
+                                LakeStoragePluginSetUp.fromDataLakeFormat(
+                                        TEST_LAKE_PLUGIN_FORMAT + "1",
+                                        new 
TestingPluginManager(lakeStoragePlugins)))
+                .isInstanceOf(UnsupportedOperationException.class)
+                .hasMessage("No LakeStoragePlugin can be found for datalake 
format: test-plugin1");
+    }
+
+    @Test
+    void testWithPluginManager() throws Exception {
+        final Map<Class<?>, Iterator<?>> lakeStoragePlugins = new HashMap<>();
+
+        lakeStoragePlugins.put(
+                LakeStoragePlugin.class,
+                Collections.singletonList(new 
LakeStorageTest.TestPluginLakeStoragePlugin())
+                        .iterator());
+
+        LakeStoragePlugin lakeStoragePlugin =
+                LakeStoragePluginSetUp.fromDataLakeFormat(
+                        TEST_LAKE_PLUGIN_FORMAT, new 
TestingPluginManager(lakeStoragePlugins));
+
+        
assertThat(lakeStoragePlugin).isInstanceOf(PluginLakeStorageWrapper.class);
+        LakeStorage lakeStorage = lakeStoragePlugin.createLakeStorage(new 
Configuration());
+
+        // the LakeStorage should wrap TestPaimonLakeStorage
+        assertThat(lakeStorage)
+                
.isInstanceOf(PluginLakeStorageWrapper.ClassLoaderFixingLakeStorage.class);
+        assertThat(
+                        
((PluginLakeStorageWrapper.ClassLoaderFixingLakeStorage) lakeStorage)
+                                .getWrappedDelegate())
+                .isInstanceOf(TestPaimonLakeStorage.class);
+
+        // the LakeCatalog should wrap TestPaimonLakeCatalog
+        LakeCatalog lakeCatalog = lakeStorage.createLakeCatalog();
+        assertThat(lakeCatalog)
+                
.isInstanceOf(PluginLakeStorageWrapper.ClassLoaderFixingLakeCatalog.class);
+        assertThat(
+                        
((PluginLakeStorageWrapper.ClassLoaderFixingLakeCatalog) lakeCatalog)
+                                .getWrappedDelegate())
+                .isInstanceOf(TestPaimonLakeCatalog.class);
+    }
+
+    private static class TestingPluginManager implements PluginManager {
+
+        private final Map<Class<?>, Iterator<?>> plugins;
+
+        private TestingPluginManager(Map<Class<?>, Iterator<?>> plugins) {
+            this.plugins = plugins;
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public <P> Iterator<P> load(Class<P> service) {
+            return (Iterator<P>) plugins.get(service);
+        }
+    }
+
+    private static class TestPluginLakeStoragePlugin implements 
LakeStoragePlugin {
+
+        private static final String IDENTIFIER = TEST_LAKE_PLUGIN_FORMAT;
+
+        @Override
+        public String identifier() {
+            return IDENTIFIER;
+        }
+
+        @Override
+        public LakeStorage createLakeStorage(Configuration configuration) {
+            return new TestPaimonLakeStorage();
+        }
+    }
+
+    private static class TestPaimonLakeStorage implements LakeStorage {
+
+        public TestPaimonLakeStorage() {}
+
+        @Override
+        public LakeTieringFactory<?, ?> createLakeTieringFactory() {
+            return null;
+        }
+
+        @Override
+        public TestPaimonLakeCatalog createLakeCatalog() {
+            return new TestPaimonLakeCatalog();
+        }
+    }
+
+    private static class TestPaimonLakeCatalog implements LakeCatalog {
+
+        @Override
+        public void createTable(TablePath tablePath, TableDescriptor 
tableDescriptor)
+                throws TableAlreadyExistException {}
+    }
+}
diff --git 
a/fluss-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/tiering/LakeTieringJobBuilder.java
 
b/fluss-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/tiering/LakeTieringJobBuilder.java
index e1156b99..3fc776a7 100644
--- 
a/fluss-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/tiering/LakeTieringJobBuilder.java
+++ 
b/fluss-flink/fluss-flink-common/src/main/java/com/alibaba/fluss/flink/tiering/LakeTieringJobBuilder.java
@@ -17,7 +17,6 @@
 
 package com.alibaba.fluss.flink.tiering;
 
-import com.alibaba.fluss.config.ConfigOptions;
 import com.alibaba.fluss.config.Configuration;
 import com.alibaba.fluss.flink.tiering.committer.CommittableMessageTypeInfo;
 import com.alibaba.fluss.flink.tiering.committer.TieringCommitOperatorFactory;
@@ -35,8 +34,6 @@ import 
org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
 
-import java.util.Collections;
-
 import static 
com.alibaba.fluss.flink.tiering.source.TieringSource.TIERING_SOURCE_TRANSFORMATION_UID;
 import static 
com.alibaba.fluss.flink.tiering.source.TieringSourceOptions.POLL_TIERING_TABLE_INTERVAL;
 import static com.alibaba.fluss.utils.Preconditions.checkNotNull;
@@ -74,11 +71,7 @@ public class LakeTieringJobBuilder {
     public JobClient build() throws Exception {
         // get the lake storage plugin
         LakeStoragePlugin lakeStoragePlugin =
-                LakeStoragePluginSetUp.fromConfiguration(
-                        Configuration.fromMap(
-                                Collections.singletonMap(
-                                        ConfigOptions.DATALAKE_FORMAT.key(), 
dataLakeFormat)),
-                        null);
+                LakeStoragePluginSetUp.fromDataLakeFormat(dataLakeFormat, 
null);
         // create lake storage from configurations
         LakeStorage lakeStorage = 
checkNotNull(lakeStoragePlugin).createLakeStorage(dataLakeConfig);
 
diff --git 
a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java
 
b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java
index 26ce667e..227d05d3 100644
--- 
a/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java
+++ 
b/fluss-server/src/main/java/com/alibaba/fluss/server/coordinator/CoordinatorServer.java
@@ -27,6 +27,7 @@ import com.alibaba.fluss.lake.lakestorage.LakeCatalog;
 import com.alibaba.fluss.lake.lakestorage.LakeStorage;
 import com.alibaba.fluss.lake.lakestorage.LakeStoragePlugin;
 import com.alibaba.fluss.lake.lakestorage.LakeStoragePluginSetUp;
+import com.alibaba.fluss.metadata.DataLakeFormat;
 import com.alibaba.fluss.metadata.DatabaseDescriptor;
 import com.alibaba.fluss.metrics.registry.MetricRegistry;
 import com.alibaba.fluss.rpc.RpcClient;
@@ -247,11 +248,12 @@ public class CoordinatorServer extends ServerBase {
 
     @Nullable
     private LakeCatalog createLakeCatalog() {
-        LakeStoragePlugin lakeStoragePlugin =
-                LakeStoragePluginSetUp.fromConfiguration(conf, pluginManager);
-        if (lakeStoragePlugin == null) {
+        DataLakeFormat dataLakeFormat = 
conf.get(ConfigOptions.DATALAKE_FORMAT);
+        if (dataLakeFormat == null) {
             return null;
         }
+        LakeStoragePlugin lakeStoragePlugin =
+                
LakeStoragePluginSetUp.fromDataLakeFormat(dataLakeFormat.toString(), 
pluginManager);
         Map<String, String> lakeProperties = extractLakeProperties(conf);
         LakeStorage lakeStorage =
                 lakeStoragePlugin.createLakeStorage(

Reply via email to