From 7ece580fcf18422f3049801c48c69822c5fcba1a Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Thu, 22 May 2025 21:17:43 +0800 Subject: [PATCH 1/5] branch-3.1: [feat](refactor-param)Integrate New Storage System Support for BACKUP/RESTORE/LOAD/TVF (#50849) #50849 --- fe/fe-core/pom.xml | 4 + .../org/apache/doris/analysis/BrokerDesc.java | 63 ++-- .../org/apache/doris/analysis/LoadStmt.java | 116 +------ .../apache/doris/analysis/OutFileClause.java | 116 +++---- .../apache/doris/analysis/StorageBackend.java | 82 ++--- .../apache/doris/analysis/StorageDesc.java | 26 ++ .../apache/doris/backup/BackupHandler.java | 196 +++++++---- .../org/apache/doris/backup/BackupJob.java | 5 +- .../org/apache/doris/backup/Repository.java | 156 +++++---- .../apache/doris/backup/RepositoryMgr.java | 7 +- .../org/apache/doris/backup/RestoreJob.java | 88 +++-- .../doris/common/util/BrokerReader.java | 4 +- .../apache/doris/common/util/BrokerUtil.java | 24 +- .../doris/datasource/FileQueryScanNode.java | 3 + .../doris/datasource/LoadScanProvider.java | 5 +- .../property/ConnectionProperties.java | 10 +- .../metastore/AliyunDLFProperties.java | 3 +- .../property/metastore/HMSProperties.java | 3 +- .../AbstractS3CompatibleProperties.java | 13 +- .../property/storage/AzureProperties.java | 2 +- .../property/storage/BrokerProperties.java | 48 +++ .../property/storage/COSProperties.java | 14 +- .../property/storage/HdfsProperties.java | 27 +- .../property/storage/HdfsPropertiesUtils.java | 135 ++++---- .../property/storage/MinioProperties.java | 78 +++++ .../property/storage/OBSProperties.java | 17 +- .../property/storage/OSSHdfsProperties.java | 2 +- .../property/storage/OSSProperties.java | 14 +- .../property/storage/S3Properties.java | 21 +- .../property/storage/S3PropertyUtils.java | 63 +++- .../property/storage/StorageProperties.java | 12 +- .../exception/StoragePropertiesException.java | 47 +++ .../datasource/tvf/source/TVFScanNode.java | 2 +- .../apache/doris/fsv2/FileSystemFactory.java | 29 ++ .../apache/doris/fsv2/StorageTypeMapper.java | 2 + .../apache/doris/fsv2/obj/S3ObjStorage.java | 130 ++++--- .../doris/fsv2/remote/BrokerFileSystem.java | 3 + .../doris/fsv2/remote/RemoteFileSystem.java | 2 +- .../doris/fsv2/remote/dfs/DFSFileSystem.java | 3 + .../java/org/apache/doris/load/ExportJob.java | 2 +- .../doris/load/loadv2/SparkLoadJob.java | 2 +- .../trees/plans/commands/LoadCommand.java | 5 +- .../org/apache/doris/planner/ExportSink.java | 2 +- .../java/org/apache/doris/qe/DdlExecutor.java | 4 +- .../ExternalFileTableValuedFunction.java | 22 +- .../GroupCommitTableValuedFunction.java | 2 +- .../HdfsTableValuedFunction.java | 38 +-- .../HttpStreamTableValuedFunction.java | 2 +- .../LocalTableValuedFunction.java | 2 +- .../tablefunction/S3TableValuedFunction.java | 128 +------ .../apache/doris/backup/BackupJobTest.java | 9 +- .../apache/doris/backup/RepositoryTest.java | 49 +-- .../apache/doris/backup/RestoreJobTest.java | 11 +- .../property/PropertyConverterTest.java | 16 +- .../property/PropertyPassThroughTest.java | 6 +- .../property/storage/AzurePropertiesTest.java | 7 +- .../property/storage/COSPropertiesTest.java | 3 +- .../property/storage/HdfsPropertiesTest.java | 9 +- .../storage/HdfsPropertiesUtilsTest.java | 23 +- .../property/storage/MinioPropertiesTest.java | 99 ++++++ .../property/storage/OBSPropertyTest.java | 4 +- .../storage/OSSHdfsPropertiesTest.java | 2 +- .../property/storage/OSSPropertiesTest.java | 6 +- .../property/storage/S3PropertiesTest.java | 3 +- .../property/storage/S3PropertyUtilsTest.java | 17 +- .../{fs => fsv2}/obj/S3FileSystemTest.java | 22 +- fe/pom.xml | 6 + regression-test/conf/regression-conf.groovy | 1 + .../test_outfile_s3_storage.out | 85 +++++ .../external/conf/regression-conf.groovy | 2 + .../backup_restore_azure.groovy | 182 ++++++++++ .../backup_restore_object_storage.groovy | 300 ++++++++++++++++ .../hdfs_all_test.groovy | 319 ++++++++++++++++++ .../refactor_storage_param/s3_load.groovy | 285 ++++++++++++++++ .../test_outfile_s3_storage.groovy | 218 ++++++++++++ .../test_s3_tvf_s3_storage.groovy | 265 +++++++++++++++ .../tvf/test_hdfs_tvf.groovy | 4 +- ...ain_connection_and_ak_sk_correction.groovy | 38 ++- 78 files changed, 2920 insertions(+), 855 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/BrokerProperties.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/MinioProperties.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/exception/StoragePropertiesException.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/MinioPropertiesTest.java rename fe/fe-core/src/test/java/org/apache/doris/{fs => fsv2}/obj/S3FileSystemTest.java (92%) create mode 100644 regression-test/data/external_table_p0/refactor_storage_param/test_outfile_s3_storage.out create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_azure.groovy create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_object_storage.groovy create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/hdfs_all_test.groovy create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/test_outfile_s3_storage.groovy create mode 100644 regression-test/suites/external_table_p0/refactor_storage_param/test_s3_tvf_s3_storage.groovy diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml index 083dd9ee751e6c..c2321f5f6c0c10 100644 --- a/fe/fe-core/pom.xml +++ b/fe/fe-core/pom.xml @@ -102,6 +102,10 @@ under the License. commons-pool commons-pool + + com.google.re2j + re2j + org.apache.commons commons-text diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java index 4755159379f5fe..850f321f4b4413 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java @@ -20,17 +20,19 @@ import org.apache.doris.analysis.StorageBackend.StorageType; import org.apache.doris.catalog.Env; import org.apache.doris.common.FeMetaVersion; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.PrintableMap; -import org.apache.doris.datasource.property.S3ClientBEProperties; -import org.apache.doris.datasource.property.constants.BosProperties; -import org.apache.doris.fs.PersistentFileSystem; +import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.thrift.TFileType; import com.google.common.collect.Maps; import com.google.gson.annotations.SerializedName; +import org.apache.commons.collections.MapUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -53,6 +55,7 @@ public class BrokerDesc extends StorageDesc implements Writable { // just for multi load public static final String MULTI_LOAD_BROKER = "__DORIS_MULTI_LOAD_BROKER__"; public static final String MULTI_LOAD_BROKER_BACKEND_KEY = "__DORIS_MULTI_LOAD_BROKER_BACKEND__"; + @Deprecated @SerializedName("cts3") private boolean convertedToS3 = false; @@ -75,34 +78,48 @@ public BrokerDesc(String name, Map properties) { if (properties != null) { this.properties.putAll(properties); } + // Assume the storage type is BROKER by default + // If it's a multi-load broker, override the storage type to LOCAL if (isMultiLoadBroker()) { this.storageType = StorageBackend.StorageType.LOCAL; } else { this.storageType = StorageBackend.StorageType.BROKER; } - this.properties.putAll(S3ClientBEProperties.getBeFSProperties(this.properties)); - this.convertedToS3 = BosProperties.tryConvertBosToS3(this.properties, this.storageType); - if (this.convertedToS3) { - this.storageType = StorageBackend.StorageType.S3; + + // Try to determine the actual storage type from properties if available + if (MapUtils.isNotEmpty(this.properties)) { + try { + // Create primary storage properties from the given configuration + this.storageProperties = StorageProperties.createPrimary(this.properties); + // Override the storage type based on property configuration + this.storageType = StorageBackend.StorageType.valueOf(storageProperties.getStorageName()); + } catch (StoragePropertiesException e) { + // Currently ignored: these properties might be broker-specific. + // Support for broker properties will be added in the future. + LOG.info("Failed to create storage properties for broker: {}, properties: {}", name, properties, e); + } + } + //only storage type is broker + if (StringUtils.isBlank(this.name) && (this.getStorageType() != StorageType.BROKER)) { + this.name = this.storageType().name(); } } public BrokerDesc(String name, StorageBackend.StorageType storageType, Map properties) { this.name = name; this.properties = Maps.newHashMap(); + this.storageType = storageType; if (properties != null) { this.properties.putAll(properties); } - this.storageType = storageType; - this.properties.putAll(S3ClientBEProperties.getBeFSProperties(this.properties)); - this.convertedToS3 = BosProperties.tryConvertBosToS3(this.properties, this.storageType); - if (this.convertedToS3) { - this.storageType = StorageBackend.StorageType.S3; + if (MapUtils.isNotEmpty(this.properties) && StorageType.REFACTOR_STORAGE_TYPES.contains(storageType)) { + this.storageProperties = StorageProperties.createPrimary(properties); } + } - public String getFileLocation(String location) { - return this.convertedToS3 ? BosProperties.convertPathToS3(location) : location; + public String getFileLocation(String location) throws UserException { + return (null != storageProperties) ? storageProperties.validateAndNormalizeUri(location) : location; } public static BrokerDesc createForStreamLoad() { @@ -110,7 +127,7 @@ public static BrokerDesc createForStreamLoad() { } public boolean isMultiLoadBroker() { - return this.name.equalsIgnoreCase(MULTI_LOAD_BROKER); + return StringUtils.isNotBlank(this.name) && this.name.equalsIgnoreCase(MULTI_LOAD_BROKER); } public TFileType getFileType() { @@ -150,16 +167,18 @@ public void readFields(DataInput in) throws IOException { final String val = Text.readString(in); properties.put(key, val); } - StorageBackend.StorageType st = StorageBackend.StorageType.BROKER; - String typeStr = properties.remove(PersistentFileSystem.STORAGE_TYPE); - if (typeStr != null) { + if (MapUtils.isNotEmpty(properties)) { try { - st = StorageBackend.StorageType.valueOf(typeStr); - } catch (IllegalArgumentException e) { - LOG.warn("set to BROKER, because of exception", e); + this.storageProperties = StorageProperties.createPrimary(properties); + this.storageType = StorageBackend.StorageType.valueOf(storageProperties.getStorageName()); + } catch (RuntimeException e) { + // Currently ignored: these properties might be broker-specific. + // Support for broker properties will be added in the future. + LOG.warn("Failed to create storage properties for broker: {}, properties: {}", name, properties, e); + this.storageType = StorageBackend.StorageType.BROKER; } + } - storageType = st; } public static BrokerDesc read(DataInput in) throws IOException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java index 6abe6fdc860425..ccd35cdd91f0f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LoadStmt.java @@ -21,10 +21,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; -import org.apache.doris.cloud.proto.Cloud.ObjectStoreInfoPB; import org.apache.doris.cloud.security.SecurityChecker; -import org.apache.doris.cloud.storage.RemoteBase; -import org.apache.doris.cloud.storage.RemoteBase.ObjectInfo; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; @@ -32,18 +29,18 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.PrintableMap; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.datasource.property.constants.AzureProperties; -import org.apache.doris.datasource.property.constants.S3Properties; +import org.apache.doris.datasource.property.storage.ObjectStorageProperties; +import org.apache.doris.fsv2.FileSystemFactory; import org.apache.doris.load.EtlJobType; import org.apache.doris.load.loadv2.LoadTask; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.TFileType; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import org.apache.commons.lang3.tuple.Pair; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.checkerframework.checker.nullness.qual.Nullable; @@ -108,10 +105,6 @@ public class LoadStmt extends DdlStmt implements NotFallbackInParser { public static final String BOS_ACCESSKEY = "bos_accesskey"; public static final String BOS_SECRET_ACCESSKEY = "bos_secret_accesskey"; - // for S3 load check - public static final List PROVIDERS = - new ArrayList<>(Arrays.asList("cos", "oss", "s3", "obs", "bos", "azure")); - // mini load params public static final String KEY_IN_PARAM_COLUMNS = "columns"; public static final String KEY_IN_PARAM_SET = "set"; @@ -455,8 +448,6 @@ public void analyze(Analyzer analyzer) throws UserException { for (int i = 0; i < dataDescription.getFilePaths().size(); i++) { String location = brokerDesc.getFileLocation(dataDescription.getFilePaths().get(i)); dataDescription.getFilePaths().set(i, location); - StorageBackend.checkPath(dataDescription.getFilePaths().get(i), - brokerDesc.getStorageType(), "DATA INFILE must be specified."); dataDescription.getFilePaths().set(i, dataDescription.getFilePaths().get(i)); } } @@ -523,31 +514,6 @@ public void analyze(Analyzer analyzer) throws UserException { user = ConnectContext.get().getQualifiedUser(); } - private String getProviderFromEndpoint() { - Map properties = brokerDesc.getProperties(); - for (Map.Entry entry : properties.entrySet()) { - if (entry.getKey().equalsIgnoreCase(S3Properties.PROVIDER)) { - // S3 Provider properties should be case insensitive. - return entry.getValue().toUpperCase(); - } - } - return S3Properties.S3_PROVIDER; - } - - private Pair getBucketAndObjectFromPath(String filePath) throws UserException { - String[] parts = filePath.split("\\/\\/"); - if (parts.length < 2) { - throw new UserException("Invalid file path format: " + filePath); - } - - String[] bucketAndObject = parts[1].split("\\/", 2); - if (bucketAndObject.length < 2) { - throw new UserException("Cannot extract bucket and object from path: " + filePath); - } - - return Pair.of(bucketAndObject[0], bucketAndObject[1]); - } - public String getComment() { return comment; } @@ -606,8 +572,9 @@ public RedirectStatus getRedirectStatus() { private void checkEndpoint(String endpoint) throws UserException { HttpURLConnection connection = null; try { - SecurityChecker.getInstance().startSSRFChecking(endpoint); - URL url = new URL(endpoint); + String urlStr = "http://" + endpoint; + SecurityChecker.getInstance().startSSRFChecking(urlStr); + URL url = new URL(urlStr); connection = (HttpURLConnection) url.openConnection(); connection.setConnectTimeout(10000); connection.connect(); @@ -634,29 +601,21 @@ private void checkEndpoint(String endpoint) throws UserException { } public void checkS3Param() throws UserException { - Map brokerDescProperties = brokerDesc.getProperties(); - if (brokerDescProperties.containsKey(S3Properties.Env.ENDPOINT) - && brokerDescProperties.containsKey(S3Properties.Env.ACCESS_KEY) - && brokerDescProperties.containsKey(S3Properties.Env.SECRET_KEY) - && brokerDescProperties.containsKey(S3Properties.Env.REGION)) { - String endpoint = brokerDescProperties.get(S3Properties.Env.ENDPOINT); + if (brokerDesc.getFileType() != null && brokerDesc.getFileType().equals(TFileType.FILE_S3)) { + + ObjectStorageProperties storageProperties = (ObjectStorageProperties) brokerDesc.getStorageProperties(); + String endpoint = storageProperties.getEndpoint(); + checkEndpoint(endpoint); checkWhiteList(endpoint); - // Add default protocol if not specified - if (!endpoint.startsWith("http://") && !endpoint.startsWith("https://")) { - endpoint = "http://" + endpoint; + //should add connectivity test + boolean connectivityTest = FileSystemFactory.get(brokerDesc.getStorageProperties()).connectivityTest(); + if (!connectivityTest) { + throw new UserException("Failed to access object storage, message=connectivity test failed"); } - brokerDescProperties.put(S3Properties.Env.ENDPOINT, endpoint); - if (AzureProperties.checkAzureProviderPropertyExist(brokerDescProperties)) { - return; - } - checkEndpoint(endpoint); - checkAkSk(); } } public void checkWhiteList(String endpoint) throws UserException { - endpoint = endpoint.replaceFirst("^http://", ""); - endpoint = endpoint.replaceFirst("^https://", ""); List whiteList = new ArrayList<>(Arrays.asList(Config.s3_load_endpoint_white_list)); whiteList.removeIf(String::isEmpty); if (!whiteList.isEmpty() && !whiteList.contains(endpoint)) { @@ -665,51 +624,6 @@ public void checkWhiteList(String endpoint) throws UserException { } } - private void checkAkSk() throws UserException { - RemoteBase remote = null; - ObjectInfo objectInfo = null; - String curFile = null; - try { - Map brokerDescProperties = brokerDesc.getProperties(); - String provider = getProviderFromEndpoint(); - for (DataDescription dataDescription : dataDescriptions) { - for (String filePath : dataDescription.getFilePaths()) { - curFile = filePath; - Pair pair = getBucketAndObjectFromPath(filePath); - String bucket = pair.getLeft(); - String object = pair.getRight(); - objectInfo = new ObjectInfo(ObjectStoreInfoPB.Provider.valueOf(provider.toUpperCase()), - brokerDescProperties.get(S3Properties.Env.ACCESS_KEY), - brokerDescProperties.get(S3Properties.Env.SECRET_KEY), - bucket, brokerDescProperties.get(S3Properties.Env.ENDPOINT), - brokerDescProperties.get(S3Properties.Env.REGION), ""); - remote = RemoteBase.newInstance(objectInfo); - // Verify read permissions by calling headObject() on the S3 object. - // RemoteBase#headObject does not throw exception if key does not exist. - remote.headObject(object); - // Verify list permissions by calling listObjects() on the S3 bucket. - remote.listObjects(null); - remote.close(); - } - } - } catch (Exception e) { - LOG.warn("Failed to access object storage, file={}, proto={}, err={}", curFile, objectInfo, e.toString()); - String msg; - if (e instanceof UserException) { - msg = ((UserException) e).getDetailMessage(); - } else { - msg = e.getMessage(); - } - throw new UserException(InternalErrorCode.GET_REMOTE_DATA_ERROR, - "Failed to access object storage, message=" + msg, e); - } finally { - if (remote != null) { - remote.close(); - } - } - - } - @Override public StmtType stmtType() { return StmtType.LOAD; diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java index bb4e9f81328fe9..8211f84a21c9c1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/OutFileClause.java @@ -32,10 +32,10 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.ParseUtil; import org.apache.doris.common.util.PrintableMap; -import org.apache.doris.datasource.property.PropertyConverter; -import org.apache.doris.datasource.property.constants.S3Properties; import org.apache.doris.datasource.property.fileformat.CsvFileFormatProperties; import org.apache.doris.datasource.property.fileformat.FileFormatProperties; +import org.apache.doris.datasource.property.storage.HdfsProperties; +import org.apache.doris.datasource.property.storage.HdfsPropertiesUtils; import org.apache.doris.thrift.TFileFormatType; import org.apache.doris.thrift.TParquetDataType; import org.apache.doris.thrift.TParquetRepetitionType; @@ -47,14 +47,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import org.apache.hadoop.fs.Path; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -114,10 +112,7 @@ public class OutFileClause { } public static final String LOCAL_FILE_PREFIX = "file:///"; - private static final String S3_FILE_PREFIX = "S3://"; private static final String HDFS_FILE_PREFIX = "hdfs://"; - private static final String HADOOP_FS_PROP_PREFIX = "dfs."; - private static final String HADOOP_PROP_PREFIX = "hadoop."; private static final String BROKER_PROP_PREFIX = "broker."; public static final String PROP_BROKER_NAME = "broker.name"; public static final String PROP_COLUMN_SEPARATOR = "column_separator"; @@ -520,7 +515,7 @@ private void analyzeProperties() throws UserException { if (copiedProps.containsKey(PROP_DELETE_EXISTING_FILES)) { deleteExistingFiles = Boolean.parseBoolean(copiedProps.get(PROP_DELETE_EXISTING_FILES)) - & Config.enable_delete_existing_files; + & Config.enable_delete_existing_files; copiedProps.remove(PROP_DELETE_EXISTING_FILES); } @@ -540,12 +535,6 @@ private void analyzeProperties() throws UserException { copiedProps.remove(PROP_SUCCESS_FILE_NAME); } - // For Azure compatibility, this is temporarily added to the map without further processing. - // The validity of each provider's value will be checked later in S3Properties' check. - if (copiedProps.containsKey(S3Properties.PROVIDER)) { - copiedProps.remove(S3Properties.PROVIDER); - } - if (fileFormatProperties.getFileFormatType() == TFileFormatType.FORMAT_PARQUET) { getParquetProperties(copiedProps); } @@ -553,10 +542,6 @@ private void analyzeProperties() throws UserException { if (fileFormatProperties.getFileFormatType() == TFileFormatType.FORMAT_ORC) { getOrcProperties(copiedProps); } - - if (!copiedProps.isEmpty()) { - throw new AnalysisException("Unknown properties: " + copiedProps.keySet()); - } } /** @@ -565,63 +550,48 @@ private void analyzeProperties() throws UserException { * 2. s3: with s3 pattern path, without broker name */ private void analyzeBrokerDesc(Map copiedProps) throws UserException { - String brokerName = copiedProps.get(PROP_BROKER_NAME); - StorageBackend.StorageType storageType; - if (copiedProps.containsKey(PROP_BROKER_NAME)) { - copiedProps.remove(PROP_BROKER_NAME); - storageType = StorageBackend.StorageType.BROKER; - } else if (filePath.toUpperCase().startsWith(S3_FILE_PREFIX)) { - brokerName = StorageBackend.StorageType.S3.name(); - storageType = StorageBackend.StorageType.S3; - } else if (filePath.toUpperCase().startsWith(HDFS_FILE_PREFIX.toUpperCase())) { - brokerName = StorageBackend.StorageType.HDFS.name(); - storageType = StorageBackend.StorageType.HDFS; - } else { + /** + * If the output is intended to be written to the local file system, skip BrokerDesc analysis. + * This is because Broker properties are not required when writing files locally, + * and the upper layer logic ensures that brokerDesc must be null in this case. + */ + if (isLocalOutput) { return; } - - Map brokerProps = Maps.newHashMap(); - Iterator> iterator = copiedProps.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - if (entry.getKey().startsWith(BROKER_PROP_PREFIX) && !entry.getKey().equals(PROP_BROKER_NAME)) { - brokerProps.put(entry.getKey().substring(BROKER_PROP_PREFIX.length()), entry.getValue()); - iterator.remove(); - } else if (entry.getKey().toLowerCase().startsWith(S3Properties.S3_PREFIX) - || entry.getKey().toUpperCase().startsWith(S3Properties.Env.PROPERTIES_PREFIX)) { - brokerProps.put(entry.getKey(), entry.getValue()); - iterator.remove(); - } else if (entry.getKey().contains(HdfsResource.HADOOP_FS_NAME) - && storageType == StorageBackend.StorageType.HDFS) { - brokerProps.put(entry.getKey(), entry.getValue()); - iterator.remove(); - } else if ((entry.getKey().startsWith(HADOOP_FS_PROP_PREFIX) - || entry.getKey().startsWith(HADOOP_PROP_PREFIX)) - && storageType == StorageBackend.StorageType.HDFS) { - brokerProps.put(entry.getKey(), entry.getValue()); - iterator.remove(); - } - } - - if (storageType == StorageBackend.StorageType.S3) { - if (copiedProps.containsKey(PropertyConverter.USE_PATH_STYLE)) { - brokerProps.put(PropertyConverter.USE_PATH_STYLE, copiedProps.get(PropertyConverter.USE_PATH_STYLE)); - copiedProps.remove(PropertyConverter.USE_PATH_STYLE); - } - S3Properties.requiredS3Properties(brokerProps); - } else if (storageType == StorageBackend.StorageType.HDFS) { - if (!brokerProps.containsKey(HdfsResource.HADOOP_FS_NAME)) { - brokerProps.put(HdfsResource.HADOOP_FS_NAME, getFsName(filePath)); - } + String brokerName = copiedProps.get(PROP_BROKER_NAME); + brokerDesc = new BrokerDesc(brokerName, copiedProps); + /* + * Note on HDFS export behavior and URI handling: + * + * 1. Currently, URI extraction from user input supports case-insensitive key matching + * (e.g., "URI", "Uri", "uRI", etc.), to tolerate non-standard input from users. + * + * 2. In OUTFILE scenarios, if FE fails to pass 'fs.defaultFS' to the BE , + * it may lead to data export failure *without* triggering an actual error (appears as success), + * which is misleading and can cause silent data loss or inconsistencies. + * + * 3. As a temporary safeguard, the following logic forcibly extracts the default FS from the provided + * file path and injects it into the broker descriptor config: + * + * if (brokerDesc.getStorageType() == HDFS) { + * extract default FS from file path + * and put into BE config as 'fs.defaultFS' + * } + * + * 4. Long-term solution: We should define and enforce a consistent parameter specification + * across all user-facing entry points (including FE input validation, broker desc normalization, etc.), + * to prevent missing critical configs like 'fs.defaultFS'. + * + * 5. Suggested improvements: + * - Normalize all parameter keys (e.g., to lowercase) + * - Centralize HDFS URI parsing logic + * - Add validation in FE to reject incomplete or malformed configs + */ + if (null != brokerDesc.getStorageType() && brokerDesc.getStorageType() + .equals(StorageBackend.StorageType.HDFS)) { + String defaultFs = HdfsPropertiesUtils.extractDefaultFsFromPath(filePath); + brokerDesc.getBackendConfigProperties().put(HdfsProperties.HDFS_DEFAULT_FS_NAME, defaultFs); } - brokerDesc = new BrokerDesc(brokerName, storageType, brokerProps); - } - - public static String getFsName(String path) { - Path hdfsPath = new Path(path); - String fullPath = hdfsPath.toUri().toString(); - String filePath = hdfsPath.toUri().getPath(); - return fullPath.replace(filePath, ""); } /** @@ -759,7 +729,7 @@ public TResultFileSinkOptions toSinkOptions() { sinkOptions.setWithBom(withBom); if (brokerDesc != null) { - sinkOptions.setBrokerProperties(brokerDesc.getProperties()); + sinkOptions.setBrokerProperties(brokerDesc.getBackendConfigProperties()); // broker_addresses of sinkOptions will be set in Coordinator. // Because we need to choose the nearest broker with the result sink node. } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java index 67a76cec450e36..9fa2a1203bfb20 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java @@ -22,15 +22,15 @@ import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.PrintableMap; -import org.apache.doris.common.util.URI; -import org.apache.doris.datasource.property.constants.BosProperties; import org.apache.doris.thrift.TStorageBackendType; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; import com.google.gson.annotations.SerializedName; import org.apache.commons.lang3.StringUtils; import java.util.Map; +import java.util.Set; public class StorageBackend implements ParseNode { private String location; @@ -41,58 +41,21 @@ public static void checkPath(String path, StorageBackend.StorageType type, Strin if (Strings.isNullOrEmpty(path)) { throw new AnalysisException(exceptionMsg == null ? "No destination path specified." : exceptionMsg); } - checkUri(URI.create(path), type); + //checkUri(URI.create(path), type); } - public static void checkUri(URI uri, StorageBackend.StorageType type) throws AnalysisException { - String schema = uri.getScheme(); - if (schema == null) { - throw new AnalysisException( - "Invalid export path, there is no schema of URI found. please check your path."); - } - if (type == StorageBackend.StorageType.BROKER) { - if (!schema.equalsIgnoreCase("bos") - && !schema.equalsIgnoreCase("afs") - && !schema.equalsIgnoreCase("hdfs") - && !schema.equalsIgnoreCase("viewfs") - && !schema.equalsIgnoreCase("ofs") - && !schema.equalsIgnoreCase("obs") - && !schema.equalsIgnoreCase("oss") - && !schema.equalsIgnoreCase("s3a") - && !schema.equalsIgnoreCase("cosn") - && !schema.equalsIgnoreCase("gfs") - && !schema.equalsIgnoreCase("jfs") - && !schema.equalsIgnoreCase("azure") - && !schema.equalsIgnoreCase("gs")) { - throw new AnalysisException( - "Invalid broker path " + uri.toString() + ". please use valid 'hdfs://', 'viewfs://', 'afs://'," - + " 'bos://', 'ofs://', 'obs://', 'oss://', 's3a://', 'cosn://', 'gfs://', 'gs://'" - + " or 'jfs://' path."); - } - } else if (type == StorageBackend.StorageType.S3 && !schema.equalsIgnoreCase("s3")) { - throw new AnalysisException("Invalid export path " + uri.toString() + ". please use valid 's3://' path."); - } else if (type == StorageBackend.StorageType.AZURE && !schema.equalsIgnoreCase("azure")) { - throw new AnalysisException("Invalid export path. please use valid 'azure://' path."); - } else if (type == StorageBackend.StorageType.HDFS && !schema.equalsIgnoreCase("hdfs") - && !schema.equalsIgnoreCase("viewfs")) { - throw new AnalysisException("Invalid export path. please use valid 'HDFS://' or 'viewfs://' path."); - } else if (type == StorageBackend.StorageType.LOCAL && !schema.equalsIgnoreCase("file")) { - throw new AnalysisException( - "Invalid export path. please use valid '" + OutFileClause.LOCAL_FILE_PREFIX + "' path."); - } - } public StorageBackend(String storageName, String location, - StorageType storageType, Map properties) { + StorageType storageType, Map properties) { this.storageDesc = new StorageDesc(storageName, storageType, properties); this.location = location; - boolean convertedToS3 = BosProperties.tryConvertBosToS3(properties, storageType); + /*boolean convertedToS3 = BosProperties.tryConvertBosToS3(properties, storageType); if (convertedToS3) { this.storageDesc.setStorageType(StorageBackend.StorageType.S3); this.location = BosProperties.convertPathToS3(location); } else { this.location = location; - } + }*/ } public void setStorageDesc(StorageDesc storageDesc) { @@ -138,8 +101,8 @@ public String toSql() { sb.append(" `").append(storageDesc.getName()).append("`"); } sb.append(" ON LOCATION ").append(location).append(" PROPERTIES(") - .append(new PrintableMap<>(storageDesc.getProperties(), " = ", true, false, true)) - .append(")"); + .append(new PrintableMap<>(storageDesc.getProperties(), " = ", true, false, true)) + .append(")"); return sb.toString(); } @@ -185,6 +148,35 @@ public TStorageBackendType toThrift() { return TStorageBackendType.BROKER; } } + + /** + * A set of storage types that currently support parameter refactoring. + *

+ * Includes: S3 (referring to all systems compatible with the S3 protocol), + * HDFS, OFS, JFS, and AZURE. For S3, this is a generalized type that matches + * any system whose storage type name is returned as "s3" (or compatible) + * by {@link org.apache.doris.datasource.property.storage.StorageProperties#getStorageName()}. + *

+ * This set is a temporary solution. Once parameter refactoring is fully supported + * across all storage systems, this class can be removed. + */ + public static final Set REFACTOR_STORAGE_TYPES = + ImmutableSet.of(StorageType.S3, StorageType.HDFS, StorageType.OFS, StorageType.JFS, StorageType.AZURE); + + public static StorageType convertToStorageType(String storageName) { + switch (storageName.toLowerCase()) { + case "hdfs": + return StorageType.HDFS; + case "s3": + return StorageType.S3; + case "jfs": + return StorageType.JFS; + case "local": + return StorageType.LOCAL; + default: + throw new IllegalArgumentException("Invalid storage type: " + storageName); + } + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageDesc.java index 902f5d24b91820..bbfcd8e9d2c45b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageDesc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageDesc.java @@ -17,7 +17,11 @@ package org.apache.doris.analysis; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.StorageProperties; + import com.google.gson.annotations.SerializedName; +import lombok.Getter; import java.util.Map; @@ -32,18 +36,33 @@ * The broker's StorageBackend.StorageType desc */ public class StorageDesc extends ResourceDesc { + + @Deprecated @SerializedName("st") protected StorageBackend.StorageType storageType; + @Getter + protected StorageProperties storageProperties; + public StorageDesc() { } public StorageDesc(String name, StorageBackend.StorageType storageType, Map properties) { this.name = name; this.storageType = storageType; + if (!storageType.equals(StorageBackend.StorageType.BROKER)) { + this.storageProperties = StorageProperties.createPrimary(properties); + } this.properties = properties; } + public StorageDesc(String name, Map properties) throws UserException { + this.name = name; + this.properties = properties; + this.storageProperties = StorageProperties.createPrimary(properties); + this.storageType = StorageBackend.StorageType.convertToStorageType(storageProperties.getStorageName()); + } + public void setName(String name) { this.name = name; } @@ -67,4 +86,11 @@ public StorageBackend.StorageType getStorageType() { public Map getProperties() { return properties; } + + public Map getBackendConfigProperties() { + if (null == storageProperties) { + return properties; + } + return storageProperties.getBackendConfigProperties(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index 1f8f48370a7eac..dbf197f9fe9254 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -19,7 +19,6 @@ import org.apache.doris.analysis.AbstractBackupStmt; import org.apache.doris.analysis.AbstractBackupTableRefClause; -import org.apache.doris.analysis.AlterRepositoryStmt; import org.apache.doris.analysis.BackupStmt; import org.apache.doris.analysis.BackupStmt.BackupType; import org.apache.doris.analysis.CancelBackupStmt; @@ -45,15 +44,17 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.DebugPointUtil; import org.apache.doris.common.util.MasterDaemon; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.common.util.Util; -import org.apache.doris.fs.FileSystemFactory; -import org.apache.doris.fs.remote.AzureFileSystem; -import org.apache.doris.fs.remote.RemoteFileSystem; -import org.apache.doris.fs.remote.S3FileSystem; +import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fsv2.FileSystemFactory; +import org.apache.doris.fsv2.remote.AzureFileSystem; +import org.apache.doris.fsv2.remote.RemoteFileSystem; +import org.apache.doris.fsv2.remote.S3FileSystem; import org.apache.doris.persist.BarrierLog; import org.apache.doris.task.DirMoveTask; import org.apache.doris.task.DownloadTask; @@ -214,10 +215,18 @@ public void createRepository(CreateRepositoryStmt stmt) throws DdlException { "broker does not exist: " + stmt.getBrokerName()); } - RemoteFileSystem fileSystem = FileSystemFactory.get(stmt.getBrokerName(), stmt.getStorageType(), - stmt.getProperties()); + RemoteFileSystem fileSystem; + try { + fileSystem = FileSystemFactory.get(stmt.getStorageType(), stmt.getProperties()); + } catch (UserException e) { + throw new DdlException("Failed to initialize remote file system: " + e.getMessage()); + } + org.apache.doris.fs.remote.RemoteFileSystem oldfs = org.apache.doris.fs.FileSystemFactory + .get(stmt.getBrokerName(), stmt.getStorageType(), + stmt.getProperties()); long repoId = env.getNextId(); - Repository repo = new Repository(repoId, stmt.getName(), stmt.isReadOnly(), stmt.getLocation(), fileSystem); + Repository repo = new Repository(repoId, stmt.getName(), stmt.isReadOnly(), stmt.getLocation(), + fileSystem, oldfs); Status st = repoMgr.addAndInitRepoIfNotExist(repo, false); if (!st.ok()) { @@ -230,62 +239,115 @@ public void createRepository(CreateRepositoryStmt stmt) throws DdlException { } } - public void alterRepository(AlterRepositoryStmt stmt) throws DdlException { + /** + * Alters an existing repository by applying the given new properties. + * + * @param repoName The name of the repository to alter. + * @param newProps The new properties to apply to the repository. + * @param strictCheck If true, only allows altering S3 or Azure repositories and validates properties accordingly. + * TODO: Investigate why only S3 and Azure repositories are supported for alter operation + * @throws DdlException if the repository does not exist, fails to apply properties, or cannot connect + * to the updated repository. + */ + public void alterRepository(String repoName, Map newProps, boolean strictCheck) + throws DdlException { tryLock(); try { - Repository repo = repoMgr.getRepo(stmt.getName()); - if (repo == null) { - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, "Repository does not exist"); + Repository oldRepo = repoMgr.getRepo(repoName); + if (oldRepo == null) { + throw new DdlException("Repository does not exist"); } - - if (repo.getRemoteFileSystem() instanceof S3FileSystem - || repo.getRemoteFileSystem() instanceof AzureFileSystem) { - Map oldProperties = new HashMap<>(stmt.getProperties()); - Status status = repo.alterRepositoryS3Properties(oldProperties); - if (!status.ok()) { - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, status.getErrMsg()); - } - RemoteFileSystem fileSystem = null; - if (repo.getRemoteFileSystem() instanceof S3FileSystem) { - fileSystem = FileSystemFactory.get(repo.getRemoteFileSystem().getName(), - StorageBackend.StorageType.S3, oldProperties); - } else if (repo.getRemoteFileSystem() instanceof AzureFileSystem) { - fileSystem = FileSystemFactory.get(repo.getRemoteFileSystem().getName(), - StorageBackend.StorageType.AZURE, oldProperties); - } - - Repository newRepo = new Repository(repo.getId(), repo.getName(), repo.isReadOnly(), - repo.getLocation(), fileSystem); - if (!newRepo.ping()) { - LOG.warn("Failed to connect repository {}. msg: {}", repo.getName(), repo.getErrorMsg()); - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "Repo can not ping with new s3 properties"); - } - - Status st = repoMgr.alterRepo(newRepo, false /* not replay */); - if (!st.ok()) { - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "Failed to alter repository: " + st.getErrMsg()); - } - for (AbstractJob job : getAllCurrentJobs()) { - if (!job.isDone() && job.getRepoId() == repo.getId()) { - job.updateRepo(newRepo); - } - } - } else { - ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, - "Only support alter s3 or azure repository"); + // Merge new properties with the existing repository's properties + Map mergedProps = mergeProperties(oldRepo, newProps, strictCheck); + // Create new remote file system with merged properties + RemoteFileSystem fileSystem = FileSystemFactory.get(StorageProperties.createPrimary(mergedProps)); + org.apache.doris.fs.remote.RemoteFileSystem oldfs = null; + if (oldRepo.getRemoteFileSystem() instanceof S3FileSystem) { + oldfs = org.apache.doris.fs.FileSystemFactory.get(oldRepo.getRemoteFileSystem().getName(), + StorageBackend.StorageType.S3, mergedProps); + } else if (oldRepo.getRemoteFileSystem() instanceof AzureFileSystem) { + oldfs = org.apache.doris.fs.FileSystemFactory.get(oldRepo.getRemoteFileSystem().getName(), + StorageBackend.StorageType.AZURE, mergedProps); + } + // Create new Repository instance with updated file system + Repository newRepo = new Repository( + oldRepo.getId(), oldRepo.getName(), oldRepo.isReadOnly(), + oldRepo.getLocation(), fileSystem, oldfs + ); + // Verify the repository can be connected with new settings + if (!newRepo.ping()) { + LOG.warn("Failed to connect repository {}. msg: {}", repoName, newRepo.getErrorMsg()); + throw new DdlException("Repository ping failed with new properties"); + } + // Apply the new repository metadata + Status st = repoMgr.alterRepo(newRepo, false /* not replay */); + if (!st.ok()) { + throw new DdlException("Failed to alter repository: " + st.getErrMsg()); } + // Update all running jobs that are using this repository + updateOngoingJobs(oldRepo.getId(), newRepo); } finally { seqlock.unlock(); } } + /** + * Merges new user-provided properties into the existing repository's configuration. + * In strict mode, only supports S3 or Azure repositories and applies internal S3 merge logic. + * + * @param repo The existing repository. + * @param newProps New user-specified properties. + * @param strictCheck Whether to enforce S3/Azure-only and validate the new properties. + * @return A complete set of merged properties. + * @throws DdlException if the merge fails or the repository type is unsupported. + */ + private Map mergeProperties(Repository repo, Map newProps, boolean strictCheck) + throws DdlException { + if (strictCheck) { + if (!(repo.getRemoteFileSystem() instanceof S3FileSystem + || repo.getRemoteFileSystem() instanceof AzureFileSystem)) { + throw new DdlException("Only support altering S3 or Azure repository"); + } + // Let the repository validate and enrich the new S3/Azure properties + Map propsCopy = new HashMap<>(newProps); + Status status = repo.alterRepositoryS3Properties(propsCopy); + if (!status.ok()) { + throw new DdlException("Failed to merge S3 properties: " + status.getErrMsg()); + } + return propsCopy; + } else { + // General case: just override old props with new ones + Map combined = new HashMap<>(repo.getRemoteFileSystem().getProperties()); + combined.putAll(newProps); + return combined; + } + } + + /** + * Updates all currently running jobs associated with the given repository ID. + * Used to ensure that all jobs operate on the new repository instance after alteration. + * + * @param repoId The ID of the altered repository. + * @param newRepo The new repository instance. + */ + private void updateOngoingJobs(long repoId, Repository newRepo) { + for (AbstractJob job : getAllCurrentJobs()) { + if (!job.isDone() && job.getRepoId() == repoId) { + job.updateRepo(newRepo); + } + } + } + // handle drop repository stmt public void dropRepository(DropRepositoryStmt stmt) throws DdlException { + dropRepository(stmt.getRepoName()); + } + + // handle drop repository stmt + public void dropRepository(String repoName) throws DdlException { tryLock(); try { - Repository repo = repoMgr.getRepo(stmt.getRepoName()); + Repository repo = repoMgr.getRepo(repoName); if (repo == null) { ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, "Repository does not exist"); } @@ -576,18 +638,17 @@ private void restore(Repository repository, Database db, RestoreStmt stmt) throw jobInfo.getBackupTime(), TimeUtils.getDatetimeFormatWithHyphenWithTimeZone()); restoreJob = new RestoreJob(stmt.getLabel(), backupTimestamp, db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), - stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), + stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), stmt.reserveColocate(), stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), stmt.isForceReplace(), env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); } else { restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(), - db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), - stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), - stmt.reserveDynamicPartitionEnable(), - stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), - stmt.isForceReplace(), - env, repository.getId()); + db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), + stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), stmt.reserveColocate(), + stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), stmt.isCleanTables(), + stmt.isCleanPartitions(), stmt.isAtomicRestore(), stmt.isForceReplace(), + env, repository.getId()); } env.getEditLog().logRestoreJob(restoreJob); @@ -746,6 +807,25 @@ public void checkAndFilterRestoreOlapTableExistInSnapshot(Map properties) { + if (this.fileSystem instanceof S3FileSystem) { + Map oldProperties = new HashMap<>(this.getRemoteFileSystem().getProperties()); + oldProperties.remove(S3Properties.ACCESS_KEY); + oldProperties.remove(S3Properties.SECRET_KEY); + oldProperties.remove(S3Properties.SESSION_TOKEN); + oldProperties.remove(S3Properties.Env.ACCESS_KEY); + oldProperties.remove(S3Properties.Env.SECRET_KEY); + oldProperties.remove(S3Properties.Env.TOKEN); + for (Map.Entry entry : properties.entrySet()) { + if (Objects.equals(entry.getKey(), S3Properties.ACCESS_KEY) + || Objects.equals(entry.getKey(), S3Properties.Env.ACCESS_KEY)) { + oldProperties.putIfAbsent(S3Properties.ACCESS_KEY, entry.getValue()); + } + if (Objects.equals(entry.getKey(), S3Properties.SECRET_KEY) + || Objects.equals(entry.getKey(), S3Properties.Env.SECRET_KEY)) { + oldProperties.putIfAbsent(S3Properties.SECRET_KEY, entry.getValue()); + } + if (Objects.equals(entry.getKey(), S3Properties.SESSION_TOKEN) + || Objects.equals(entry.getKey(), S3Properties.Env.TOKEN)) { + oldProperties.putIfAbsent(S3Properties.SESSION_TOKEN, entry.getValue()); + } + } + properties.clear(); + properties.putAll(oldProperties); + return Status.OK; + } else { + return new Status(ErrCode.COMMON_ERROR, "Only support alter s3 repository"); + } + } + @Override public void gsonPostProcess() { StorageBackend.StorageType type = StorageBackend.StorageType.BROKER; - if (this.fileSystem.properties.containsKey(PersistentFileSystem.STORAGE_TYPE)) { + if (this.oldfs.properties.containsKey(org.apache.doris.fs.PersistentFileSystem.STORAGE_TYPE)) { type = StorageBackend.StorageType.valueOf( - this.fileSystem.properties.get(PersistentFileSystem.STORAGE_TYPE)); - this.fileSystem.properties.remove(PersistentFileSystem.STORAGE_TYPE); + this.oldfs.properties.get(org.apache.doris.fs.PersistentFileSystem.STORAGE_TYPE)); + this.oldfs.properties.remove(org.apache.doris.fs.PersistentFileSystem.STORAGE_TYPE); } - this.fileSystem = FileSystemFactory.get(this.fileSystem.getName(), + this.oldfs = org.apache.doris.fs.FileSystemFactory.get(this.oldfs.getName(), type, - this.fileSystem.getProperties()); + this.oldfs.getProperties()); + if (!type.equals(StorageBackend.StorageType.BROKER)) { + StorageProperties storageProperties = StorageProperties.createPrimary(this.oldfs.properties); + this.fileSystem = FileSystemFactory.get(storageProperties); + } } public long getId() { @@ -229,7 +275,18 @@ public boolean isReadOnly() { } public String getLocation() { - return location; + if (null == fileSystem) { + return location; + } + try { + if (null == fileSystem.getStorageProperties()) { + return location; + } else { + return fileSystem.getStorageProperties().validateAndNormalizeUri(location); + } + } catch (UserException e) { + throw new RuntimeException(e); + } } public String getErrorMsg() { @@ -277,7 +334,7 @@ public Status initRepository() { if (name.compareTo((String) root.get("name")) != 0) { return new Status(ErrCode.COMMON_ERROR, "Invalid repository __repo_info, expected repo '" + name + "', but get name '" - + (String) root.get("name") + "' from " + repoInfoFilePath); + + (String) root.get("name") + "' from " + repoInfoFilePath); } name = (String) root.get("name"); createTime = TimeUtils.timeStringToLong((String) root.get("create_time")); @@ -307,54 +364,23 @@ public Status initRepository() { } } - public Status alterRepositoryS3Properties(Map properties) { - if (fileSystem instanceof S3FileSystem) { - Map oldProperties = new HashMap<>(this.getRemoteFileSystem().getProperties()); - oldProperties.remove(S3Properties.ACCESS_KEY); - oldProperties.remove(S3Properties.SECRET_KEY); - oldProperties.remove(S3Properties.SESSION_TOKEN); - oldProperties.remove(S3Properties.Env.ACCESS_KEY); - oldProperties.remove(S3Properties.Env.SECRET_KEY); - oldProperties.remove(S3Properties.Env.TOKEN); - for (Map.Entry entry : properties.entrySet()) { - if (Objects.equals(entry.getKey(), S3Properties.ACCESS_KEY) - || Objects.equals(entry.getKey(), S3Properties.Env.ACCESS_KEY)) { - oldProperties.putIfAbsent(S3Properties.ACCESS_KEY, entry.getValue()); - } - if (Objects.equals(entry.getKey(), S3Properties.SECRET_KEY) - || Objects.equals(entry.getKey(), S3Properties.Env.SECRET_KEY)) { - oldProperties.putIfAbsent(S3Properties.SECRET_KEY, entry.getValue()); - } - if (Objects.equals(entry.getKey(), S3Properties.SESSION_TOKEN) - || Objects.equals(entry.getKey(), S3Properties.Env.TOKEN)) { - oldProperties.putIfAbsent(S3Properties.SESSION_TOKEN, entry.getValue()); - } - } - properties.clear(); - properties.putAll(oldProperties); - return Status.OK; - } else { - return new Status(ErrCode.COMMON_ERROR, "Only support alter s3 repository"); - } - } - // eg: location/__palo_repository_repo_name/__repo_info public String assembleRepoInfoFilePath() { - return Joiner.on(PATH_DELIMITER).join(location, + return Joiner.on(PATH_DELIMITER).join(getLocation(), joinPrefix(PREFIX_REPO, name), FILE_REPO_INFO); } // eg: location/__palo_repository_repo_name/__my_sp1/__meta public String assembleMetaInfoFilePath(String label) { - return Joiner.on(PATH_DELIMITER).join(location, joinPrefix(PREFIX_REPO, name), + return Joiner.on(PATH_DELIMITER).join(getLocation(), joinPrefix(PREFIX_REPO, name), joinPrefix(PREFIX_SNAPSHOT_DIR, label), FILE_META_INFO); } // eg: location/__palo_repository_repo_name/__my_sp1/__info_2018-01-01-08-00-00 public String assembleJobInfoFilePath(String label, long createTime) { - return Joiner.on(PATH_DELIMITER).join(location, joinPrefix(PREFIX_REPO, name), + return Joiner.on(PATH_DELIMITER).join(getLocation(), joinPrefix(PREFIX_REPO, name), joinPrefix(PREFIX_SNAPSHOT_DIR, label), jobInfoFileNameWithTimestamp(createTime)); } @@ -362,7 +388,7 @@ public String assembleJobInfoFilePath(String label, long createTime) { // eg: // __palo_repository_repo_name/__ss_my_ss1/__ss_content/__db_10001/__tbl_10020/__part_10031/__idx_10020/__10022/ public String getRepoTabletPathBySnapshotInfo(String label, SnapshotInfo info) { - String path = Joiner.on(PATH_DELIMITER).join(location, joinPrefix(PREFIX_REPO, name), + String path = Joiner.on(PATH_DELIMITER).join(getLocation(), joinPrefix(PREFIX_REPO, name), joinPrefix(PREFIX_SNAPSHOT_DIR, label), DIR_SNAPSHOT_CONTENT, joinPrefix(PREFIX_DB, info.getDbId()), @@ -381,7 +407,7 @@ public String getRepoTabletPathBySnapshotInfo(String label, SnapshotInfo info) { } public String getRepoPath(String label, String childPath) { - String path = Joiner.on(PATH_DELIMITER).join(location, joinPrefix(PREFIX_REPO, name), + String path = Joiner.on(PATH_DELIMITER).join(getLocation(), joinPrefix(PREFIX_REPO, name), joinPrefix(PREFIX_SNAPSHOT_DIR, label), DIR_SNAPSHOT_CONTENT, childPath); @@ -568,7 +594,7 @@ public Status upload(String localFilePath, String remoteFilePath) { if (!st.ok()) { return st; } - } else if (fileSystem instanceof S3FileSystem || fileSystem instanceof AzureFileSystem) { + } else { if (LOG.isDebugEnabled()) { LOG.debug("get md5sum of file: {}. final remote path: {}", localFilePath, finalRemotePath); } @@ -577,20 +603,6 @@ public Status upload(String localFilePath, String remoteFilePath) { return st; } - // upload final file - st = fileSystem.upload(localFilePath, finalRemotePath); - if (!st.ok()) { - return st; - } - } else if (fileSystem instanceof DFSFileSystem) { - if (LOG.isDebugEnabled()) { - LOG.debug("hdfs get md5sum of file: {}. final remote path: {}", localFilePath, finalRemotePath); - } - st = fileSystem.delete(finalRemotePath); - if (!st.ok()) { - return st; - } - // upload final file st = fileSystem.upload(localFilePath, finalRemotePath); if (!st.ok()) { @@ -637,7 +649,7 @@ public Status download(String remoteFilePath, String localFilePath) { // 2. download status = fileSystem.downloadWithFileSize(remoteFilePathWithChecksum, localFilePath, - remoteFiles.get(0).getSize()); + remoteFiles.get(0).getSize()); if (!status.ok()) { return status; } @@ -855,7 +867,13 @@ public void readFields(DataInput in) throws IOException { name = Text.readString(in); isReadOnly = in.readBoolean(); location = Text.readString(in); - fileSystem = PersistentFileSystem.read(in); + oldfs = org.apache.doris.fs.PersistentFileSystem.read(in); + try { + fileSystem = FileSystemFactory.get(oldfs.getStorageType(), oldfs.getProperties()); + } catch (UserException e) { + // do we ignore this exception? + throw new IOException("Failed to create file system: " + e.getMessage()); + } createTime = in.readLong(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RepositoryMgr.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RepositoryMgr.java index 853c1841449046..d57593c5443098 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RepositoryMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RepositoryMgr.java @@ -23,8 +23,8 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.Daemon; -import org.apache.doris.fs.remote.AzureFileSystem; -import org.apache.doris.fs.remote.S3FileSystem; +import org.apache.doris.fsv2.remote.AzureFileSystem; +import org.apache.doris.fsv2.remote.S3FileSystem; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; @@ -104,6 +104,9 @@ public Repository getRepo(long repoId) { return repoIdMap.get(repoId); } + /** + * todo: why not support alter other file system like hdfs + */ public Status alterRepo(Repository newRepo, boolean isReplay) { lock.lock(); try { diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 83569e10eabfaa..2643d0efacd5e4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -68,7 +68,7 @@ import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.datasource.property.S3ClientBEProperties; +import org.apache.doris.persist.ColocatePersistInfo; import org.apache.doris.persist.gson.GsonPostProcessable; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.resource.Tag; @@ -119,6 +119,7 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { private static final String PROP_RESERVE_REPLICA = RestoreStmt.PROP_RESERVE_REPLICA; + private static final String PROP_RESERVE_COLOCATE = RestoreStmt.PROP_RESERVE_COLOCATE; private static final String PROP_RESERVE_DYNAMIC_PARTITION_ENABLE = RestoreStmt.PROP_RESERVE_DYNAMIC_PARTITION_ENABLE; private static final String PROP_IS_BEING_SYNCED = PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED; @@ -175,6 +176,7 @@ public enum RestoreJobState { private ReplicaAllocation replicaAlloc; private boolean reserveReplica = false; + private boolean reserveColocate = false; private boolean reserveDynamicPartitionEnable = false; private long createReplicasTimeStamp = -1; // this 2 members is to save all newly restored objs @@ -196,6 +198,8 @@ public enum RestoreJobState { private Map unfinishedSignatureToId = Maps.newConcurrentMap(); + private List colocatePersistInfos = Lists.newArrayList(); + // the meta version is used when reading backup meta from file. // we do not persist this field, because this is just a temporary solution. // the true meta version should be get from backup job info, which is saved when doing backup job. @@ -232,8 +236,9 @@ public RestoreJob(JobType jobType) { public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, - boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, long repoId) { + boolean reserveColocate, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, + boolean isCleanTables, boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, + long repoId) { super(JobType.RESTORE, label, dbId, dbName, timeoutMs, env, repoId); this.backupTimestamp = backupTs; this.jobInfo = jobInfo; @@ -242,8 +247,8 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu this.state = RestoreJobState.PENDING; this.metaVersion = metaVersion; this.reserveReplica = reserveReplica; - // if backup snapshot is come from a cluster with force replication allocation, - // ignore the origin allocation + this.reserveColocate = reserveColocate; + // if backup snapshot is come from a cluster with force replication allocation, ignore the origin allocation if (jobInfo.isForceReplicationAllocation) { this.reserveReplica = false; } @@ -256,6 +261,7 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu this.isForceReplace = isForceReplace; } properties.put(PROP_RESERVE_REPLICA, String.valueOf(reserveReplica)); + properties.put(PROP_RESERVE_COLOCATE, String.valueOf(reserveColocate)); properties.put(PROP_RESERVE_DYNAMIC_PARTITION_ENABLE, String.valueOf(reserveDynamicPartitionEnable)); properties.put(PROP_IS_BEING_SYNCED, String.valueOf(isBeingSynced)); properties.put(PROP_CLEAN_TABLES, String.valueOf(isCleanTables)); @@ -266,13 +272,14 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, - boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, long repoId, + boolean reserveColocate, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, + boolean isCleanTables, boolean isCleanPartitions, boolean isAtomicRestore, boolean isForeReplace, Env env, + long repoId, BackupMeta backupMeta) { this(label, backupTs, dbId, dbName, jobInfo, allowLoad, replicaAlloc, timeoutMs, metaVersion, reserveReplica, - reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, isAtomicRestore, - isForceReplace, env, - repoId); + reserveColocate, reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, + isAtomicRestore, isForeReplace, env, repoId); + this.backupMeta = backupMeta; } @@ -292,6 +299,10 @@ public boolean isBeingSynced() { return isBeingSynced; } + public List getColocatePersistInfos() { + return colocatePersistInfos; + } + public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishTaskRequest request) { if (checkTaskStatus(task, task.getJobId(), request)) { return false; @@ -426,7 +437,7 @@ public synchronized Status updateRepo(Repository repo) { continue; } ((DownloadTask) task).updateBrokerProperties( - S3ClientBEProperties.getBeFSProperties(repo.getRemoteFileSystem().getProperties())); + repo.getRemoteFileSystem().getStorageProperties().getBackendConfigProperties()); AgentTaskQueue.updateTask(beId, TTaskType.DOWNLOAD, signature, task); } LOG.info("finished to update download job properties. {}", this); @@ -706,6 +717,12 @@ private void checkAndPrepareMeta() { OlapTable localOlapTbl = (OlapTable) localTbl; OlapTable remoteOlapTbl = (OlapTable) remoteTbl; + if (localOlapTbl.isColocateTable() || (reserveColocate && remoteOlapTbl.isColocateTable())) { + status = new Status(ErrCode.COMMON_ERROR, "Not support to restore to local table " + + tableName + " with colocate group."); + return; + } + localOlapTbl.readLock(); try { List intersectPartNames = Lists.newArrayList(); @@ -839,7 +856,8 @@ private void checkAndPrepareMeta() { // reset all ids in this table String srcDbName = jobInfo.dbName; - Status st = remoteOlapTbl.resetIdsForRestore(env, db, replicaAlloc, reserveReplica, srcDbName); + Status st = remoteOlapTbl.resetIdsForRestore(env, db, replicaAlloc, reserveReplica, + reserveColocate, colocatePersistInfos, srcDbName); if (!st.ok()) { status = st; return; @@ -1190,11 +1208,11 @@ private Status bindLocalAndRemoteOlapTableReplicas( List localTablets = localIndex.getTablets(); List remoteTablets = index.getTablets(); if (localTablets.size() != remoteTablets.size()) { - return new Status(ErrCode.COMMON_ERROR, String.format( - "the size of local tablet %s is not equals to the remote %s, " - + "is_atomic_restore=true, remote table=%d, remote index=%d, " - + "local table=%d, local index=%d", localTablets.size(), remoteTablets.size(), - remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId)); + LOG.warn("skip bind replicas because the size of local tablet {} is not equals to " + + "the remote {}, is_atomic_restore=true, remote table={}, remote index={}, " + + "local table={}, local index={}", localTablets.size(), remoteTablets.size(), + remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId); + continue; } for (int i = 0; i < remoteTablets.size(); i++) { Tablet localTablet = localTablets.get(i); @@ -1202,13 +1220,13 @@ private Status bindLocalAndRemoteOlapTableReplicas( List localReplicas = localTablet.getReplicas(); List remoteReplicas = remoteTablet.getReplicas(); if (localReplicas.size() != remoteReplicas.size()) { - return new Status(ErrCode.COMMON_ERROR, String.format( - "the size of local replicas %s is not equals to the remote %s, " - + "is_atomic_restore=true, remote table=%d, remote index=%d, " - + "local table=%d, local index=%d, local replicas=%d, remote replicas=%d", - localTablets.size(), remoteTablets.size(), remoteOlapTbl.getId(), - index.getId(), localOlapTbl.getId(), localIndexId, localReplicas.size(), - remoteReplicas.size())); + LOG.warn("skip bind replicas because the size of local replicas {} is not equals to " + + "the remote {}, is_atomic_restore=true, remote table={}, remote index={}, " + + "local table={}, local index={}, local tablet={}, remote tablet={}", + localReplicas.size(), remoteReplicas.size(), remoteOlapTbl.getId(), + index.getId(), localOlapTbl.getId(), localIndexId, localTablet.getId(), + remoteTablet.getId()); + continue; } for (int j = 0; j < remoteReplicas.size(); j++) { long backendId = localReplicas.get(j).getBackendIdWithoutException(); @@ -1366,9 +1384,9 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc MaterializedIndexMeta indexMeta = localTbl.getIndexMetaByIndexId(restoredIdx.getId()); List indexes = restoredIdx.getId() == localTbl.getBaseIndexId() ? localTbl.getCopiedIndexes() : null; - List clusterKeyIndexes = null; + List clusterKeyUids = null; if (indexMeta.getIndexId() == localTbl.getBaseIndexId() || localTbl.isShadowIndex(indexMeta.getIndexId())) { - clusterKeyIndexes = OlapTable.getClusterKeyIndexes(indexMeta.getSchema()); + clusterKeyUids = OlapTable.getClusterKeyUids(indexMeta.getSchema()); } for (Tablet restoreTablet : restoredIdx.getTablets()) { TabletRef baseTabletRef = tabletBases == null ? null : tabletBases.get(restoreTablet.getId()); @@ -1420,11 +1438,11 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc LOG.info("set base tablet {} for replica {} in restore job {}, tablet id={}", baseTabletRef.tabletId, restoreReplica.getId(), jobId, restoreTablet.getId()); } - if (!CollectionUtils.isEmpty(clusterKeyIndexes)) { - task.setClusterKeyIndexes(clusterKeyIndexes); - LOG.info("table: {}, partition: {}, index: {}, tablet: {}, cluster key indexes: {}", + if (!CollectionUtils.isEmpty(clusterKeyUids)) { + task.setClusterKeyUids(clusterKeyUids); + LOG.info("table: {}, partition: {}, index: {}, tablet: {}, cluster key uids: {}", localTbl.getId(), restorePart.getId(), restoredIdx.getId(), restoreTablet.getId(), - clusterKeyIndexes); + clusterKeyUids); } batchTask.addTask(task); } @@ -1696,6 +1714,9 @@ private void waitingAllSnapshotsFinished() { state = RestoreJobState.DOWNLOAD; env.getEditLog().logRestoreJob(this); + for (ColocatePersistInfo info : colocatePersistInfos) { + env.getEditLog().logColocateAddTable(info); + } LOG.info("finished making snapshots. {}", this); return; } @@ -1842,7 +1863,7 @@ private void downloadRemoteSnapshots() { long signature = env.getNextId(); DownloadTask task = new DownloadTask(null, beId, signature, jobId, dbId, srcToDest, brokerAddrs.get(0), - S3ClientBEProperties.getBeFSProperties(repo.getRemoteFileSystem().getProperties()), + repo.getRemoteFileSystem().getStorageProperties().getBackendConfigProperties(), repo.getRemoteFileSystem().getStorageType(), repo.getLocation()); batchTask.addTask(task); unfinishedSignatureToId.put(signature, beId); @@ -2443,6 +2464,11 @@ private void cancelInternal(boolean isReplay) { state = RestoreJobState.CANCELLED; // log env.getEditLog().logRestoreJob(this); + for (ColocatePersistInfo info : colocatePersistInfos) { + Env.getCurrentColocateIndex().removeTable(info.getTableId()); + env.getEditLog().logColocateRemoveTable(info); + } + colocatePersistInfos.clear(); LOG.info("finished to cancel restore job. current state: {}. is replay: {}. {}", curState.name(), isReplay, this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerReader.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerReader.java index 592d9f94d114b3..bb3162aa1e7f01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerReader.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerReader.java @@ -103,7 +103,7 @@ public TBrokerFD open(String path) throws IOException { String clientId = NetUtils .getHostPortInAccessibleFormat(FrontendOptions.getLocalHostAddress(), Config.rpc_port); TBrokerOpenReaderRequest tOpenReaderRequest = new TBrokerOpenReaderRequest( - TBrokerVersion.VERSION_ONE, path, 0, clientId, brokerDesc.getProperties()); + TBrokerVersion.VERSION_ONE, path, 0, clientId, brokerDesc.getBackendConfigProperties()); TBrokerOpenReaderResponse tOpenReaderResponse = null; try { tOpenReaderResponse = client.openReader(tOpenReaderRequest); @@ -137,7 +137,7 @@ public void close(TBrokerFD fd) { public long getFileLength(String path) throws IOException { TBrokerListPathRequest request = new TBrokerListPathRequest( - TBrokerVersion.VERSION_ONE, path, false, brokerDesc.getProperties()); + TBrokerVersion.VERSION_ONE, path, false, brokerDesc.getBackendConfigProperties()); TBrokerListResponse tBrokerListResponse = null; try { tBrokerListResponse = client.listPath(request); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java index 617d346ba3daa3..e043b7a9fd6406 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/BrokerUtil.java @@ -28,9 +28,9 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.datasource.hive.HiveMetaStoreCache; -import org.apache.doris.fs.FileSystemFactory; -import org.apache.doris.fs.remote.RemoteFile; -import org.apache.doris.fs.remote.RemoteFileSystem; +import org.apache.doris.fsv2.FileSystemFactory; +import org.apache.doris.fsv2.remote.RemoteFile; +import org.apache.doris.fsv2.remote.RemoteFileSystem; import org.apache.doris.service.FrontendOptions; import org.apache.doris.thrift.TBrokerCheckPathExistRequest; import org.apache.doris.thrift.TBrokerCheckPathExistResponse; @@ -85,8 +85,7 @@ public class BrokerUtil { public static void parseFile(String path, BrokerDesc brokerDesc, List fileStatuses) throws UserException { List rfiles = new ArrayList<>(); - try (RemoteFileSystem fileSystem = FileSystemFactory.get( - brokerDesc.getName(), brokerDesc.getStorageType(), brokerDesc.getProperties())) { + try (RemoteFileSystem fileSystem = FileSystemFactory.get(brokerDesc)) { Status st = fileSystem.globList(path, rfiles, false); if (!st.ok()) { throw new UserException(st.getErrMsg()); @@ -107,8 +106,7 @@ public static void parseFile(String path, BrokerDesc brokerDesc, List getLocationProperties() throws UserException; @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java index b4b123760e698c..ed4e93b65a3648 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java @@ -96,9 +96,10 @@ public FileLoadScanNode.ParamCreateContext createContext(Analyzer analyzer) thro && fileGroupInfo.getFileGroup().getFileFormat().equals("hive_text")) { params.setTextSerdeType(TTextSerdeType.HIVE_TEXT_SERDE); } - params.setProperties(fileGroupInfo.getBrokerDesc().getProperties()); + params.setProperties(fileGroupInfo.getBrokerDesc().getBackendConfigProperties()); if (fileGroupInfo.getBrokerDesc().getFileType() == TFileType.FILE_HDFS) { - THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(fileGroupInfo.getBrokerDesc().getProperties()); + THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(fileGroupInfo.getBrokerDesc() + .getBackendConfigProperties()); params.setHdfsParams(tHdfsParams); } TFileAttributes fileAttributes = new TFileAttributes(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/ConnectionProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/ConnectionProperties.java index 7fe00cedec7984..2c86c67e4ce4a8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/ConnectionProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/ConnectionProperties.java @@ -18,7 +18,7 @@ package org.apache.doris.datasource.property; import org.apache.doris.common.CatalogConfigFileUtils; -import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import com.google.common.base.Strings; import com.google.common.collect.Maps; @@ -55,7 +55,7 @@ protected ConnectionProperties(Map origProps) { this.origProps = origProps; } - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { List supportedProps = PropertyUtils.getConnectorProperties(this.getClass()); for (Field field : supportedProps) { field.setAccessible(true); @@ -67,7 +67,8 @@ protected void initNormalizeAndCheckProps() throws UserException { field.set(this, origProps.get(name)); matchedProperties.put(name, origProps.get(name)); } catch (IllegalAccessException e) { - throw new RuntimeException("Failed to set property " + name + ", " + e.getMessage(), e); + throw new StoragePropertiesException("Failed to set property " + name + + ", " + e.getMessage(), e); } break; } @@ -115,7 +116,8 @@ protected void checkRequiredProperties() { throw new IllegalArgumentException("Property " + names[0] + " is required."); } } catch (IllegalAccessException e) { - throw new RuntimeException(e); + throw new StoragePropertiesException("Failed to get property " + names[0] + + ", " + e.getMessage(), e); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AliyunDLFProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AliyunDLFProperties.java index c0096baddc467b..37b79b7dcc7f22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AliyunDLFProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/AliyunDLFProperties.java @@ -17,7 +17,6 @@ package org.apache.doris.datasource.property.metastore; -import org.apache.doris.common.UserException; import org.apache.doris.datasource.property.ConnectorProperty; import com.google.common.base.Strings; @@ -67,7 +66,7 @@ public AliyunDLFProperties(Map origProps) { } @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); // Other properties that start with "dlf." will be saved in otherDlfProps, // and passed to the DLF client. diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/HMSProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/HMSProperties.java index b7c18a3e74a808..2702ae474918ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/HMSProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/metastore/HMSProperties.java @@ -18,7 +18,6 @@ package org.apache.doris.datasource.property.metastore; import org.apache.doris.common.CatalogConfigFileUtils; -import org.apache.doris.common.UserException; import org.apache.doris.datasource.property.ConnectorProperty; import com.google.common.base.Strings; @@ -95,7 +94,7 @@ protected void checkRequiredProperties() { } @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); hiveConfParams = loadConfigFromFile(getResourceConfigPropName()); initHmsConnectionProperties(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AbstractS3CompatibleProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AbstractS3CompatibleProperties.java index 5685c278347867..3c0422954e54e7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AbstractS3CompatibleProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AbstractS3CompatibleProperties.java @@ -80,6 +80,12 @@ public abstract class AbstractS3CompatibleProperties extends StorageProperties i @Getter protected String forceParsingByStandardUrl = "false"; + @Getter + @ConnectorProperty(names = {"s3.session_token", "session_token"}, + required = false, + description = "The session token of S3.") + protected String sessionToken = ""; + /** * Constructor to initialize the object storage properties with the provided type and original properties map. * @@ -135,6 +141,9 @@ private Map doBuildS3Configuration(String maxConnections, s3Props.put("AWS_REQUEST_TIMEOUT_MS", requestTimeoutMs); s3Props.put("AWS_CONNECTION_TIMEOUT_MS", connectionTimeoutMs); s3Props.put("use_path_style", usePathStyle); + if (StringUtils.isNotBlank(getSessionToken())) { + s3Props.put("AWS_TOKEN", getSessionToken()); + } return s3Props; } @@ -145,7 +154,7 @@ public Map getBackendConfigProperties() { @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); setEndpointIfNotSet(); if (!isValidEndpoint(getEndpoint())) { @@ -184,7 +193,7 @@ private boolean isValidEndpoint(String endpoint) { return endpointPattern().matcher(endpoint).matches(); } - private void setEndpointIfNotSet() throws UserException { + private void setEndpointIfNotSet() { if (StringUtils.isNotBlank(getEndpoint())) { return; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AzureProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AzureProperties.java index 512ed92cca9054..9d85c3657668c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AzureProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/AzureProperties.java @@ -100,7 +100,7 @@ public AzureProperties(Map origProps) { private static final String AZURE_ENDPOINT_SUFFIX = ".blob.core.windows.net"; @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); //check endpoint if (!endpoint.endsWith(AZURE_ENDPOINT_SUFFIX)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/BrokerProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/BrokerProperties.java new file mode 100644 index 00000000000000..b53a65ecdc7ac9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/BrokerProperties.java @@ -0,0 +1,48 @@ +// 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.datasource.property.storage; + +import org.apache.doris.common.UserException; + +import java.util.Map; + +public class BrokerProperties extends StorageProperties { + public BrokerProperties(Map origProps) { + super(Type.BROKER, origProps); + } + + @Override + public Map getBackendConfigProperties() { + return origProps; + } + + @Override + public String validateAndNormalizeUri(String url) throws UserException { + return url; + } + + @Override + public String validateAndGetUri(Map loadProps) throws UserException { + return loadProps.get("uri"); + } + + @Override + public String getStorageName() { + return "BROKER"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/COSProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/COSProperties.java index 6dc72e1b27cc97..730a4bdb7b2fac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/COSProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/COSProperties.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -45,7 +46,7 @@ public class COSProperties extends AbstractS3CompatibleProperties { protected String region = ""; @Getter - @ConnectorProperty(names = {"cos.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", "access_key"}, + @ConnectorProperty(names = {"cos.access_key", "s3.access_key", "AWS_ACCESS_KEY", "access_key", "ACCESS_KEY"}, description = "The access key of COS.") protected String accessKey = ""; @@ -75,12 +76,13 @@ protected static boolean guessIsMe(Map origProps) { .findFirst() .orElse(null); if (!Strings.isNullOrEmpty(value)) { - return ENDPOINT_PATTERN.matcher(value).matches(); + return value.contains("myqcloud.com"); } - if (!origProps.containsKey("uri")) { - return false; - } - return origProps.get("uri").contains("myqcloud.com"); + Optional uriValue = origProps.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase("uri")) + .map(Map.Entry::getValue) + .findFirst(); + return uriValue.isPresent() && uriValue.get().contains("myqcloud.com"); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsProperties.java index d4722a3c1ad0f5..97c6c28db145a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsProperties.java @@ -58,6 +58,11 @@ public class HdfsProperties extends HdfsCompatibleProperties { description = "Whether to enable the impersonation of HDFS.") private boolean hdfsImpersonationEnabled = false; + @ConnectorProperty(names = {"ipc.client.fallback-to-simple-auth-allowed"}, + required = false, + description = "Whether to allow fallback to simple authentication.") + private String allowFallbackToSimpleAuth = ""; + private Map backendConfigProperties; /** @@ -69,6 +74,8 @@ public class HdfsProperties extends HdfsCompatibleProperties { */ private Map userOverriddenHdfsConfig; + public static final String HDFS_DEFAULT_FS_NAME = "fs.defaultFS"; + private static final List HDFS_PROPERTIES_KEYS = Arrays.asList("hdfs.authentication.type", "hadoop.security.authentication", "hadoop.username", "hdfs.authentication.kerberos.principal", "hadoop.kerberos.principal", "dfs.nameservices"); @@ -81,6 +88,9 @@ public static boolean guessIsMe(Map props) { if (MapUtils.isEmpty(props)) { return false; } + if (HdfsPropertiesUtils.validateUriIsHdfsUri(props)) { + return true; + } if (HDFS_PROPERTIES_KEYS.stream().anyMatch(props::containsKey)) { return true; } @@ -98,7 +108,7 @@ public static boolean guessIsMe(Map props) { } @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); extractUserOverriddenHdfsConfig(origProps); initHadoopConfiguration(); @@ -111,7 +121,7 @@ private void extractUserOverriddenHdfsConfig(Map origProps) { } userOverriddenHdfsConfig = new HashMap<>(); origProps.forEach((key, value) -> { - if (key.startsWith("hadoop.") || key.startsWith("dfs.") || key.equals("fs.defaultFS")) { + if (key.startsWith("hadoop.") || key.startsWith("dfs.") || key.startsWith("fs.")) { userOverriddenHdfsConfig.put(key, value); } }); @@ -132,11 +142,7 @@ protected void checkRequiredProperties() { // fsDefaultFS is not strictly required here. // This is a best-effort fallback to populate fsDefaultFS when possible. if (StringUtils.isBlank(fsDefaultFS)) { - try { - this.fsDefaultFS = HdfsPropertiesUtils.validateAndGetUri(origProps); - } catch (UserException e) { - //ignore - } + this.fsDefaultFS = HdfsPropertiesUtils.extractDefaultFsFromUri(origProps); } } @@ -148,7 +154,12 @@ private void initHadoopConfiguration() { userOverriddenHdfsConfig.forEach(conf::set); } if (StringUtils.isNotBlank(fsDefaultFS)) { - conf.set("fs.defaultFS", fsDefaultFS); + conf.set(HDFS_DEFAULT_FS_NAME, fsDefaultFS); + } + if (StringUtils.isNotBlank(allowFallbackToSimpleAuth)) { + conf.set("ipc.client.fallback-to-simple-auth-allowed", allowFallbackToSimpleAuth); + } else { + conf.set("ipc.client.fallback-to-simple-auth-allowed", "true"); } conf.set("hdfs.security.authentication", hdfsAuthenticationType); if ("kerberos".equalsIgnoreCase(hdfsAuthenticationType)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java index 5e97a2fd639fb4..77822f8426d093 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtils.java @@ -20,120 +20,101 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.UserException; import org.apache.doris.common.util.URI; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; +import com.google.common.collect.ImmutableSet; import org.apache.commons.lang3.StringUtils; -import java.util.HashSet; import java.util.Map; import java.util.Set; public class HdfsPropertiesUtils { private static final String URI_KEY = "uri"; + private static final Set supportSchema = ImmutableSet.of("hdfs", "viewfs"); - private static Set supportSchema = new HashSet<>(); - - static { - supportSchema.add("hdfs"); - supportSchema.add("viewfs"); - } - - /** - * Validates that the 'uri' property exists in the provided props map, and normalizes it. - * - * @param props the map of properties that must include a 'uri' entry - * @return a normalized URI string like 'hdfs://host/path' - * @throws UserException if the map is empty or does not contain the required 'uri' key - *

- * Example: - * Input: {"uri": "hdfs://namenode:9000/data/input"} - * Output: "hdfs://namenode:9000/data/input" - */ public static String validateAndGetUri(Map props) throws UserException { if (props.isEmpty()) { throw new UserException("props is empty"); } - if (!props.containsKey(URI_KEY)) { - throw new UserException("props must contain uri"); + String uriStr = getUri(props); + if (StringUtils.isBlank(uriStr)) { + throw new StoragePropertiesException("props must contain uri"); } - String uriStr = props.get(URI_KEY); return validateAndNormalizeUri(uriStr); } - /** - * Validates and normalizes a raw URI string. - * - * @param uriStr the URI string to validate - * @return a normalized URI in the form of 'scheme://authority/path' - * @throws UserException if the URI is invalid or unsupported - *

- * Example: - * Input: "viewfs://ns1/path/to/file" - * Output: "viewfs://ns1/path/to/file" - */ - public static String convertUrlToFilePath(String uriStr) throws UserException { - return validateAndNormalizeUri(uriStr); + public static boolean validateUriIsHdfsUri(Map props) { + String uriStr = getUri(props); + if (StringUtils.isBlank(uriStr)) { + return false; + } + try { + URI uri = URI.create(uriStr); + String schema = uri.getScheme(); + if (StringUtils.isBlank(schema)) { + throw new IllegalArgumentException("Invalid uri: " + uriStr + ", extract schema is null"); + } + return isSupportedSchema(schema); + } catch (AnalysisException e) { + throw new IllegalArgumentException("Invalid uri: " + uriStr, e); + } } - /** - * Constructs the default filesystem URI (scheme + authority) from a full URI string in the props map. - * - * @param props the map of properties, expected to contain a valid 'uri' entry - * @return a URI prefix like 'hdfs://host:port', or null if the URI is missing or invalid - *

- * Example: - * Input: {"uri": "hdfs://namenode:8020/data"} - * Output: "hdfs://namenode:8020" - */ - public static String constructDefaultFsFromUri(Map props) { - if (props.isEmpty()) { + public static String extractDefaultFsFromPath(String filePath) { + if (StringUtils.isBlank(filePath)) { return null; } - if (!props.containsKey(URI_KEY)) { - return null; + try { + URI uri = URI.create(filePath); + return uri.getScheme() + "://" + uri.getAuthority(); + } catch (AnalysisException e) { + throw new IllegalArgumentException("Invalid file path: " + filePath, e); } - String uriStr = props.get(URI_KEY); + } + + public static String extractDefaultFsFromUri(Map props) { + String uriStr = getUri(props); if (StringUtils.isBlank(uriStr)) { return null; } - URI uri = null; try { - uri = URI.create(uriStr); + URI uri = URI.create(uriStr); + if (!isSupportedSchema(uri.getScheme())) { + return null; + } + return uri.getScheme() + "://" + uri.getAuthority(); } catch (AnalysisException e) { - return null; - } - String schema = uri.getScheme(); - if (StringUtils.isBlank(schema)) { - throw new IllegalArgumentException("Invalid uri: " + uriStr + "extract schema is null"); - } - if (!supportSchema.contains(schema.toLowerCase())) { - throw new IllegalArgumentException("Invalid export path:" - + schema + " , please use valid 'hdfs://' or 'viewfs://' path."); + throw new IllegalArgumentException("Invalid uri: " + uriStr, e); } - return uri.getScheme() + "://" + uri.getAuthority(); } - /** - * Internal method that validates and normalizes a URI string. - * Ensures it has a valid scheme and is supported (e.g., hdfs, viewfs). - * - * @param uriStr the URI string to validate - * @return the normalized URI string - * @throws AnalysisException if the URI is blank or has an unsupported scheme - *

- * Example: - * Input: "hdfs://host:8020/user/data" - * Output: "hdfs://host:8020/user/data" - */ + public static String convertUrlToFilePath(String uriStr) throws UserException { + return validateAndNormalizeUri(uriStr); + } + + private static String getUri(Map props) { + return props.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase(URI_KEY)) + .map(Map.Entry::getValue) + .filter(StringUtils::isNotBlank) + .findFirst() + .orElse(null); + } + + private static boolean isSupportedSchema(String schema) { + return schema != null && supportSchema.contains(schema.toLowerCase()); + } + private static String validateAndNormalizeUri(String uriStr) throws AnalysisException { if (StringUtils.isBlank(uriStr)) { - throw new IllegalArgumentException("uri is null, pls check your params"); + throw new IllegalArgumentException("Properties 'uri' is required"); } URI uri = URI.create(uriStr); String schema = uri.getScheme(); if (StringUtils.isBlank(schema)) { - throw new IllegalArgumentException("Invalid uri: " + uriStr + "extract schema is null"); + throw new IllegalArgumentException("Invalid uri: " + uriStr + ", extract schema is null"); } - if (!supportSchema.contains(schema.toLowerCase())) { + if (!isSupportedSchema(schema)) { throw new IllegalArgumentException("Invalid export path:" + schema + " , please use valid 'hdfs://' or 'viewfs://' path."); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/MinioProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/MinioProperties.java new file mode 100644 index 00000000000000..8f5c3f3dfa324a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/MinioProperties.java @@ -0,0 +1,78 @@ +// 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.datasource.property.storage; + +import org.apache.doris.datasource.property.ConnectorProperty; + +import com.google.common.collect.ImmutableSet; +import lombok.Getter; +import lombok.Setter; + +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class MinioProperties extends AbstractS3CompatibleProperties { + @Setter + @Getter + @ConnectorProperty(names = {"minio.endpoint", "s3.endpoint", "AWS_ENDPOINT", "endpoint", "ENDPOINT"}, + required = false, description = "The endpoint of Minio.") + protected String endpoint = ""; + + @Getter + @Setter + protected String region = "us-east-1"; + + @Getter + @ConnectorProperty(names = {"minio.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", "access_key", "s3.access_key"}, + description = "The access key of Minio.") + protected String accessKey = ""; + + @Getter + @ConnectorProperty(names = {"minio.secret_key", "s3.secret_key", "AWS_SECRET_KEY", "secret_key", "SECRET_KEY"}, + description = "The secret key of Minio.") + protected String secretKey = ""; + + private static final Set IDENTIFIERS = ImmutableSet.of("minio.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", + "access_key", "s3.access_key", "minio.endpoint", "s3.endpoint", "AWS_ENDPOINT", "endpoint", "ENDPOINT"); + + /** + * Constructor to initialize the object storage properties with the provided type and original properties map. + * + * @param origProps the original properties map. + */ + protected MinioProperties(Map origProps) { + super(Type.MINIO, origProps); + } + + public static boolean guessIsMe(Map origProps) { + //ugly, but we need to check if the user has set any of the identifiers + if (AzureProperties.guessIsMe(origProps) || COSProperties.guessIsMe(origProps) + || OSSProperties.guessIsMe(origProps) || S3Properties.guessIsMe(origProps)) { + return false; + } + + return IDENTIFIERS.stream().map(origProps::get).anyMatch(value -> value != null && !value.isEmpty()); + } + + + @Override + protected Pattern endpointPattern() { + return Pattern.compile("^(?:https?://)?[a-zA-Z0-9.-]+(?::\\d+)?$"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OBSProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OBSProperties.java index 87ad9b5761c7ab..b99f36bcd30303 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OBSProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OBSProperties.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -38,12 +39,12 @@ public class OBSProperties extends AbstractS3CompatibleProperties { protected String endpoint = ""; @Getter - @ConnectorProperty(names = {"obs.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", "access_key"}, + @ConnectorProperty(names = {"obs.access_key", "s3.access_key", "AWS_ACCESS_KEY", "access_key", "ACCESS_KEY"}, description = "The access key of OBS.") protected String accessKey = ""; @Getter - @ConnectorProperty(names = {"obs.secret_key", "secret_key", "s3.secret_key"}, + @ConnectorProperty(names = {"obs.secret_key", "s3.secret_key", "AWS_SECRET_KEY", "secret_key", "SECRET_KEY"}, description = "The secret key of OBS.") protected String secretKey = ""; @@ -80,13 +81,13 @@ protected static boolean guessIsMe(Map origProps) { .orElse(null); if (!Strings.isNullOrEmpty(value)) { - return ENDPOINT_PATTERN.matcher(value).matches(); + return value.contains("myhuaweicloud.com"); } - if (!origProps.containsKey("uri")) { - return false; - } - // Check if the uri property contains "myhuaweicloud.com" - return origProps.get("uri").contains("myhuaweicloud.com"); + Optional uriValue = origProps.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase("uri")) + .map(Map.Entry::getValue) + .findFirst(); + return uriValue.isPresent() && uriValue.get().contains("myhuaweicloud.com"); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSHdfsProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSHdfsProperties.java index e0e0def53f6d01..0a09a8602532dc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSHdfsProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSHdfsProperties.java @@ -82,7 +82,7 @@ protected void checkRequiredProperties() { } @Override - protected void initNormalizeAndCheckProps() throws UserException { + protected void initNormalizeAndCheckProps() { super.initNormalizeAndCheckProps(); initConfigurationParams(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSProperties.java index 34982f4b690039..26170dcef8bbad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/OSSProperties.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -38,7 +39,7 @@ public class OSSProperties extends AbstractS3CompatibleProperties { protected String endpoint = ""; @Getter - @ConnectorProperty(names = {"oss.access_key", "s3.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", "access_key"}, + @ConnectorProperty(names = {"oss.access_key", "s3.access_key", "AWS_ACCESS_KEY", "access_key", "ACCESS_KEY"}, description = "The access key of OSS.") protected String accessKey = ""; @@ -78,12 +79,17 @@ protected static boolean guessIsMe(Map origProps) { .findFirst() .orElse(null); if (!Strings.isNullOrEmpty(value)) { - return ENDPOINT_PATTERN.matcher(value).matches(); + return value.contains("aliyuncs.com"); } - if (!origProps.containsKey("uri")) { + Optional uriValue = origProps.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase("uri")) + .map(Map.Entry::getValue) + .findFirst(); + if (!uriValue.isPresent()) { return false; } - return origProps.get("uri").contains("aliyuncs.com"); + String uri = uriValue.get(); + return uri.contains("aliyuncs.com") && (!uri.contains("oss-dls.aliyuncs.com")); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3Properties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3Properties.java index 478f45ee3d06b5..343b4b29d39ef5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3Properties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3Properties.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.regex.Pattern; import java.util.stream.Stream; @@ -49,7 +50,7 @@ public class S3Properties extends AbstractS3CompatibleProperties { protected String region = ""; @Getter - @ConnectorProperty(names = {"s3.access_key", "AWS_ACCESS_KEY", "ACCESS_KEY", "access_key"}, + @ConnectorProperty(names = {"s3.access_key", "AWS_ACCESS_KEY", "access_key", "ACCESS_KEY"}, description = "The access key of S3.") protected String accessKey = ""; @@ -134,14 +135,20 @@ protected static boolean guessIsMe(Map origProps) { .filter(Objects::nonNull) .findFirst() .orElse(null); + /** + * Check if the endpoint contains "amazonaws.com" to determine if it's an S3-compatible storage. + * Note: This check should not be overly strict, as a malformed or misconfigured endpoint may + * cause the type detection to fail, leading to missed recognition of valid S3 properties. + * A more robust approach would allow further validation downstream rather than failing early here. + */ if (!Strings.isNullOrEmpty(endpoint)) { - return ENDPOINT_PATTERN.matcher(endpoint).matches(); + return endpoint.contains("amazonaws.com"); } - if (!origProps.containsKey("uri")) { - return false; - } - String uri = origProps.get("uri"); - return uri.contains("amazonaws.com"); + Optional uriValue = origProps.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase("uri")) + .map(Map.Entry::getValue) + .findFirst(); + return uriValue.isPresent() && uriValue.get().contains("amazonaws.com"); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3PropertyUtils.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3PropertyUtils.java index 745838438dd012..0a4fda0bcdfe8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3PropertyUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/S3PropertyUtils.java @@ -19,10 +19,12 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.S3URI; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.apache.commons.lang3.StringUtils; import java.util.Map; +import java.util.Optional; public class S3PropertyUtils { @@ -43,14 +45,28 @@ public class S3PropertyUtils { */ public static String constructEndpointFromUrl(Map props, String stringUsePathStyle, - String stringForceParsingByStandardUri) throws UserException { - String uri = props.get(URI_KEY); - if (uri == null || uri.isEmpty()) { + String stringForceParsingByStandardUri) { + Optional uriOptional = props.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase(URI_KEY)) + .map(Map.Entry::getValue) + .findFirst(); + + if (!uriOptional.isPresent()) { + return null; + } + String uri = uriOptional.get(); + if (StringUtils.isBlank(uri)) { return null; } boolean usePathStyle = Boolean.parseBoolean(stringUsePathStyle); boolean forceParsingByStandardUri = Boolean.parseBoolean(stringForceParsingByStandardUri); - S3URI s3uri = S3URI.create(uri, usePathStyle, forceParsingByStandardUri); + S3URI s3uri; + try { + s3uri = S3URI.create(uri, usePathStyle, forceParsingByStandardUri); + } catch (UserException e) { + throw new IllegalArgumentException("Invalid S3 URI: " + uri + ",usePathStyle: " + usePathStyle + + " forceParsingByStandardUri: " + forceParsingByStandardUri, e); + } return s3uri.getEndpoint().orElse(null); } @@ -68,14 +84,28 @@ public static String constructEndpointFromUrl(Map props, */ public static String constructRegionFromUrl(Map props, String stringUsePathStyle, - String stringForceParsingByStandardUri) throws UserException { - String uri = props.get(URI_KEY); - if (uri == null || uri.isEmpty()) { + String stringForceParsingByStandardUri) { + Optional uriOptional = props.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase(URI_KEY)) + .map(Map.Entry::getValue) + .findFirst(); + + if (!uriOptional.isPresent()) { + return null; + } + String uri = uriOptional.get(); + if (StringUtils.isBlank(uri)) { return null; } boolean usePathStyle = Boolean.parseBoolean(stringUsePathStyle); boolean forceParsingByStandardUri = Boolean.parseBoolean(stringForceParsingByStandardUri); - S3URI s3uri = S3URI.create(uri, usePathStyle, forceParsingByStandardUri); + S3URI s3uri = null; + try { + s3uri = S3URI.create(uri, usePathStyle, forceParsingByStandardUri); + } catch (UserException e) { + throw new IllegalArgumentException("Invalid S3 URI: " + uri + ",usePathStyle: " + usePathStyle + + " forceParsingByStandardUri: " + forceParsingByStandardUri, e); + } return s3uri.getRegion().orElse(null); } @@ -99,7 +129,7 @@ public static String validateAndNormalizeUri(String path, String stringUsePathStyle, String stringForceParsingByStandardUri) throws UserException { if (StringUtils.isBlank(path)) { - throw new UserException("path is null"); + throw new StoragePropertiesException("path is null"); } if (path.startsWith("s3://")) { return path; @@ -122,13 +152,18 @@ public static String validateAndNormalizeUri(String path, * Input: {"uri": "s3://my-bucket/my-key"} * Output: "s3://my-bucket/my-key" */ - public static String validateAndGetUri(Map props) throws UserException { + public static String validateAndGetUri(Map props) { if (props.isEmpty()) { - throw new UserException("props is empty"); + throw new StoragePropertiesException("props is empty"); } - if (!props.containsKey(URI_KEY)) { - throw new UserException("props must contain uri"); + Optional uriOptional = props.entrySet().stream() + .filter(e -> e.getKey().equalsIgnoreCase(URI_KEY)) + .map(Map.Entry::getValue) + .findFirst(); + + if (!uriOptional.isPresent()) { + throw new StoragePropertiesException("props must contain uri"); } - return props.get(URI_KEY); + return uriOptional.get(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java index 58cdedde869a5e..ae69deea400ba0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/StorageProperties.java @@ -20,6 +20,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.datasource.property.ConnectionProperties; import org.apache.doris.datasource.property.ConnectorProperty; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import lombok.Getter; @@ -35,6 +36,7 @@ public abstract class StorageProperties extends ConnectionProperties { public static final String FS_HDFS_SUPPORT = "fs.hdfs.support"; public static final String FS_S3_SUPPORT = "fs.s3.support"; public static final String FS_GCS_SUPPORT = "fs.gcs.support"; + public static final String FS_MINIO_SUPPORT = "fs.minio.support"; public static final String FS_AZURE_SUPPORT = "fs.azure.support"; public static final String FS_OSS_SUPPORT = "fs.oss.support"; public static final String FS_OBS_SUPPORT = "fs.obs.support"; @@ -50,7 +52,9 @@ public enum Type { OSS, OBS, COS, + MINIO, AZURE, + BROKER, UNKNOWN } @@ -98,7 +102,7 @@ public static List createAll(Map origProps) t * @return a StorageProperties instance for the primary storage type * @throws RuntimeException if no supported storage type is found */ - public static StorageProperties createPrimary(Map origProps) throws UserException { + public static StorageProperties createPrimary(Map origProps) { for (Function, StorageProperties> func : PROVIDERS) { StorageProperties p = func.apply(origProps); if (p != null) { @@ -106,7 +110,7 @@ public static StorageProperties createPrimary(Map origProps) thr return p; } } - throw new RuntimeException("No supported storage type found."); + throw new StoragePropertiesException("No supported storage type found. Please check your configuration."); } private static final List, StorageProperties>> PROVIDERS = @@ -125,7 +129,9 @@ public static StorageProperties createPrimary(Map origProps) thr props -> (isFsSupport(props, FS_COS_SUPPORT) || COSProperties.guessIsMe(props)) ? new COSProperties(props) : null, props -> (isFsSupport(props, FS_AZURE_SUPPORT) - || AzureProperties.guessIsMe(props)) ? new AzureProperties(props) : null + || AzureProperties.guessIsMe(props)) ? new AzureProperties(props) : null, + props -> (isFsSupport(props, FS_MINIO_SUPPORT) + || MinioProperties.guessIsMe(props)) ? new MinioProperties(props) : null ); protected StorageProperties(Type type, Map origProps) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/exception/StoragePropertiesException.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/exception/StoragePropertiesException.java new file mode 100644 index 00000000000000..00d8c46411f097 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/storage/exception/StoragePropertiesException.java @@ -0,0 +1,47 @@ +// 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. + +/** + * Custom exception class for handling storage property-related errors. + * This exception class extends RuntimeException and is used to handle errors related + * to storage properties at runtime. It provides two constructors: one that accepts only + * an error message, and another that also accepts a cause exception. + */ + +package org.apache.doris.datasource.property.storage.exception; + +public class StoragePropertiesException extends RuntimeException { + + /** + * Constructor that initializes the exception with an error message. + * + * @param message The error message describing the reason for the exception. + */ + public StoragePropertiesException(String message) { + super(message); + } + + /** + * Constructor that initializes the exception with a message and a cause. + * + * @param message The error message describing the reason for the exception. + * @param cause The underlying cause of the exception, typically another Throwable. + */ + public StoragePropertiesException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java index 5e65093036517e..82f42ee110546b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/tvf/source/TVFScanNode.java @@ -115,7 +115,7 @@ protected boolean isFileStreamType() { @Override public Map getLocationProperties() { - return tableValuedFunction.getLocationProperties(); + return tableValuedFunction.getBackendConnectProperties(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/FileSystemFactory.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/FileSystemFactory.java index 4e418ac217e057..319188c91091d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/FileSystemFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/FileSystemFactory.java @@ -17,8 +17,11 @@ package org.apache.doris.fsv2; +import org.apache.doris.analysis.BrokerDesc; +import org.apache.doris.analysis.StorageBackend; import org.apache.doris.common.UserException; import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fsv2.remote.BrokerFileSystem; import org.apache.doris.fsv2.remote.RemoteFileSystem; import java.util.Map; @@ -30,7 +33,33 @@ public static RemoteFileSystem get(Map properties) throws UserEx return get(storageProperties); } + public static RemoteFileSystem get(StorageBackend.StorageType storageType, Map properties) + throws UserException { + if (storageType.equals(StorageBackend.StorageType.BROKER)) { + return new BrokerFileSystem("broker", properties); + } + StorageProperties storageProperties = StorageProperties.createPrimary(properties); + return get(storageProperties); + } + public static RemoteFileSystem get(StorageProperties storageProperties) { return StorageTypeMapper.create(storageProperties); } + + // This method is a temporary workaround for handling properties. + // It will be removed when broker properties are officially supported. + public static RemoteFileSystem get(String name, Map properties) { + return new BrokerFileSystem(name, properties); + } + + public static RemoteFileSystem get(BrokerDesc brokerDesc) { + if (null != brokerDesc.getStorageProperties()) { + return get(brokerDesc.getStorageProperties()); + } + if (null != brokerDesc.getStorageType() + && brokerDesc.getStorageType().equals(StorageBackend.StorageType.BROKER)) { + return new BrokerFileSystem(brokerDesc.getName(), brokerDesc.getProperties()); + } + throw new RuntimeException("Unexpected storage type: " + brokerDesc.getStorageType()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/StorageTypeMapper.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/StorageTypeMapper.java index 10c8daec002e78..5683934d9606d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/StorageTypeMapper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/StorageTypeMapper.java @@ -20,6 +20,7 @@ import org.apache.doris.datasource.property.storage.AzureProperties; import org.apache.doris.datasource.property.storage.COSProperties; import org.apache.doris.datasource.property.storage.HdfsProperties; +import org.apache.doris.datasource.property.storage.MinioProperties; import org.apache.doris.datasource.property.storage.OBSProperties; import org.apache.doris.datasource.property.storage.OSSHdfsProperties; import org.apache.doris.datasource.property.storage.OSSProperties; @@ -37,6 +38,7 @@ public enum StorageTypeMapper { OSS(OSSProperties.class, S3FileSystem::new), OBS(OBSProperties.class, S3FileSystem::new), COS(COSProperties.class, S3FileSystem::new), + MINIO(MinioProperties.class, S3FileSystem::new), AZURE(AzureProperties.class, AzureFileSystem::new), S3(S3Properties.class, S3FileSystem::new), HDFS(HdfsProperties.class, DFSFileSystem::new), diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/obj/S3ObjStorage.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/obj/S3ObjStorage.java index 9d04a077b622ba..76c12a85834df3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/obj/S3ObjStorage.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/obj/S3ObjStorage.java @@ -56,11 +56,14 @@ import java.io.File; import java.io.InputStream; import java.net.URI; +import java.nio.file.FileSystems; +import java.nio.file.PathMatcher; +import java.nio.file.Paths; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.regex.Pattern; import java.util.stream.Collectors; public class S3ObjStorage implements ObjStorage { @@ -101,57 +104,98 @@ public S3Client getClient() throws UserException { CloudCredential credential = new CloudCredential(); credential.setAccessKey(s3Properties.getAccessKey()); credential.setSecretKey(s3Properties.getSecretKey()); - - /* if (properties.containsKey(S3Properties.SESSION_TOKEN)) { - credential.setSessionToken(properties.get(S3Properties.SESSION_TOKEN)); - }*/ + if (StringUtils.isNotBlank(s3Properties.getSessionToken())) { + credential.setSessionToken(s3Properties.getSessionToken()); + } client = S3Util.buildS3Client(endpoint, s3Properties.getRegion(), credential, isUsePathStyle); } return client; } - - public Status globList(String remotePath, List result, boolean fileNameOnly) { + long roundCnt = 0; + long elementCnt = 0; + long matchCnt = 0; + long startTime = System.nanoTime(); try { - remotePath = s3Properties.validateAndNormalizeUri(remotePath); - URI uri = new URI(remotePath); - String bucketName = uri.getHost(); - String prefix = uri.getPath().substring(1); - int wildcardIndex = prefix.indexOf('*'); - String searchPrefix = wildcardIndex > 0 ? prefix.substring(0, wildcardIndex) : prefix; - try (S3Client s3 = getClient()) { - ListObjectsV2Request listRequest = ListObjectsV2Request.builder() - .bucket(bucketName) - .prefix(searchPrefix) - .build(); + S3URI uri = S3URI.create(remotePath, isUsePathStyle, forceParsingByStandardUri); + String bucket = uri.getBucket(); + String globPath = uri.getKey(); // eg: path/to/*.csv + if (LOG.isDebugEnabled()) { + LOG.info("globList globPath:{}, remotePath:{}", globPath, remotePath); + } - ListObjectsV2Response listResponse = s3.listObjectsV2(listRequest); - String regex = prefix.replace(".", "\\.") - .replace("*", ".*") - .replace("?", "."); - Pattern pattern = Pattern.compile(regex); - List matchedFiles = listResponse.contents().stream() - .filter(obj -> pattern.matcher(obj.key()).matches()) - .map(obj -> { - String fullKey = obj.key(); - String fullPath = "s3://" + bucketName + "/" + fullKey; - return new RemoteFile( - fileNameOnly ? fullPath.substring(fullPath.lastIndexOf('/') + 1) : fullPath, - true, - obj.size(), - -1, - obj.lastModified().toEpochMilli() - ); - }) - .collect(Collectors.toList()); + java.nio.file.Path pathPattern = Paths.get(globPath); + PathMatcher matcher = FileSystems.getDefault().getPathMatcher("glob:" + pathPattern); + HashSet directorySet = new HashSet<>(); + String listPrefix = S3Util.getLongestPrefix(globPath); // similar to Azure + if (LOG.isDebugEnabled()) { + LOG.info("globList listPrefix: {}", listPrefix); + } + ListObjectsV2Request request = ListObjectsV2Request.builder() + .bucket(bucket) + .prefix(listPrefix) + .build(); + boolean isTruncated; + do { + roundCnt++; + ListObjectsV2Response response = getClient().listObjectsV2(request); + for (S3Object obj : response.contents()) { + elementCnt++; + java.nio.file.Path objPath = Paths.get(obj.key()); - result.addAll(matchedFiles); + boolean isPrefix = false; + while (objPath != null && objPath.normalize().toString().startsWith(listPrefix)) { + if (!matcher.matches(objPath)) { + isPrefix = true; + objPath = objPath.getParent(); + continue; + } + if (directorySet.contains(objPath.normalize().toString())) { + break; + } + if (isPrefix) { + directorySet.add(objPath.normalize().toString()); + } + + matchCnt++; + RemoteFile remoteFile = new RemoteFile( + fileNameOnly ? objPath.getFileName().toString() : + "s3://" + bucket + "/" + objPath, + !isPrefix, + isPrefix ? -1 : obj.size(), + isPrefix ? -1 : obj.size(), + isPrefix ? 0 : obj.lastModified().toEpochMilli() + ); + result.add(remoteFile); + objPath = objPath.getParent(); + isPrefix = true; + } + } + + isTruncated = response.isTruncated(); + if (isTruncated) { + request = request.toBuilder() + .continuationToken(response.nextContinuationToken()) + .build(); + } + } while (isTruncated); + + if (LOG.isDebugEnabled()) { + LOG.debug("remotePath:{}, result:{}", remotePath, result); } return Status.OK; } catch (Exception e) { LOG.warn("Errors while getting file status", e); return new Status(Status.ErrCode.COMMON_ERROR, "Errors while getting file status " + e.getMessage()); + } finally { + long endTime = System.nanoTime(); + long duration = endTime - startTime; + if (LOG.isDebugEnabled()) { + LOG.debug("process {} elements under prefix {} for {} round, match {} elements, take {} ms", + elementCnt, remotePath, roundCnt, matchCnt, + duration / 1000); + } } } @@ -167,7 +211,7 @@ public Status headObject(String remotePath) { S3URI uri = S3URI.create(remotePath, isUsePathStyle, forceParsingByStandardUri); HeadObjectResponse response = getClient() .headObject(HeadObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build()); - LOG.info("head file " + remotePath + " success: " + response.toString()); + LOG.info("head file {} success: {}", remotePath, response); return Status.OK; } catch (S3Exception e) { if (e.statusCode() == HttpStatus.SC_NOT_FOUND) { @@ -189,7 +233,7 @@ public Status getObject(String remoteFilePath, File localFile) { S3URI uri = S3URI.create(remoteFilePath, isUsePathStyle, forceParsingByStandardUri); GetObjectResponse response = getClient().getObject( GetObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(), localFile.toPath()); - LOG.info("get file " + remoteFilePath + " success: " + response.toString()); + LOG.info("get file {} success: {}", remoteFilePath, response); return Status.OK; } catch (S3Exception s3Exception) { return new Status( @@ -214,13 +258,13 @@ public Status putObject(String remotePath, @Nullable InputStream content, long c .putObject( PutObjectRequest.builder().bucket(uri.getBucket()).key(uri.getKey()).build(), body); - LOG.info("put object success: " + response.toString()); + LOG.info("put object success: {}", response.toString()); return Status.OK; } catch (S3Exception e) { - LOG.error("put object failed:", e); + LOG.warn("put object failed:", e); return new Status(Status.ErrCode.COMMON_ERROR, "put object failed: " + e.getMessage()); } catch (Exception ue) { - LOG.error("connect to s3 failed: ", ue); + LOG.warn("connect to s3 failed: ", ue); return new Status(Status.ErrCode.COMMON_ERROR, "connect to s3 failed: " + ue.getMessage()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/BrokerFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/BrokerFileSystem.java index b5398f0102e6ea..88e0f90fc1911d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/BrokerFileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/BrokerFileSystem.java @@ -27,6 +27,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.BrokerUtil; import org.apache.doris.datasource.property.PropertyConverter; +import org.apache.doris.datasource.property.storage.BrokerProperties; import org.apache.doris.fs.operations.BrokerFileOperations; import org.apache.doris.fs.operations.OpParams; import org.apache.doris.service.FrontendOptions; @@ -81,6 +82,8 @@ public BrokerFileSystem(String name, Map properties) { properties.putAll(PropertyConverter.convertToHadoopFSProperties(properties)); this.properties = properties; this.operations = new BrokerFileOperations(name, properties); + // support broker properties in future + this.storageProperties = new BrokerProperties(properties); } public Pair getBroker() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/RemoteFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/RemoteFileSystem.java index d418fe90b19e1c..c7877d9ed8db69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/RemoteFileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/RemoteFileSystem.java @@ -42,7 +42,7 @@ public abstract class RemoteFileSystem extends PersistentFileSystem implements C // this field will be visited by multi-threads, better use volatile qualifier protected volatile org.apache.hadoop.fs.FileSystem dfsFileSystem = null; private final ReentrantLock fsLock = new ReentrantLock(); - protected static final AtomicBoolean closed = new AtomicBoolean(false); + protected AtomicBoolean closed = new AtomicBoolean(false); public RemoteFileSystem(String name, StorageBackend.StorageType type) { super(name, type); diff --git a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/dfs/DFSFileSystem.java b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/dfs/DFSFileSystem.java index 53fcc9ba17c5e1..8c708399eadb64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/dfs/DFSFileSystem.java +++ b/fe/fe-core/src/main/java/org/apache/doris/fsv2/remote/dfs/DFSFileSystem.java @@ -90,6 +90,9 @@ public FileSystem nativeFileSystem(String remotePath) throws UserException { } if (dfsFileSystem == null) { Configuration conf = hdfsProperties.getHadoopConfiguration(); + // TODO: Temporarily disable the HDFS file system cache to prevent instances from being closed by + // each other in V1. This line can be removed once V1 and V2 are unified. + conf.set("fs.hdfs.impl.disable.cache", "true"); authenticator = HadoopAuthenticator.getHadoopAuthenticator(conf); try { dfsFileSystem = authenticator.doAs(() -> { diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java index b9e1ca2ead31ef..434deaf440be08 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/ExportJob.java @@ -597,7 +597,7 @@ private Map convertOutfileProperties() { outfileProperties.put(BROKER_PROPERTY_PREFIXES + "name", brokerDesc.getName()); brokerDesc.getProperties().forEach((k, v) -> outfileProperties.put(BROKER_PROPERTY_PREFIXES + k, v)); } else { - for (Entry kv : brokerDesc.getProperties().entrySet()) { + for (Entry kv : brokerDesc.getBackendConfigProperties().entrySet()) { outfileProperties.put(kv.getKey(), kv.getValue()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java index 1abc8b76b544ea..ccc067ec1c8a70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java @@ -951,7 +951,7 @@ private void initTBrokerScanRange(DescriptorTable descTable, TupleDescriptor des // scan range params TBrokerScanRangeParams params = new TBrokerScanRangeParams(); params.setStrictMode(false); - params.setProperties(brokerDesc.getProperties()); + params.setProperties(brokerDesc.getBackendConfigProperties()); TupleDescriptor srcTupleDesc = descTable.createTupleDescriptor(); Map srcSlotDescByName = Maps.newHashMap(); for (Column column : columns) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java index 69cbf762c2afe6..71e24dfc469dfc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/LoadCommand.java @@ -30,7 +30,6 @@ import org.apache.doris.common.util.FileFormatConstants; import org.apache.doris.common.util.FileFormatUtils; import org.apache.doris.common.util.TimeUtils; -import org.apache.doris.datasource.property.constants.S3Properties; import org.apache.doris.job.base.JobExecuteType; import org.apache.doris.job.base.JobExecutionConfiguration; import org.apache.doris.job.extensions.insert.InsertJob; @@ -469,10 +468,8 @@ private static Map getTvfProperties(BulkLoadDataDesc dataDesc, B // TODO: support multi location by union String listFilePath = filePaths.get(0); if (bulkStorageDesc.getStorageType() == BulkStorageDesc.StorageType.S3) { - S3Properties.convertToStdProperties(tvfProperties); - tvfProperties.keySet().removeIf(S3Properties.Env.FS_KEYS::contains); // TODO: check file path by s3 fs list status - tvfProperties.put(S3TableValuedFunction.PROP_URI, listFilePath); + tvfProperties.put("uri", listFilePath); } final Map dataDescProps = dataDesc.getProperties(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ExportSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ExportSink.java index 0fd2535b62bff7..9f307a376b2f01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ExportSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ExportSink.java @@ -82,7 +82,7 @@ protected TDataSink toThrift() { tExportSink.addToBrokerAddresses(new TNetworkAddress(broker.host, broker.port)); } } - tExportSink.setProperties(brokerDesc.getProperties()); + tExportSink.setProperties(brokerDesc.getStorageProperties().getBackendConfigProperties()); tExportSink.setHeader(header); result.setExportSink(tExportSink); return result; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java index 5b9cfd602092bc..9d8a34a7612e70 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java @@ -421,7 +421,9 @@ public static void execute(Env env, DdlStmt ddlStmt) throws Exception { DropAnalyzeJobStmt analyzeJobStmt = (DropAnalyzeJobStmt) ddlStmt; Env.getCurrentEnv().getAnalysisManager().dropAnalyzeJob(analyzeJobStmt); } else if (ddlStmt instanceof AlterRepositoryStmt) { - env.getBackupHandler().alterRepository((AlterRepositoryStmt) ddlStmt); + AlterRepositoryStmt alterRepositoryStmt = (AlterRepositoryStmt) ddlStmt; + env.getBackupHandler().alterRepository(alterRepositoryStmt.getName(), alterRepositoryStmt.getProperties(), + false); } else if (ddlStmt instanceof CreateStorageVaultStmt) { env.getStorageVaultMgr().createStorageVaultResource((CreateStorageVaultStmt) ddlStmt); } else if (ddlStmt instanceof CreateStageStmt) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java index 9ab6d302c3e13c..d6cf657ceda078 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java @@ -42,6 +42,7 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.property.fileformat.CsvFileFormatProperties; import org.apache.doris.datasource.property.fileformat.FileFormatProperties; +import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.datasource.tvf.source.TVFScanNode; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.planner.PlanNodeId; @@ -81,6 +82,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -93,6 +95,7 @@ */ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctionIf { public static final Logger LOG = LogManager.getLogger(ExternalFileTableValuedFunction.class); + protected static final String URI_KEY = "uri"; public static final String PROP_TABLE_ID = "table_id"; @@ -105,7 +108,10 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio private List pathPartitionKeys; protected List fileStatuses = Lists.newArrayList(); - protected Map locationProperties = Maps.newHashMap(); + protected Map backendConnectProperties = Maps.newHashMap(); + protected StorageProperties storageProperties; + // Processed parameters derived from user input; includes normalization and default value filling. + Map processedParams; protected String filePath; protected Optional resourceName = Optional.empty(); @@ -127,8 +133,8 @@ public TFileCompressType getTFileCompressType() { return fileFormatProperties.getCompressionType(); } - public Map getLocationProperties() { - return locationProperties; + public Map getBackendConnectProperties() { + return backendConnectProperties; } public List getPathPartitionKeys() { @@ -181,7 +187,7 @@ protected Map parseCommonProperties(Map properti .map(String::trim) .collect(Collectors.toList())) .orElse(Lists.newArrayList()); - + this.processedParams = new HashMap<>(copiedProps); return copiedProps; } @@ -370,7 +376,9 @@ private PFetchTableSchemaRequest getFetchTableStructureRequest() throws TExcepti // set TFileScanRangeParams TFileScanRangeParams fileScanRangeParams = new TFileScanRangeParams(); fileScanRangeParams.setFormatType(fileFormatProperties.getFileFormatType()); - fileScanRangeParams.setProperties(locationProperties); + Map beProperties = new HashMap<>(); + beProperties.putAll(backendConnectProperties); + fileScanRangeParams.setProperties(beProperties); if (fileFormatProperties instanceof CsvFileFormatProperties) { fileScanRangeParams.setTextSerdeType(((CsvFileFormatProperties) fileFormatProperties).getTextSerdeType()); } @@ -384,8 +392,8 @@ private PFetchTableSchemaRequest getFetchTableStructureRequest() throws TExcepti } if (getTFileType() == TFileType.FILE_HDFS) { - THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(locationProperties); - String fsName = getLocationProperties().get(HdfsResource.HADOOP_FS_NAME); + THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(storageProperties.getBackendConfigProperties()); + String fsName = storageProperties.getBackendConfigProperties().get(HdfsResource.HADOOP_FS_NAME); tHdfsParams.setFsName(fsName); fileScanRangeParams.setHdfsParams(tHdfsParams); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/GroupCommitTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/GroupCommitTableValuedFunction.java index 60336d91b9770b..111c3b9370d63b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/GroupCommitTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/GroupCommitTableValuedFunction.java @@ -106,7 +106,7 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return new BrokerDesc("GroupCommitTvfBroker", StorageType.STREAM, locationProperties); + return new BrokerDesc("GroupCommitTvfBroker", StorageType.STREAM, processedParams); } // =========== implement abstract methods of TableValuedFunctionIf ================= diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java index 80149e3d1380ae..e99340eb105bec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java @@ -18,15 +18,13 @@ package org.apache.doris.tablefunction; import org.apache.doris.analysis.BrokerDesc; -import org.apache.doris.analysis.StorageBackend; import org.apache.doris.analysis.StorageBackend.StorageType; -import org.apache.doris.catalog.HdfsResource; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; -import org.apache.doris.common.util.URI; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.thrift.TFileType; -import com.google.common.base.Strings; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -39,7 +37,6 @@ public class HdfsTableValuedFunction extends ExternalFileTableValuedFunction { public static final Logger LOG = LogManager.getLogger(HdfsTableValuedFunction.class); public static final String NAME = "hdfs"; - private static final String PROP_URI = "uri"; public HdfsTableValuedFunction(Map properties) throws AnalysisException { init(properties); @@ -47,28 +44,15 @@ public HdfsTableValuedFunction(Map properties) throws AnalysisEx private void init(Map properties) throws AnalysisException { // 1. analyze common properties - Map otherProps = super.parseCommonProperties(properties); - + Map props = super.parseCommonProperties(properties); // 2. analyze uri - String uriStr = getOrDefaultAndRemove(otherProps, PROP_URI, null); - if (Strings.isNullOrEmpty(uriStr)) { - throw new AnalysisException(String.format("Properties '%s' is required.", PROP_URI)); - } - URI uri = URI.create(uriStr); - StorageBackend.checkUri(uri, StorageType.HDFS); - filePath = uri.getScheme() + "://" + uri.getAuthority() + uri.getPath(); - - // 3. analyze other properties - for (String key : otherProps.keySet()) { - if (HdfsResource.HADOOP_FS_NAME.equalsIgnoreCase(key)) { - locationProperties.put(HdfsResource.HADOOP_FS_NAME, otherProps.get(key)); - } else { - locationProperties.put(key, otherProps.get(key)); - } - } - // If the user does not specify the HADOOP_FS_NAME, we will use the uri's scheme and authority - if (!locationProperties.containsKey(HdfsResource.HADOOP_FS_NAME)) { - locationProperties.put(HdfsResource.HADOOP_FS_NAME, uri.getScheme() + "://" + uri.getAuthority()); + try { + this.storageProperties = StorageProperties.createPrimary(props); + backendConnectProperties.putAll(storageProperties.getBackendConfigProperties()); + String uri = storageProperties.validateAndGetUri(props); + filePath = storageProperties.validateAndNormalizeUri(uri); + } catch (UserException e) { + throw new AnalysisException("Failed check storage props, " + e.getMessage(), e); } if (!FeConstants.runningUnitTest) { @@ -91,7 +75,7 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return new BrokerDesc("HdfsTvfBroker", StorageType.HDFS, locationProperties); + return new BrokerDesc("HdfsTvfBroker", StorageType.HDFS, processedParams); } // =========== implement abstract methods of TableValuedFunctionIf ================= diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java index 72573a2355f451..c473e174189975 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HttpStreamTableValuedFunction.java @@ -60,7 +60,7 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return new BrokerDesc("HttpStreamTvfBroker", StorageType.STREAM, locationProperties); + return new BrokerDesc("HttpStreamTvfBroker", StorageType.STREAM, processedParams); } // =========== implement abstract methods of TableValuedFunctionIf ================= diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java index 32f76b233fad68..c82552f1862a14 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/LocalTableValuedFunction.java @@ -141,7 +141,7 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return new BrokerDesc("LocalTvfBroker", StorageType.LOCAL, locationProperties); + return new BrokerDesc("LocalTvfBroker", StorageType.LOCAL, processedParams); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java index 93dc28d84a35b2..7d38c50f565ac2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java @@ -18,22 +18,12 @@ package org.apache.doris.tablefunction; import org.apache.doris.analysis.BrokerDesc; -import org.apache.doris.analysis.StorageBackend.StorageType; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.UserException; -import org.apache.doris.common.credentials.CloudCredentialWithEndpoint; -import org.apache.doris.common.util.S3URI; -import org.apache.doris.datasource.property.PropertyConverter; -import org.apache.doris.datasource.property.S3ClientBEProperties; -import org.apache.doris.datasource.property.constants.AzureProperties; -import org.apache.doris.datasource.property.constants.S3Properties; -import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.datasource.property.storage.StorageProperties; import org.apache.doris.thrift.TFileType; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableSet; - import java.util.Map; /** @@ -49,120 +39,28 @@ */ public class S3TableValuedFunction extends ExternalFileTableValuedFunction { public static final String NAME = "s3"; - public static final String PROP_URI = "uri"; - - private static final ImmutableSet DEPRECATED_KEYS = - ImmutableSet.of("access_key", "secret_key", "session_token", "region", - "ACCESS_KEY", "SECRET_KEY", "SESSION_TOKEN", "REGION"); public S3TableValuedFunction(Map properties) throws AnalysisException { // 1. analyze common properties - Map otherProps = super.parseCommonProperties(properties); - - // 2. analyze uri and other properties - String uriStr = getOrDefaultAndRemove(otherProps, PROP_URI, null); - if (Strings.isNullOrEmpty(uriStr)) { - throw new AnalysisException(String.format("Properties '%s' is required.", PROP_URI)); - } - forwardCompatibleDeprecatedKeys(otherProps); - - String usePathStyle = getOrDefaultAndRemove(otherProps, PropertyConverter.USE_PATH_STYLE, - PropertyConverter.USE_PATH_STYLE_DEFAULT_VALUE); - String forceParsingByStandardUri = getOrDefaultAndRemove(otherProps, - PropertyConverter.FORCE_PARSING_BY_STANDARD_URI, - PropertyConverter.FORCE_PARSING_BY_STANDARD_URI_DEFAULT_VALUE); - - S3URI s3uri = getS3Uri(uriStr, Boolean.parseBoolean(usePathStyle.toLowerCase()), - Boolean.parseBoolean(forceParsingByStandardUri.toLowerCase())); - - // get endpoint first from properties, if not present, get it from s3 uri. - // If endpoint is missing, exception will be thrown. - String endpoint = constructEndpoint(otherProps, s3uri); - if (!otherProps.containsKey(S3Properties.REGION)) { - String region; - if (AzureProperties.checkAzureProviderPropertyExist(properties)) { - // Azure could run without region - region = s3uri.getRegion().orElse("DUMMY-REGION"); - } else { - region = s3uri.getRegion().orElseThrow(() -> new AnalysisException( - String.format("Properties '%s' is required.", S3Properties.REGION))); - } - otherProps.put(S3Properties.REGION, region); - } - checkNecessaryS3Properties(otherProps); - CloudCredentialWithEndpoint credential = new CloudCredentialWithEndpoint(endpoint, - getOrDefaultAndRemove(otherProps, S3Properties.REGION, ""), - getOrDefaultAndRemove(otherProps, S3Properties.ACCESS_KEY, ""), - getOrDefaultAndRemove(otherProps, S3Properties.SECRET_KEY, "")); - if (otherProps.containsKey(S3Properties.SESSION_TOKEN)) { - credential.setSessionToken(getOrDefaultAndRemove(otherProps, S3Properties.SESSION_TOKEN, "")); - } - - locationProperties = S3Properties.credentialToMap(credential); - locationProperties.put(PropertyConverter.USE_PATH_STYLE, usePathStyle); - if (AzureProperties.checkAzureProviderPropertyExist(properties)) { - // For Azure's compatibility, we need bucket to connect to the blob storage's container - locationProperties.put(S3Properties.BUCKET, s3uri.getBucket()); - } - - if (properties.containsKey(S3Properties.ROLE_ARN)) { - locationProperties.put(S3Properties.ROLE_ARN, properties.get(S3Properties.ROLE_ARN)); - if (properties.containsKey(S3Properties.EXTERNAL_ID)) { - locationProperties.put(S3Properties.EXTERNAL_ID, properties.get(S3Properties.EXTERNAL_ID)); - } + Map props = super.parseCommonProperties(properties); + try { + this.storageProperties = StorageProperties.createPrimary(props); + this.backendConnectProperties.putAll(storageProperties.getBackendConfigProperties()); + String uri = storageProperties.validateAndGetUri(props); + filePath = storageProperties.validateAndNormalizeUri(uri); + this.backendConnectProperties.put(URI_KEY, filePath); + } catch (UserException e) { + throw new RuntimeException(e); } - - locationProperties.putAll(S3ClientBEProperties.getBeFSProperties(locationProperties)); - locationProperties.putAll(otherProps); - - filePath = NAME + S3URI.SCHEME_DELIM + s3uri.getBucket() + S3URI.PATH_DELIM + s3uri.getKey(); - if (FeConstants.runningUnitTest) { // Just check - FileSystemFactory.getS3FileSystem(locationProperties); + // Fixme wait to be done #50320 + // FileSystemFactory.get(storageProperties); } else { parseFile(); } } - private String constructEndpoint(Map properties, S3URI s3uri) throws AnalysisException { - // get endpoint first from properties, if not present, get it from s3 uri. - String endpoint = getOrDefaultAndRemove(properties, S3Properties.ENDPOINT, s3uri.getEndpoint().orElse("")); - if (AzureProperties.checkAzureProviderPropertyExist(properties)) { - String accountName = properties.getOrDefault(S3Properties.ACCESS_KEY, ""); - if (accountName.isEmpty()) { - throw new AnalysisException(String.format("Properties '%s' is required.", S3Properties.ACCESS_KEY)); - } - endpoint = AzureProperties.formatAzureEndpoint(endpoint, accountName); - } else if (Strings.isNullOrEmpty(endpoint)) { - throw new AnalysisException(String.format("Properties '%s' is required.", S3Properties.ENDPOINT)); - } - return endpoint; - } - - private void forwardCompatibleDeprecatedKeys(Map props) { - for (String deprecatedKey : DEPRECATED_KEYS) { - String value = props.remove(deprecatedKey); - if (!Strings.isNullOrEmpty(value)) { - props.put("s3." + deprecatedKey.toLowerCase(), value); - } - } - } - - private void checkNecessaryS3Properties(Map props) throws AnalysisException { - if (Strings.isNullOrEmpty(props.get(S3Properties.REGION))) { - throw new AnalysisException(String.format("Properties '%s' is required.", S3Properties.REGION)); - } - // do not check ak and sk, because we can read them from system environment. - } - - private S3URI getS3Uri(String uri, boolean isPathStyle, boolean forceParsingStandardUri) throws AnalysisException { - try { - return S3URI.create(uri, isPathStyle, forceParsingStandardUri); - } catch (UserException e) { - throw new AnalysisException("parse s3 uri failed, uri = " + uri, e); - } - } // =========== implement abstract methods of ExternalFileTableValuedFunction ================= @Override @@ -178,7 +76,7 @@ public String getFilePath() { @Override public BrokerDesc getBrokerDesc() { - return new BrokerDesc("S3TvfBroker", StorageType.S3, locationProperties); + return new BrokerDesc("S3TvfBroker", processedParams); } // =========== implement abstract methods of TableValuedFunctionIf ================= diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java index cbee4a8bcecd5b..a8c2960868adde 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/BackupJobTest.java @@ -18,7 +18,6 @@ package org.apache.doris.backup; import org.apache.doris.analysis.BackupStmt; -import org.apache.doris.analysis.StorageBackend; import org.apache.doris.analysis.TableName; import org.apache.doris.analysis.TableRef; import org.apache.doris.backup.BackupJob.BackupJobState; @@ -29,10 +28,11 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; +import org.apache.doris.common.UserException; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.common.util.UnitTestUtil; import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fsv2.FileSystemFactory; import org.apache.doris.persist.EditLog; import org.apache.doris.task.AgentBatchTask; import org.apache.doris.task.AgentTask; @@ -96,6 +96,9 @@ public class BackupJobTest { private MockRepositoryMgr repoMgr; + public BackupJobTest() throws UserException { + } + // Thread is not mockable in Jmockit, use subclass instead private final class MockBackupHandler extends BackupHandler { public MockBackupHandler(Env env) { @@ -124,7 +127,7 @@ public Repository getRepo(long repoId) { private EditLog editLog; private Repository repo = new Repository(repoId, "repo", false, "my_repo", - FileSystemFactory.get("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap())); + FileSystemFactory.get("broker", Maps.newHashMap()), null); @BeforeClass public static void start() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java index a49d7e4328e94b..c8fec2b64033f6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java @@ -18,13 +18,13 @@ package org.apache.doris.backup; import org.apache.doris.analysis.ShowRepositoriesStmt; -import org.apache.doris.analysis.StorageBackend; import org.apache.doris.catalog.BrokerMgr; import org.apache.doris.catalog.FsBroker; import org.apache.doris.common.AnalysisException; -import org.apache.doris.fs.FileSystemFactory; -import org.apache.doris.fs.remote.RemoteFile; -import org.apache.doris.fs.remote.RemoteFileSystem; +import org.apache.doris.common.UserException; +import org.apache.doris.fsv2.FileSystemFactory; +import org.apache.doris.fsv2.remote.RemoteFile; +import org.apache.doris.fsv2.remote.RemoteFileSystem; import org.apache.doris.service.FrontendOptions; import com.google.common.collect.Lists; @@ -36,6 +36,7 @@ import mockit.Mocked; import org.junit.Assert; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.io.DataInputStream; @@ -80,6 +81,13 @@ String getLocalHostAddress() { return "127.0.0.1"; } }; + new Expectations() { + { + fileSystem.getStorageProperties(); + minTimes = 0; + result = null; + } + }; new MockUp() { @Mock @@ -92,7 +100,7 @@ public FsBroker getBroker(String name, String host) throws AnalysisException { @Test public void testGet() { - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); Assert.assertEquals(repoId, repo.getId()); Assert.assertEquals(name, repo.getName()); @@ -103,7 +111,7 @@ public void testGet() { } @Test - public void testInit() { + public void testInit() throws UserException { new Expectations() { { fileSystem.globList(anyString, (List) any); @@ -114,14 +122,13 @@ public Status list(String remotePath, List result) { return Status.OK; } }; - fileSystem.directUpload(anyString, anyString); minTimes = 0; result = Status.OK; } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); Status st = repo.initRepository(); System.out.println(st); @@ -130,7 +137,7 @@ public Status list(String remotePath, List result) { @Test public void testassemnblePath() throws MalformedURLException, URISyntaxException { - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); // job info String label = "label"; @@ -171,7 +178,7 @@ public void testPing() { } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); Assert.assertTrue(repo.ping()); Assert.assertTrue(repo.getErrorMsg() == null); } @@ -192,7 +199,7 @@ public Status list(String remotePath, List result) { } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); List snapshotNames = Lists.newArrayList(); Status st = repo.listSnapshots(snapshotNames); Assert.assertTrue(st.ok()); @@ -218,7 +225,7 @@ public void testUpload() { } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); String localFilePath = "./tmp_" + System.currentTimeMillis(); try (PrintWriter out = new PrintWriter(localFilePath)) { out.print("a"); @@ -265,7 +272,7 @@ public Status list(String remotePath, List result) { } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); String remoteFilePath = location + "/remote_file"; Status st = repo.download(remoteFilePath, localFilePath); Assert.assertTrue(st.ok()); @@ -276,7 +283,7 @@ public Status list(String remotePath, List result) { @Test public void testGetInfo() { - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); List infos = repo.getInfo(); Assert.assertTrue(infos.size() == ShowRepositoriesStmt.TITLE_NAMES.size()); } @@ -304,7 +311,7 @@ public Status list(String remotePath, List result) { } }; - repo = new Repository(10000, "repo", false, location, fileSystem); + repo = new Repository(10000, "repo", false, location, fileSystem, null); String snapshotName = ""; String timestamp = ""; try { @@ -317,14 +324,15 @@ public Status list(String remotePath, List result) { } } + @Ignore("wait support") @Test - public void testPersist() { + public void testPersist() throws UserException { Map properties = Maps.newHashMap(); properties.put("bos_endpoint", "http://gz.bcebos.com"); properties.put("bos_accesskey", "a"); properties.put("bos_secret_accesskey", "b"); - RemoteFileSystem fs = FileSystemFactory.get(brokerName, StorageBackend.StorageType.BROKER, properties); - repo = new Repository(10000, "repo", false, location, fs); + RemoteFileSystem fs = FileSystemFactory.get(properties); + repo = new Repository(10000, "repo", false, location, fs, null); File file = new File("./Repository"); try { @@ -352,8 +360,9 @@ public void testPersist() { @Test public void testPathNormalize() { + String newLoc = "bos://cmy_bucket/bos_repo/"; - repo = new Repository(10000, "repo", false, newLoc, fileSystem); + repo = new Repository(10000, "repo", false, newLoc, fileSystem, null); String path = repo.getRepoPath("label1", "/_ss_my_ss/_ss_content/__db_10000/"); Assert.assertEquals("bos://cmy_bucket/bos_repo/__palo_repository_repo/__ss_label1/__ss_content/_ss_my_ss/_ss_content/__db_10000/", path); @@ -361,7 +370,7 @@ public void testPathNormalize() { Assert.assertEquals("bos://cmy_bucket/bos_repo/__palo_repository_repo/__ss_label1/__ss_content/_ss_my_ss/_ss_content/__db_10000", path); newLoc = "hdfs://path/to/repo"; - repo = new Repository(10000, "repo", false, newLoc, fileSystem); + repo = new Repository(10000, "repo", false, newLoc, fileSystem, null); SnapshotInfo snapshotInfo = new SnapshotInfo(1, 2, 3, 4, 5, 6, 7, "/path", Lists.newArrayList()); path = repo.getRepoTabletPathBySnapshotInfo("label1", snapshotInfo); Assert.assertEquals("hdfs://path/to/repo/__palo_repository_repo/__ss_label1/__ss_content/__db_1/__tbl_2/__part_3/__idx_4/__5", path); diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java index 300e46a2b4ce9e..50ca8943178cf1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java @@ -17,7 +17,6 @@ package org.apache.doris.backup; -import org.apache.doris.analysis.StorageBackend; import org.apache.doris.backup.BackupJobInfo.BackupIndexInfo; import org.apache.doris.backup.BackupJobInfo.BackupOlapTableInfo; import org.apache.doris.backup.BackupJobInfo.BackupPartitionInfo; @@ -38,9 +37,10 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.MarkedCountDownLatch; +import org.apache.doris.common.UserException; import org.apache.doris.common.jmockit.Deencapsulation; import org.apache.doris.datasource.InternalCatalog; -import org.apache.doris.fs.FileSystemFactory; +import org.apache.doris.fsv2.FileSystemFactory; import org.apache.doris.persist.EditLog; import org.apache.doris.resource.Tag; import org.apache.doris.system.SystemInfoService; @@ -91,6 +91,9 @@ public class RestoreJobTest { private MockRepositoryMgr repoMgr; + public RestoreJobTest() throws UserException { + } + // Thread is not mockable in Jmockit, use subclass instead private final class MockBackupHandler extends BackupHandler { public MockBackupHandler(Env env) { @@ -122,7 +125,7 @@ public Repository getRepo(long repoId) { @Injectable private Repository repo = new Repository(repoId, "repo", false, "bos://my_repo", - FileSystemFactory.get("broker", StorageBackend.StorageType.BROKER, Maps.newHashMap())); + FileSystemFactory.get("broker", Maps.newHashMap()), null); private BackupMeta backupMeta; @@ -245,7 +248,7 @@ boolean await(long timeout, TimeUnit unit) { for (Tablet tablet : index.getTablets()) { List files = Lists.newArrayList(tablet.getId() + ".dat", - tablet.getId() + ".idx", tablet.getId() + ".hdr"); + tablet.getId() + ".idx", tablet.getId() + ".hdr"); BackupTabletInfo tabletInfo = new BackupTabletInfo(tablet.getId(), files); idxInfo.sortedTabletInfoList.add(tabletInfo); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java index ce8158d1bd6efd..b7912e449faa10 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java @@ -58,6 +58,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.Arrays; @@ -93,10 +94,10 @@ public void testOutFileS3PropertiesConverter() throws Exception { + "into outfile 's3://bucket/mock_dir'\n" + "format as csv\n" + "properties(\n" - + " 'AWS_ENDPOINT' = 'http://127.0.0.1:9000',\n" + + " 'AWS_ENDPOINT' = 's3.ap-northeast-1.amazonaws.com',\n" + " 'AWS_ACCESS_KEY' = 'akk',\n" + " 'AWS_SECRET_KEY'='akk',\n" - + " 'AWS_REGION' = 'mock',\n" + + " 'AWS_REGION' = 'ap-northeast-1',\n" + " 'use_path_style' = 'true'\n" + ");"; QueryStmt analyzedOutStmt = createStmt(query); @@ -112,7 +113,7 @@ public void testOutFileS3PropertiesConverter() throws Exception { + "into outfile 's3://bucket/mock_dir'\n" + "format as csv\n" + "properties(\n" - + " 's3.endpoint' = 'http://127.0.0.1:9000',\n" + + " 's3.endpoint' = 'https://s3.ap-northeast-1.amazonaws.com',\n" + " 's3.access_key' = 'akk',\n" + " 's3.secret_key'='akk',\n" + " 'use_path_style' = 'true'\n" @@ -181,7 +182,7 @@ public void testS3RepositoryPropertiesConverter() throws Exception { CreateRepositoryStmt analyzedStmt = createStmt(s3Repo); Assertions.assertEquals(analyzedStmt.getProperties().size(), 4); Repository repository = getRepository(analyzedStmt, "s3_repo"); - Assertions.assertEquals(9, repository.getRemoteFileSystem().getProperties().size()); + Assertions.assertEquals(4, repository.getRemoteFileSystem().getProperties().size()); String s3RepoNew = "CREATE REPOSITORY `s3_repo_new`\n" + "WITH S3\n" @@ -195,7 +196,7 @@ public void testS3RepositoryPropertiesConverter() throws Exception { CreateRepositoryStmt analyzedStmtNew = createStmt(s3RepoNew); Assertions.assertEquals(analyzedStmtNew.getProperties().size(), 3); Repository repositoryNew = getRepository(analyzedStmtNew, "s3_repo_new"); - Assertions.assertEquals(repositoryNew.getRemoteFileSystem().getProperties().size(), 5); + Assertions.assertEquals(repositoryNew.getRemoteFileSystem().getProperties().size(), 3); } private static Repository getRepository(CreateRepositoryStmt analyzedStmt, String name) throws DdlException { @@ -203,6 +204,7 @@ private static Repository getRepository(CreateRepositoryStmt analyzedStmt, Strin return Env.getCurrentEnv().getBackupHandler().getRepoMgr().getRepo(name); } + @Disabled("not support") @Test public void testBosBrokerRepositoryPropertiesConverter() throws Exception { FeConstants.runningUnitTest = true; @@ -241,7 +243,7 @@ public void testS3TVFPropertiesConverter() throws Exception { Assertions.assertEquals(analyzedStmt.getTableRefs().size(), 1); TableValuedFunctionRef oldFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0); S3TableValuedFunction s3Tvf = (S3TableValuedFunction) oldFuncTable.getTableFunction(); - Assertions.assertEquals(10, s3Tvf.getBrokerDesc().getProperties().size()); + Assertions.assertEquals(5, s3Tvf.getBrokerDesc().getProperties().size()); String queryNew = "select * from s3(\n" + " 'uri' = 'http://s3.us-east-1.amazonaws.com/my-bucket/test.parquet',\n" @@ -254,7 +256,7 @@ public void testS3TVFPropertiesConverter() throws Exception { Assertions.assertEquals(analyzedStmtNew.getTableRefs().size(), 1); TableValuedFunctionRef newFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0); S3TableValuedFunction newS3Tvf = (S3TableValuedFunction) newFuncTable.getTableFunction(); - Assertions.assertEquals(10, newS3Tvf.getBrokerDesc().getProperties().size()); + Assertions.assertEquals(5, newS3Tvf.getBrokerDesc().getProperties().size()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyPassThroughTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyPassThroughTest.java index 32a212c5cf7bdc..1f953f6aefc604 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyPassThroughTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyPassThroughTest.java @@ -45,8 +45,8 @@ public void testS3TVFPropertiesPassThrough() throws Exception { Assertions.assertEquals(analyzedStmt.getTableRefs().size(), 1); TableValuedFunctionRef oldFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0); S3TableValuedFunction s3Tvf = (S3TableValuedFunction) oldFuncTable.getTableFunction(); - Assertions.assertTrue(s3Tvf.getBrokerDesc().getProperties().containsKey("fs.s3a.list.version")); - Assertions.assertTrue(s3Tvf.getBrokerDesc().getProperties().containsKey("test_property")); + Assertions.assertFalse(s3Tvf.getBrokerDesc().getBackendConfigProperties().containsKey("fs.s3a.list.version")); + Assertions.assertFalse(s3Tvf.getBrokerDesc().getBackendConfigProperties().containsKey("test_property")); } @Test @@ -63,6 +63,6 @@ public void testHdfsTVFPropertiesPassThrough() throws Exception { Assertions.assertEquals(analyzedStmt.getTableRefs().size(), 1); TableValuedFunctionRef oldFuncTable = (TableValuedFunctionRef) analyzedStmt.getTableRefs().get(0); HdfsTableValuedFunction hdfsTvf = (HdfsTableValuedFunction) oldFuncTable.getTableFunction(); - Assertions.assertTrue(hdfsTvf.getBrokerDesc().getProperties().containsKey("test_property")); + Assertions.assertFalse(hdfsTvf.getBrokerDesc().getBackendConfigProperties().containsKey("test_property")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/AzurePropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/AzurePropertiesTest.java index fadefd43f57c30..fd36521a82875d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/AzurePropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/AzurePropertiesTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -81,8 +82,6 @@ public void testMissingProvider() throws UserException { Assertions.assertEquals(HdfsProperties.class, storagePropertiesList.get(1).getClass()); Assertions.assertEquals(AzureProperties.class, storagePropertiesList.get(0).getClass()); origProps.put("s3.endpoint", "https://mystorageaccount.net"); - Assertions.assertThrows(RuntimeException.class, () -> - StorageProperties.createPrimary(origProps), "No supported storage type found."); // Expect an exception due to missing provider origProps.put("provider", "azure"); Assertions.assertThrows(IllegalArgumentException.class, () -> @@ -130,7 +129,7 @@ public void testParsingUri() throws Exception { AzureProperties azureProperties = (AzureProperties) StorageProperties.createPrimary(origProps); Assertions.assertEquals("s3://mycontainer/blob.txt", azureProperties.validateAndNormalizeUri("https://mystorageaccount.blob.core.windows.net/mycontainer/blob.txt")); - Assertions.assertThrowsExactly(UserException.class, () -> + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> azureProperties.validateAndGetUri(origProps), "props must contain uri"); origProps.put("uri", "https://mystorageaccount.blob.core.windows.net/mycontainer/blob.txt"); @@ -188,7 +187,7 @@ public void testEmptyPath() throws UserException { AzureProperties azureProperties = (AzureProperties) StorageProperties.createPrimary(origProps); // Expect an exception when the path is empty - Assertions.assertThrows(UserException.class, () -> + Assertions.assertThrows(StoragePropertiesException.class, () -> azureProperties.validateAndNormalizeUri(""), "Path cannot be empty."); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/COSPropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/COSPropertiesTest.java index d6f8dce27a5816..294f9c2bea97b1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/COSPropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/COSPropertiesTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -140,6 +141,6 @@ public void testGetRegionWithDefault() throws UserException { cosNoEndpointProps.put("cos.region", "ap-beijing"); origProps.put("uri", "s3://examplebucket-1250000000/test/file.txt"); //not support this case - Assertions.assertThrowsExactly(RuntimeException.class, () -> StorageProperties.createPrimary(cosNoEndpointProps), "Property cos.endpoint is required."); + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> StorageProperties.createPrimary(cosNoEndpointProps), "Property cos.endpoint is required."); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesTest.java index a4dff23d2766d1..84f4c40cbdc53e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesTest.java @@ -19,6 +19,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; @@ -32,10 +33,12 @@ public class HdfsPropertiesTest { - @Test public void testBasicHdfsCreate() throws UserException { // Test 1: Check default authentication type (should be "simple") + Map simpleHdfsProperties = new HashMap<>(); + simpleHdfsProperties.put("uri", "hdfs://test/1.orc"); + Assertions.assertEquals(HdfsProperties.class, StorageProperties.createPrimary(simpleHdfsProperties).getClass()); Map origProps = createBaseHdfsProperties(); List storageProperties = StorageProperties.createAll(origProps); HdfsProperties hdfsProperties = (HdfsProperties) storageProperties.get(0); @@ -101,9 +104,9 @@ public void testBasicHdfsPropertiesCreateByConfigFile() throws UserException { @Test public void testNonParamsException() throws UserException { Map origProps = new HashMap<>(); - Assertions.assertThrowsExactly(RuntimeException.class, () -> StorageProperties.createPrimary(origProps)); + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> StorageProperties.createPrimary(origProps)); origProps.put("nonhdfs", "hdfs://localhost:9000"); - Assertions.assertThrowsExactly(RuntimeException.class, () -> { + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> { StorageProperties.createPrimary(origProps); }); origProps.put(StorageProperties.FS_HDFS_SUPPORT, "true"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtilsTest.java index b01688cd264478..e150af31f3b62d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/HdfsPropertiesUtilsTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -51,10 +52,10 @@ public void testCheckLoadPropsAndReturnUri_missingUriKey() { Map props = new HashMap<>(); props.put("path", "xxx"); - Exception exception = Assertions.assertThrows(UserException.class, () -> { + Exception exception = Assertions.assertThrows(StoragePropertiesException.class, () -> { HdfsPropertiesUtils.validateAndGetUri(props); }); - Assertions.assertEquals("errCode = 2, detailMessage = props must contain uri", exception.getMessage()); + Assertions.assertEquals("props must contain uri", exception.getMessage()); } @Test @@ -81,7 +82,7 @@ public void testConvertUrlToFilePath_blankUri() { Exception exception = Assertions.assertThrows(IllegalArgumentException.class, () -> { HdfsPropertiesUtils.convertUrlToFilePath(uri); }); - Assertions.assertTrue(exception.getMessage().contains("uri is null")); + Assertions.assertTrue(exception.getMessage().contains("Properties 'uri' is required")); } @Test @@ -89,7 +90,7 @@ public void testConstructDefaultFsFromUri_valid() { Map props = new HashMap<>(); props.put("uri", "hdfs://localhost:8020/data"); - String result = HdfsPropertiesUtils.constructDefaultFsFromUri(props); + String result = HdfsPropertiesUtils.extractDefaultFsFromUri(props); Assertions.assertEquals("hdfs://localhost:8020", result); } @@ -98,7 +99,7 @@ public void testConstructDefaultFsFromUri_viewfs() { Map props = new HashMap<>(); props.put("uri", "viewfs://cluster/path"); - String result = HdfsPropertiesUtils.constructDefaultFsFromUri(props); + String result = HdfsPropertiesUtils.extractDefaultFsFromUri(props); Assertions.assertEquals("viewfs://cluster", result); } @@ -106,17 +107,13 @@ public void testConstructDefaultFsFromUri_viewfs() { public void testConstructDefaultFsFromUri_invalidSchema() { Map props = new HashMap<>(); props.put("uri", "obs://bucket/test"); - - Exception exception = Assertions.assertThrows(IllegalArgumentException.class, () -> { - HdfsPropertiesUtils.constructDefaultFsFromUri(props); - }); - Assertions.assertTrue(exception.getMessage().contains("Invalid export path")); + Assertions.assertNull(HdfsPropertiesUtils.extractDefaultFsFromUri(props)); } @Test public void testConstructDefaultFsFromUri_emptyProps() { Map props = new HashMap<>(); - String result = HdfsPropertiesUtils.constructDefaultFsFromUri(props); + String result = HdfsPropertiesUtils.extractDefaultFsFromUri(props); Assertions.assertNull(result); } @@ -125,7 +122,7 @@ public void testConstructDefaultFsFromUri_missingUri() { Map props = new HashMap<>(); props.put("x", "y"); - String result = HdfsPropertiesUtils.constructDefaultFsFromUri(props); + String result = HdfsPropertiesUtils.extractDefaultFsFromUri(props); Assertions.assertNull(result); } @@ -134,7 +131,7 @@ public void testConstructDefaultFsFromUri_blankUri() { Map props = new HashMap<>(); props.put("uri", " "); - String result = HdfsPropertiesUtils.constructDefaultFsFromUri(props); + String result = HdfsPropertiesUtils.extractDefaultFsFromUri(props); Assertions.assertNull(result); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/MinioPropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/MinioPropertiesTest.java new file mode 100644 index 00000000000000..994a00ad1c8b11 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/MinioPropertiesTest.java @@ -0,0 +1,99 @@ +// 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.datasource.property.storage; + +import org.apache.doris.common.UserException; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +public class MinioPropertiesTest { + + private Map origProps; + + @BeforeEach + public void setup() { + origProps = new HashMap<>(); + } + + @Test + public void testValidMinioConfiguration() throws UserException { + origProps.put("s3.endpoint", "http://localhost:9000"); + origProps.put("s3.access_key", "minioAccessKey"); + origProps.put("s3.secret_key", "minioSecretKey"); + + MinioProperties minioProperties = (MinioProperties) StorageProperties.createPrimary(origProps); + + Assertions.assertEquals("http://localhost:9000", minioProperties.getEndpoint()); + Assertions.assertEquals("minioAccessKey", minioProperties.getAccessKey()); + Assertions.assertEquals("minioSecretKey", minioProperties.getSecretKey()); + Assertions.assertEquals("us-east-1", minioProperties.getRegion()); + origProps.remove("s3.endpoint"); + origProps.put("uri", "http://localhost:9000/test/"); + Assertions.assertThrows(IllegalArgumentException.class, () -> + StorageProperties.createPrimary(origProps), "Property endpoint is required."); + origProps.put("s3.endpoint", "http://localhost:9000"); + Assertions.assertDoesNotThrow(() -> StorageProperties.createPrimary(origProps)); + + + } + + @Test + public void testGuessIsMeWithMinio() { + origProps.put("s3.access_key", "minioAccessKey"); + Assertions.assertTrue(MinioProperties.guessIsMe(origProps)); + } + + @Test + public void testMissingAccessKey() { + origProps.put("s3.endpoint", "http://localhost:9000"); + origProps.put("s3.secret_key", "minioSecretKey"); + Assertions.assertThrows(IllegalArgumentException.class, () -> + StorageProperties.createPrimary(origProps), "Property s3.access_key is required."); + } + + @Test + public void testEndpoint() { + origProps.put("s3.endpoint", "not-a-valid-url"); + origProps.put("s3.access_key", "a"); + origProps.put("s3.secret_key", "b"); + Assertions.assertDoesNotThrow(() -> StorageProperties.createPrimary(origProps)); + origProps.put("s3.endpoint", "http://localhost:9000"); + Assertions.assertDoesNotThrow(() -> StorageProperties.createPrimary(origProps)); + } + + @Test + public void testBackendConfigProperties() { + origProps.put("s3.endpoint", "http://localhost:9000"); + origProps.put("s3.access_key", "minioAccessKey"); + origProps.put("s3.secret_key", "minioSecretKey"); + + MinioProperties minioProperties = (MinioProperties) StorageProperties.createPrimary(origProps); + Map backendProps = minioProperties.getBackendConfigProperties(); + + Assertions.assertEquals("http://localhost:9000", backendProps.get("AWS_ENDPOINT")); + Assertions.assertEquals("minioAccessKey", backendProps.get("AWS_ACCESS_KEY")); + Assertions.assertEquals("minioSecretKey", backendProps.get("AWS_SECRET_KEY")); + Assertions.assertEquals("us-east-1", backendProps.get("AWS_REGION")); + } +} + diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OBSPropertyTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OBSPropertyTest.java index 03914f76693d3e..a5993742837c6d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OBSPropertyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OBSPropertyTest.java @@ -117,9 +117,9 @@ public void testGetRegionWithDefault() throws UserException { cosNoEndpointProps.put("obs.access_key", "myCOSAccessKey"); cosNoEndpointProps.put("obs.secret_key", "myCOSSecretKey"); cosNoEndpointProps.put("obs.region", "ap-beijing"); - cosNoEndpointProps.put("uri", "s3://examplebucket-1250000000/test/file.txt"); + cosNoEndpointProps.put("uri", "s3://examplebucket-1250000000/myhuaweicloud.com/test/file.txt"); //not support - Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> StorageProperties.createAll(cosNoEndpointProps), "Property cos.endpoint is required."); + Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> StorageProperties.createPrimary(cosNoEndpointProps), "Property cos.endpoint is required."); } private static String obsAccessKey = ""; diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSHdfsPropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSHdfsPropertiesTest.java index 6a076fdc664270..d499027904e5d2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSHdfsPropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSHdfsPropertiesTest.java @@ -116,7 +116,7 @@ public void testGetStorageName() throws UserException { origProps.put("oss.secret_key", "testSecretKey"); origProps.put("oss.region", "cn-shanghai"); - OSSHdfsProperties props = (OSSHdfsProperties) StorageProperties.createAll(origProps).get(0); + OSSHdfsProperties props = (OSSHdfsProperties) StorageProperties.createPrimary(origProps); Assertions.assertEquals("HDFS", props.getStorageName()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSPropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSPropertiesTest.java index ad0d89f1db8acd..b52e888358b81f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSPropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/OSSPropertiesTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -46,8 +47,7 @@ public void testBasicCreateTest() { origProps = new HashMap<>(); origProps.put("oss.endpoint", "https://oss.aliyuncs.com"); Map finalOrigProps2 = origProps; - Assertions.assertThrowsExactly(RuntimeException.class, () -> StorageProperties.createPrimary(finalOrigProps2)); - + Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> StorageProperties.createPrimary(finalOrigProps2)); } @@ -123,6 +123,6 @@ public void testGetRegionWithDefault() throws UserException { cosNoEndpointProps.put("oss.region", "cn-hangzhou"); origProps.put("uri", "s3://examplebucket-1250000000/test/file.txt"); // not support - Assertions.assertThrowsExactly(RuntimeException.class, () -> StorageProperties.createPrimary(cosNoEndpointProps), "Property cos.endpoint is required."); + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> StorageProperties.createPrimary(cosNoEndpointProps), "Property cos.endpoint is required."); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertiesTest.java index 14a449ab91f4e9..9c397c358aac02 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertiesTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -175,6 +176,6 @@ public void testGetRegionWithDefault() throws UserException { s3EndpointProps.put("oss.region", "cn-hangzhou"); origProps.put("uri", "s3://examplebucket-1250000000/test/file.txt"); //not support - Assertions.assertThrowsExactly(RuntimeException.class, () -> StorageProperties.createPrimary(s3EndpointProps), "Property cos.endpoint is required."); + Assertions.assertThrowsExactly(StoragePropertiesException.class, () -> StorageProperties.createPrimary(s3EndpointProps), "Property cos.endpoint is required."); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertyUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertyUtilsTest.java index 64795761383c3e..6427abd8fda5f9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertyUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/storage/S3PropertyUtilsTest.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.property.storage; import org.apache.doris.common.UserException; +import org.apache.doris.datasource.property.storage.exception.StoragePropertiesException; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -40,13 +41,13 @@ void testCheckLoadPropsAndReturnUri_success() throws UserException { void testCheckLoadPropsAndReturnUri_missingKey() { Map props = new HashMap<>(); Executable executable = () -> S3PropertyUtils.validateAndGetUri(props); - UserException exception = Assertions.assertThrows(UserException.class, executable); - Assertions.assertEquals("errCode = 2, detailMessage = props is empty", exception.getMessage()); + StoragePropertiesException exception = Assertions.assertThrows(StoragePropertiesException.class, executable); + Assertions.assertEquals("props is empty", exception.getMessage()); props.put("someKey", "value"); executable = () -> S3PropertyUtils.validateAndGetUri(props); - exception = Assertions.assertThrows(UserException.class, executable); - Assertions.assertEquals("errCode = 2, detailMessage = props must contain uri", exception.getMessage()); + exception = Assertions.assertThrows(StoragePropertiesException.class, executable); + Assertions.assertEquals("props must contain uri", exception.getMessage()); } @Test @@ -66,7 +67,7 @@ void testConstructEndpointFromUrl_nullOrBlank() throws UserException { Assertions.assertNull(S3PropertyUtils.constructEndpointFromUrl(props, "false", "true")); props.put("uri", "invalid uri without scheme"); - Assertions.assertThrowsExactly(UserException.class, () -> S3PropertyUtils.constructEndpointFromUrl(props, "true", "true")); + Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> S3PropertyUtils.constructEndpointFromUrl(props, "true", "true")); } @Test @@ -86,7 +87,7 @@ void testConstructRegionFromUrl_nullOrInvalid() throws UserException { Assertions.assertNull(S3PropertyUtils.constructRegionFromUrl(props, "false", "true")); props.put("uri", "not a uri"); - Assertions.assertThrowsExactly(UserException.class, () -> S3PropertyUtils.constructRegionFromUrl(props, "false", "true")); + Assertions.assertThrowsExactly(IllegalArgumentException.class, () -> S3PropertyUtils.constructRegionFromUrl(props, "false", "true")); props.put("uri", "https://my-bucket.s3.us-west-1.amazonaws.com/test.txt"); Assertions.assertEquals("us-west-1", S3PropertyUtils.constructRegionFromUrl(props, "false", "true")); } @@ -103,8 +104,8 @@ void testConvertToS3Address_success() throws UserException { @Test void testConvertToS3Address_invalid() { - Assertions.assertThrows(UserException.class, () -> S3PropertyUtils.validateAndNormalizeUri(null, "false", "true")); - Assertions.assertThrows(UserException.class, () -> S3PropertyUtils.validateAndNormalizeUri("", "false", "false")); + Assertions.assertThrows(StoragePropertiesException.class, () -> S3PropertyUtils.validateAndNormalizeUri(null, "false", "true")); + Assertions.assertThrows(StoragePropertiesException.class, () -> S3PropertyUtils.validateAndNormalizeUri("", "false", "false")); Assertions.assertThrows(UserException.class, () -> S3PropertyUtils.validateAndNormalizeUri("not a uri", "true", "true")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/fs/obj/S3FileSystemTest.java b/fe/fe-core/src/test/java/org/apache/doris/fsv2/obj/S3FileSystemTest.java similarity index 92% rename from fe/fe-core/src/test/java/org/apache/doris/fs/obj/S3FileSystemTest.java rename to fe/fe-core/src/test/java/org/apache/doris/fsv2/obj/S3FileSystemTest.java index 442883573ce49a..b36d0b4aba46b9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/fs/obj/S3FileSystemTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/fsv2/obj/S3FileSystemTest.java @@ -15,16 +15,19 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.fs.obj; +package org.apache.doris.fsv2.obj; import org.apache.doris.backup.Repository; import org.apache.doris.backup.Status; import org.apache.doris.common.UserException; import org.apache.doris.common.util.S3URI; import org.apache.doris.datasource.property.PropertyConverter; -import org.apache.doris.fs.FileSystemFactory; -import org.apache.doris.fs.remote.RemoteFile; -import org.apache.doris.fs.remote.S3FileSystem; +import org.apache.doris.datasource.property.storage.AbstractS3CompatibleProperties; +import org.apache.doris.datasource.property.storage.StorageProperties; +import org.apache.doris.fs.obj.MockedS3Client; +import org.apache.doris.fsv2.FileSystemFactory; +import org.apache.doris.fsv2.remote.RemoteFile; +import org.apache.doris.fsv2.remote.S3FileSystem; import mockit.Mock; import mockit.MockUp; @@ -70,7 +73,7 @@ public void setUp() throws Exception { properties = new HashMap<>(); properties.put("AWS_ACCESS_KEY", System.getenv().getOrDefault("AWS_AK", "")); properties.put("AWS_SECRET_KEY", System.getenv().getOrDefault("AWS_SK", "")); - properties.put("AWS_ENDPOINT", "http://s3.bj.bcebos.com"); + properties.put("AWS_ENDPOINT", "http://s3.ap-northeast-1.amazonaws.com"); properties.put(PropertyConverter.USE_PATH_STYLE, "false"); properties.put("AWS_REGION", "bj"); content = @@ -102,10 +105,10 @@ public S3Client getClient() throws UserException { return mockedClient; } }; - S3ObjStorage mockedStorage = new S3ObjStorage(properties); + S3ObjStorage mockedStorage = new S3ObjStorage((AbstractS3CompatibleProperties) StorageProperties.createPrimary(properties)); Assertions.assertTrue(mockedStorage.getClient() instanceof MockedS3Client); // inject storage to file system. - fileSystem = new S3FileSystem(mockedStorage); + fileSystem = (S3FileSystem) FileSystemFactory.get(properties); new MockUp(S3FileSystem.class) { @Mock public Status globList(String remotePath, List result, boolean fileNameOnly) { @@ -124,7 +127,7 @@ public Status globList(String remotePath, List result, boolean fileN }; } else { // can also real file system to test. - fileSystem = (S3FileSystem) FileSystemFactory.getS3FileSystem(properties); + fileSystem = (S3FileSystem) FileSystemFactory.get(properties); } testFile = bucket + basePath + "/Ode_to_the_West_Wind"; Assertions.assertEquals(Status.OK, fileSystem.directUpload(content, testFile)); @@ -168,7 +171,8 @@ public void upload() throws IOException { @Test public void testRepositoryUpload() throws IOException { - Repository repo = new Repository(10000, "repo", false, bucket + basePath, fileSystem); + Repository repo = new Repository(10000, "repo", false, bucket + basePath, fileSystem, + null); File localFile = File.createTempFile("s3unittest", ".dat"); localFile.deleteOnExit(); String remote = bucket + basePath + "/" + localFile.getName(); diff --git a/fe/pom.xml b/fe/pom.xml index 7c4dbb0551c42d..83dbd1d764c4cc 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -339,6 +339,7 @@ under the License. 1.12.669 3.0.9 3.3.6 + 1.8 1.2.0 1.1.1 2.4.9 @@ -628,6 +629,11 @@ under the License. + + com.google.re2j + re2j + ${re2j.version} + org.apache.hadoop hadoop-client-api diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy index 39b3c50f94b658..98649d51b25f85 100644 --- a/regression-test/conf/regression-conf.groovy +++ b/regression-test/conf/regression-conf.groovy @@ -248,6 +248,7 @@ enableTrinoConnectorTest = false enableKerberosTest=false kerberosHmsPort=9883 kerberosHdfsPort=8820 +enableRefactorParamsHdfsTest=true // cloud metaServiceToken = "greedisgood9999" diff --git a/regression-test/data/external_table_p0/refactor_storage_param/test_outfile_s3_storage.out b/regression-test/data/external_table_p0/refactor_storage_param/test_outfile_s3_storage.out new file mode 100644 index 00000000000000..bb72b202ca35ec --- /dev/null +++ b/regression-test/data/external_table_p0/refactor_storage_param/test_outfile_s3_storage.out @@ -0,0 +1,85 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_export -- +1 ftw-1 19 +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 +10 \N \N + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + +-- !s3_tvf_1_http -- +1 ftw-1 19 +10 \N \N +2 ftw-2 20 +3 ftw-3 21 +4 ftw-4 22 +5 ftw-5 23 +6 ftw-6 24 +7 ftw-7 25 +8 ftw-8 26 +9 ftw-9 27 + diff --git a/regression-test/pipeline/external/conf/regression-conf.groovy b/regression-test/pipeline/external/conf/regression-conf.groovy index 7a3f5b768522de..782fb456799506 100644 --- a/regression-test/pipeline/external/conf/regression-conf.groovy +++ b/regression-test/pipeline/external/conf/regression-conf.groovy @@ -123,6 +123,8 @@ hive3PgPort=5732 // See `docker/thirdparties/start-thirdparties-docker.sh` enableKafkaTest=true kafka_port=19193 +// refactor params +enableRefactorParamsTest=true // iceberg test config iceberg_rest_uri_port=18181 diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_azure.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_azure.groovy new file mode 100644 index 00000000000000..6c5b4547fba90a --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_azure.groovy @@ -0,0 +1,182 @@ +// 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. +import org.awaitility.Awaitility; +import static java.util.concurrent.TimeUnit.SECONDS; +import static groovy.test.GroovyAssert.shouldFail + +suite("refactor_storage_backup_restore_azure", "p0,external") { + + String enabled = context.config.otherConfigs.get("enableAzureBackupRestoreTest") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return ; + } + String objPrefix = "azure" + String container = context.config.otherConfigs.get("azure.container") + String account =context.config.otherConfigs.get("azure.account") + String s3_endpoint = "${account}.blob.core.windows.net" + String ak = context.config.otherConfigs.get("azure.ak") + String sk = context.config.otherConfigs.get("azure.sk") + + def s3table = "test_backup_restore_azure"; + + def databaseQueryResult = sql """ + select database(); + """ + println databaseQueryResult + def currentDBName = databaseQueryResult.get(0).get(0) + println currentDBName + // cos + + def createDBAndTbl = { String dbName -> + + sql """ + drop database if exists ${dbName} + """ + + sql """ + create database ${dbName} + """ + + sql """ + use ${dbName} + """ + sql """ + CREATE TABLE ${s3table}( + user_id BIGINT NOT NULL COMMENT "user id", + name VARCHAR(20) COMMENT "name", + age INT COMMENT "age" + ) + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${s3table} values (1, 'a', 10); + """ + + def insertResult = sql """ + SELECT count(1) FROM ${s3table} + """ + + println "insertResult: ${insertResult}" + + assert insertResult.get(0).get(0) == 1 + } + + def createRepository = { String repoName, String endpointName, String endpoint, String regionName, String region, String accessKeyName, String accessKey, String secretKeyName, String secretKey, String usePathStyle, String location -> + try { + sql """ + drop repository ${repoName}; + """ + } catch (Exception e) { + // ignore exception, repo may not exist + } + + sql """ + CREATE REPOSITORY ${repoName} + WITH S3 + ON LOCATION "${location}" + PROPERTIES ( + "${endpointName}" = "${endpoint}", + "${regionName}" = "${region}", + "${accessKeyName}" = "${accessKey}", + "${secretKeyName}" = "${secretKey}", + "provider"="azure", + "use_path_style" = "${usePathStyle}" + ); + """ + } + + def backupAndRestore = { String repoName, String dbName, String tableName, String backupLabel -> + sql """ + BACKUP SNAPSHOT ${dbName}.${backupLabel} + TO ${repoName} + ON (${tableName}) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + def backupResult = sql """ + show backup from ${dbName} where SnapshotName = '${backupLabel}'; + """ + println "backupResult: ${backupResult}" + return backupResult.get(0).get(3) == "FINISHED" + }) + + def querySnapshotResult = sql """ + SHOW SNAPSHOT ON ${repoName} WHERE SNAPSHOT = '${backupLabel}'; + """ + println querySnapshotResult + def snapshotTimes = querySnapshotResult.get(0).get(1).split('\n') + def snapshotTime = snapshotTimes[0] + + sql """ + drop table if exists ${tableName}; + """ + + sql """ + RESTORE SNAPSHOT ${dbName}.${backupLabel} + FROM ${repoName} + ON (`${tableName}`) + PROPERTIES + ( + "backup_timestamp"="${snapshotTime}", + "replication_num" = "1" + ); + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + try { + + sql """ + use ${dbName} + """ + def restoreResult = sql """ + SELECT count(1) FROM ${tableName} + """ + println "restoreResult: ${restoreResult}" + def count = restoreResult.get(0).get(0) + println "count: ${count}" + return restoreResult.get(0).get(0) == 1 + } catch (Exception e) { + // tbl not found + println "tbl not found" + e.getMessage() + return false + } + }) + } + + + def s3repoName1 = "azure_repo_1" + createRepository("${s3repoName1}", "s3.endpoint", s3_endpoint, "s3.region", "", "s3.access_key", ak, "s3.secret_key", sk, "true", "s3://${container}/test_" + System.currentTimeMillis()) + + def dbName1 = currentDBName + "${objPrefix}_1" + createDBAndTbl("${dbName1}") + backupAndRestore("${s3repoName1}", dbName1, s3table, "backup_${s3repoName1}_test") + def s3repoName2 = "${objPrefix}_repo_2" + createRepository("${s3repoName2}", "s3.endpoint", s3_endpoint, "s3.region", "", "s3.access_key", ak, "s3.secret_key", sk, "true", "https://${s3_endpoint}/${container}/test_" + System.currentTimeMillis()) + def dbName2 = currentDBName + "${objPrefix}_2" + createDBAndTbl("${dbName2}") + backupAndRestore("${s3repoName2}", dbName2, s3table, "backup_${s3repoName2}_test") + String failedRepoName = "azure_failed_repo" + shouldFail { + createRepository("${failedRepoName}", "s3.endpoint", s3_endpoint, "s3.region", "", "s3.access_key", ak, "s3.secret_key", sk, "false", "https://${s3_endpoint}/${container}/test_" + System.currentTimeMillis()) + } + + +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_object_storage.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_object_storage.groovy new file mode 100644 index 00000000000000..179a33c19d1beb --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/backup_restore_object_storage.groovy @@ -0,0 +1,300 @@ +// 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. +import org.awaitility.Awaitility; +import static java.util.concurrent.TimeUnit.SECONDS; +import static groovy.test.GroovyAssert.shouldFail + +suite("refactor_storage_backup_restore_object_storage", "p0,external,external_docker") { + String enabled = context.config.otherConfigs.get("enableRefactorParamsTest") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return + } + def s3table = "test_backup_restore"; + + def databaseQueryResult = sql """ + select database(); + """ + println databaseQueryResult + def currentDBName = 'refactor_repo' + println currentDBName + // cos + + def createDBAndTbl = { String dbName -> + + sql """ + drop database if exists ${dbName} + """ + + sql """ + create database ${dbName} + """ + + sql """ + use ${dbName} + """ + sql """ + CREATE TABLE ${s3table}( + user_id BIGINT NOT NULL COMMENT "user id", + name VARCHAR(20) COMMENT "name", + age INT COMMENT "age" + ) + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${s3table} values (1, 'a', 10); + """ + + def insertResult = sql """ + SELECT count(1) FROM ${s3table} + """ + + println "insertResult: ${insertResult}" + + assert insertResult.get(0).get(0) == 1 + } + + def createRepository = { String repoName, String endpointName, String endpoint, String regionName, String region, String accessKeyName, String accessKey, String secretKeyName, String secretKey, String usePathStyle, String location -> + try { + sql """ + drop repository ${repoName}; + """ + } catch (Exception e) { + // ignore exception, repo may not exist + } + + sql """ + CREATE REPOSITORY ${repoName} + WITH S3 + ON LOCATION "${location}" + PROPERTIES ( + "${endpointName}" = "${endpoint}", + "${regionName}" = "${region}", + "${accessKeyName}" = "${accessKey}", + "${secretKeyName}" = "${secretKey}", + "use_path_style" = "${usePathStyle}" + ); + """ + } + + def backupAndRestore = { String repoName, String dbName, String tableName, String backupLabel -> + sql """ + BACKUP SNAPSHOT ${dbName}.${backupLabel} + TO ${repoName} + ON (${tableName}) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + def backupResult = sql """ + show backup from ${dbName} where SnapshotName = '${backupLabel}'; + """ + println "backupResult: ${backupResult}" + return backupResult.get(0).get(3) == "FINISHED" + }) + + def querySnapshotResult = sql """ + SHOW SNAPSHOT ON ${repoName} WHERE SNAPSHOT = '${backupLabel}'; + """ + println querySnapshotResult + def snapshotTimes = querySnapshotResult.get(0).get(1).split('\n') + def snapshotTime = snapshotTimes[0] + + sql """ + drop table if exists ${tableName}; + """ + + sql """ + RESTORE SNAPSHOT ${dbName}.${backupLabel} + FROM ${repoName} + ON (`${tableName}`) + PROPERTIES + ( + "backup_timestamp"="${snapshotTime}", + "replication_num" = "1" + ); + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + try { + + sql """ + use ${dbName} + """ + def restoreResult = sql """ + SELECT count(1) FROM ${tableName} + """ + println "restoreResult: ${restoreResult}" + def count = restoreResult.get(0).get(0) + println "count: ${count}" + return restoreResult.get(0).get(0) == 1 + } catch (Exception e) { + // tbl not found + println "tbl not found"+e.getMessage() + return false + } + }) + } + + + + def test_backup_restore= {String ak,String sk,String s3_endpoint,String region,String bucket,String objPrefix -> + def s3repoName1 = "${objPrefix}_repo_1" + createRepository("${s3repoName1}", "s3.endpoint", s3_endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true", "s3://${bucket}/test_" + System.currentTimeMillis()) + + def dbName1 = currentDBName + "${objPrefix}_1" + createDBAndTbl("${dbName1}") + backupAndRestore("${s3repoName1}", dbName1, s3table, "backup_${s3repoName1}_test") + def s3repoName2 = "${objPrefix}_repo_2" + createRepository("${s3repoName2}", "s3.endpoint", s3_endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false", "s3://${bucket}/test_" + System.currentTimeMillis()) + def dbName2 = currentDBName + "${objPrefix}_2" + createDBAndTbl("${dbName2}") + backupAndRestore("${s3repoName2}", dbName2, s3table, "backup_${s3repoName2}_test") + + def s3repoName3 = "${objPrefix}_repo_3" + createRepository("${s3repoName3}", "s3.endpoint", s3_endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "", "s3://${bucket}/test_" + System.currentTimeMillis()) + def dbName3 = currentDBName + "${objPrefix}_3" + createDBAndTbl("${dbName3}") + backupAndRestore("${s3repoName3}", dbName3, s3table, "backup_${s3repoName3}_test") + + def s3repoName4 = "${objPrefix}_s3_repo_4" + createRepository("${s3repoName4}", "s3.endpoint", s3_endpoint, "s3.region", region, "AWS_ACCESS_KEY", ak, "AWS_SECRET_KEY", sk, "true", "s3://${bucket}/test_" + System.currentTimeMillis()) + def dbName4 = currentDBName + "${objPrefix}_4" + createDBAndTbl("${dbName4}") + backupAndRestore("${s3repoName4}", dbName4, s3table, "backup_${s3repoName4}_test") + def s3repoName5 = "${objPrefix}_s3_repo_5" + createRepository("${s3repoName5}", "s3.endpoint", s3_endpoint, "s3.region", region, "AWS_ACCESS_KEY", ak, "AWS_SECRET_KEY", sk, "false", "s3://${bucket}/test_" + System.currentTimeMillis()) + def dbName5 = currentDBName + "${objPrefix}_5" + createDBAndTbl("${dbName5}") + backupAndRestore("${s3repoName5}", dbName5, s3table, "backup_${s3repoName5}_test") + def s3repoName6 = "${objPrefix}_s3_repo_6" + createRepository("${s3repoName6}", "AWS_ENDPOINT", s3_endpoint, "AWS_REGION", region, "AWS_ACCESS_KEY", ak, "AWS_SECRET_KEY", sk, "false", "s3://${bucket}/test_" + System.currentTimeMillis()) + def dbName6 = currentDBName + "${objPrefix}_6" + createDBAndTbl("${dbName6}") + backupAndRestore("${s3repoName6}", dbName6, s3table, "backup_${s3repoName6}_test") + def s3repoName7 = "${objPrefix}_s3_repo_7" + createRepository("${s3repoName7}", "s3.endpoint", s3_endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "", "https://${bucket}/test_" + System.currentTimeMillis()) + def dbName7 = currentDBName + "${objPrefix}_7" + + createDBAndTbl("${dbName7}") + backupAndRestore("${s3repoName7}", dbName7, s3table, "backup_${s3repoName7}_test") + def failedRepoName = "s3_repo_failed" + // wrong address + shouldFail { + createRepository("${failedRepoName}", "s3.endpoint", s3_endpoint, "s3.region", region, "AWS_ACCESS_KEY", ak, "AWS_SECRET_KEY", sk, "true", "s3://ck/" + System.currentTimeMillis()) + } + //endpoint is empty + shouldFail { + createRepository("${failedRepoName}", "s3.endpoint", "", "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "", "s3://${bucket}/test_" + System.currentTimeMillis()) + } + //region is empty + shouldFail { + createRepository("${failedRepoName}", "s3.endpoint", "", "s3.region", "", "s3.access_key", ak, "s3.secret_key", sk, "", "s3://${bucket}/test_" + System.currentTimeMillis()) + } + } + /*-------------AWS S3--------------------------------*/ + String ak = context.config.otherConfigs.get("AWSAK") + String sk = context.config.otherConfigs.get("AWSSK") + String s3_endpoint = "s3.ap-northeast-1.amazonaws.com" + String region = "ap-northeast-1" + String bucket = "selectdb-qa-datalake-test" + String objPrefix="s3" + test_backup_restore(ak,sk,s3_endpoint,region,bucket,objPrefix) + //todo When the new fs is fully enabled, we need to open this startup + String enabledOtherObjectStorageTest = context.config.otherConfigs.get("enabledOtherObjectStorageTest") + if (enabledOtherObjectStorageTest == null || enabledOtherObjectStorageTest.equalsIgnoreCase("false")) { + return + } + /*-----------------Tencent COS----------------*/ + ak = context.config.otherConfigs.get("txYunAk") + sk = context.config.otherConfigs.get("txYunSk") + s3_endpoint = "cos.ap-beijing.myqcloud.com" + region = "ap-beijing" + bucket = "doris-build-1308700295"; + + objPrefix="cos" + test_backup_restore(ak,sk,s3_endpoint,region,bucket,objPrefix) + /* cos_url */ + def cos_repoName1 = "${objPrefix}_repo_cos_prefix_1" + // url is : cos://bucket/prefix/ + createRepository("${cos_repoName1}", "cos.endpoint", s3_endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "true", "cos://${bucket}/test_" + System.currentTimeMillis()) + + def cosDbName1 = currentDBName + "${objPrefix}_cos_1" + createDBAndTbl("${cosDbName1}") + backupAndRestore("${cos_repoName1}", cosDbName1, s3table, "backup_${cos_repoName1}_test") + def cos_repoName2 = "${objPrefix}_repo_cos_prefix_2" + // url is : cos://bucket/prefix/ + createRepository("${cos_repoName2}", "cos.endpoint", s3_endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false", "https://${bucket}.${s3_endpoint}/test_" + System.currentTimeMillis()) + + def cosDbName2 = currentDBName + "${objPrefix}_cos_2" + createDBAndTbl("${cosDbName2}") + backupAndRestore("${cos_repoName2}", cosDbName2, s3table, "backup_${cos_repoName1}_test") + + + + /*-----------------Huawei OBS----------------*/ + ak = context.config.otherConfigs.get("hwYunAk") + sk = context.config.otherConfigs.get("hwYunSk") + s3_endpoint = "obs.cn-north-4.myhuaweicloud.com" + region = "cn-north-4" + bucket = "doris-build"; + objPrefix="obs" + test_backup_restore(ak,sk,s3_endpoint,region,bucket,objPrefix) + def obs_repoName1 = "${objPrefix}_repo_obs_prefix_1" + // url is : cos://bucket/prefix/ + createRepository("${obs_repoName1}", "obs.endpoint", s3_endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "true", "obs://${bucket}/test_" + System.currentTimeMillis()) + + def obsDbName1 = currentDBName + "${objPrefix}_obs_1" + createDBAndTbl("${obsDbName1}") + backupAndRestore("${obs_repoName1}", obsDbName1, s3table, "backup_${obs_repoName1}_test") + def obs_repoName2 = "${objPrefix}_repo_obs_prefix_2" + // url is : cos://bucket/prefix/ + createRepository("${obs_repoName2}", "obs.endpoint", s3_endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false", "https://${bucket}.${s3_endpoint}/test_" + System.currentTimeMillis()) + + def obsDbName2 = currentDBName + "${objPrefix}_obs_2" + createDBAndTbl("${obsDbName2}") + backupAndRestore("${obs_repoName2}", obsDbName2, s3table, "backup_${obs_repoName1}_test") + + + /*-----------------Aliyun OSS----------------*/ + ak = context.config.otherConfigs.get("aliYunAk") + sk = context.config.otherConfigs.get("aliYunSk") + s3_endpoint = "oss-cn-hongkong.aliyuncs.com" + region = "oss-cn-hongkong" + bucket = "doris-regression-hk"; + objPrefix="oss" + // oss has some problem, so we comment it. + //test_backup_restore(ak,sk,s3_endpoint,region,bucket,objPrefix) + def oss_repoName1 = "${objPrefix}_repo_oss_prefix_1" + // url is : cos://bucket/prefix/ + createRepository("${oss_repoName1}", "oss.endpoint", s3_endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false", "oss://${bucket}/test_" + System.currentTimeMillis()) + + def ossDbName1 = currentDBName + "${objPrefix}_oss_1" + createDBAndTbl("${ossDbName1}") + backupAndRestore("${oss_repoName1}", ossDbName1, s3table, "backup_${oss_repoName1}_test") + def oss_repoName2 = "${objPrefix}_repo_oss_prefix_2" + // url is : cos://bucket/prefix/ + createRepository("${oss_repoName2}", "oss.endpoint", s3_endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false", "https://${bucket}.${s3_endpoint}/test_" + System.currentTimeMillis()) + + def ossDbName2 = currentDBName + "${objPrefix}_oss_2" + createDBAndTbl("${ossDbName2}") + backupAndRestore("${oss_repoName2}", ossDbName2, s3table, "backup_${oss_repoName1}_test") + + +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/hdfs_all_test.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/hdfs_all_test.groovy new file mode 100644 index 00000000000000..7c0a79ee7748b4 --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/hdfs_all_test.groovy @@ -0,0 +1,319 @@ +// 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. +import org.awaitility.Awaitility; +import static java.util.concurrent.TimeUnit.SECONDS; +import static groovy.test.GroovyAssert.shouldFail + +suite("refactor_params_hdfs_all_test", "p0,external,kerberos,external_docker,external_docker_kerberos") { + String enabled = context.config.otherConfigs.get("refactor_params_hdfs_kerberos_test") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return + } + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def keytab_root_dir = "/keytabs" + def keytab_dir = "${keytab_root_dir}/hive-presto-master.keytab" + def table = "hdfs_all_test"; + + def databaseQueryResult = sql """ + select database(); + """ + println databaseQueryResult + def currentDBName = 'refactor_params_hdfs_all_test' + println currentDBName + // cos + + def createDBAndTbl = { String dbName -> + + sql """ + drop database if exists ${dbName} + """ + + sql """ + create database ${dbName} + """ + + sql """ + use ${dbName} + """ + sql """ + CREATE TABLE ${table}( + user_id BIGINT NOT NULL COMMENT "user id", + name VARCHAR(20) COMMENT "name", + age INT COMMENT "age" + ) + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${table} values (1, 'a', 10); + """ + + def insertResult = sql """ + SELECT count(1) FROM ${table} + """ + + println "insertResult: ${insertResult}" + + assert insertResult.get(0).get(0) == 1 + } + + def hdfsNonXmlParams = "\"fs.defaultFS\" = \"hdfs://${externalEnvIp}:8520\",\n" + + "\"hadoop.kerberos.min.seconds.before.relogin\" = \"5\",\n" + + "\"hadoop.security.authentication\" = \"kerberos\",\n" + + "\"hadoop.kerberos.principal\"=\"hive/presto-master.docker.cluster@LABS.TERADATA.COM\",\n" + + "\"hadoop.kerberos.keytab\" = \"${keytab_dir}\",\n" + + "\"hive.metastore.sasl.enabled \" = \"true\",\n" + + "\"hadoop.security.auth_to_local\" = \"RULE:[2:\\\$1@\\\$0](.*@LABS.TERADATA.COM)s/@.*//\n" + + " RULE:[2:\\\$1@\\\$0](.*@OTHERLABS.TERADATA.COM)s/@.*//\n" + + " RULE:[2:\\\$1@\\\$0](.*@OTHERREALM.COM)s/@.*//\n" + + " DEFAULT\"" + + def createRepository = { String repoName, String location, String hdfsParams -> + try { + sql """ + drop repository ${repoName}; + """ + } catch (Exception e) { + // ignore exception, repo may not exist + } + + sql """ + CREATE REPOSITORY ${repoName} + WITH HDFS + ON LOCATION "${location}" + PROPERTIES ( + ${hdfsParams} + ); + """ + } + + def backupAndRestore = { String repoName, String dbName, String tableName, String backupLabel -> + sql """ + BACKUP SNAPSHOT ${dbName}.${backupLabel} + TO ${repoName} + ON (${tableName}) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + def backupResult = sql """ + show backup from ${dbName} where SnapshotName = '${backupLabel}'; + """ + println "backupResult: ${backupResult}" + return backupResult.get(0).get(3) == "FINISHED" + }) + + def querySnapshotResult = sql """ + SHOW SNAPSHOT ON ${repoName} WHERE SNAPSHOT = '${backupLabel}'; + """ + println querySnapshotResult + def snapshotTimes = querySnapshotResult.get(0).get(1).split('\n') + def snapshotTime = snapshotTimes[0] + + sql """ + drop table if exists ${tableName}; + """ + + sql """ + RESTORE SNAPSHOT ${dbName}.${backupLabel} + FROM ${repoName} + ON (`${tableName}`) + PROPERTIES + ( + "backup_timestamp"="${snapshotTime}", + "replication_num" = "1" + ); + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until( + { + try { + + sql """ + use ${dbName} + """ + def restoreResult = sql """ + SELECT count(1) FROM ${tableName} + """ + println "restoreResult: ${restoreResult}" + def count = restoreResult.get(0).get(0) + println "count: ${count}" + return restoreResult.get(0).get(0) == 1 + } catch (Exception e) { + // tbl not found + println "tbl not found" + e.getMessage() + return false + } + }) + } + def hdfs_tvf = { filePath, hdfsParam -> + + def hdfs_tvf_sql = sql """ + select * from hdfs + + ( + 'uri'='${filePath}', + "format" = "csv", + ${hdfsParam} + ); + """ + } + def export_hdfs = { defaultFs, hdfsParams -> + def exportPath = defaultFs + "/test/_export/" + System.currentTimeMillis() + def exportLabel = "export_" + System.currentTimeMillis(); + sql """ + EXPORT TABLE ${table} + TO "${exportPath}" + PROPERTIES + ( + "label"="${exportLabel}", + "line_delimiter" = "," + ) + with HDFS + ( + + ${hdfsParams} + ); + """ + + databaseQueryResult = sql """ + select database(); + """ + currentDBName = databaseQueryResult.get(0).get(0) + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until({ + def exportResult = sql """ + SHOW EXPORT FROM ${currentDBName} WHERE LABEL = "${exportLabel}"; + + """ + + println exportResult + + if (null == exportResult || exportResult.isEmpty() || null == exportResult.get(0) || exportResult.get(0).size() < 3) { + return false; + } + if (exportResult.get(0).get(2) == 'CANCELLED' || exportResult.get(0).get(2) == 'FAILED') { + throw new RuntimeException("load failed") + } + + return exportResult.get(0).get(2) == 'FINISHED' + }) + + } + def outfile_to_hdfs = { defaultFs, hdfsParams -> + def outFilePath = "${defaultFs}/outfile_different_hdfs/exp_" + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${table} ORDER BY user_id + INTO OUTFILE "${outFilePath}" + FORMAT AS CSV + PROPERTIES ( + ${hdfsParams} + ); + """ + return res[0][3] + } + def hdfsLoad = { filePath, hdfsParams -> + databaseQueryResult = sql """ + select database(); + """ + println databaseQueryResult + def dataCountResult = sql """ + SELECT count(*) FROM ${table} + """ + def dataCount = dataCountResult[0][0] + def label = "hdfs_load_label_" + System.currentTimeMillis() + def load = sql """ + LOAD LABEL `${label}` ( + data infile ("${filePath}") + into table ${table} + COLUMNS TERMINATED BY "\\\t" + FORMAT AS "CSV" + ( + user_id, + name, + age + )) + with hdfs + ( + ${hdfsParams} + ) + PROPERTIES + ( + "timeout" = "3600" + ); + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until({ + def loadResult = sql """ + show load where label = '${label}'; + """ + println 'test' + println loadResult + + if (null == loadResult || loadResult.isEmpty() || null == loadResult.get(0) || loadResult.get(0).size() < 3) { + return false; + } + if (loadResult.get(0).get(2) == 'CANCELLED' || loadResult.get(0).get(2) == 'FAILED') { + throw new RuntimeException("load failed") + } + + return loadResult.get(0).get(2) == 'FINISHED' + }) + + + def expectedCount = dataCount + 1 + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until({ + def loadResult = sql """ + select count(*) from ${table} + """ + println "loadResult: ${loadResult} " + return loadResult.get(0).get(0) == expectedCount + }) + + } + def repoName = 'hdfs_repo'; + // create repo + createRepository(repoName,"hdfs://${externalEnvIp}:8520/test_repo",hdfsNonXmlParams); + def dbName1 = currentDBName + "${repoName}_1" + createDBAndTbl(dbName1) + def backupLabel=repoName+System.currentTimeMillis() + //backup and restore + backupAndRestore(repoName,dbName1,table,backupLabel) + def failedRepoName='failedRepo' + shouldFail { + createRepository(failedRepoName,"s3://172.20.32.136:8520",hdfsNonXmlParams); + } + shouldFail { + createRepository(failedRepoName," ",hdfsNonXmlParams); + } + + //outfile + dbName1 = currentDBName + 'outfile_test_1' + createDBAndTbl(dbName1) + def outfile = outfile_to_hdfs("hdfs://${externalEnvIp}:8520", hdfsNonXmlParams); + println outfile + //hdfs tvf + def hdfsTvfResult = hdfs_tvf(outfile, hdfsNonXmlParams) + println hdfsTvfResult + + //hdfsLoad(outfile,hdfsNonXmlParams) + + //export + export_hdfs("hdfs://${externalEnvIp}:8520", hdfsNonXmlParams) + + +} \ No newline at end of file diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy new file mode 100644 index 00000000000000..d9ed6a401896c8 --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy @@ -0,0 +1,285 @@ +// 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. +import org.awaitility.Awaitility + +import static groovy.test.GroovyAssert.shouldFail; +import static java.util.concurrent.TimeUnit.SECONDS; + +suite("refactor_storage_param_s3_load", "p0,external,external_docker") { + String enabled = context.config.otherConfigs.get("enableRefactorParamsTest") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return + } + String ak = context.config.otherConfigs.get("AWSAK") + String sk = context.config.otherConfigs.get("AWSSK") + String endpoint = "s3.ap-northeast-1.amazonaws.com" + String region = "ap-northeast-1" + String bucket = "selectdb-qa-datalake-test" + + def s3table = "test_s3load"; + sql """ + drop table if exists ${s3table}; + """ + sql """ + CREATE TABLE ${s3table}( + user_id BIGINT NOT NULL COMMENT "user id", + name VARCHAR(20) COMMENT "name", + age INT COMMENT "age" + ) + DUPLICATE KEY(user_id) + DISTRIBUTED BY HASH(user_id) BUCKETS 10 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${s3table} values (1, 'a', 10); + """ + + def insertResult = sql """ + SELECT count(1) FROM ${s3table} + """ + + println "insertResult: ${insertResult}" + assert insertResult.get(0).get(0) == 1 + + def outfile_to_S3 = { objBucket, objEndpoint, objRegion, objAk, objSk -> + def outFilePath = "${objBucket}/outfile_different_s3/exp_" + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${s3table} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS CSV + PROPERTIES ( + "s3.endpoint" = "${objEndpoint}", + "s3.region" = "${objRegion}", + "s3.secret_key"="${objSk}", + "s3.access_key" = "${objAk}" + ); + """ + return res[0][3] + } + def outfile_path = outfile_to_S3(bucket, endpoint, region, ak, sk); + def filePath = outfile_path.replace("s3://${bucket}", "") + + def s3Load = { String objFilePath, String objBucket, String objEndpointName, String objEndpoint, String objRegionName, String objRegion, String objAkName, String objAk, String objSkName, String objSk, String usePathStyle -> + + def dataCountResult = sql """ + SELECT count(*) FROM ${s3table} + """ + def label = "s3_load_label_" + System.currentTimeMillis() + def load = sql """ + LOAD LABEL `${label}` ( + data infile ("${objFilePath}") + into table ${s3table} + COLUMNS TERMINATED BY "\\\t" + FORMAT AS "CSV" + ( + user_id, + name, + age + )) + with s3 + ( + "${objEndpointName}" = "${objEndpoint}", + "${objRegionName}" = "${objRegion}", + "${objSkName}"="${objSk}", + "use_path_style" = "${usePathStyle}", + "${objAkName}" = "${objAk}" + ) + PROPERTIES + ( + "timeout" = "3600" + ); + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(5, SECONDS).until({ + def loadResult = sql """ + show load where label = '${label}' + """ + if (loadResult.get(0).get(2) == 'CANCELLED' || loadResult.get(0).get(2) == 'FAILED') { + throw new RuntimeException("load failed") + } + return loadResult.get(0).get(2) == 'FINISHED' + }) + + } + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "AWS_ACCESS_KEY", ak, "AWS_SECRET_KEY", sk, "") + s3Load("http://${bucket}.${endpoint}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + s3Load("http://${bucket}.${endpoint}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "") + s3Load("https://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + } + + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "s3.region", region, "s3.access_key", "", "s3.secret_key", sk, "false") + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "") + + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true") + } + shouldFail { + s3Load("s3://${endpoint}/${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + } + shouldFail { + s3Load("s3://${bucket}/${endpoint}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + } + shouldFail { + s3Load("s3://${endpoint}/${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + } + /*----------obs---------------*/ + ak = context.config.otherConfigs.get("hwYunAk") + sk = context.config.otherConfigs.get("hwYunSk") + endpoint = "obs.cn-north-4.myhuaweicloud.com" + region = "cn-north-4" + bucket = "doris-build"; + outfile_path = outfile_to_S3(bucket, endpoint, region, ak, sk); + filePath = outfile_path.replace("s3://${bucket}", "") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + s3Load("s3://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + s3Load("obs://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "true") + s3Load("obs://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + s3Load("obs://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "") + s3Load("s3://${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "") + s3Load("http://${bucket}.${endpoint}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "") + s3Load("https://${bucket}.${endpoint}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "") + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + } + + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "obs.region", region, "obs.access_key", "", "obs.secret_key", sk, "false") + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "") + + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "true") + } + shouldFail { + s3Load("s3://${endpoint}/${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + } + shouldFail { + s3Load("obs://${bucket}/${endpoint}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + } + shouldFail { + s3Load("obs://${endpoint}/${bucket}${filePath}", bucket, "obs.endpoint", endpoint, "obs.region", region, "obs.access_key", ak, "obs.secret_key", sk, "false") + } + + /*-------------Tencent COS ----------*/ + ak = context.config.otherConfigs.get("txYunAk") + sk = context.config.otherConfigs.get("txYunSk") + endpoint = "cos.ap-beijing.myqcloud.com" + region = "ap-beijing" + bucket = "doris-build-1308700295"; + + outfile_path = outfile_to_S3(bucket, endpoint, region, ak, sk); + filePath = outfile_path.replace("s3://${bucket}", "") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + s3Load("s3://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + s3Load("cos://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "true") + s3Load("cos://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + s3Load("cos://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "") + s3Load("s3://${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "") + s3Load("http://${bucket}.${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "") + s3Load("https://${bucket}.${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "") + s3Load("http://${bucket}.${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + } + + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "cos.region", region, "cos.access_key", "", "cos.secret_key", sk, "false") + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "obs.secret_key", sk, "") + + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "true") + } + shouldFail { + s3Load("s3://${endpoint}/${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + } + shouldFail { + s3Load("cos://${bucket}/${endpoint}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + } + shouldFail { + s3Load("cos://${endpoint}/${bucket}${filePath}", bucket, "cos.endpoint", endpoint, "cos.region", region, "cos.access_key", ak, "cos.secret_key", sk, "false") + } + /************ Aliyun OSS ************/ + /*-----------------Aliyun OSS----------------*/ +/* ak = context.config.otherConfigs.get("aliYunAk") + sk = context.config.otherConfigs.get("aliYunSk") + endpoint = "oss-cn-hongkong.aliyuncs.com" + region = "oss-cn-hongkong" + bucket = "doris-regression-hk"; + + outfile_path = outfile_to_S3(bucket, endpoint, region, ak, sk); + filePath = outfile_path.replace("s3://${bucket}", "") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "s3.endpoint", endpoint, "s3.region", region, "s3.access_key", ak, "s3.secret_key", sk, "false") + s3Load("s3://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "true") + s3Load("s3://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + s3Load("cos://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "true") + s3Load("cos://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + s3Load("cos://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "") + s3Load("s3://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "") + s3Load("http://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "") + s3Load("https://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "") + s3Load("http://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "true") + s3Load("http://${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + } + + shouldFail { + s3Load("https://${bucket}${filePath}", bucket, "", endpoint, "oss.region", region, "oss.access_key", "", "oss.secret_key", sk, "false") + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "") + + } + shouldFail { + s3Load("https://${bucket}/${endpoint}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "true") + } + shouldFail { + s3Load("s3://${endpoint}/${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + } + shouldFail { + s3Load("oss://${bucket}/${endpoint}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + } + shouldFail { + s3Load("oss://${endpoint}/${bucket}${filePath}", bucket, "oss.endpoint", endpoint, "oss.region", region, "oss.access_key", ak, "oss.secret_key", sk, "false") + } + */ + + +} + + diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/test_outfile_s3_storage.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/test_outfile_s3_storage.groovy new file mode 100644 index 00000000000000..326d4235d0a1bc --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/test_outfile_s3_storage.groovy @@ -0,0 +1,218 @@ +// 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. + +suite("test_outfile_s3_storage", "p0,external,external_docker") { + String enabled = context.config.otherConfigs.get("enableRefactorParamsTest") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return + } + def export_table_name = "test_outfile_s3_storage" + + def s3_tvf = {bucket, s3_endpoint, region, ak, sk, path -> + // http schema + order_qt_s3_tvf_1_http """ SELECT * FROM S3 ( + "uri" = "http://${bucket}.${s3_endpoint}${path}0.parquet", + "s3.access_key"= "${ak}", + "s3.secret_key" = "${sk}", + "format" = "parquet", + "region" = "${region}" + ); + """ + } + + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE IF NOT EXISTS ${export_table_name} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `Name` STRING COMMENT "用户年龄", + `Age` int(11) NULL + ) + DISTRIBUTED BY HASH(user_id) BUCKETS 3 + PROPERTIES("replication_num" = "1"); + """ + StringBuilder sb = new StringBuilder() + int i = 1 + for (; i < 10; i ++) { + sb.append(""" + (${i}, 'ftw-${i}', ${i + 18}), + """) + } + sb.append(""" + (${i}, NULL, NULL) + """) + sql """ INSERT INTO ${export_table_name} VALUES + ${sb.toString()} + """ + qt_select_export """ SELECT * FROM ${export_table_name} t ORDER BY user_id; """ + + + String ak = "" + String sk = "" + String s3_endpoint = "" + String region = "" + String bucket = "" + + /******************************************************************************************************* + ***************************** TEST AWS ***************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("AWSAK") + sk = context.config.otherConfigs.get("AWSSK") + s3_endpoint = "s3.ap-northeast-1.amazonaws.com" + region = "ap-northeast-1" + bucket = "selectdb-qa-datalake-test" + + // 1. test s3 schema + def outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + def res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + def outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + + // 2. test AWS_ENDPOINT + outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "AWS_ENDPOINT" = "${s3_endpoint}", + "AWS_REGION" = "${region}", + "AWS_SECRET_KEY"="${sk}", + "AWS_ACCESS_KEY" = "${ak}" + ); + """ + outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + + } finally { + } + + /******************************************************************************************************* + ***************************** TEST COS & COSN ************************************************* + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("txYunAk") + sk = context.config.otherConfigs.get("txYunSk") + s3_endpoint = "cos.ap-beijing.myqcloud.com" + region = "ap-beijing" + bucket = "doris-build-1308700295"; + + // 1. test s3 schema + def outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + def res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + def outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + + // 2. test AWS_ENDPOINT + outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "AWS_ENDPOINT" = "${s3_endpoint}", + "AWS_REGION" = "${region}", + "AWS_SECRET_KEY"="${sk}", + "AWS_ACCESS_KEY" = "${ak}" + ); + """ + outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + + } finally { + } + + /******************************************************************************************************* + ***************************** TEST OSS ******************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("aliYunAk") + sk = context.config.otherConfigs.get("aliYunSk") + s3_endpoint = "oss-cn-hongkong.aliyuncs.com" + region = "oss-cn-hongkong" + bucket = "doris-regression-hk"; + + // 1. test s3 schema + def outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + def res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + def outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + + // 2. test AWS_ENDPOINT + outFilePath = "${bucket}/test_outfile_s3_storage/exp_" + res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "AWS_ENDPOINT" = "${s3_endpoint}", + "AWS_REGION" = "${region}", + "AWS_SECRET_KEY"="${sk}", + "AWS_ACCESS_KEY" = "${ak}" + ); + """ + outfile_url = res[0][3]; + s3_tvf(bucket, s3_endpoint, region, ak, sk, outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)); + } finally { + + } + + + /******************************************************************************************************* + ***************************** TEST OBS ******************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("hwYunAk") + sk = context.config.otherConfigs.get("hwYunSk") + s3_endpoint = "obs.cn-north-4.myhuaweicloud.com" + region = "cn-north-4" + bucket = "doris-build"; + } finally { + } + +} diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/test_s3_tvf_s3_storage.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/test_s3_tvf_s3_storage.groovy new file mode 100644 index 00000000000000..c4ce3595bf8647 --- /dev/null +++ b/regression-test/suites/external_table_p0/refactor_storage_param/test_s3_tvf_s3_storage.groovy @@ -0,0 +1,265 @@ +// 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. +import static groovy.test.GroovyAssert.shouldFail + +suite("test_s3_tvf_s3_storage", "p0,external,external_docker") { + String enabled = context.config.otherConfigs.get("enableRefactorParamsTest") + if (enabled == null || enabled.equalsIgnoreCase("false")) { + return + } + def export_table_name = "test_s3_tvf_s3_storage" + + def outfile_to_S3 = { bucket, s3_endpoint, region, ak, sk -> + def outFilePath = "${bucket}/outfile_different_s3/exp_" + // select ... into outfile ... + def res = sql """ + SELECT * FROM ${export_table_name} t ORDER BY user_id + INTO OUTFILE "s3://${outFilePath}" + FORMAT AS parquet + PROPERTIES ( + "s3.endpoint" = "${s3_endpoint}", + "s3.region" = "${region}", + "s3.secret_key"="${sk}", + "s3.access_key" = "${ak}" + ); + """ + return res[0][3] + } + + + sql """ DROP TABLE IF EXISTS ${export_table_name} """ + sql """ + CREATE TABLE IF NOT EXISTS ${export_table_name} ( + `user_id` LARGEINT NOT NULL COMMENT "用户id", + `Name` STRING COMMENT "用户年龄", + `Age` int(11) NULL + ) + DISTRIBUTED BY HASH(user_id) BUCKETS 3 + PROPERTIES("replication_num" = "1"); + """ + StringBuilder sb = new StringBuilder() + int i = 1 + for (; i < 10; i++) { + sb.append(""" + (${i}, 'ftw-${i}', ${i + 18}), + """) + } + sb.append(""" + (${i}, NULL, NULL) + """) + sql """ INSERT INTO ${export_table_name} VALUES + ${sb.toString()} + """ + def insert_result = sql """ SELECT * FROM ${export_table_name} t ORDER BY user_id; """ + assert insert_result.size() == 10 + + String ak = "" + String sk = "" + String s3_endpoint = "" + String region = "" + String bucket = "" + String outfile_url = "" + + def s3_tvf = { uri_prefix, endpoint_key, ak_key, sk_key, region_key, is_path_style -> + // http schema + def queryResult= sql """ SELECT * FROM S3 ( + "uri" = "${uri_prefix}${outfile_url.substring(5 + bucket.length(), outfile_url.length() - 1)}0.parquet", + "${endpoint_key}" = "${s3_endpoint}", + "${ak_key}"= "${ak}", + "${sk_key}" = "${sk}", + "${region_key}" = "${region}", + "use_path_style" = "${is_path_style}", + "format" = "parquet" + ); + """ + assert queryResult.size() == 10 + } + + + /******************************************************************************************************* + ***************************** TEST AWS ***************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("AWSAK") + sk = context.config.otherConfigs.get("AWSSK") + s3_endpoint = "s3.ap-northeast-1.amazonaws.com" + region = "ap-northeast-1" + bucket = "selectdb-qa-datalake-test" + + outfile_url = outfile_to_S3(bucket, s3_endpoint, region, ak, sk) + + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "s3.region", "false"); + + //s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key" , "s3.secret_key", "region", "true"); + s3_tvf("http://${bucket}.${s3_endpoint}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key", "s3.secret_key", "region", "true"); + //s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "false"); + // s3_tvf("s3://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "s3.region", "true"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "s3.region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "AWS_REGION", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + } finally { + } + + /******************************************************************************************************* + ***************************** TEST COS & COSN ************************************************* + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("txYunAk") + sk = context.config.otherConfigs.get("txYunSk") + s3_endpoint = "cos.ap-beijing.myqcloud.com" + region = "ap-beijing" + bucket = "doris-build-1308700295"; + + + outfile_url = outfile_to_S3(bucket, s3_endpoint, region, ak, sk) + + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "s3.region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "cos.access_key", "cos.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "cos.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key", "s3.secret_key", "region", "true"); + shouldFail { + s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key", "s3.secret_key", "region", "false"); + } + + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "s3.region", "true"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "s3.region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "AWS_REGION", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "cos.endpoint", "cos.access_key", "cos.secret_key", "cos.region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "cos.access_key", "cos.secret_key", "cos.region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + } finally { + } + + /******************************************************************************************************* + ***************************** TEST OSS ******************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("aliYunAk") + sk = context.config.otherConfigs.get("aliYunSk") + s3_endpoint = "oss-cn-hongkong.aliyuncs.com" + region = "oss-cn-hongkong" + bucket = "doris-regression-hk"; + + + outfile_url = outfile_to_S3(bucket, s3_endpoint, region, ak, sk) + + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "s3.region", "false"); + shouldFail { + // it's OSS + s3_tvf("http://${bucket}.${s3_endpoint}", "", "cos.access_key", "cos.secret_key", "region", "false"); + } + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + //endpoint field is no valid, so we extract the endpoint from uri + s3_tvf("http://${bucket}.${s3_endpoint}", "cos.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + // TODO(ftw): Note this case + // s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "true"); + + // s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "false"); + // s3_tvf("s3://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + // TODO(ftw): Note this case + // s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key" , "s3.secret_key", "s3.region", "true"); + + // s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY" , "AWS_SECRET_KEY", "region", "false"); + // s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY" , "AWS_SECRET_KEY", "s3.region", "false"); + // s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY" , "AWS_SECRET_KEY", "AWS_REGION", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + // s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key" , "AWS_SECRET_KEY", "region", "false"); + // s3_tvf("s3://${bucket}", "cos.endpoint", "cos.access_key" , "cos.secret_key", "cos.region", "false"); + // s3_tvf("s3://${bucket}", "s3.endpoint", "cos.access_key" , "cos.secret_key", "cos.region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + } finally { + + } + + + /******************************************************************************************************* + ***************************** TEST OBS ******************************************************** + *******************************************************************************************************/ + try { + ak = context.config.otherConfigs.get("hwYunAk") + sk = context.config.otherConfigs.get("hwYunSk") + s3_endpoint = "obs.cn-north-4.myhuaweicloud.com" + region = "cn-north-4" + bucket = "doris-build"; + + + outfile_url = outfile_to_S3(bucket, s3_endpoint, region, ak, sk) + + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "s3.region", "false"); + shouldFail { + s3_tvf("http://${bucket}.${s3_endpoint}", "", "cos.access_key", "cos.secret_key", "region", "false"); + } + s3_tvf("http://${bucket}.${s3_endpoint}", "", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("http://${bucket}.${s3_endpoint}", "cos.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + shouldFail { + s3_tvf("http://${bucket}.${s3_endpoint}", "cos.endpoint", "s3.access_key", "s3.secret_key", "region", "true"); + } + + s3_tvf("http://${bucket}.${s3_endpoint}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + + s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key", "s3.secret_key", "region", "true"); + shouldFail { + s3_tvf("http://${s3_endpoint}/${bucket}", "", "s3.access_key", "s3.secret_key", "region", "false"); + } + // should support in 2.1&3.0 s3_tvf("s3://${s3_endpoint}/${bucket}", "", "s3.access_key" , "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "s3.region", "true"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "s3.region", "false"); + s3_tvf("s3://${bucket}", "AWS_ENDPOINT", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "AWS_REGION", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "AWS_ACCESS_KEY", "AWS_SECRET_KEY", "region", "false"); + s3_tvf("s3://${bucket}", "s3.endpoint", "s3.access_key", "AWS_SECRET_KEY", "region", "false"); + shouldFail { + s3_tvf("s3://${bucket}", "cos.endpoint", "cos.access_key", "cos.secret_key", "cos.region", "false"); + } + shouldFail{ + s3_tvf("s3://${bucket}", "s3.endpoint", "cos.access_key", "cos.secret_key", "cos.region", "false"); + } + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + s3_tvf("cos://${bucket}", "s3.endpoint", "s3.access_key", "s3.secret_key", "region", "false"); + } finally { + } +} diff --git a/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy b/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy index 74cb1e320aaa16..f6b3538f021704 100644 --- a/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy +++ b/regression-test/suites/external_table_p0/tvf/test_hdfs_tvf.groovy @@ -368,7 +368,7 @@ suite("test_hdfs_tvf","external,hive,tvf,external_docker") { """ // check exception - exception """Properties 'uri' is required""" + exception """props must contain uri""" } // test exception @@ -381,7 +381,7 @@ suite("test_hdfs_tvf","external,hive,tvf,external_docker") { """ // check exception - exception """Invalid export path, there is no schema of URI found. please check your path""" + exception """Invalid uri: xx""" } // test exception diff --git a/regression-test/suites/load_p0/broker_load/test_domain_connection_and_ak_sk_correction.groovy b/regression-test/suites/load_p0/broker_load/test_domain_connection_and_ak_sk_correction.groovy index 36073df4b07a2c..c88b6c8fcf31be 100644 --- a/regression-test/suites/load_p0/broker_load/test_domain_connection_and_ak_sk_correction.groovy +++ b/regression-test/suites/load_p0/broker_load/test_domain_connection_and_ak_sk_correction.groovy @@ -102,7 +102,7 @@ suite("test_domain_connection_and_ak_sk_correction", "load_p0") { assertTrue(false. "The endpoint is wrong, so the connection test should fale") } catch (Exception e) { logger.info("the second sql exception result is {}", e.getMessage()) - assertTrue(e.getMessage().contains("Failed to access object storage, message="), e.getMessage()) + assertTrue(e.getMessage().contains("Invalid endpoint format"), e.getMessage()) } label = UUID.randomUUID().toString().replace("-", "") @@ -132,7 +132,7 @@ suite("test_domain_connection_and_ak_sk_correction", "load_p0") { } label = UUID.randomUUID().toString().replace("-", "") - try { + result = sql """ LOAD LABEL ${label} ( @@ -155,10 +155,36 @@ suite("test_domain_connection_and_ak_sk_correction", "load_p0") { ); """ logger.info("the fourth sql result is {}", result) - assertTrue(false. "in the second DATA INFILE, the first bucket is wrong, so the sql should fail") - } catch (Exception e) { - logger.info("the fourth sql exception result is {}", e.getMessage()) - assertTrue(e.getMessage().contains("Failed to access object storage, message="), e.getMessage()) + int totalWaitTime = 0 + int pollInterval = 5 + int timeout = 120 + while (totalWaitTime < timeout) { + def loadResult = sql """ + SHOW LOAD WHERE label="${label}" + """ + + if (loadResult == null || loadResult.isEmpty()) { + return false + } else if (loadResult.get(0).get(2) in ['CANCELLED', 'FAILED']) { + break + } else if (loadResult.get(0).get(2) == 'FINISHED') { + throw new RuntimeException("load success, but the first bucket is wrong, so the sql should fail") + } else { + println("load status is ${loadResult.get(0).get(2)}") + Thread.sleep(pollInterval * 1000L) + totalWaitTime += pollInterval + } + + + } + + if (totalWaitTime >= timeout) { + def queryLoadResult = sql """ + SHOW LOAD WHERE label="${label}" + """ + if (queryLoadResult != null && queryLoadResult.get(0).get(2) == 'FINISHED') { + throw new RuntimeException("load success, but the first bucket is wrong, so the sql should fail") + } } sql """ DROP TABLE IF EXISTS ${tableName} FORCE""" sql """ DROP TABLE IF EXISTS ${tableNameOrders} FORCE""" From 649659a2e43d1cc406683e1482b857d75513552d Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Mon, 30 Jun 2025 11:15:31 +0800 Subject: [PATCH 2/5] fix --- .../apache/doris/backup/BackupHandler.java | 32 ++----- .../org/apache/doris/backup/RestoreJob.java | 83 +++++++------------ 2 files changed, 35 insertions(+), 80 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index dbf197f9fe9254..a65269d4f0b19d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -638,17 +638,18 @@ private void restore(Repository repository, Database db, RestoreStmt stmt) throw jobInfo.getBackupTime(), TimeUtils.getDatetimeFormatWithHyphenWithTimeZone()); restoreJob = new RestoreJob(stmt.getLabel(), backupTimestamp, db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), - stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), stmt.reserveColocate(), + stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), stmt.isForceReplace(), env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); } else { restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(), - db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), - stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), stmt.reserveColocate(), - stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), stmt.isCleanTables(), - stmt.isCleanPartitions(), stmt.isAtomicRestore(), stmt.isForceReplace(), - env, repository.getId()); + db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), + stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), + stmt.reserveDynamicPartitionEnable(), + stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), + stmt.isForceReplace(), + env, repository.getId()); } env.getEditLog().logRestoreJob(restoreJob); @@ -807,25 +808,6 @@ public void checkAndFilterRestoreOlapTableExistInSnapshot(Map unfinishedSignatureToId = Maps.newConcurrentMap(); - private List colocatePersistInfos = Lists.newArrayList(); - // the meta version is used when reading backup meta from file. // we do not persist this field, because this is just a temporary solution. // the true meta version should be get from backup job info, which is saved when doing backup job. @@ -236,9 +231,8 @@ public RestoreJob(JobType jobType) { public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, - boolean reserveColocate, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, - boolean isCleanTables, boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, - long repoId) { + boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, + boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, long repoId) { super(JobType.RESTORE, label, dbId, dbName, timeoutMs, env, repoId); this.backupTimestamp = backupTs; this.jobInfo = jobInfo; @@ -247,8 +241,8 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu this.state = RestoreJobState.PENDING; this.metaVersion = metaVersion; this.reserveReplica = reserveReplica; - this.reserveColocate = reserveColocate; - // if backup snapshot is come from a cluster with force replication allocation, ignore the origin allocation + // if backup snapshot is come from a cluster with force replication allocation, + // ignore the origin allocation if (jobInfo.isForceReplicationAllocation) { this.reserveReplica = false; } @@ -261,7 +255,6 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu this.isForceReplace = isForceReplace; } properties.put(PROP_RESERVE_REPLICA, String.valueOf(reserveReplica)); - properties.put(PROP_RESERVE_COLOCATE, String.valueOf(reserveColocate)); properties.put(PROP_RESERVE_DYNAMIC_PARTITION_ENABLE, String.valueOf(reserveDynamicPartitionEnable)); properties.put(PROP_IS_BEING_SYNCED, String.valueOf(isBeingSynced)); properties.put(PROP_CLEAN_TABLES, String.valueOf(isCleanTables)); @@ -272,14 +265,13 @@ public RestoreJob(String label, String backupTs, long dbId, String dbName, Backu public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, - boolean reserveColocate, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, - boolean isCleanTables, boolean isCleanPartitions, boolean isAtomicRestore, boolean isForeReplace, Env env, - long repoId, + boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, + boolean isCleanPartitions, boolean isAtomicRestore, boolean isForceReplace, Env env, long repoId, BackupMeta backupMeta) { this(label, backupTs, dbId, dbName, jobInfo, allowLoad, replicaAlloc, timeoutMs, metaVersion, reserveReplica, - reserveColocate, reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, - isAtomicRestore, isForeReplace, env, repoId); - + reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, isAtomicRestore, + isForceReplace, env, + repoId); this.backupMeta = backupMeta; } @@ -299,10 +291,6 @@ public boolean isBeingSynced() { return isBeingSynced; } - public List getColocatePersistInfos() { - return colocatePersistInfos; - } - public synchronized boolean finishTabletSnapshotTask(SnapshotTask task, TFinishTaskRequest request) { if (checkTaskStatus(task, task.getJobId(), request)) { return false; @@ -717,12 +705,6 @@ private void checkAndPrepareMeta() { OlapTable localOlapTbl = (OlapTable) localTbl; OlapTable remoteOlapTbl = (OlapTable) remoteTbl; - if (localOlapTbl.isColocateTable() || (reserveColocate && remoteOlapTbl.isColocateTable())) { - status = new Status(ErrCode.COMMON_ERROR, "Not support to restore to local table " - + tableName + " with colocate group."); - return; - } - localOlapTbl.readLock(); try { List intersectPartNames = Lists.newArrayList(); @@ -856,8 +838,7 @@ private void checkAndPrepareMeta() { // reset all ids in this table String srcDbName = jobInfo.dbName; - Status st = remoteOlapTbl.resetIdsForRestore(env, db, replicaAlloc, reserveReplica, - reserveColocate, colocatePersistInfos, srcDbName); + Status st = remoteOlapTbl.resetIdsForRestore(env, db, replicaAlloc, reserveReplica, srcDbName); if (!st.ok()) { status = st; return; @@ -1208,11 +1189,11 @@ private Status bindLocalAndRemoteOlapTableReplicas( List localTablets = localIndex.getTablets(); List remoteTablets = index.getTablets(); if (localTablets.size() != remoteTablets.size()) { - LOG.warn("skip bind replicas because the size of local tablet {} is not equals to " - + "the remote {}, is_atomic_restore=true, remote table={}, remote index={}, " - + "local table={}, local index={}", localTablets.size(), remoteTablets.size(), - remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId); - continue; + return new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local tablet %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d", localTablets.size(), remoteTablets.size(), + remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId)); } for (int i = 0; i < remoteTablets.size(); i++) { Tablet localTablet = localTablets.get(i); @@ -1220,13 +1201,13 @@ private Status bindLocalAndRemoteOlapTableReplicas( List localReplicas = localTablet.getReplicas(); List remoteReplicas = remoteTablet.getReplicas(); if (localReplicas.size() != remoteReplicas.size()) { - LOG.warn("skip bind replicas because the size of local replicas {} is not equals to " - + "the remote {}, is_atomic_restore=true, remote table={}, remote index={}, " - + "local table={}, local index={}, local tablet={}, remote tablet={}", - localReplicas.size(), remoteReplicas.size(), remoteOlapTbl.getId(), - index.getId(), localOlapTbl.getId(), localIndexId, localTablet.getId(), - remoteTablet.getId()); - continue; + return new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local replicas %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d, local replicas=%d, remote replicas=%d", + localTablets.size(), remoteTablets.size(), remoteOlapTbl.getId(), + index.getId(), localOlapTbl.getId(), localIndexId, localReplicas.size(), + remoteReplicas.size())); } for (int j = 0; j < remoteReplicas.size(); j++) { long backendId = localReplicas.get(j).getBackendIdWithoutException(); @@ -1384,9 +1365,9 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc MaterializedIndexMeta indexMeta = localTbl.getIndexMetaByIndexId(restoredIdx.getId()); List indexes = restoredIdx.getId() == localTbl.getBaseIndexId() ? localTbl.getCopiedIndexes() : null; - List clusterKeyUids = null; + List clusterKeyIndexes = null; if (indexMeta.getIndexId() == localTbl.getBaseIndexId() || localTbl.isShadowIndex(indexMeta.getIndexId())) { - clusterKeyUids = OlapTable.getClusterKeyUids(indexMeta.getSchema()); + clusterKeyIndexes = OlapTable.getClusterKeyIndexes(indexMeta.getSchema()); } for (Tablet restoreTablet : restoredIdx.getTablets()) { TabletRef baseTabletRef = tabletBases == null ? null : tabletBases.get(restoreTablet.getId()); @@ -1438,11 +1419,11 @@ private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable loc LOG.info("set base tablet {} for replica {} in restore job {}, tablet id={}", baseTabletRef.tabletId, restoreReplica.getId(), jobId, restoreTablet.getId()); } - if (!CollectionUtils.isEmpty(clusterKeyUids)) { - task.setClusterKeyUids(clusterKeyUids); - LOG.info("table: {}, partition: {}, index: {}, tablet: {}, cluster key uids: {}", + if (!CollectionUtils.isEmpty(clusterKeyIndexes)) { + task.setClusterKeyIndexes(clusterKeyIndexes); + LOG.info("table: {}, partition: {}, index: {}, tablet: {}, cluster key indexes: {}", localTbl.getId(), restorePart.getId(), restoredIdx.getId(), restoreTablet.getId(), - clusterKeyUids); + clusterKeyIndexes); } batchTask.addTask(task); } @@ -1714,9 +1695,6 @@ private void waitingAllSnapshotsFinished() { state = RestoreJobState.DOWNLOAD; env.getEditLog().logRestoreJob(this); - for (ColocatePersistInfo info : colocatePersistInfos) { - env.getEditLog().logColocateAddTable(info); - } LOG.info("finished making snapshots. {}", this); return; } @@ -2464,11 +2442,6 @@ private void cancelInternal(boolean isReplay) { state = RestoreJobState.CANCELLED; // log env.getEditLog().logRestoreJob(this); - for (ColocatePersistInfo info : colocatePersistInfos) { - Env.getCurrentColocateIndex().removeTable(info.getTableId()); - env.getEditLog().logColocateRemoveTable(info); - } - colocatePersistInfos.clear(); LOG.info("finished to cancel restore job. current state: {}. is replay: {}. {}", curState.name(), isReplay, this); From 9508ee5ee9136348c5ea3c7076ccc03bba8b3337 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Tue, 1 Jul 2025 15:10:44 +0800 Subject: [PATCH 3/5] fix --- .../doris/tablefunction/ExternalFileTableValuedFunction.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java index 26ca67c43d6f53..636db0943f67f4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java @@ -381,9 +381,6 @@ private PFetchTableSchemaRequest getFetchTableStructureRequest() throws TExcepti Map beProperties = new HashMap<>(); beProperties.putAll(backendConnectProperties); fileScanRangeParams.setProperties(beProperties); - if (fileFormatProperties instanceof CsvFileFormatProperties) { - fileScanRangeParams.setTextSerdeType(((CsvFileFormatProperties) fileFormatProperties).getTextSerdeType()); - } fileScanRangeParams.setFileAttributes(getFileAttributes()); ConnectContext ctx = ConnectContext.get(); fileScanRangeParams.setLoadId(ctx.queryId()); From 9ff1717f4ffb7eb9e7b41cd63f56b5f9407ff1b4 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Wed, 2 Jul 2025 11:12:46 +0800 Subject: [PATCH 4/5] add log --- .../external_table_p0/refactor_storage_param/s3_load.groovy | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy b/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy index d9ed6a401896c8..df36c202e16d30 100644 --- a/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy +++ b/regression-test/suites/external_table_p0/refactor_storage_param/s3_load.groovy @@ -111,7 +111,8 @@ suite("refactor_storage_param_s3_load", "p0,external,external_docker") { show load where label = '${label}' """ if (loadResult.get(0).get(2) == 'CANCELLED' || loadResult.get(0).get(2) == 'FAILED') { - throw new RuntimeException("load failed") + println("load failed: " + loadResult.get(0)) + throw new RuntimeException("load failed"+ loadResult.get(0)) } return loadResult.get(0).get(2) == 'FINISHED' }) From c390135d8bc523d18185834045cab15afc064e86 Mon Sep 17 00:00:00 2001 From: Calvin Kirs Date: Mon, 7 Jul 2025 10:46:58 +0800 Subject: [PATCH 5/5] fix --- .../main/java/org/apache/doris/datasource/LoadScanProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java index af746592222d4b..a4efb019e67483 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/LoadScanProvider.java @@ -91,7 +91,7 @@ public FileLoadScanNode.ParamCreateContext createContext(Analyzer analyzer) thro params.setFormatType(formatType(fileGroupInfo.getFileGroup().getFileFormat())); params.setCompressType(fileGroupInfo.getFileGroup().getCompressType()); params.setStrictMode(fileGroupInfo.isStrictMode()); - params.setProperties(fileGroupInfo.getBrokerDesc().getProperties()); + params.setProperties(fileGroupInfo.getBrokerDesc().getBackendConfigProperties()); if (fileGroupInfo.getBrokerDesc().getFileType() == TFileType.FILE_HDFS) { THdfsParams tHdfsParams = HdfsResource.generateHdfsParam(fileGroupInfo.getBrokerDesc() .getBackendConfigProperties());