Copilot commented on code in PR #54304:
URL: https://github.com/apache/doris/pull/54304#discussion_r2271436149


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateFunctionCommand.java:
##########
@@ -377,6 +384,34 @@ private void computeObjectChecksum() throws IOException, 
NoSuchAlgorithmExceptio
         }
     }
 
+    private String getRealUrl(String url) {
+        if (!url.contains(":/")) {
+            return checkAndReturnDefaultJavaUdfUrl(url);
+        }
+        return url;
+    }
+
+    private String checkAndReturnDefaultJavaUdfUrl(String url) {
+        String dorisHome = System.getenv("DORIS_HOME");
+        String defaultUrl = dorisHome + "/plugins/java_udf";
+        // In cloud mode, try cloud download first
+        if (Config.isCloudMode()) {
+            String targetPath = defaultUrl + "/" + url;
+            try {
+                String downloadedPath = 
CloudPluginDownloader.downloadFromCloud(
+                        CloudPluginDownloader.PluginType.JAVA_UDF, url, 
targetPath, null);
+                if (!downloadedPath.isEmpty()) {
+                    return "file://" + downloadedPath;
+                }
+            } catch (Exception e) {
+                throw new RuntimeException("Can't download UDF from cloud: " + 
url

Review Comment:
   The error message uses a contraction "Can't" which should be "Cannot" for 
consistency with professional error messaging standards.
   ```suggestion
                   throw new RuntimeException("Cannot download UDF from cloud: 
" + url
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java:
##########
@@ -322,14 +332,33 @@ public static String getFullDriverUrl(String driverUrl) 
throws IllegalArgumentEx
         }
     }
 
-    private static String checkAndReturnDefaultDriverUrl(String driverUrl) {
+    private static String checkAndReturnDefaultDriverUrl(String driverUrl, 
String expectedMd5) {
         final String defaultDriverUrl = EnvUtils.getDorisHome() + 
"/plugins/jdbc_drivers";
         final String defaultOldDriverUrl = EnvUtils.getDorisHome() + 
"/jdbc_drivers";
         if (Config.jdbc_drivers_dir.equals(defaultDriverUrl)) {
             // If true, which means user does not set `jdbc_drivers_dir` and 
use the default one.
             // Because in new version, we change the default value of 
`jdbc_drivers_dir`
             // from `DORIS_HOME/jdbc_drivers` to 
`DORIS_HOME/plugins/jdbc_drivers`,
             // so we need to check the old default dir for compatibility.
+            // In cloud mode, always try cloud download first for validation 
and updates
+            if (Config.isCloudMode()) {
+                try {
+                    String downloadedPath = 
CloudPluginDownloader.downloadFromCloud(
+                            PluginType.JDBC_DRIVERS,
+                            driverUrl,
+                            defaultDriverUrl + "/" + driverUrl,
+                            expectedMd5);
+                    if (!downloadedPath.isEmpty()) {
+                        return "file://" + downloadedPath;
+                    }
+                } catch (Exception e) {
+                    LOG.warn("Failed to download JDBC driver from cloud: {}, 
error: {}", driverUrl, e.getMessage());
+                    throw new RuntimeException("Can't download JDBC driver 
from cloud: " + driverUrl

Review Comment:
   The error message uses a contraction "Can't" which should be "Cannot" for 
consistency with professional error messaging standards.
   ```suggestion
                       throw new RuntimeException("Cannot download JDBC driver 
from cloud: " + driverUrl
   ```



##########
be/src/vec/exec/vjdbc_connector.cpp:
##########
@@ -635,33 +642,55 @@ Status JdbcConnector::_get_java_table_type(JNIEnv* env, 
TOdbcTableType::type tab
     return Status::OK();
 }
 
-std::string JdbcConnector::_get_real_url(const std::string& url) {
+Status JdbcConnector::_get_real_url(const std::string& url, const std::string& 
expected_md5,
+                                    std::string* result_url) {
     if (url.find(":/") == std::string::npos) {
-        return _check_and_return_default_driver_url(url);
+        return _check_and_return_default_driver_url(url, expected_md5, 
result_url);
     }
-    return url;
+    *result_url = url;
+    return Status::OK();
 }
 
-std::string JdbcConnector::_check_and_return_default_driver_url(const 
std::string& url) {
+Status JdbcConnector::_check_and_return_default_driver_url(const std::string& 
url,
+                                                           const std::string& 
expected_md5,
+                                                           std::string* 
result_url) {
     const char* doris_home = std::getenv("DORIS_HOME");
-
     std::string default_url = std::string(doris_home) + 
"/plugins/jdbc_drivers";
     std::string default_old_url = std::string(doris_home) + "/jdbc_drivers";
-
     if (config::jdbc_drivers_dir == default_url) {
         // If true, which means user does not set `jdbc_drivers_dir` and use 
the default one.
         // Because in 2.1.8, we change the default value of `jdbc_drivers_dir`
         // from `DORIS_HOME/jdbc_drivers` to `DORIS_HOME/plugins/jdbc_drivers`,
         // so we need to check the old default dir for compatibility.
         std::filesystem::path file = default_url + "/" + url;
+
+        if (config::is_cloud_mode()) {
+            std::string target_path = default_url + "/" + url;
+            std::string downloaded_path;
+            Status status = CloudPluginDownloader::download_from_cloud(
+                    CloudPluginDownloader::PluginType::JDBC_DRIVERS, url, 
target_path,
+                    &downloaded_path, expected_md5);
+            if (status.ok() && !downloaded_path.empty()) {
+                *result_url = "file://" + downloaded_path;
+                return Status::OK();
+            } else {
+                LOG(WARNING) << "Failed to download JDBC driver from cloud: " 
<< status.to_string();
+                return Status::RuntimeError(
+                        "Can't download JDBC driver from cloud: {}. "

Review Comment:
   The error message uses a contraction "Can't" which should be "Cannot" for 
consistency with professional error messaging standards.
   ```suggestion
                           "Cannot download JDBC driver from cloud: {}. "
   ```



##########
be/src/runtime/plugin/s3_plugin_downloader.cpp:
##########
@@ -0,0 +1,185 @@
+// 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.
+
+#include "runtime/plugin/s3_plugin_downloader.h"
+
+#include <fmt/format.h>
+
+#include <filesystem>
+#include <fstream>
+#include <memory>
+#include <string>
+#include <thread>
+
+#include "common/logging.h"
+#include "common/status.h"
+#include "io/fs/local_file_system.h"
+#include "io/fs/s3_file_system.h"
+#include "runtime/plugin/plugin_file_cache.h"
+#include "util/s3_util.h"
+
+namespace doris {
+
+std::string S3PluginDownloader::S3Config::to_string() const {
+    return fmt::format("S3Config{{endpoint='{}', region='{}', bucket='{}', 
access_key='{}'}}",
+                       endpoint, region, bucket, access_key.empty() ? "null" : 
"***");
+}
+
+S3PluginDownloader::S3PluginDownloader(const S3Config& config) : 
config_(config) {
+    s3_fs_ = create_s3_filesystem(config_);
+    // Note: We don't throw here. The caller should check via download_file 
return status
+}
+
+S3PluginDownloader::~S3PluginDownloader() = default;
+
+Status S3PluginDownloader::download_file(const std::string& remote_s3_path,
+                                         const std::string& local_target_path,
+                                         std::string* local_path, const 
std::string& expected_md5) {
+    // Check if S3 filesystem is initialized
+    if (!s3_fs_) {
+        return Status::InternalError("S3 filesystem not initialized");
+    }
+
+    // Check if download is needed first
+    if (PluginFileCache::is_file_valid(local_target_path, expected_md5)) {
+        *local_path = local_target_path;
+        return Status::OK(); // Local file is valid, return directly
+    }
+
+    // Execute download retry logic
+    Status last_status;
+    for (int attempt = 1; attempt <= MAX_RETRY_ATTEMPTS; ++attempt) {
+        last_status = execute_download(remote_s3_path, local_target_path, 
expected_md5);
+        if (last_status.ok()) {
+            *local_path = local_target_path;
+            return Status::OK();
+        }
+        if (attempt < MAX_RETRY_ATTEMPTS) {
+            sleep_for_retry(attempt);
+        }
+    }
+
+    // All retries failed
+    return last_status;
+}
+
+Status S3PluginDownloader::execute_download(const std::string& remote_s3_path,
+                                            const std::string& local_path,
+                                            const std::string& expected_md5) {
+    // Create parent directory
+    Status status = create_parent_directory(local_path);
+    RETURN_IF_ERROR(status);
+
+    // Use S3FileSystem's public download method
+    status = s3_fs_->download(remote_s3_path, local_path);
+    RETURN_IF_ERROR(status);
+
+    // MD5 verification and cache update
+    std::string actual_md5 = calculate_file_md5(local_path);
+
+    // If user provided MD5, must verify consistency
+    if (!expected_md5.empty()) {
+        if (actual_md5.empty() || expected_md5 != actual_md5) {
+            std::filesystem::remove(local_path); // Delete invalid file
+            return Status::InvalidArgument("MD5 mismatch: expected={}, 
actual={}", expected_md5,
+                                           actual_md5);
+        }
+    }
+
+    // Update cache
+    update_cache_after_download(local_path, actual_md5);
+
+    LOG(INFO) << "Successfully downloaded " << remote_s3_path << " to " << 
local_path;
+    return Status::OK();
+}
+
+void S3PluginDownloader::sleep_for_retry(int attempt) {
+    try {
+        std::this_thread::sleep_for(std::chrono::milliseconds(RETRY_DELAY_MS * 
attempt));
+    } catch (const std::exception& e) {
+        throw std::runtime_error("Download interrupted: " + 
std::string(e.what()));

Review Comment:
   The exception handling converts a std::exception to a RuntimeException but 
doesn't preserve the original exception type information, which could make 
debugging more difficult.
   ```suggestion
           std::throw_with_nested(std::runtime_error("Download interrupted: " + 
std::string(e.what())));
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/common/plugin/S3PluginDownloader.java:
##########
@@ -0,0 +1,210 @@
+// 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.doris.common.plugin;
+
+import org.apache.doris.backup.Status;
+import org.apache.doris.datasource.property.storage.S3Properties;
+import org.apache.doris.fs.obj.S3ObjStorage;
+
+import com.google.common.base.Strings;
+import org.apache.commons.codec.binary.Hex;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.MessageDigest;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * S3PluginDownloader is an independent, generic S3 downloader.
+ * <p>
+ * Design principles:
+ * 1. Single responsibility: Only downloads files from S3, no business logic
+ * 2. Complete decoupling: No dependency on cloud mode or Doris-specific 
configuration
+ * 3. Reusable: Supports both cloud mode auto-configuration and manual S3 
parameters
+ * 4. Features: Single file download, batch directory download, MD5 
verification, retry mechanism
+ */
+public class S3PluginDownloader implements AutoCloseable {
+    private static final int MAX_RETRY_ATTEMPTS = 3;
+    private static final long RETRY_DELAY_MS = 1000;
+
+    private final S3ObjStorage s3Storage;
+
+    /**
+     * S3 configuration info (completely independent, no dependency on any 
Doris internal types)
+     */
+    public static class S3Config {
+        public final String endpoint;
+        public final String region;
+        public final String bucket;
+        public final String accessKey;
+        public final String secretKey;
+
+        public S3Config(String endpoint, String region, String bucket,
+                String accessKey, String secretKey) {
+            this.endpoint = endpoint;
+            this.region = region;
+            this.bucket = bucket;
+            this.accessKey = accessKey;
+            this.secretKey = secretKey;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("S3Config{endpoint='%s', region='%s', 
bucket='%s', accessKey='%s'}",
+                    endpoint, region, bucket, accessKey != null ? "***" : 
"null");
+        }
+    }
+
+    /**
+     * Constructor - pass in S3 configuration
+     */
+    public S3PluginDownloader(S3Config config) {
+        this.s3Storage = createS3Storage(config);
+    }
+
+    // ======================== Core Download Methods ========================
+
+    /**
+     * Download single file (supports MD5 verification and retry)
+     *
+     * @param remoteS3Path complete S3 path like "s3://bucket/path/to/file.jar"
+     * @param localPath local target file path
+     * @param expectedMd5 optional MD5 verification value, null to skip 
verification
+     * @return returns local file path on success
+     * @throws RuntimeException if download fails after all retries
+     */
+    public String downloadFile(String remoteS3Path, String localPath, String 
expectedMd5) {
+        // Check if you need to download it first
+        if (PluginFileCache.isFileValid(localPath, expectedMd5)) {
+            return localPath; // The local file is valid and returns directly
+        }
+
+        // Execute the download retry logic
+        Exception lastException = null;
+        for (int attempt = 1; attempt <= MAX_RETRY_ATTEMPTS; attempt++) {
+            try {
+                return executeDownload(remoteS3Path, localPath, expectedMd5);
+            } catch (Exception e) {
+                lastException = e;
+                if (attempt < MAX_RETRY_ATTEMPTS) {
+                    sleepForRetry(attempt);
+                }
+            }
+        }
+        // All retries failed
+        throw new RuntimeException("Download failed after " + 
MAX_RETRY_ATTEMPTS + " attempts: "
+                + lastException.getMessage(), lastException);
+    }
+
+    private String executeDownload(String remoteS3Path, String localPath, 
String expectedMd5) throws Exception {
+        // Create a parent directory
+        Path parentDir = Paths.get(localPath).getParent();
+        if (parentDir != null && !Files.exists(parentDir)) {
+            Files.createDirectories(parentDir);
+        }
+
+        // Perform the download
+        File localFile = new File(localPath);
+        Status status = s3Storage.getObject(remoteS3Path, localFile);
+        if (status != Status.OK) {
+            throw new RuntimeException("Download failed: " + 
status.getErrMsg());
+        }
+
+        // MD5 checksum cache update
+        String actualMd5 = calculateFileMD5(localFile);
+
+        // If the user provides MD5, the consistency must be verified
+        if (!Strings.isNullOrEmpty(expectedMd5)) {
+            if (!expectedMd5.equalsIgnoreCase(actualMd5)) {
+                localFile.delete(); // Delete invalid files
+                throw new RuntimeException(String.format(
+                        "MD5 mismatch: expected=%s, actual=%s", expectedMd5, 
actualMd5));
+            }
+        }
+        // Update cache
+        updateCacheAfterDownload(localPath, actualMd5);
+
+        return localPath;
+    }
+
+    private void sleepForRetry(int attempt) {
+        try {
+            Thread.sleep(RETRY_DELAY_MS * attempt);
+        } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException("Download interrupted", ie);
+        }
+    }
+
+    /**
+     * Update cache after download - save MD5 and file size
+     *
+     * @param localPath local file path
+     * @param actualMd5 calculated MD5 hash of the downloaded file
+     */
+    private void updateCacheAfterDownload(String localPath, String actualMd5) {
+        try {
+            File localFile = new File(localPath);
+            long fileSize = localFile.exists() ? localFile.length() : 0;
+            PluginFileCache.updateCache(localPath, actualMd5, fileSize);
+        } catch (Exception e) {
+            // Ignore cache update failures - not critical
+        }
+    }
+
+    private String calculateFileMD5(File file) {
+        try (FileInputStream inputStream = new FileInputStream(file)) {
+            MessageDigest digest = MessageDigest.getInstance("MD5");
+            byte[] buf = new byte[4096];
+            int bytesRead;
+            do {
+                bytesRead = inputStream.read(buf);
+                if (bytesRead < 0) {
+                    break;
+                }
+                digest.update(buf, 0, bytesRead);
+            } while (true);

Review Comment:
   [nitpick] The do-while loop with a `while (true)` condition and a break 
statement inside is less readable than a standard while loop. Consider 
refactoring to use `while (bytesRead >= 0)` or similar.
   ```suggestion
               while ((bytesRead = inputStream.read(buf)) >= 0) {
                   digest.update(buf, 0, bytesRead);
               }
   ```



##########
be/src/runtime/user_function_cache.cpp:
##########
@@ -381,4 +389,46 @@ std::vector<std::string> 
UserFunctionCache::_split_string_by_checksum(const std:
 
     return result;
 }
+
+Status UserFunctionCache::_get_real_url(const std::string& url, const 
std::string& checksum,
+                                        std::string* result_url) {
+    if (url.find(":/") == std::string::npos) {
+        return _check_and_return_default_java_udf_url(url, checksum, 
result_url);
+    }
+    *result_url = url;
+    return Status::OK();
+}
+
+Status UserFunctionCache::_check_and_return_default_java_udf_url(const 
std::string& url,
+                                                                 const 
std::string& checksum,
+                                                                 std::string* 
result_url) {
+    const char* doris_home = std::getenv("DORIS_HOME");
+    std::string default_url = std::string(doris_home) + "/plugins/java_udf";
+
+    std::filesystem::path file = default_url + "/" + url;
+
+    // In cloud mode, always try cloud download first (prioritize cloud mode)
+    if (config::is_cloud_mode()) {
+        std::string target_path = default_url + "/" + url;
+        std::string downloaded_path;
+        Status status = CloudPluginDownloader::download_from_cloud(
+                CloudPluginDownloader::PluginType::JAVA_UDF, url, target_path, 
&downloaded_path,
+                checksum);
+        if (status.ok() && !downloaded_path.empty()) {
+            *result_url = "file://" + downloaded_path;
+            return Status::OK();
+        } else {
+            LOG(WARNING) << "Failed to download Java UDF from cloud: " << 
status.to_string();
+            return Status::RuntimeError(
+                    "Can't download Java UDF from cloud: {}. "

Review Comment:
   The error message uses a contraction "Can't" which should be "Cannot" for 
consistency with professional error messaging standards.
   ```suggestion
                       "Cannot download Java UDF from cloud: {}. "
   ```



##########
be/src/runtime/plugin/cloud_plugin_config_provider.cpp:
##########
@@ -0,0 +1,125 @@
+// 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.
+
+#include "runtime/plugin/cloud_plugin_config_provider.h"
+
+#include <fmt/format.h>
+
+#include "cloud/cloud_meta_mgr.h"
+#include "cloud/cloud_storage_engine.h"
+#include "cloud/config.h"
+#include "common/logging.h"
+#include "runtime/exec_env.h"
+#include "util/s3_util.h"
+
+namespace doris {
+
+std::unique_ptr<S3PluginDownloader::S3Config> 
CloudPluginConfigProvider::get_cloud_s3_config() {
+    S3PluginDownloader::S3Config s3_config("", "", "", "", "");
+    Status status = get_default_storage_vault_info(&s3_config);
+    if (!status.ok()) {
+        throw std::runtime_error("Cannot get default storage vault info for 
plugin download: " +
+                                 status.to_string());
+    }
+
+    if (s3_config.bucket.empty() || s3_config.access_key.empty() || 
s3_config.secret_key.empty()) {
+        throw std::runtime_error(
+                "Incomplete S3 configuration: bucket=" + s3_config.bucket +
+                ", access_key=" + (s3_config.access_key.empty() ? "empty" : 
"***") +
+                ", secret_key=" + (s3_config.secret_key.empty() ? "empty" : 
"***"));
+    }
+
+    return std::make_unique<S3PluginDownloader::S3Config>(s3_config.endpoint, 
s3_config.region,
+                                                          s3_config.bucket, 
s3_config.access_key,
+                                                          
s3_config.secret_key);
+}
+
+std::string CloudPluginConfigProvider::get_cloud_instance_id() {
+    if (config::cluster_id == -1) {

Review Comment:
   The magic number -1 should be defined as a named constant to improve code 
readability and maintainability.
   ```suggestion
       if (config::cluster_id == INVALID_CLUSTER_ID) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/common/plugin/CloudPluginConfigProvider.java:
##########
@@ -0,0 +1,140 @@
+// 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.doris.common.plugin;
+
+import org.apache.doris.catalog.Env;
+import org.apache.doris.cloud.catalog.CloudEnv;
+import org.apache.doris.cloud.proto.Cloud;
+import org.apache.doris.cloud.rpc.MetaServiceProxy;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.Pair;
+import org.apache.doris.rpc.RpcException;
+
+import com.google.common.base.Strings;
+
+/**
+ * CloudPluginConfigProvider retrieves S3 authentication info and plugin base 
paths
+ * from Doris cloud mode environment.
+ * <p>
+ * Responsibilities:
+ * - Configuration retrieval only, no download logic
+ * - Converts complex cloud mode configuration to simple S3 config objects
+ * - Provides unified configuration interface, hiding implementation details
+ * <p>
+ * Uses StorageVaultMgr to get default storage configuration, more stable and 
reliable.
+ */
+public class CloudPluginConfigProvider {
+
+    /**
+     * Get S3 configuration from cloud mode
+     *
+     * @return S3 config object
+     * @throws RuntimeException if configuration retrieval fails
+     */
+    public static S3PluginDownloader.S3Config getCloudS3Config() {
+        Cloud.ObjectStoreInfoPB objInfo = getDefaultStorageVaultInfo();
+
+        if (Strings.isNullOrEmpty(objInfo.getBucket())
+                || Strings.isNullOrEmpty(objInfo.getAk())
+                || Strings.isNullOrEmpty(objInfo.getSk())) {
+            throw new RuntimeException("Incomplete S3 configuration: bucket=" 
+ objInfo.getBucket()
+                    + ", ak=" + (Strings.isNullOrEmpty(objInfo.getAk()) ? 
"empty" : "***")
+                    + ", sk=" + (Strings.isNullOrEmpty(objInfo.getSk()) ? 
"empty" : "***"));
+        }
+
+        return new S3PluginDownloader.S3Config(
+                objInfo.getEndpoint(),
+                objInfo.getRegion(),
+                objInfo.getBucket(),
+                objInfo.getAk(),
+                objInfo.getSk()
+        );
+    }
+
+    /**
+     * Get cloud instance ID - exposed for direct path construction
+     *
+     * @return cloud instance ID
+     */
+    public static String getCloudInstanceId() {
+        // Get from CloudEnv first
+        if (Env.getCurrentEnv() instanceof CloudEnv) {
+            String instanceId = ((CloudEnv) 
Env.getCurrentEnv()).getCloudInstanceId();
+            if (!Strings.isNullOrEmpty(instanceId)) {
+                return instanceId;
+            } else {
+                throw new RuntimeException("CloudEnv instance ID is null or 
empty");
+            }
+        } else {
+            throw new RuntimeException("CloudEnv instance ID is null or 
empty");

Review Comment:
   The code throws RuntimeException when CloudEnv instance ID is null, but the 
condition checks the same thing twice in the if-else chain, making the else 
branch unreachable.
   ```suggestion
               throw new RuntimeException("Current environment is not an 
instance of CloudEnv");
   ```



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to